From 92db25fabc3c5d824808d39a25e9789c12b95f3e Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 12 Jan 2017 13:56:21 -0600 Subject: [PATCH 01/35] coordinator lookups mgmt improvements --- .../lookup/KafkaLookupExtractorFactory.java | 2 +- .../NamespaceLookupExtractorFactory.java | 2 +- .../server/lookup/LoadingLookupFactory.java | 11 - .../server/lookup/PollingLookupFactory.java | 12 - .../query/dimension/LookupDimensionSpec.java | 2 +- .../io/druid/query/lookup/LookupBean.java | 6 + .../query/lookup/LookupExtractorFactory.java | 7 - .../LookupExtractorFactoryContainer.java | 106 +++ .../query/lookup/LookupReferencesManager.java | 535 ++++++------ .../io/druid/query/lookup/LookupsState.java | 75 ++ .../lookup/RegisteredLookupExtractionFn.java | 4 +- .../LookupExtractorFactoryContainerTest.java | 87 ++ .../RegisteredLookupExtractionFnTest.java | 11 +- .../lookup/LookupIntrospectionResource.java | 10 +- .../io/druid/query/lookup/LookupModule.java | 71 +- .../lookup/MapLookupExtractorFactory.java | 13 - .../http/LookupCoordinatorResource.java | 20 +- .../resource/AbstractListenerHandler.java | 6 +- .../listener/resource/ListenerHandler.java | 4 + .../listener/resource/ListenerResource.java | 21 + .../cache/LookupCoordinatorManager.java | 696 +++++++-------- .../cache/LookupCoordinatorManagerConfig.java | 55 +- .../LookupExtractorFactoryMapContainer.java | 99 +++ .../lookup/cache/LookupsStateWithMap.java | 108 +++ .../dimension/LookupDimensionSpecTest.java | 7 +- .../LookupIntrospectionResourceImplTest.java | 7 +- .../LookupIntrospectionResourceTest.java | 20 +- .../lookup/LookupReferencesManagerTest.java | 572 +++--------- .../lookup/MapLookupExtractorFactoryTest.java | 14 +- .../http/LookupCoordinatorResourceTest.java | 46 +- .../resource/AbstractListenerHandlerTest.java | 6 + .../resource/ListenerResourceTest.java | 6 + .../LookupCoordinatorManagerConfigTest.java | 20 +- .../cache/LookupCoordinatorManagerTest.java | 825 +++++++----------- 34 files changed, 1757 insertions(+), 1729 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java create mode 100644 processing/src/main/java/io/druid/query/lookup/LookupsState.java create mode 100644 processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java create mode 100644 server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java create mode 100644 server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java index ebd99c430ade..6b16ccc77333 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java @@ -327,7 +327,7 @@ public boolean close() } @Override - public boolean replaces(@Nullable LookupExtractorFactory other) + public boolean equals(@Nullable Object other) { if (this == other) { return false; diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index 54badfcde0e7..60de8e5be66a 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -150,7 +150,7 @@ public boolean close() } @Override - public boolean replaces(@Nullable LookupExtractorFactory other) + public boolean equals(@Nullable Object other) { if (other != null && other instanceof NamespaceLookupExtractorFactory) { NamespaceLookupExtractorFactory that = (NamespaceLookupExtractorFactory) other; diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java index 4c6b57bffa42..3e9ffdf39aeb 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java @@ -92,17 +92,6 @@ public synchronized boolean close() return !started.get(); } - @Override - public boolean replaces( - @Nullable LookupExtractorFactory lookupExtractorFactory - ) - { - if (lookupExtractorFactory == null) { - return true; - } - return !this.equals(lookupExtractorFactory); - } - @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java index 2333ea48d163..9322e4450f5f 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java @@ -107,18 +107,6 @@ public boolean close() } } - @Override - public boolean replaces( - @Nullable LookupExtractorFactory lookupExtractorFactory - ) - { - if (lookupExtractorFactory == null) { - return true; - } - - return !this.equals(lookupExtractorFactory); - } - @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() 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..8e038be3c1e4 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupBean.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupBean.java @@ -24,8 +24,14 @@ //TODO merge this code to the same definition when pr/1576 is merged class LookupBean { + //kept for backward compatibility with druid ver <= 0.9.2 persisted snapshots + @Deprecated @JsonProperty LookupExtractorFactory factory; + + @JsonProperty + LookupExtractorFactoryContainer container; + @JsonProperty String name; 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..113d9914a738 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java @@ -50,13 +50,6 @@ 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 - */ - boolean replaces(@Nullable LookupExtractorFactory other); /** * @return Returns the actual introspection request handler, can return {@code null} if it is not supported. 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..ebab64de9fb9 --- /dev/null +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java @@ -0,0 +1,106 @@ +/* + * 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; + +/** + */ +public class LookupExtractorFactoryContainer +{ + private String version; + private 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.equals(other.getLookupExtractorFactory()); + } + + if (version == null && other.getVersion() != null) { + return false; + } + + if (version != null && other.getVersion() == null) { + return true; + } + + return version.compareTo(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; + + if (version != null ? !version.equals(that.version) : that.version != null) { + return false; + } + return lookupExtractorFactory.equals(that.lookupExtractorFactory); + + } + + @Override + public int hashCode() + { + int result = version != null ? version.hashCode() : 0; + result = 31 * result + lookupExtractorFactory.hashCode(); + return result; + } +} 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..c3326c6f8514 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -21,27 +21,33 @@ import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; 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.inject.Inject; +import com.metamx.emitter.EmittingLogger; +import io.druid.concurrent.Execs; 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 javax.annotation.concurrent.GuardedBy; import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; 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.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -52,44 +58,112 @@ * 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 static final EmittingLogger LOG = new EmittingLogger(LookupReferencesManager.class); + private final ReadWriteLock startStopLock = new ReentrantReadWriteLock(true); - private final AtomicBoolean started = new AtomicBoolean(false); + + @VisibleForTesting + volatile boolean started = false; + + @GuardedBy("startStopLock") + private final Map lookupMap = new HashMap<>(); private final LookupSnapshotTaker lookupSnapshotTaker; + @VisibleForTesting + final BlockingQueue queue = new ArrayBlockingQueue<>(10000); + + private volatile ExecutorService exec; + + //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(); + try { - if (!started.getAndSet(true)) { + if (!started) { + LOG.info("LookupReferencesManager is starting."); + if (lookupSnapshotTaker != null) { final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); for (LookupBean lookupBean : lookupBeanList) { - this.put(lookupBean.name, lookupBean.factory); + LookupExtractorFactoryContainer container = lookupBean.container; + + //for backward compatibility with druid ver <= 0.9.2 persisted snapshots + if (lookupBean.container == null) { + container = new LookupExtractorFactoryContainer(null, lookupBean.factory); + } + + if (container.getLookupExtractorFactory().start()) { + lookupMap.put(lookupBean.name, container); + } else { + throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.name, container); + } } } - LOGGER.info("Started lookup factory references manager"); + + if (exec == null && !testMode) { + exec = Execs.singleThreaded("lookup-reference-manager-%d"); + exec.execute( + new Runnable() + { + @Override + public void run() + { + try { + while (started && !Thread.currentThread().isInterrupted()) { + try { + queue.take().handle(); + } + catch (InterruptedException ex) { + LOG.warn("interrupted, going down... lookups are not managed anymore"); + Thread.currentThread().interrupt(); + } + catch (Exception ex) { + LOG.makeAlert(ex, "Exception occured while lookup notice handling.").emit(); + } + catch (Throwable t) { + LOG.makeAlert(t, "Fatal error occured while lookup notice handling.").emit(); + throw t; + } + } + } + finally { + LOG.info("Lookup Mgmt loop exited, Lookup notices are not handled anymore."); + } + } + } + ); + } + + started = true; + + LOG.info("LookupReferencesManager is started."); } - } - finally { + } finally { startStopLock.writeLock().unlock(); } } @@ -98,287 +172,250 @@ public void start() public void stop() { startStopLock.writeLock().lock(); - try { - if (started.getAndSet(false)) { - if (lookupSnapshotTaker != null) { - lookupSnapshotTaker.takeSnapshot(getAllAsList()); + + if (started) { + try { + LOG.info("LookupReferencesManager is stopping."); + started = false; + + if (exec != null) { + exec.shutdownNow(); + exec = null; } - LOGGER.info("Stopping lookup factory references manager"); - for (String lookupName : lookupMap.keySet()) { - lookupMap.remove(lookupName).close(); + + for (Map.Entry e : lookupMap.entrySet()) { + try { + LOG.info("Closing lookup [%s]", e.getKey()); + if (!e.getValue().getLookupExtractorFactory().close()) { + LOG.error("Failed to close lookup [%s]."); + } + } + catch (Exception ex) { + LOG.error(ex, "Failed to close lookup [%s].", e.getKey()); + } } + + lookupMap.clear(); } - } - finally { + finally { + startStopLock.writeLock().unlock(); + } + LOG.info("LookupReferencesManager is stopped."); + } else { startStopLock.writeLock().unlock(); } } - /** - * @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) + public void add(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) { + assertStarted(); + try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - try { - assertStarted(); - if (lookupMap.containsKey(lookupName)) { - LOGGER.warn("lookup [%s] is not add, another lookup with the same name already exist", lookupName); - return false; - } - if (!lookupExtractorFactory.start()) { - throw new ISE("start method returned false for lookup [%s]", lookupName); - } - 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); - } + if (!queue.offer(new LoadNotice(lookupName, lookupExtractorFactoryContainer), 1, TimeUnit.MILLISECONDS)) { + throw new ISE("notice queue add timedout to add [%s] lookup drop notice", lookupName); } - return noPrior; - } - finally { - startStopLock.readLock().unlock(); + } catch (InterruptedException ex) { + throw new ISE(ex, "failed to add [%s] lookup load notice", lookupName); } } - /** - * @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 remove(String lookupName) { - Map failedExtractorFactoryMap = new HashMap<>(); - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } + assertStarted(); + 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); - } - continue; - } - if (lookupSnapshotTaker != null) { - lookupSnapshotTaker.takeSnapshot(getAllAsList()); - } + if (!queue.offer(new DropNotice(lookupName), 1, TimeUnit.MILLISECONDS)) { + throw new ISE("notice queue add timedout to add [%s] lookup drop notice", lookupName); } - if (!failedExtractorFactoryMap.isEmpty()) { - throw new ISE( - "was not able to start the following lookup(s) [%s]", - failedExtractorFactoryMap.keySet().toString() - ); - } - } - finally { - startStopLock.readLock().unlock(); + } catch (InterruptedException ex) { + throw new ISE(ex, "failed to add [%s] lookup drop notice", lookupName); } } - /** - * 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) + @Nullable + public LookupExtractorFactoryContainer get(String lookupName) { - boolean update = false; + assertStarted(); + + startStopLock.readLock().lock(); try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); + return lookupMap.get(lookupName); + } finally { + startStopLock.readLock().unlock(); } - 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); + public LookupsState getAllLookupsState() + { + assertStarted(); - if (prior != null && update) { - if (!prior.close()) { - LOGGER.error("Error closing [%s]:[%s]", lookupName, prior); - } - } + startStopLock.readLock().lock(); + try { + Map lookupsToLoad = new HashMap<>(); + Set lookupsToDrop = new HashSet<>(); - if (!update) { - // We started the lookup, failed a race, and now need to cleanup - if (!lookupExtractorFactory.close()) { - LOGGER.error("Error closing [%s]:[%s]", lookupExtractorFactory); - } + Iterator iter = queue.iterator(); + while (iter.hasNext()) { + Notice notice = iter.next(); + 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()); } } - } - finally { + + return new LookupsState(Maps.newHashMap(lookupMap), lookupsToLoad, lookupsToDrop); + } 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 void takeSnapshot() { - 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(); + if (lookupSnapshotTaker != null) { + startStopLock.readLock().lock(); + + List lookups; + try { + lookups = Lists.newArrayList( + Collections2.transform( + lookupMap.entrySet(), + new Function, LookupBean>() + { + @Nullable + @Override + public LookupBean apply( + @Nullable + Map.Entry input + ) + { + final LookupBean lookupBean = new LookupBean(); + lookupBean.container = input.getValue(); + lookupBean.name = input.getKey(); + return lookupBean; + } + } + ) + ); } + finally { + startStopLock.readLock().unlock(); + } + + lookupSnapshotTaker.takeSnapshot(lookups); } - 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) + private void assertStarted() { - 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(); + if (!started) { + throw new ISE("LookupReferencesManager is not started."); } } - /** - * @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() + @VisibleForTesting + interface Notice { - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - try { - assertStarted(); - return Maps.newHashMap(lookupMap); - } - finally { - startStopLock.readLock().unlock(); - } + void handle(); } - private void assertStarted() throws ISE + private class LoadNotice implements Notice { - if (isClosed()) { - throw new ISE("lookup manager is closed"); + String lookupName; + LookupExtractorFactoryContainer lookupExtractorFactoryContainer; + + public LoadNotice(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) + { + this.lookupName = lookupName; + this.lookupExtractorFactoryContainer = lookupExtractorFactoryContainer; } - } - public boolean isClosed() - { - return !started.get(); + @Override + public void handle() + { + startStopLock.readLock().lock(); + + try { + 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; + } + } finally { + startStopLock.readLock().unlock(); + } + + if (!lookupExtractorFactoryContainer.getLookupExtractorFactory().start()) { + throw new ISE("start method returned false for lookup [%s]:[%s]", lookupName, lookupExtractorFactoryContainer); + } + + startStopLock.writeLock().lock(); + final LookupExtractorFactoryContainer old; + try { + assertStarted(); + old = lookupMap.put(lookupName, lookupExtractorFactoryContainer); + } finally { + startStopLock.writeLock().unlock(); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Loaded lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer); + } + + takeSnapshot(); + + if (old != null) { + if (!old.getLookupExtractorFactory().close()) { + throw new ISE("close method returned false for lookup [%s]:[%s]", lookupName, old); + } + } + } } - private List getAllAsList() + private class DropNotice implements Notice { - 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; - } - } - )); + String lookupName; + + public DropNotice(String lookupName) + { + this.lookupName = lookupName; + } + + @Override + public void handle() + { + startStopLock.writeLock().lock(); + + final LookupExtractorFactoryContainer lookupExtractorFactoryContainer; + + try { + assertStarted(); + lookupExtractorFactoryContainer = lookupMap.remove(lookupName); + } finally { + startStopLock.writeLock().unlock(); + } + + if (lookupExtractorFactoryContainer != null) { + takeSnapshot(); + + if (LOG.isDebugEnabled()) { + 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); + } + } + } } } 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..140c919ec701 --- /dev/null +++ b/processing/src/main/java/io/druid/query/lookup/LookupsState.java @@ -0,0 +1,75 @@ +/* +* 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.Map; +import java.util.Set; + +/** + */ +public class LookupsState +{ + private Map current; + private Map toLoad; + private Set toDrop; + + @JsonCreator + public LookupsState( + @JsonProperty("current") Map current, + @JsonProperty("toLoad") Map toLoad, + @JsonProperty("toDrop") Set toDrop + ) + { + this.current = current; + this.toLoad = toLoad; + this.toDrop = 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 + + '}'; + } +} 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..f0e2e2bebb54 --- /dev/null +++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java @@ -0,0 +1,87 @@ +/* + * 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("v1", actual.getVersion()); + Assert.assertTrue(actual.getLookupExtractorFactory() instanceof TestLookupExtractorFactory); + } + + @JsonTypeName("test") + private static class TestLookupExtractorFactory implements LookupExtractorFactory + { + + @Override + public boolean start() + { + return false; + } + + @Override + public boolean close() + { + return false; + } + + @Nullable + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + return null; + } + + @Override + public LookupExtractor get() + { + return null; + } + } +} 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..7afb627760e8 100644 --- a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java @@ -226,7 +226,8 @@ public void testEquals() private void managerReturnsMap(LookupReferencesManager manager) { - EasyMock.expect(manager.get(EasyMock.eq(LOOKUP_NAME))).andReturn(new LookupExtractorFactory() + EasyMock.expect(manager.get(EasyMock.eq(LOOKUP_NAME))).andReturn( + new LookupExtractorFactoryContainer("v0", new LookupExtractorFactory() { @Override public boolean start() @@ -240,12 +241,6 @@ public boolean close() return false; } - @Override - public boolean replaces(@Nullable LookupExtractorFactory other) - { - return false; - } - @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() @@ -258,6 +253,6 @@ public LookupExtractor get() { return LOOKUP_EXTRACTOR; } - }).anyTimes(); + })).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..051cc7a24673 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; @@ -118,23 +119,50 @@ public LookupListeningResource( { }) { + @Override + public Response handleUpdates( + InputStream inputStream, ObjectMapper mapper + ) + { + final LookupsState state; + try { + state = mapper.readValue(inputStream, LookupsState.class); + } + catch (final IOException ex) { + LOG.debug(ex, "Bad request"); + return Response.status(Response.Status.BAD_REQUEST).entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + + try { + for (Map.Entry e : state.getToLoad().entrySet()) { + manager.add(e.getKey(), e.getValue()); + } + + for (String lookupToDrop : state.getToDrop()) { + manager.remove(lookupToDrop); + } + + 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/query/lookup/MapLookupExtractorFactory.java b/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java index 5ac15478ca38..76d4211d52ae 100644 --- a/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java +++ b/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java @@ -67,19 +67,6 @@ public boolean close() return true; } - /** - * For MapLookups, the replaces consideration is very easy, it simply considers if the other is the same as this one - * - * @param other Some other LookupExtractorFactory which might need replaced - * - * @return true - should replace, false - should not replace - */ - @Override - public boolean replaces(@Nullable LookupExtractorFactory other) - { - return !equals(other); - } - @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() 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..f0a7ab72e5bc 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 @@ -118,6 +118,8 @@ public ObjType apply(Object input) } } + + @Override public final Response handleGET(String id) { @@ -138,7 +140,7 @@ public final Response handleGET(String id) @Override public final Response handleGETAll() { - final Map all; + final Object all; try { all = getAll(); if (all == null) { @@ -200,7 +202,7 @@ public final void use_AbstractListenerHandler_instead() protected abstract @Nullable - Map getAll(); + Object getAll(); /** * Process a POST request of the input items 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..850b15db4ee7 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,7 +19,6 @@ 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; @@ -72,13 +71,15 @@ 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.Random; +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; @@ -91,9 +92,7 @@ public class LookupCoordinatorManager // 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; 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 @@ -120,10 +119,12 @@ public URL apply(HostAndPort input) private final ObjectMapper smileMapper; private final JacksonConfigManager configManager; private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; + private final AtomicReference> knownOldState = new AtomicReference<>(); private final Object startStopSync = new Object(); + // Updated by config watching service - private AtomicReference>>> lookupMapConfigRef; - private volatile Map>> prior_update = ImmutableMap.of(); + private AtomicReference>> lookupMapConfigRef; + private volatile boolean started = false; private volatile ListenableScheduledFuture backgroundManagerFuture = null; private final CountDownLatch backgroundManagerExitedLatch = new CountDownLatch(1); @@ -151,283 +152,28 @@ public LookupCoordinatorManager( ); } - 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 - ) - { - 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() - ) - ); - } - 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(); + 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) { @@ -437,14 +183,24 @@ public boolean updateLookups(final Map>> // Needs update updatedSpec = new HashMap<>(priorSpec); for (final String tier : updateSpec.keySet()) { - final Map> priorTierSpec = priorSpec.get(tier); - final Map> updateTierSpec = updateSpec.get(tier); + final Map priorTierSpec = priorSpec.get(tier); + final Map updateTierSpec = updateSpec.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] can't replace existing spec [%s].", + e.getKey(), + updatedTierSpec.get(e.getKey()) + ); + } + } updatedTierSpec.putAll(updateTierSpec); updatedSpec.put(tier, updatedTierSpec); } @@ -454,7 +210,7 @@ public boolean updateLookups(final Map>> } } - public Map>> getKnownLookups() + public Map> getKnownLookups() { if (!started) { throw new ISE("Not started"); @@ -465,13 +221,13 @@ public Map>> getKnownLookups() public boolean deleteLookup(final String tier, final String lookup, AuditInfo auditInfo) { synchronized (startStopSync) { - final Map>> priorSpec = getKnownLookups(); + 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 +238,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 +262,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,7 +278,6 @@ Map getLookup(final String tier, final String lookupName) return tierLookups.get(lookupName); } - @LifecycleStart public void start() { @@ -534,87 +288,51 @@ public void start() if (executorService.isShutdown()) { throw new ISE("Cannot restart after stop!"); } + lookupMapConfigRef = configManager.watch( LOOKUP_CONFIG_KEY, - new TypeReference>>>() + new TypeReference>>() { }, null ); - final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( + + 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; - } - 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; + lookupMgmtLoop(); } }, 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"); - } + 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.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(); + @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(); + } + } } - } - }); + ); started = true; LOG.debug("Started"); } @@ -640,6 +358,258 @@ public void stop() } } + private void lookupMgmtLoop() + { + // Sanity check for if we are shutting down + if (Thread.currentThread().isInterrupted()) { + LOG.info("Not updating lookups because process was interrupted"); + return; + } + + final Map currState = new HashMap<>(); + + final Map> allLookupTiers = lookupMapConfigRef.get(); + + if (allLookupTiers == null) { + + if (LOG.isDebugEnabled()) { + LOG.debug("Not updating lookups because no data exists"); + } + return; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Starting lookup sync for on all nodes."); + } + + final Random rand = new Random(); + + try { + List> futures = new ArrayList<>(); + for (String tier : allLookupTiers.keySet()) { + + if (LOG.isDebugEnabled()) { + LOG.debug("Starting lookup mgmt for tier [%s].", tier); + } + + final Map tierLookups = allLookupTiers.get(tier); + for (final HostAndPort node : listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tier))) { + + if (LOG.isDebugEnabled()) { + LOG.debug("Starting lookup mgmt for tier [%s] and host [%s:%s].", tier, node.getHostText(), node.getPort()); + } + + futures.add( + executorService.submit( + new Runnable() + { + @Override + public void run() + { + try { + + if (LOG.isDebugEnabled()) { + LOG.debug("Starting lookup sync for node [%s].", node); + } + + LookupsStateWithMap lookupsState = knownOldState.get().get(node); + if (lookupsState == null + || !lookupsState.getToLoad().isEmpty() + || !lookupsState.getToDrop().isEmpty() + || rand.nextBoolean()) { + lookupsState = getLookupStateForNode(node); + + if (LOG.isDebugEnabled()) { + LOG.debug("Received lookups state from node [%s].", node); + } + } + currState.put(node, lookupsState); + + Map toLoad = new HashMap<>(); + for (Map.Entry e : tierLookups.entrySet()) { + String name = e.getKey(); + LookupExtractorFactoryMapContainer lookupToBe = e.getValue(); + + LookupExtractorFactoryMapContainer current = lookupsState.getToLoad().get(name); + if (current == null) { + current = lookupsState.getCurrent().get(name); + } + + if (current == null || lookupToBe.replaces(current)) { + toLoad.put(name, lookupToBe); + } + } + + Set toDrop = new HashSet<>(); + toDrop.addAll(lookupsState.getCurrent().keySet()); + toDrop.addAll(lookupsState.getToLoad().keySet()); + toDrop = Sets.difference(toDrop, lookupsState.getToDrop()); + toDrop = Sets.difference(toDrop, tierLookups.keySet()); + + if (!toLoad.isEmpty() || !toDrop.isEmpty()) { + currState.put(node, updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop))); + + if (LOG.isDebugEnabled()) { + LOG.debug("Sent lookup updates to node [%s].", node); + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Finished lookup sync for node [%s].", node); + } + } + catch (Exception ex) { + LOG.makeAlert(ex, "Failed to manage lookups on node [%s].", node).emit(); + } + } + } + ) + ); + } + } + + final ListenableFuture allFuture = Futures.allAsList(futures); + try { + allFuture.get(lookupCoordinatorManagerConfig.getAllHostTimeout().getMillis(), TimeUnit.MILLISECONDS); + knownOldState.set(currState); + } + catch (Exception ex) { + allFuture.cancel(true); + throw ex; + } + + } catch (Exception ex) { + LOG.makeAlert(ex, "Failed to finish lookup management loop.").emit(); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Finished lookup sync for on all nodes."); + } + } + + @VisibleForTesting + LookupsStateWithMap updateNode( + HostAndPort node, + LookupsStateWithMap lookupsUpdate + ) + throws IOException, InterruptedException, ExecutionException + { + final AtomicInteger returnCode = new AtomicInteger(0); + final AtomicReference reasonString = new AtomicReference<>(null); + + final URL url = getLookupsUpdateURL(node); + + if (LOG.isDebugEnabled()) { + 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())) { + 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 { + try { + final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Update on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), + response + ); + } + return response; + } catch (IOException ex) { + throw new IOException( + String.format("Failed to parse update response from [%s]. response [%s]", url, result), + ex + ); + } + } + } + } + + @VisibleForTesting + LookupsStateWithMap getLookupStateForNode( + HostAndPort node + ) throws IOException, InterruptedException, ExecutionException + { + final URL url = getLookupsURL(node); + final AtomicInteger returnCode = new AtomicInteger(0); + final AtomicReference reasonString = new AtomicReference<>(null); + + if (LOG.isDebugEnabled()) { + 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() == 200) { + try { + final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Get on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), + response + ); + } + return response; + } catch(IOException ex) { + throw new IOException( + String.format( + "Failed to parser GET lookups response from [%s]. response [%s].", + url, + result + ), + ex + ); + } + } 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 IOException( + String.format( + "GET request failed to [%s] : [%d] : [%s] Response: [%s]", + url, + returnCode.get(), + reasonString.get(), + StringUtils.fromUtf8(baos.toByteArray()) + ) + ); + } + } + } + static URL getLookupsURL(HostAndPort druidNode) throws MalformedURLException { return new URL( @@ -650,6 +620,16 @@ static URL getLookupsURL(HostAndPort druidNode) throws MalformedURLException ); } + static URL getLookupsUpdateURL(HostAndPort druidNode) throws MalformedURLException + { + return new URL( + "http", + druidNode.getHostText(), + druidNode.getPortOrDefault(-1), + ListenerResource.BASE_PATH + "/" + LOOKUP_LISTEN_ANNOUNCE_KEY + "/" + "updates" + ); + } + private static boolean httpStatusIsSuccess(int statusCode) { return statusCode >= 200 && statusCode < 300; @@ -672,4 +652,36 @@ boolean waitForBackgroundTermination(long timeout) throws InterruptedException { return backgroundManagerExitedLatch.await(timeout, TimeUnit.MILLISECONDS); } + + @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); + } + }; + } + + @VisibleForTesting + Collection getAllHostsAnnounceEndpoint(final String tier) throws IOException + { + return ImmutableList.copyOf( + Collections2.filter( + Collections2.transform( + listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tier)), + HOST_TO_URL + ), + Predicates.notNull() + ) + ); + } } 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..eed843a3ff41 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,38 @@ 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(10_000L); + public static final Duration DEFAULT_ALL_HOST_TIMEOUT = Duration.millis(300_000L); + @JsonProperty - private Duration deleteAllTimeout = null; + private Duration hostTimeout = null; @JsonProperty - private Duration updateAllTimeout = null; + private Duration allHostTimeout = null; @JsonProperty - @Min(1) + @Min(2) 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; - } - - public Duration getHostUpdateTimeout() - { - return hostUpdateTimeout == null ? DEFAULT_HOST_UPDATE_TIMEOUT : hostUpdateTimeout; - } - - public void setHostUpdateTimeout(Duration hostUpdateTimeout) - { - this.hostUpdateTimeout = hostUpdateTimeout; - } + private long period = 120_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() 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..56782c3628ac --- /dev/null +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java @@ -0,0 +1,99 @@ +/* + * 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 java.util.Map; + +/** + * 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 String version; + private Map lookupExtractorFactory; + + @JsonCreator + public LookupExtractorFactoryMapContainer( + @JsonProperty("version") String version, + @JsonProperty("lookupExtractorFactory") Map lookupExtractorFactory + ) + { + this.version = Preconditions.checkNotNull(version, "null 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) { + return version.compareTo(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; + + if (!version.equals(that.version)) { + return false; + } + return lookupExtractorFactory.equals(that.lookupExtractorFactory); + + } + + @Override + public int hashCode() + { + int result = version.hashCode(); + result = 31 * result + lookupExtractorFactory.hashCode(); + return result; + } +} diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java new file mode 100644 index 000000000000..5f4bac250207 --- /dev/null +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java @@ -0,0 +1,108 @@ +/* + * 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 java.util.Map; +import java.util.Set; + +/** + * Same as LookupsState except that it uses LookupExtractorFactoryMapContainer instead of + * LookupExtractorFactoryContainer to refer to lookup specs. + */ +public class LookupsStateWithMap +{ + private Map current; + private Map toLoad; + private Set toDrop; + + @JsonCreator + public LookupsStateWithMap( + @JsonProperty("current") Map current, + @JsonProperty("toLoad") Map toLoad, + @JsonProperty("toDrop") Set toDrop + ) + { + this.current = current; + this.toLoad = toLoad; + this.toDrop = 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; + } + + LookupsStateWithMap that = (LookupsStateWithMap) o; + + if (current != null ? !current.equals(that.current) : that.current != null) { + return false; + } + if (toLoad != null ? !toLoad.equals(that.toLoad) : that.toLoad != null) { + return false; + } + return !(toDrop != null ? !toDrop.equals(that.toDrop) : that.toDrop != null); + + } + + @Override + public int hashCode() + { + int result = current != null ? current.hashCode() : 0; + result = 31 * result + (toLoad != null ? toLoad.hashCode() : 0); + result = 31 * result + (toDrop != null ? toDrop.hashCode() : 0); + return result; + } +} 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..d8e6068f7346 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); } @@ -100,12 +105,6 @@ public boolean close() return true; } - @Override - public boolean replaces(@Nullable LookupExtractorFactory other) - { - return true; - } - @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() @@ -121,7 +120,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..1338ad8df24d 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java @@ -27,12 +27,11 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; - +import com.metamx.emitter.EmittingLogger; import io.druid.concurrent.Execs; 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; @@ -41,17 +40,10 @@ 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; @@ -62,29 +54,34 @@ public class LookupReferencesManagerTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); ObjectMapper mapper = new DefaultObjectMapper(); - private final ListeningExecutorService executorService = MoreExecutors.listeningDecorator(Execs.multiThreaded( - CONCURRENT_THREADS, - "hammer-time-%s" - )); + 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 + mapper, + true ); - Assert.assertTrue("must be closed before start call", lookupReferencesManager.isClosed()); + Assert.assertTrue("must be closed before start call", !lookupReferencesManager.started); lookupReferencesManager.start(); - Assert.assertFalse("must start after start call", lookupReferencesManager.isClosed()); + Assert.assertTrue("must start after start call", lookupReferencesManager.started); } @After public void tearDown() { lookupReferencesManager.stop(); - Assert.assertTrue("stop call should close it", lookupReferencesManager.isClosed()); + Assert.assertTrue("stop call should close it", !lookupReferencesManager.started); executorService.shutdownNow(); } @@ -99,51 +96,73 @@ public void testGetExceptionWhenClosed() public void testAddExceptionWhenClosed() { lookupReferencesManager.stop(); - lookupReferencesManager.put("test", EasyMock.createMock(LookupExtractorFactory.class)); + lookupReferencesManager.add("test", EasyMock.createMock(LookupExtractorFactoryContainer.class)); + } + + @Test(expected = ISE.class) + public void testRemoveExceptionWhenClosed() + { + lookupReferencesManager.stop(); + lookupReferencesManager.remove("test"); + } + + @Test(expected = ISE.class) + public void testGetAllLookupsStateExceptionWhenClosed() + { + lookupReferencesManager.stop(); + 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); 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); + handleOneNotice(lookupReferencesManager); + + Assert.assertEquals(testContainer, lookupReferencesManager.get("test")); + + lookupReferencesManager.remove("test"); + handleOneNotice(lookupReferencesManager); + 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.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); + handleOneNotice(lookupReferencesManager); + 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.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); + handleOneNotice(lookupReferencesManager); + lookupReferencesManager.remove("testMock"); - EasyMock.verify(lookupExtractorFactory); - } + handleOneNotice(lookupReferencesManager); - @Test - public void testRemoveInExisting() - { - Assert.assertFalse(lookupReferencesManager.remove("notThere")); + EasyMock.verify(lookupExtractorFactory); } @Test @@ -153,149 +172,125 @@ public void testGetNotThere() } @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); - } + EasyMock.replay(lookupExtractorFactory1, lookupExtractorFactory2); - @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); + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); + handleOneNotice(lookupReferencesManager); + + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("2", lookupExtractorFactory2)); + handleOneNotice(lookupReferencesManager); + + EasyMock.verify(lookupExtractorFactory1, lookupExtractorFactory2); } @Test - public void testUpdateIfNewOnlyIfIsNew() + 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(); - EasyMock.expect(oldFactory.replaces(EasyMock.eq(oldFactory))).andReturn(false).once(); - // Add new + LookupExtractorFactory lookupExtractorFactory2 = EasyMock.createNiceMock(LookupExtractorFactory.class); - 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(lookupExtractorFactory1, lookupExtractorFactory2); - EasyMock.replay(oldFactory, newFactory); + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); + handleOneNotice(lookupReferencesManager); - Assert.assertTrue(lookupReferencesManager.updateIfNew(lookupName, oldFactory)); - Assert.assertFalse(lookupReferencesManager.updateIfNew(lookupName, oldFactory)); - Assert.assertTrue(lookupReferencesManager.updateIfNew(lookupName, newFactory)); + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("0", lookupExtractorFactory2)); + handleOneNotice(lookupReferencesManager); - // Remove now or else EasyMock gets confused on lazy lookup manager stop handling - lookupReferencesManager.remove(lookupName); + EasyMock.verify(lookupExtractorFactory1, lookupExtractorFactory2); + } - EasyMock.verify(oldFactory, newFactory); + @Test + public void testRemoveNonExisting() throws Exception + { + lookupReferencesManager.remove("test"); + handleOneNotice(lookupReferencesManager); } - @Test(expected = ISE.class) - public void testUpdateIfNewExceptional() + @Test + public void testBootstrapFromFile() throws Exception { - 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); - } + LookupExtractorFactory lookupExtractorFactory = new MapLookupExtractorFactory( + ImmutableMap.of( + "key", + "value" + ), true + ); + LookupExtractorFactoryContainer container = new LookupExtractorFactoryContainer("v0", lookupExtractorFactory); + lookupReferencesManager.add("testMockForBootstrap", container); + handleOneNotice(lookupReferencesManager); + lookupReferencesManager.stop(); + lookupReferencesManager.start(); + Assert.assertEquals(container, lookupReferencesManager.get("testMockForBootstrap")); } @Test - public void testUpdateIfNewSuppressOldCloseProblem() + public void testGetAllLookupsState() throws Exception { - final String lookupName = "some lookup"; - LookupExtractorFactory oldFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); - LookupExtractorFactory newFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); + LookupExtractorFactoryContainer container1 = new LookupExtractorFactoryContainer( + "0", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key1", + "value1" + ), true + ) + ); - 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(); + LookupExtractorFactoryContainer container2 = new LookupExtractorFactoryContainer( + "0", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key2", + "value2" + ), true + ) + ); - EasyMock.replay(oldFactory, newFactory); + LookupExtractorFactoryContainer container3 = new LookupExtractorFactoryContainer( + "0", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key3", + "value3" + ), true + ) + ); - lookupReferencesManager.updateIfNew(lookupName, oldFactory); - lookupReferencesManager.updateIfNew(lookupName, newFactory); + lookupReferencesManager.add("one", container1); + lookupReferencesManager.add("two", container2); + lookupReferencesManager.remove("one"); + lookupReferencesManager.add("three", container3); - // Remove now or else EasyMock gets confused on lazy lookup manager stop handling - lookupReferencesManager.remove(lookupName); + handleOneNotice(lookupReferencesManager); + handleOneNotice(lookupReferencesManager); - EasyMock.verify(oldFactory, newFactory); - } + LookupsState state = lookupReferencesManager.getAllLookupsState(); - @Test - public void testBootstrapFromFile() throws IOException - { - LookupExtractorFactory lookupExtractorFactory = new MapLookupExtractorFactory(ImmutableMap.of( - "key", - "value" - ), true); - lookupReferencesManager.put("testMockForBootstrap", lookupExtractorFactory); - lookupReferencesManager.stop(); - lookupReferencesManager.start(); - Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("testMockForBootstrap")); + Assert.assertEquals(2, state.getCurrent().size()); + Assert.assertEquals(container1, state.getCurrent().get("one")); + Assert.assertEquals(container2, state.getCurrent().get("two")); + + Assert.assertEquals(1, state.getToLoad().size()); + Assert.assertEquals(container3, state.getToLoad().get("three")); + Assert.assertEquals(1, state.getToDrop().size()); + Assert.assertTrue(state.getToDrop().contains("one")); } @Test - public void testConcurrencyStaaaaaaaaaaartStop() throws Exception + public void testConcurrencyStaaaaaaaaartStop() throws Exception { lookupReferencesManager.stop(); final CyclicBarrier cyclicBarrier = new CyclicBarrier(CONCURRENT_THREADS); @@ -354,294 +349,7 @@ public void run() } } - @Test(timeout = 10000L) - public void testConcurrencySequentialChaos() 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()); - } - - 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); - - lookupReferencesManager.start(); - - 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()); - } - - @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()); - } - - 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); - } -} - -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; - } - - @Override - public void run() - { - try { - startLatch.await(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - lookupReferencesManager.updateIfNew(name, factory); - } -} - -class NamedIntrospectionHandler implements LookupIntrospectHandler -{ - final int position; - - NamedIntrospectionHandler(final int position) - { - this.position = position; + private void handleOneNotice(LookupReferencesManager mgr) throws Exception { + mgr.queue.take().handle(); } } diff --git a/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java b/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java index c8fefe6e6704..f76fe49bc6bf 100644 --- a/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java +++ b/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java @@ -41,14 +41,14 @@ public void testSimpleExtraction() } @Test - public void testReplaces() + public void testEquals() { - Assert.assertFalse(factory.replaces(factory)); - Assert.assertFalse(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), true))); - Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), false))); - Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY + "1", VALUE), true))); - Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE + "1"), true))); - Assert.assertTrue(factory.replaces(null)); + Assert.assertTrue(factory.equals(factory)); + Assert.assertTrue(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), true))); + Assert.assertFalse(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), false))); + Assert.assertFalse(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY + "1", VALUE), true))); + Assert.assertFalse(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE + "1"), true))); + Assert.assertFalse(factory.equals(null)); } @Test 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..a34ba63d3274 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,7 +19,6 @@ 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; @@ -36,11 +35,10 @@ import io.druid.audit.AuditInfo; import io.druid.common.config.JacksonConfigManager; 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.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 +51,13 @@ 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.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -78,19 +74,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 LookupsStateWithMap LOOKUPS_STATE = new LookupsStateWithMap( + 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; @@ -126,18 +142,17 @@ 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(), @@ -160,41 +175,33 @@ HttpResponseHandler makeResponseHandler( final AtomicReference reasonString ) { - returnCode.set(200); + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); reasonString.set(""); return responseHandler; } }; - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + + LookupsStateWithMap resp = manager.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); @@ -212,41 +219,37 @@ HttpResponseHandler makeResponseHandler( final AtomicReference reasonString ) { - returnCode.set(200); + returnCode.set(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode()); reasonString.set(""); return responseHandler; } }; - expectedException.expectMessage("Lookups failed to update: [\"" + failedLookup + "\"]"); + try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + manager.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); @@ -264,41 +267,36 @@ HttpResponseHandler makeResponseHandler( final AtomicReference reasonString ) { - returnCode.set(200); + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); reasonString.set(""); return responseHandler; } }; - expectedException.expectMessage(String.format( - "Update result did not have field for [%s]", - LookupModule.FAILED_UPDATES_KEY - )); + try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + manager.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); @@ -316,69 +314,50 @@ HttpResponseHandler makeResponseHandler( final AtomicReference reasonString ) { - returnCode.set(500); + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); 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) - { - } - }); + Thread.currentThread().interrupt(); try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + manager.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 URL url = LookupCoordinatorManager.getLookupsURL(HostAndPort.fromString("localhost")); - - EasyMock.reset(configManager); - EasyMock.expect(configManager.watch(EasyMock.anyString(), EasyMock.anyObject())) - .andReturn(configVal); - - final JsonProcessingException ex = EasyMock.createStrictMock(JsonProcessingException.class); - - final ObjectMapper mapper = EasyMock.createStrictMock(ObjectMapper.class); - EasyMock.expect(mapper.writeValueAsBytes(EasyMock.eq(SINGLE_LOOKUP_MAP))).andThrow(ex); - - expectedException.expectCause(new BaseMatcher() - { - @Override - public boolean matches(Object o) - { - return ex == o; - } - - @Override - public void describeTo(Description description) - { - - } - }); + 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(); - EasyMock.replay(mapper); + EasyMock.replay(client, responseHandler); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, @@ -386,23 +365,44 @@ public void describeTo(Description description) mapper, configManager, lookupCoordinatorManagerConfig + ) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.OK.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; + + LookupsStateWithMap resp = manager.getLookupStateForNode( + HostAndPort.fromString("localhost") ); - try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP - ); - } - finally { - EasyMock.verify(mapper); - } + + EasyMock.verify(client, responseHandler); + Assert.assertEquals(resp, LOOKUPS_STATE); } @Test - public void testUpdateAll() throws Exception + public void testGetLookupsStateNodeRespondedWithNotOkErrorCode() throws Exception { - final List urls = ImmutableList.of(new URL("http://foo.bar")); + 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(client, responseHandler); + final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -412,30 +412,44 @@ public void testUpdateAll() throws Exception ) { @Override - Collection getAllHostsAnnounceEndpoint(String tier) - { - return urls; - } - - @Override - void updateAllOnHost(final URL url, Map> updatedLookups) - throws IOException, InterruptedException, ExecutionException + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) { - if (!urls.get(0).equals(url) || updatedLookups != SINGLE_LOOKUP_MAP) { - throw new RuntimeException("Not matched"); - } + returnCode.set(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode()); + reasonString.set(""); + return responseHandler; } }; - // Should be no-ops - manager.updateAllOnTier(null, null); - } + try { + manager.getLookupStateForNode( + HostAndPort.fromString("localhost") + ); + Assert.fail(); + } + catch (IOException ex) { + } + + 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 HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); + + final SettableFuture future = SettableFuture.create(); + future.set(new ByteArrayInputStream("weird".getBytes())); + EasyMock.expect(client.go( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); + + EasyMock.replay(client, responseHandler); + final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -445,30 +459,43 @@ public void testUpdateAllIOException() throws Exception ) { @Override - Collection getAllHostsAnnounceEndpoint(String Tier) - { - return urls; - } - - @Override - void updateAllOnHost(final URL url, Map> updatedLookups) - throws IOException, InterruptedException, ExecutionException + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) { - throw ex; + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); + reasonString.set(""); + return responseHandler; } }; - // 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); + + try { + manager.getLookupStateForNode( + HostAndPort.fromString("localhost") + ); + Assert.fail(); + } + catch (IOException ex) { + } + + EasyMock.verify(client, responseHandler); } @Test - public void testUpdateAllInterrupted() throws Exception + public void testGetLookupsStateNodeInterrupted() throws Exception { - final InterruptedException ex = new InterruptedException("interruption test"); - final List urls = ImmutableList.of(new URL("http://foo.bar")); + final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); + + final SettableFuture future = SettableFuture.create(); + EasyMock.expect(client.go( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); + + EasyMock.replay(client, responseHandler); + final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -478,107 +505,32 @@ public void testUpdateAllInterrupted() throws Exception ) { @Override - Collection getAllHostsAnnounceEndpoint(String tier) - { - return urls; - } - - @Override - void updateAllOnHost(final URL url, Map> knownLookups) - throws IOException, InterruptedException, ExecutionException + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) { - throw ex; + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); + reasonString.set(""); + return responseHandler; } }; - 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; - } - @Override - public void describeTo(Description description) - { - - } - }); + Thread.currentThread().interrupt(); try { - manager.updateAllOnTier(LOOKUP_TIER, SINGLE_LOOKUP_MAP); + manager.getLookupStateForNode( + HostAndPort.fromString("localhost") + ); + Assert.fail(); + } + catch (InterruptedException ex) { } finally { - // Clear status + //clear the interrupt Thread.interrupted(); } - } - @Test - public void testGetAllHostsAnnounceEndpoint() throws Exception - { - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ); - - 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); - - 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); - } - - @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") - ) - ); - - 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") - ) - ); + EasyMock.verify(client, responseHandler); } @Test @@ -593,7 +545,7 @@ public void testUpdateLookupAdds() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { return EMPTY_TIERED_LOOKUP; } @@ -602,11 +554,11 @@ public Map>> getKnownLookups() EasyMock.reset(configManager); EasyMock.expect(configManager.set( EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(TIERED_LOOKUP_MAP), + EasyMock.eq(TIERED_LOOKUP_MAP_V0), EasyMock.eq(auditInfo) )).andReturn(true).once(); EasyMock.replay(configManager); - manager.updateLookup(LOOKUP_TIER, SINGLE_LOOKUP_NAME, SINGLE_LOOKUP_SPEC, auditInfo); + manager.updateLookup(LOOKUP_TIER, SINGLE_LOOKUP_NAME, SINGLE_LOOKUP_SPEC_V0, auditInfo); EasyMock.verify(configManager); } @@ -623,20 +575,19 @@ public void testUpdateLookupFailsUnitialized() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { return null; } }; 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 { - final Map ignore = ImmutableMap.of("prop", "old"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -646,46 +597,27 @@ 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 TIERED_LOOKUP_MAP_V0; } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" - ); - final Map> lookup = ImmutableMap.>of( - "foo", newSpec, - "ignore", ignore - ); - final Map>> tier = ImmutableMap.of( - LOOKUP_TIER, - lookup - ); + 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(TIERED_LOOKUP_MAP_V1), EasyMock.eq(auditInfo) )).andReturn(true).once(); EasyMock.replay(configManager); - manager.updateLookups(tier, auditInfo); + manager.updateLookups(TIERED_LOOKUP_MAP_V1, auditInfo); EasyMock.verify(configManager); } - - @Test - public void testUpdateLookupFailsBadUpdates() throws Exception + @Test(expected = IAE.class) + public void testUpdateLookupFailsSameVersionUpdates() throws Exception { - final Map ignore = ImmutableMap.of("prop", "old"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -695,45 +627,23 @@ 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 TIERED_LOOKUP_MAP_V0; } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" - ); - final Map> lookup = ImmutableMap.>of( - "foo", newSpec, - "ignore", ignore - ); - final Map>> tier = ImmutableMap.of( - LOOKUP_TIER, - lookup - ); + 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.replay(configManager); - Assert.assertFalse(manager.updateLookups(tier, auditInfo)); - EasyMock.verify(configManager); + manager.updateLookups(TIERED_LOOKUP_MAP_V0, auditInfo); } @Test public void testUpdateLookupsOnlyAddsToTier() 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, @@ -744,25 +654,25 @@ public void testUpdateLookupsOnlyAddsToTier() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of( + return ImmutableMap.>of( LOOKUP_TIER + "1", - ImmutableMap.>of("foo", ImmutableMap.of("prop", "old")), + ImmutableMap.of("foo", new LookupExtractorFactoryMapContainer("v0", ImmutableMap.of("prop", "old"))), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" + 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( + EasyMock.eq(ImmutableMap.>of( LOOKUP_TIER + "1", ImmutableMap.of("foo", newSpec), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) )), @@ -770,8 +680,8 @@ public Map>> getKnownLookups() ) ).andReturn(true).once(); EasyMock.replay(configManager); - Assert.assertTrue(manager.updateLookups(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.>of( + Assert.assertTrue(manager.updateLookups(ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of( "foo", newSpec ) @@ -782,7 +692,11 @@ public Map>> getKnownLookups() @Test public void testUpdateLookupsAddsNewTier() 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, @@ -793,23 +707,23 @@ public void testUpdateLookupsAddsNewTier() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of( + return ImmutableMap.>of( LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" + 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( + EasyMock.eq(ImmutableMap.>of( LOOKUP_TIER + "1", ImmutableMap.of("foo", newSpec), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) )), @@ -817,8 +731,8 @@ public Map>> getKnownLookups() ) ).andReturn(true).once(); EasyMock.replay(configManager); - Assert.assertTrue(manager.updateLookups(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.>of( + Assert.assertTrue(manager.updateLookups(ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of( "foo", newSpec ) @@ -829,7 +743,11 @@ public Map>> getKnownLookups() @Test public void testUpdateLookupsAddsNewLookup() 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,27 +758,31 @@ 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( + "foo1", new LookupExtractorFactoryMapContainer( + "v0", ImmutableMap.of("prop", "old") + ) + ), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" + 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( + EasyMock.eq(ImmutableMap.>of( LOOKUP_TIER + "1", ImmutableMap.of( - "foo1", ImmutableMap.of("prop", "old"), + "foo1", ignore, "foo2", newSpec ), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) @@ -869,8 +791,8 @@ public Map>> getKnownLookups() ) ).andReturn(true).once(); EasyMock.replay(configManager); - Assert.assertTrue(manager.updateLookups(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.>of( + Assert.assertTrue(manager.updateLookups(ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of( "foo2", newSpec ) @@ -881,8 +803,15 @@ public Map>> getKnownLookups() @Test public void testDeleteLookup() throws Exception { - final Map ignore = ImmutableMap.of("lookup", "ignore"); - final Map lookup = ImmutableMap.of("lookup", "foo"); + 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, @@ -892,9 +821,9 @@ public void testDeleteLookup() 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, "ignore", ignore )); @@ -917,7 +846,10 @@ public Map>> getKnownLookups() @Test public void testDeleteLookupIgnoresMissing() throws Exception { - final Map ignore = ImmutableMap.of("lookup", "ignore"); + final LookupExtractorFactoryMapContainer ignore = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("lookup", "ignore") + ); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -927,11 +859,12 @@ public void testDeleteLookupIgnoresMissing() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of(LOOKUP_TIER, ImmutableMap.of( - "ignore", ignore - )); + return ImmutableMap.>of( + LOOKUP_TIER, + ImmutableMap.of("ignore", ignore) + ); } }; final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); @@ -950,7 +883,7 @@ public void testDeleteLookupIgnoresNotReady() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { return null; } @@ -959,150 +892,13 @@ public Map>> getKnownLookups() Assert.assertFalse(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); } - @Test - public void testDeleteAllTier() throws Exception - { - final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - 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; - } - }; - 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); - } - - @Test - public void testDeleteAllTierMissing() throws Exception - { - final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) - { - returnCode.set(404); - reasonString.set(""); - return responseHandler; - } - }; - 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); - } - - @Test - public void testDeleteAllTierContinuesOnMissing() throws Exception - { - final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final AtomicInteger responseHandlerCalls = new AtomicInteger(0); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) - { - if (responseHandlerCalls.getAndIncrement() == 0) { - returnCode.set(404); - reasonString.set("Not Found"); - } else { - returnCode.set(202); - reasonString.set(""); - } - return responseHandler; - } - }; - 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()); - } - @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 +908,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 +921,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 +937,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 +962,7 @@ public void testGetLookupIgnoresNotReady() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { return null; } @@ -1301,7 +1099,6 @@ public void testLookupDiscoverAll() throws Exception EasyMock.verify(discoverer); } - @Test public void testLookupDiscoverAllExceptional() throws Exception { From cab7c01adfcf045b69d60de192b4463e6a9d73f6 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 9 Feb 2017 11:19:00 -0600 Subject: [PATCH 02/35] revert replaces removal, deprecate it instead --- docs/content/querying/lookups.md | 251 +++++++++--------- .../lookup/KafkaLookupExtractorFactory.java | 2 +- .../NamespaceLookupExtractorFactory.java | 2 +- .../server/lookup/LoadingLookupFactory.java | 11 + .../server/lookup/PollingLookupFactory.java | 12 + .../query/lookup/LookupExtractorFactory.java | 7 + .../LookupExtractorFactoryContainer.java | 2 +- .../io/druid/query/lookup/LookupsState.java | 34 +-- .../LookupExtractorFactoryContainerTest.java | 6 + .../RegisteredLookupExtractionFnTest.java | 57 ++-- .../lookup/MapLookupExtractorFactory.java | 13 + .../LookupIntrospectionResourceTest.java | 6 + .../lookup/MapLookupExtractorFactoryTest.java | 14 +- .../cache/LookupCoordinatorManagerTest.java | 207 ++++++++------- 14 files changed, 346 insertions(+), 278 deletions(-) diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index cd57d92e5021..aba82a09d4d4 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 diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java index 6b16ccc77333..ebd99c430ade 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java @@ -327,7 +327,7 @@ public boolean close() } @Override - public boolean equals(@Nullable Object other) + public boolean replaces(@Nullable LookupExtractorFactory other) { if (this == other) { return false; diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index 60de8e5be66a..54badfcde0e7 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -150,7 +150,7 @@ public boolean close() } @Override - public boolean equals(@Nullable Object other) + public boolean replaces(@Nullable LookupExtractorFactory other) { if (other != null && other instanceof NamespaceLookupExtractorFactory) { NamespaceLookupExtractorFactory that = (NamespaceLookupExtractorFactory) other; diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java index 3e9ffdf39aeb..4c6b57bffa42 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java @@ -92,6 +92,17 @@ public synchronized boolean close() return !started.get(); } + @Override + public boolean replaces( + @Nullable LookupExtractorFactory lookupExtractorFactory + ) + { + if (lookupExtractorFactory == null) { + return true; + } + return !this.equals(lookupExtractorFactory); + } + @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java index 9322e4450f5f..2333ea48d163 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java @@ -107,6 +107,18 @@ public boolean close() } } + @Override + public boolean replaces( + @Nullable LookupExtractorFactory lookupExtractorFactory + ) + { + if (lookupExtractorFactory == null) { + return true; + } + + return !this.equals(lookupExtractorFactory); + } + @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() 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 113d9914a738..6b2bea2163ac 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java @@ -51,6 +51,13 @@ public interface LookupExtractorFactory extends Supplier */ public boolean close(); + /** + * This method is deprecated and is not removed only to allow 0.9.x to 0.10.x transition. It is not used + * on a cluster that is running 0.10.x. It will be removed in the next release. + */ + @Deprecated + boolean replaces(@Nullable LookupExtractorFactory other); + /** * @return Returns the actual introspection request handler, can return {@code null} if it is not supported. * This will be called once per HTTP request to introspect the actual lookup. diff --git a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java index ebab64de9fb9..5daf9497e1cf 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java @@ -54,7 +54,7 @@ public LookupExtractorFactory getLookupExtractorFactory() public boolean replaces(LookupExtractorFactoryContainer other) { if (version == null && other.getVersion() == null) { - return !this.lookupExtractorFactory.equals(other.getLookupExtractorFactory()); + return !this.lookupExtractorFactory.replaces(other.getLookupExtractorFactory()); } if (version == null && other.getVersion() != null) { diff --git a/processing/src/main/java/io/druid/query/lookup/LookupsState.java b/processing/src/main/java/io/druid/query/lookup/LookupsState.java index 140c919ec701..4d095b962d4d 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupsState.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupsState.java @@ -1,21 +1,21 @@ /* -* 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. -*/ + * 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; diff --git a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java index f0e2e2bebb54..0998e8630ee8 100644 --- a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java +++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java @@ -71,6 +71,12 @@ public boolean close() return false; } + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return true; + } + @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() 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 7afb627760e8..f3d1abd2e529 100644 --- a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java @@ -227,32 +227,41 @@ public void testEquals() private void managerReturnsMap(LookupReferencesManager manager) { EasyMock.expect(manager.get(EasyMock.eq(LOOKUP_NAME))).andReturn( - new LookupExtractorFactoryContainer("v0", new LookupExtractorFactory() - { - @Override - public boolean start() - { - return false; - } + new LookupExtractorFactoryContainer( + "v0", new LookupExtractorFactory() + { + @Override + public boolean start() + { + return false; + } + + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return false; + } - @Override - public boolean close() - { - 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/MapLookupExtractorFactory.java b/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java index 76d4211d52ae..5ac15478ca38 100644 --- a/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java +++ b/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java @@ -67,6 +67,19 @@ public boolean close() return true; } + /** + * For MapLookups, the replaces consideration is very easy, it simply considers if the other is the same as this one + * + * @param other Some other LookupExtractorFactory which might need replaced + * + * @return true - should replace, false - should not replace + */ + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return !equals(other); + } + @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() 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 d8e6068f7346..9bd895640e77 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceTest.java @@ -105,6 +105,12 @@ public boolean close() return true; } + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return true; + } + @Nullable @Override public LookupIntrospectHandler getIntrospectHandler() diff --git a/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java b/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java index f76fe49bc6bf..c8fefe6e6704 100644 --- a/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java +++ b/server/src/test/java/io/druid/query/lookup/MapLookupExtractorFactoryTest.java @@ -41,14 +41,14 @@ public void testSimpleExtraction() } @Test - public void testEquals() + public void testReplaces() { - Assert.assertTrue(factory.equals(factory)); - Assert.assertTrue(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), true))); - Assert.assertFalse(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), false))); - Assert.assertFalse(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY + "1", VALUE), true))); - Assert.assertFalse(factory.equals(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE + "1"), true))); - Assert.assertFalse(factory.equals(null)); + Assert.assertFalse(factory.replaces(factory)); + Assert.assertFalse(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), true))); + Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE), false))); + Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY + "1", VALUE), true))); + Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(KEY, VALUE + "1"), true))); + Assert.assertTrue(factory.replaces(null)); } @Test 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 a34ba63d3274..27092a221766 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 @@ -534,37 +534,7 @@ HttpResponseHandler makeResponseHandler( } @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_V0), - EasyMock.eq(auditInfo) - )).andReturn(true).once(); - EasyMock.replay(configManager); - manager.updateLookup(LOOKUP_TIER, SINGLE_LOOKUP_NAME, SINGLE_LOOKUP_SPEC_V0, auditInfo); - EasyMock.verify(configManager); - } - - - @Test - public void testUpdateLookupFailsUnitialized() throws Exception + public void testUpdateLookupsFailsUnitialized() throws Exception { final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, @@ -586,7 +556,7 @@ public Map> getKnownLook } @Test - public void testUpdateLookupUpdates() throws Exception + public void testUpdateLookupAdds() throws Exception { final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, @@ -599,51 +569,29 @@ public void testUpdateLookupUpdates() throws Exception @Override public Map> getKnownLookups() { - return TIERED_LOOKUP_MAP_V0; + 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_V1), - EasyMock.eq(auditInfo) - )).andReturn(true).once(); + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.eq(TIERED_LOOKUP_MAP_V0), + EasyMock.eq(auditInfo) + )).andReturn(true).once(); EasyMock.replay(configManager); - manager.updateLookups(TIERED_LOOKUP_MAP_V1, auditInfo); + manager.updateLookup(LOOKUP_TIER, SINGLE_LOOKUP_NAME, SINGLE_LOOKUP_SPEC_V0, auditInfo); EasyMock.verify(configManager); } - @Test(expected = IAE.class) - public void testUpdateLookupFailsSameVersionUpdates() throws Exception - { - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - public Map> getKnownLookups() - { - return TIERED_LOOKUP_MAP_V0; - } - }; - - final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); - manager.updateLookups(TIERED_LOOKUP_MAP_V0, auditInfo); - } - @Test - public void testUpdateLookupsOnlyAddsToTier() throws Exception + public void testUpdateLookupsAddsNewLookup() 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, @@ -658,7 +606,11 @@ public Map> getKnownLook { return ImmutableMap.>of( LOOKUP_TIER + "1", - ImmutableMap.of("foo", new LookupExtractorFactoryMapContainer("v0", ImmutableMap.of("prop", "old"))), + ImmutableMap.of( + "foo1", new LookupExtractorFactoryMapContainer( + "v0", ImmutableMap.of("prop", "old") + ) + ), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); @@ -673,30 +625,32 @@ public Map> getKnownLook 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) - )), + 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)); + LOOKUP_TIER + "1", ImmutableMap.of( + "foo2", + newSpec + ) + ), auditInfo)); EasyMock.verify(configManager); } @Test - public void testUpdateLookupsAddsNewTier() throws Exception + public void testUpdateLookupsOnlyUpdatesToTier() 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, @@ -710,6 +664,13 @@ public void testUpdateLookupsAddsNewTier() throws Exception public Map> getKnownLookups() { return ImmutableMap.>of( + LOOKUP_TIER + "1", + ImmutableMap.of( + "foo", new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("prop", "new") + ) + ), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); @@ -724,24 +685,81 @@ public Map> getKnownLook 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) - )), + 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)); + LOOKUP_TIER + "1", ImmutableMap.of( + "foo", + newSpec + ) + ), auditInfo)); EasyMock.verify(configManager); } @Test - public void testUpdateLookupsAddsNewLookup() throws Exception + public void testUpdateLookupsUpdates() throws Exception + { + final LookupCoordinatorManager manager = new LookupCoordinatorManager( + client, + discoverer, + mapper, + configManager, + lookupCoordinatorManagerConfig + ) + { + @Override + public Map> getKnownLookups() + { + return TIERED_LOOKUP_MAP_V0; + } + }; + + 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_V1), + EasyMock.eq(auditInfo) + )).andReturn(true).once(); + EasyMock.replay(configManager); + manager.updateLookups(TIERED_LOOKUP_MAP_V1, auditInfo); + EasyMock.verify(configManager); + } + + @Test + public void testUpdateLookupFailsSameVersionUpdates() throws Exception + { + final LookupCoordinatorManager manager = new LookupCoordinatorManager( + client, + discoverer, + mapper, + configManager, + lookupCoordinatorManagerConfig + ) + { + @Override + public Map> getKnownLookups() + { + return TIERED_LOOKUP_MAP_V0; + } + }; + + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); + + try { + manager.updateLookups(TIERED_LOOKUP_MAP_V0, auditInfo); + Assert.fail(); + } catch (IAE ex) { + } + } + + @Test + public void testUpdateLookupsAddsNewTier() throws Exception { final LookupExtractorFactoryMapContainer ignore = new LookupExtractorFactoryMapContainer( "v0", @@ -761,12 +779,6 @@ public void testUpdateLookupsAddsNewLookup() throws Exception public Map> getKnownLookups() { return ImmutableMap.>of( - LOOKUP_TIER + "1", - ImmutableMap.of( - "foo1", new LookupExtractorFactoryMapContainer( - "v0", ImmutableMap.of("prop", "old") - ) - ), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); @@ -781,22 +793,19 @@ public Map> getKnownLook configManager.set( EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), EasyMock.eq(ImmutableMap.>of( - LOOKUP_TIER + "1", ImmutableMap.of( - "foo1", ignore, - "foo2", newSpec - ), - LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) - )), + 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)); + LOOKUP_TIER + "1", ImmutableMap.of( + "foo", + newSpec + ) + ), auditInfo)); EasyMock.verify(configManager); } From 98676eb0c432bfd82c5d1ca2df039780f043896a Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 14 Feb 2017 11:11:22 -0600 Subject: [PATCH 03/35] convert and use older specs stored in db --- .../cache/LookupCoordinatorManager.java | 50 +++++++++++++++++-- .../LookupExtractorFactoryMapContainer.java | 18 +++++-- 2 files changed, 61 insertions(+), 7 deletions(-) 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 850b15db4ee7..fda96c90a9c2 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 @@ -24,6 +24,7 @@ import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import com.google.common.base.Throwables; import com.google.common.collect.Collections2; @@ -88,9 +89,8 @@ */ 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; + public static final String LOOKUP_CONFIG_KEY = "lookupsConfig"; + public static final String LOOKUP_LISTEN_ANNOUNCE_KEY = "lookups"; private static final EmittingLogger LOG = new EmittingLogger(LookupCoordinatorManager.class); private final static Function HOST_TO_URL = new Function() @@ -167,6 +167,22 @@ public boolean updateLookup( public boolean updateLookups(final Map> updateSpec, AuditInfo auditInfo) { + if (updateSpec == null || updateSpec.isEmpty()) { + 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.9.x lookup specs + for (final String tier : updateSpec.keySet()) { + Map lookups = updateSpec.get(tier); + for (Map.Entry e : lookups.entrySet()) { + Preconditions.checkNotNull( + e.getValue().getVersion(), + String.format("lookup [%s]:[%s] does not have version.", tier, e.getKey()) + ); + } + } + synchronized (startStopSync) { final Map> priorSpec = getKnownLookups(); if (priorSpec == null && !updateSpec.isEmpty()) { @@ -195,7 +211,8 @@ public boolean updateLookups(final Map e : updateTierSpec.entrySet()) { if (updatedTierSpec.containsKey(e.getKey()) && !e.getValue().replaces(updatedTierSpec.get(e.getKey()))) { throw new IAE( - "given update for lookup [%s] can't replace existing spec [%s].", + "given update for lookup [%s]:[%s] can't replace existing spec [%s].", + tier, e.getKey(), updatedTierSpec.get(e.getKey()) ); @@ -297,6 +314,31 @@ public void start() null ); + // backward compatibility with 0.9.x + if (lookupMapConfigRef.get() == null) { + Map>> oldLookups = configManager.watch( + "lookups", + new TypeReference>>>() + { + }, + null + ).get(); + + if (oldLookups != null) { + Map> converted = new HashMap<>(); + for (String tier : oldLookups.keySet()) { + Map> oldTierLookups = oldLookups.get(tier); + if (oldLookups != null && !oldLookups.isEmpty()) { + Map convertedTierLookups = new HashMap<>(); + for (Map.Entry> e : oldTierLookups.entrySet()) { + convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); + } + } + } + configManager.set(LOOKUP_CONFIG_KEY, converted, new AuditInfo("autoConversion", "autoConversion", "127.0.0.1")); + } + } + this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( new Runnable() { 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 index 56782c3628ac..4f59679e1664 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java @@ -41,7 +41,7 @@ public LookupExtractorFactoryMapContainer( @JsonProperty("lookupExtractorFactory") Map lookupExtractorFactory ) { - this.version = Preconditions.checkNotNull(version, "null version"); + this.version = version; this.lookupExtractorFactory = Preconditions.checkNotNull(lookupExtractorFactory, "null factory"); } @@ -58,6 +58,18 @@ public Map getLookupExtractorFactory() } public boolean replaces(LookupExtractorFactoryMapContainer other) { + if (version == null && other.getVersion() == null) { + return false; + } + + if (version == null && other.getVersion() != null) { + return false; + } + + if (version != null && other.getVersion() == null) { + return true; + } + return version.compareTo(other.getVersion()) > 0; } @@ -82,7 +94,7 @@ public boolean equals(Object o) LookupExtractorFactoryMapContainer that = (LookupExtractorFactoryMapContainer) o; - if (!version.equals(that.version)) { + if (version != null ? !version.equals(that.version) : that.version != null) { return false; } return lookupExtractorFactory.equals(that.lookupExtractorFactory); @@ -92,7 +104,7 @@ public boolean equals(Object o) @Override public int hashCode() { - int result = version.hashCode(); + int result = version != null ? version.hashCode() : 0; result = 31 * result + lookupExtractorFactory.hashCode(); return result; } From 17fb95292036971d4d0c5812f3f4cfc60a731905 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 14 Feb 2017 12:00:17 -0600 Subject: [PATCH 04/35] more tests and updates --- docs/content/querying/lookups.md | 7 +- .../io/druid/query/lookup/LookupBean.java | 55 +++++- .../query/lookup/LookupReferencesManager.java | 15 +- .../io/druid/query/lookup/LookupsState.java | 38 +++- .../LookupExtractorFactoryContainerTest.java | 16 +- .../druid/query/lookup/LookupsStateTest.java | 87 +++++++++ .../cache/LookupCoordinatorManager.java | 6 +- .../cache/LookupCoordinatorManagerConfig.java | 4 +- .../lookup/cache/LookupsStateWithMap.java | 7 +- .../lookup/LookupReferencesManagerTest.java | 3 +- .../query/lookup/LookupSnapshotTakerTest.java | 66 ++++++- .../cache/LookupCoordinatorManagerTest.java | 127 +++++++++++--- ...ookupExtractorFactoryMapContainerTest.java | 114 ++++++++++++ .../lookup/cache/LookupsStateWithMapTest.java | 165 ++++++++++++++++++ 14 files changed, 644 insertions(+), 66 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java create mode 100644 server/src/test/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainerTest.java create mode 100644 server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index aba82a09d4d4..061e84d08ca9 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -289,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`(s second)| +|`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 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 8e038be3c1e4..37a6a6a7f3dd 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupBean.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupBean.java @@ -19,21 +19,52 @@ package io.druid.query.lookup; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; //TODO merge this code to the same definition when pr/1576 is merged class LookupBean { - //kept for backward compatibility with druid ver <= 0.9.2 persisted snapshots - @Deprecated @JsonProperty - LookupExtractorFactory factory; + private final LookupExtractorFactoryContainer container; @JsonProperty - LookupExtractorFactoryContainer container; + private final String name; - @JsonProperty - String name; + @JsonCreator + public LookupBean( + @JsonProperty("name") String name, + //kept for backward compatibility with druid ver <= 0.9.2 persisted snapshots + @Deprecated @JsonProperty("factory") LookupExtractorFactory factory, + @JsonProperty("container") LookupExtractorFactoryContainer container + ) + { + Preconditions.checkArgument(factory == null || container == null, "only one of factory or container should exist"); + Preconditions.checkArgument(factory != null || container != null, "one of factory or container must exist"); + + this.name = name; + this.container = container != null ? container : new LookupExtractorFactoryContainer(null, factory); + } + + public String getName() + { + return name; + } + + public LookupExtractorFactoryContainer getContainer() + { + return container; + } + + @Override + public String toString() + { + return "LookupBean{" + + "container=" + container + + ", name='" + name + '\'' + + '}'; + } @Override public boolean equals(Object o) @@ -41,16 +72,24 @@ 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; - if (!factory.equals(that.factory)) { + if (!container.equals(that.container)) { return false; } return name.equals(that.name); } + + @Override + public int hashCode() + { + int result = container.hashCode(); + result = 31 * result + name.hashCode(); + return result; + } } 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 c3326c6f8514..3b9308d6789b 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -110,17 +110,12 @@ public void start() if (lookupSnapshotTaker != null) { final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); for (LookupBean lookupBean : lookupBeanList) { - LookupExtractorFactoryContainer container = lookupBean.container; - - //for backward compatibility with druid ver <= 0.9.2 persisted snapshots - if (lookupBean.container == null) { - container = new LookupExtractorFactoryContainer(null, lookupBean.factory); - } + LookupExtractorFactoryContainer container = lookupBean.getContainer(); if (container.getLookupExtractorFactory().start()) { - lookupMap.put(lookupBean.name, container); + lookupMap.put(lookupBean.getName(), container); } else { - throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.name, container); + throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.getName(), container); } } } @@ -295,9 +290,7 @@ public LookupBean apply( Map.Entry input ) { - final LookupBean lookupBean = new LookupBean(); - lookupBean.container = input.getValue(); - lookupBean.name = input.getKey(); + final LookupBean lookupBean = new LookupBean(input.getKey(), null, input.getValue()); return lookupBean; } } diff --git a/processing/src/main/java/io/druid/query/lookup/LookupsState.java b/processing/src/main/java/io/druid/query/lookup/LookupsState.java index 4d095b962d4d..309a5f6a10c9 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupsState.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupsState.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Collections; import java.util.Map; import java.util.Set; @@ -40,9 +41,9 @@ public LookupsState( @JsonProperty("toDrop") Set toDrop ) { - this.current = current; - this.toLoad = toLoad; - this.toDrop = 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 @@ -72,4 +73,35 @@ public String toString() ", 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; + + if (!current.equals(that.current)) { + return false; + } + if (!toLoad.equals(that.toLoad)) { + return false; + } + return toDrop.equals(that.toDrop); + + } + + @Override + public int hashCode() + { + int result = current.hashCode(); + result = 31 * result + toLoad.hashCode(); + result = 31 * result + toDrop.hashCode(); + return result; + } } diff --git a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java index 0998e8630ee8..2abf258e155a 100644 --- a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java +++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java @@ -51,12 +51,17 @@ public void testSerde() throws Exception LookupExtractorFactoryContainer.class ); - Assert.assertEquals("v1", actual.getVersion()); - Assert.assertTrue(actual.getLookupExtractorFactory() instanceof TestLookupExtractorFactory); + Assert.assertEquals( + new LookupExtractorFactoryContainer( + "v1", + new TestLookupExtractorFactory() + ), + actual + ); } @JsonTypeName("test") - private static class TestLookupExtractorFactory implements LookupExtractorFactory + static class TestLookupExtractorFactory implements LookupExtractorFactory { @Override @@ -89,5 +94,10 @@ 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..be8f1686268a --- /dev/null +++ b/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java @@ -0,0 +1,87 @@ +/* + * 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.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" + + "}"; + + final ObjectMapper mapper = new DefaultObjectMapper(); + mapper.registerSubtypes(LookupExtractorFactoryContainerTest.TestLookupExtractorFactory.class); + + LookupsState actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupsState.class) + ), + LookupsState.class + ); + + 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/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index fda96c90a9c2..6e88f0f5ef54 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 @@ -89,6 +89,9 @@ */ public class LookupCoordinatorManager { + //key used in druid-0.9.x 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 EmittingLogger LOG = new EmittingLogger(LookupCoordinatorManager.class); @@ -317,7 +320,7 @@ public void start() // backward compatibility with 0.9.x if (lookupMapConfigRef.get() == null) { Map>> oldLookups = configManager.watch( - "lookups", + OLD_LOOKUP_CONFIG_KEY, new TypeReference>>>() { }, @@ -333,6 +336,7 @@ public void start() for (Map.Entry> e : oldTierLookups.entrySet()) { convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); } + converted.put(tier, convertedTierLookups); } } configManager.set(LOOKUP_CONFIG_KEY, converted, new AuditInfo("autoConversion", "autoConversion", "127.0.0.1")); 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 eed843a3ff41..bec67c44f9cc 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,8 +26,8 @@ public class LookupCoordinatorManagerConfig { - public static final Duration DEFAULT_HOST_TIMEOUT = Duration.millis(10_000L); - public static final Duration DEFAULT_ALL_HOST_TIMEOUT = Duration.millis(300_000L); + 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 hostTimeout = null; diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java index 5f4bac250207..878caa413a35 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Collections; import java.util.Map; import java.util.Set; @@ -42,9 +43,9 @@ public LookupsStateWithMap( @JsonProperty("toDrop") Set toDrop ) { - this.current = current; - this.toLoad = toLoad; - this.toDrop = 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 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 1338ad8df24d..a662171e270f 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java @@ -22,7 +22,6 @@ 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; @@ -68,7 +67,7 @@ public void setUp() throws IOException mapper.registerSubtypes(MapLookupExtractorFactory.class); lookupReferencesManager = new LookupReferencesManager( - new LookupConfig(Files.createTempDir().getAbsolutePath()), + new LookupConfig(temporaryFolder.newFolder().getAbsolutePath()), mapper, true ); 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/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 27092a221766..32d4e9f4e331 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 @@ -840,13 +840,19 @@ public Map> getKnownLook }; 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.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); @@ -982,14 +988,20 @@ public Map> getKnownLook @Test public void testStart() 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(); + )).andReturn(new AtomicReference>(null)).once(); + + EasyMock.expect(configManager.watch( + EasyMock.eq(LookupCoordinatorManager.OLD_LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn(new AtomicReference>>(null)).once(); + EasyMock.replay(configManager); final LookupCoordinatorManager manager = new LookupCoordinatorManager( @@ -1015,14 +1027,20 @@ public long getPeriod(){ @Test public void testStop() 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>(null)).once(); + + EasyMock.expect(configManager.watch( + EasyMock.eq(LookupCoordinatorManager.OLD_LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn(new AtomicReference>>(null)).once(); + EasyMock.replay(configManager); final LookupCoordinatorManager manager = new LookupCoordinatorManager( @@ -1045,14 +1063,20 @@ public void testStop() throws Exception @Test public void testStartTooMuch() 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>(null)).once(); + + EasyMock.expect(configManager.watch( + EasyMock.eq(LookupCoordinatorManager.OLD_LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn(new AtomicReference>>(null)).once(); + EasyMock.replay(configManager); final LookupCoordinatorManager manager = new LookupCoordinatorManager( @@ -1145,4 +1169,63 @@ public void describeTo(Description description) EasyMock.verify(discoverer); } } + + //tests that lookups stored in db from 0.9.x 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..e17fa6015de0 --- /dev/null +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainerTest.java @@ -0,0 +1,114 @@ +/* + * 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 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/server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java new file mode 100644 index 000000000000..89600f454263 --- /dev/null +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java @@ -0,0 +1,165 @@ +/* + * 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 com.google.common.collect.ImmutableSet; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.lookup.LookupExtractorFactoryContainer; +import io.druid.query.lookup.LookupsState; +import io.druid.query.lookup.MapLookupExtractorFactory; +import org.junit.Assert; +import org.junit.Test; + +/** + */ +public class LookupsStateWithMapTest +{ + private final ObjectMapper mapper; + private final String jsonStr; + private final LookupsStateWithMap testLookupsState; + + public LookupsStateWithMapTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerSubtypes(MapLookupExtractorFactory.class); + + jsonStr = "{\n" + + " \"current\": {\n" + + " \"l1\": {\n" + + " \"version\": \"v1\",\n" + + " \"lookupExtractorFactory\": {\n" + + " \"type\": \"map\",\n" + + " \"map\": {\"k\": \"v\"},\n" + + " \"isOneToOne\": true\n" + + " }\n" + + " }\n" + + " },\n" + + " \"toLoad\": {\n" + + " \"l2\": {\n" + + " \"version\": \"v1\",\n" + + " \"lookupExtractorFactory\": {\n" + + " \"type\": \"map\",\n" + + " \"map\": {\"k\": \"v\"},\n" + + " \"isOneToOne\": true\n" + + " }\n" + + " }\n" + + " },\n" + + " \"toDrop\": [\"l3\"]\n" + + "}"; + + testLookupsState = new LookupsStateWithMap( + ImmutableMap.of( + "l1", + new LookupExtractorFactoryMapContainer( + "v1", + ImmutableMap.of( + "type", "map", + "map", ImmutableMap.of("k", "v"), + "isOneToOne", true + ) + ) + ), + ImmutableMap.of( + "l2", + new LookupExtractorFactoryMapContainer( + "v1", + ImmutableMap.of( + "type", "map", + "map", ImmutableMap.of("k", "v"), + "isOneToOne", true + ) + ) + ), + ImmutableSet.of("l3") + ); + } + + @Test + public void testSerde() throws Exception + { + LookupsStateWithMap actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupsStateWithMap.class) + ), + LookupsStateWithMap.class + ); + Assert.assertEquals(testLookupsState, actual); + } + + + + //test interchangeability with LookupsState + //read and write as LookupsState + //then read as LookupsStateWithMap + @Test + public void testInterchangeability1() throws Exception + { + LookupsStateWithMap actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupsState.class) + ), + LookupsStateWithMap.class + ); + + Assert.assertEquals(testLookupsState, actual); + } + + //test interchangeability with LookupExtractorFactoryContainer + //read and write as LookupExtractorFactoryMapContainer + //then read as LookupExtractorFactoryContainer + @Test + public void testInterchangeability2() throws Exception + { + LookupsState actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupsStateWithMap.class) + ), + LookupsState.class + ); + + Assert.assertEquals(new LookupsState( + ImmutableMap.of( + "l1", + new LookupExtractorFactoryContainer( + "v1", + new MapLookupExtractorFactory( + ImmutableMap.of("k", "v"), + true + ) + ) + ), + ImmutableMap.of( + "l2", + new LookupExtractorFactoryContainer( + "v1", + new MapLookupExtractorFactory( + ImmutableMap.of("k", "v"), + true + ) + ) + ), + ImmutableSet.of("l3") + ), + actual + ); + } +} From c9fbd7116c96ac62eb484d22a5425d06122cab5f Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 17 Feb 2017 15:55:09 -0600 Subject: [PATCH 05/35] review comments --- docs/content/querying/lookups.md | 2 +- .../io/druid/query/lookup/LookupBean.java | 5 ++- .../query/lookup/LookupReferencesManager.java | 31 +++++++++---------- 3 files changed, 18 insertions(+), 20 deletions(-) diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 061e84d08ca9..44f7e35b52a3 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -289,7 +289,7 @@ To configure the behavior of the dynamic configuration manager, use the followin |Property|Description|Default| |--------|-----------|-------| -|`druid.manager.lookups.hostTimeout`|Timeout (in ms) PER HOST for processing request|`2000`(s second)| +|`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`| 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 37a6a6a7f3dd..ec6884b94353 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupBean.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupBean.java @@ -26,10 +26,7 @@ //TODO merge this code to the same definition when pr/1576 is merged class LookupBean { - @JsonProperty private final LookupExtractorFactoryContainer container; - - @JsonProperty private final String name; @JsonCreator @@ -47,11 +44,13 @@ public LookupBean( this.container = container != null ? container : new LookupExtractorFactoryContainer(null, factory); } + @JsonProperty public String getName() { return name; } + @JsonProperty public LookupExtractorFactoryContainer getContainer() { return container; 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 3b9308d6789b..a4f83af25535 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -66,7 +66,7 @@ public class LookupReferencesManager private final ReadWriteLock startStopLock = new ReentrantReadWriteLock(true); @VisibleForTesting - volatile boolean started = false; + boolean started = false; @GuardedBy("startStopLock") private final Map lookupMap = new HashMap<>(); @@ -147,7 +147,7 @@ public void run() } } finally { - LOG.info("Lookup Mgmt loop exited, Lookup notices are not handled anymore."); + LOG.info("Lookup Management loop exited, Lookup notices are not handled anymore."); } } } @@ -168,8 +168,8 @@ public void stop() { startStopLock.writeLock().lock(); - if (started) { - try { + try { + if (started) { LOG.info("LookupReferencesManager is stopping."); started = false; @@ -182,7 +182,7 @@ public void stop() try { LOG.info("Closing lookup [%s]", e.getKey()); if (!e.getValue().getLookupExtractorFactory().close()) { - LOG.error("Failed to close lookup [%s]."); + LOG.error("Failed to close lookup [%s].", e.getKey()); } } catch (Exception ex) { @@ -191,12 +191,11 @@ public void stop() } lookupMap.clear(); + + LOG.info("LookupReferencesManager is stopped."); } - finally { - startStopLock.writeLock().unlock(); - } - LOG.info("LookupReferencesManager is stopped."); - } else { + } + finally { startStopLock.writeLock().unlock(); } } @@ -315,7 +314,7 @@ private void assertStarted() @VisibleForTesting interface Notice { - void handle(); + void handle() throws InterruptedException; } private class LoadNotice implements Notice @@ -330,9 +329,9 @@ public LoadNotice(String lookupName, LookupExtractorFactoryContainer lookupExtra } @Override - public void handle() + public void handle() throws InterruptedException { - startStopLock.readLock().lock(); + startStopLock.readLock().lockInterruptibly(); try { LookupExtractorFactoryContainer old = lookupMap.get(lookupName); @@ -352,7 +351,7 @@ public void handle() throw new ISE("start method returned false for lookup [%s]:[%s]", lookupName, lookupExtractorFactoryContainer); } - startStopLock.writeLock().lock(); + startStopLock.writeLock().lockInterruptibly(); final LookupExtractorFactoryContainer old; try { assertStarted(); @@ -385,9 +384,9 @@ public DropNotice(String lookupName) } @Override - public void handle() + public void handle() throws InterruptedException { - startStopLock.writeLock().lock(); + startStopLock.writeLock().lockInterruptibly(); final LookupExtractorFactoryContainer lookupExtractorFactoryContainer; From 79db2271064293b8690b86413c766fc01df42f85 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 22 Feb 2017 11:00:41 -0600 Subject: [PATCH 06/35] add behavior for 0.10.0 to 0.9.2 downgrade --- docs/content/querying/lookups.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 44f7e35b52a3..53da6aa50e9d 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -306,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.9.2 to 0.10.0 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.9.2 to 0.10.0, then migration for all persisted metadata is handled automatically. +If you wanna downgrade from 0.10.0 to 0.9.2, then "snapshot" directory on all lookup nodes needs to be deleted and coordinator will not see any lookup updates done after 0.10.0 upgrade. + + From 786c21f415fc7497291ca513eb92a6e07e99987c Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 23 Feb 2017 09:49:39 -0600 Subject: [PATCH 07/35] incorporating more review comments --- .../io/druid/query/lookup/LookupBean.java | 19 ++++------- .../LookupExtractorFactoryContainer.java | 32 +++++++------------ .../query/lookup/LookupReferencesManager.java | 6 ++-- .../io/druid/query/lookup/LookupsState.java | 25 +++++---------- .../LookupExtractorFactoryMapContainer.java | 31 ++++++------------ .../lookup/cache/LookupsStateWithMap.java | 19 +++-------- 6 files changed, 43 insertions(+), 89 deletions(-) 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 ec6884b94353..abd1c8247220 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupBean.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupBean.java @@ -23,7 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -//TODO merge this code to the same definition when pr/1576 is merged +import java.util.Objects; + class LookupBean { private final LookupExtractorFactoryContainer container; @@ -37,8 +38,7 @@ public LookupBean( @JsonProperty("container") LookupExtractorFactoryContainer container ) { - Preconditions.checkArgument(factory == null || container == null, "only one of factory or container should exist"); - Preconditions.checkArgument(factory != null || container != null, "one of factory or container must exist"); + Preconditions.checkArgument(factory == null ^ container == null, "only one of factory or container should exist"); this.name = name; this.container = container != null ? container : new LookupExtractorFactoryContainer(null, factory); @@ -74,21 +74,14 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LookupBean that = (LookupBean) o; - - if (!container.equals(that.container)) { - return false; - } - return name.equals(that.name); - + return Objects.equals(container, that.container) && + Objects.equals(name, that.name); } @Override public int hashCode() { - int result = container.hashCode(); - result = 31 * result + name.hashCode(); - return result; + return Objects.hash(container, name); } } diff --git a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java index 5daf9497e1cf..0edc1764421a 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java @@ -22,13 +22,18 @@ 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 String version; - private LookupExtractorFactory lookupExtractorFactory; + private final static Ordering VERSION_COMPARATOR = Ordering.natural().nullsFirst(); + + private final String version; + private final LookupExtractorFactory lookupExtractorFactory; @JsonCreator public LookupExtractorFactoryContainer( @@ -57,15 +62,7 @@ public boolean replaces(LookupExtractorFactoryContainer other) { return !this.lookupExtractorFactory.replaces(other.getLookupExtractorFactory()); } - if (version == null && other.getVersion() != null) { - return false; - } - - if (version != null && other.getVersion() == null) { - return true; - } - - return version.compareTo(other.getVersion()) > 0; + return VERSION_COMPARATOR.compare(version, other.getVersion()) > 0; } @Override @@ -86,21 +83,14 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LookupExtractorFactoryContainer that = (LookupExtractorFactoryContainer) o; - - if (version != null ? !version.equals(that.version) : that.version != null) { - return false; - } - return lookupExtractorFactory.equals(that.lookupExtractorFactory); - + return Objects.equals(version, that.version) && + Objects.equals(lookupExtractorFactory, that.lookupExtractorFactory); } @Override public int hashCode() { - int result = version != null ? version.hashCode() : 0; - result = 31 * result + lookupExtractorFactory.hashCode(); - return result; + 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 a4f83af25535..95934cf57f4b 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -44,9 +44,9 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -74,9 +74,9 @@ public class LookupReferencesManager private final LookupSnapshotTaker lookupSnapshotTaker; @VisibleForTesting - final BlockingQueue queue = new ArrayBlockingQueue<>(10000); + final BlockingQueue queue = new LinkedBlockingDeque<>(10000); - private volatile ExecutorService exec; + private ExecutorService exec; //for unit testing only private final boolean testMode; diff --git a/processing/src/main/java/io/druid/query/lookup/LookupsState.java b/processing/src/main/java/io/druid/query/lookup/LookupsState.java index 309a5f6a10c9..fd97df653cb6 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupsState.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupsState.java @@ -24,15 +24,16 @@ import java.util.Collections; import java.util.Map; +import java.util.Objects; import java.util.Set; /** */ public class LookupsState { - private Map current; - private Map toLoad; - private Set toDrop; + private final Map current; + private final Map toLoad; + private final Set toDrop; @JsonCreator public LookupsState( @@ -83,25 +84,15 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LookupsState that = (LookupsState) o; - - if (!current.equals(that.current)) { - return false; - } - if (!toLoad.equals(that.toLoad)) { - return false; - } - return toDrop.equals(that.toDrop); - + return Objects.equals(current, that.current) && + Objects.equals(toLoad, that.toLoad) && + Objects.equals(toDrop, that.toDrop); } @Override public int hashCode() { - int result = current.hashCode(); - result = 31 * result + toLoad.hashCode(); - result = 31 * result + toDrop.hashCode(); - return result; + return Objects.hash(current, toLoad, toDrop); } } 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 index 4f59679e1664..d89d9c6315e3 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java @@ -22,8 +22,10 @@ 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 @@ -32,8 +34,10 @@ */ public class LookupExtractorFactoryMapContainer { - private String version; - private Map lookupExtractorFactory; + private final static Ordering VERSION_COMPARATOR = Ordering.natural().nullsFirst(); + + private final String version; + private final Map lookupExtractorFactory; @JsonCreator public LookupExtractorFactoryMapContainer( @@ -62,15 +66,7 @@ public boolean replaces(LookupExtractorFactoryMapContainer other) { return false; } - if (version == null && other.getVersion() != null) { - return false; - } - - if (version != null && other.getVersion() == null) { - return true; - } - - return version.compareTo(other.getVersion()) > 0; + return VERSION_COMPARATOR.compare(version, other.getVersion()) > 0; } @Override @@ -91,21 +87,14 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LookupExtractorFactoryMapContainer that = (LookupExtractorFactoryMapContainer) o; - - if (version != null ? !version.equals(that.version) : that.version != null) { - return false; - } - return lookupExtractorFactory.equals(that.lookupExtractorFactory); - + return Objects.equals(version, that.version) && + Objects.equals(lookupExtractorFactory, that.lookupExtractorFactory); } @Override public int hashCode() { - int result = version != null ? version.hashCode() : 0; - result = 31 * result + lookupExtractorFactory.hashCode(); - return result; + return Objects.hash(version, lookupExtractorFactory); } } diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java index 878caa413a35..25f9af4b7fdf 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.Map; +import java.util.Objects; import java.util.Set; /** @@ -85,25 +86,15 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LookupsStateWithMap that = (LookupsStateWithMap) o; - - if (current != null ? !current.equals(that.current) : that.current != null) { - return false; - } - if (toLoad != null ? !toLoad.equals(that.toLoad) : that.toLoad != null) { - return false; - } - return !(toDrop != null ? !toDrop.equals(that.toDrop) : that.toDrop != null); - + return Objects.equals(current, that.current) && + Objects.equals(toLoad, that.toLoad) && + Objects.equals(toDrop, that.toDrop); } @Override public int hashCode() { - int result = current != null ? current.hashCode() : 0; - result = 31 * result + (toLoad != null ? toLoad.hashCode() : 0); - result = 31 * result + (toDrop != null ? toDrop.hashCode() : 0); - return result; + return Objects.hash(current, toLoad, toDrop); } } From 424cdae36d701bc7b66e626d0f98174d827cc079 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 24 Feb 2017 09:45:08 -0600 Subject: [PATCH 08/35] remove explicit lock and use LifecycleLock in LookupReferencesManager. use LifecycleLock in LookupCoordinatorManager as well --- .../query/lookup/LookupReferencesManager.java | 327 ++++++++---------- .../cache/LookupCoordinatorManager.java | 158 +++++---- .../lookup/LookupReferencesManagerTest.java | 87 +---- 3 files changed, 241 insertions(+), 331 deletions(-) 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 95934cf57f4b..5e3034c000a9 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -23,13 +23,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Collections2; import com.google.common.collect.Lists; import com.google.common.collect.Maps; 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; @@ -37,7 +38,6 @@ import io.druid.java.util.common.lifecycle.LifecycleStop; import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -45,11 +45,9 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; /** * This class provide a basic {@link LookupExtractorFactory} references manager. @@ -63,20 +61,18 @@ public class LookupReferencesManager { private static final EmittingLogger LOG = new EmittingLogger(LookupReferencesManager.class); - private final ReadWriteLock startStopLock = new ReentrantReadWriteLock(true); + private final Map lookupMap = new ConcurrentHashMap<>(); @VisibleForTesting - boolean started = false; - - @GuardedBy("startStopLock") - private final Map lookupMap = new HashMap<>(); + final BlockingQueue queue = new LinkedBlockingDeque<>(10000); private final LookupSnapshotTaker lookupSnapshotTaker; @VisibleForTesting - final BlockingQueue queue = new LinkedBlockingDeque<>(10000); + final LifecycleLock lifecycleLock = new LifecycleLock(); - private ExecutorService exec; + @VisibleForTesting + Thread mainThread; //for unit testing only private final boolean testMode; @@ -95,114 +91,117 @@ public LookupReferencesManager(LookupConfig lookupConfig, @Json ObjectMapper obj } else { this.lookupSnapshotTaker = new LookupSnapshotTaker(objectMapper, lookupConfig.getSnapshotWorkingDir()); } + this.testMode = testMode; } @LifecycleStart public void start() { - startStopLock.writeLock().lock(); - try { - if (!started) { - LOG.info("LookupReferencesManager is starting."); - - if (lookupSnapshotTaker != null) { - final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); - for (LookupBean lookupBean : lookupBeanList) { - LookupExtractorFactoryContainer container = lookupBean.getContainer(); - - if (container.getLookupExtractorFactory().start()) { - lookupMap.put(lookupBean.getName(), container); - } else { - throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.getName(), container); - } + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + + LOG.info("LookupReferencesManager is starting."); + + if (lookupSnapshotTaker != null) { + final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); + for (LookupBean lookupBean : lookupBeanList) { + LookupExtractorFactoryContainer container = lookupBean.getContainer(); + + if (container.getLookupExtractorFactory().start()) { + lookupMap.put(lookupBean.getName(), container); + } else { + throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.getName(), container); } } + } - if (exec == null && !testMode) { - exec = Execs.singleThreaded("lookup-reference-manager-%d"); - exec.execute( - new Runnable() + if (!testMode) { + mainThread = new Thread( + new Runnable() + { + @Override + public void run() { - @Override - public void run() - { - try { - while (started && !Thread.currentThread().isInterrupted()) { - try { - queue.take().handle(); - } - catch (InterruptedException ex) { - LOG.warn("interrupted, going down... lookups are not managed anymore"); - Thread.currentThread().interrupt(); - } - catch (Exception ex) { - LOG.makeAlert(ex, "Exception occured while lookup notice handling.").emit(); - } - catch (Throwable t) { - LOG.makeAlert(t, "Fatal error occured while lookup notice handling.").emit(); - throw t; - } + try { + while (lifecycleLock.isStarted() && !Thread.interrupted()) { + try { + queue.take().handle(); + } + catch (InterruptedException ex) { + LOG.warn("interrupted, going down... lookups are not managed anymore"); + Thread.currentThread().interrupt(); + } + catch (Exception ex) { + LOG.makeAlert(ex, "Exception occured while lookup notice handling.").emit(); + } + catch (Throwable t) { + LOG.makeAlert(t, "Fatal error occured while lookup notice handling.").emit(); + throw t; } } - finally { - LOG.info("Lookup Management loop exited, Lookup notices are not handled anymore."); - } + } + finally { + LOG.info("Lookup Management loop exited, Lookup notices are not handled anymore."); } } - ); - } - - started = true; + } + ); - LOG.info("LookupReferencesManager is started."); + mainThread.setName("LookupReferencesManager-MainThread"); + mainThread.start(); } - } finally { - startStopLock.writeLock().unlock(); + + LOG.info("LookupReferencesManager is started."); + lifecycleLock.started(); + } + finally { + lifecycleLock.exitStart(); } } @LifecycleStop public void stop() { - startStopLock.writeLock().lock(); - - try { - if (started) { - LOG.info("LookupReferencesManager is stopping."); - started = false; - - if (exec != null) { - exec.shutdownNow(); - exec = null; - } + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); + } - for (Map.Entry e : lookupMap.entrySet()) { - try { - LOG.info("Closing lookup [%s]", e.getKey()); - if (!e.getValue().getLookupExtractorFactory().close()) { - LOG.error("Failed to close lookup [%s].", e.getKey()); - } - } - catch (Exception ex) { - LOG.error(ex, "Failed to close lookup [%s].", e.getKey()); - } - } + LOG.info("LookupReferencesManager is stopping."); - lookupMap.clear(); + if (!testMode) { + mainThread.interrupt(); - LOG.info("LookupReferencesManager is stopped."); + try { + mainThread.join(); + } + catch (InterruptedException ex) { + throw new ISE("failed to stop, mainThread couldn't finish."); } } - finally { - startStopLock.writeLock().unlock(); + + for (Map.Entry e : lookupMap.entrySet()) { + try { + LOG.info("Closing lookup [%s]", e.getKey()); + if (!e.getValue().getLookupExtractorFactory().close()) { + LOG.error("Failed to close lookup [%s].", e.getKey()); + } + } + catch (Exception ex) { + LOG.error(ex, "Failed to close lookup [%s].", e.getKey()); + } } + + lookupMap.clear(); + + LOG.info("LookupReferencesManager is stopped."); } public void add(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) { - assertStarted(); + Preconditions.checkState(lifecycleLock.isStarted()); try { if (!queue.offer(new LoadNotice(lookupName, lookupExtractorFactoryContainer), 1, TimeUnit.MILLISECONDS)) { @@ -215,7 +214,7 @@ public void add(String lookupName, LookupExtractorFactoryContainer lookupExtract public void remove(String lookupName) { - assertStarted(); + Preconditions.checkState(lifecycleLock.isStarted()); try { if (!queue.offer(new DropNotice(lookupName), 1, TimeUnit.MILLISECONDS)) { @@ -229,92 +228,66 @@ public void remove(String lookupName) @Nullable public LookupExtractorFactoryContainer get(String lookupName) { - assertStarted(); - - startStopLock.readLock().lock(); - try { - return lookupMap.get(lookupName); - } finally { - startStopLock.readLock().unlock(); - } + Preconditions.checkState(lifecycleLock.isStarted()); + return lookupMap.get(lookupName); } public LookupsState getAllLookupsState() { - assertStarted(); - - startStopLock.readLock().lock(); - try { - Map lookupsToLoad = new HashMap<>(); - Set lookupsToDrop = new HashSet<>(); - - Iterator iter = queue.iterator(); - while (iter.hasNext()) { - Notice notice = iter.next(); - 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()); - } + Preconditions.checkState(lifecycleLock.isStarted()); + + Map lookupsToLoad = new HashMap<>(); + Set lookupsToDrop = new HashSet<>(); + + Iterator iter = queue.iterator(); + while (iter.hasNext()) { + Notice notice = iter.next(); + 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()); } - - return new LookupsState(Maps.newHashMap(lookupMap), lookupsToLoad, lookupsToDrop); - } finally { - startStopLock.readLock().unlock(); } + + return new LookupsState(Maps.newHashMap(lookupMap), lookupsToLoad, lookupsToDrop); } private void takeSnapshot() { if (lookupSnapshotTaker != null) { - startStopLock.readLock().lock(); - - List lookups; - try { - lookups = Lists.newArrayList( - Collections2.transform( - lookupMap.entrySet(), - new Function, LookupBean>() + List lookups = Lists.newArrayList( + Collections2.transform( + lookupMap.entrySet(), + new Function, LookupBean>() + { + @Nullable + @Override + public LookupBean apply( + @Nullable + Map.Entry input + ) { - @Nullable - @Override - public LookupBean apply( - @Nullable - Map.Entry input - ) - { - final LookupBean lookupBean = new LookupBean(input.getKey(), null, input.getValue()); - return lookupBean; - } + final LookupBean lookupBean = new LookupBean(input.getKey(), null, input.getValue()); + return lookupBean; } - ) - ); - } - finally { - startStopLock.readLock().unlock(); - } + } + ) + ); lookupSnapshotTaker.takeSnapshot(lookups); } } - private void assertStarted() - { - if (!started) { - throw new ISE("LookupReferencesManager is not started."); - } - } - @VisibleForTesting interface Notice { - void handle() throws InterruptedException; + void handle(); } private class LoadNotice implements Notice @@ -329,36 +302,25 @@ public LoadNotice(String lookupName, LookupExtractorFactoryContainer lookupExtra } @Override - public void handle() throws InterruptedException + public void handle() { - startStopLock.readLock().lockInterruptibly(); - - try { - 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; - } - } finally { - startStopLock.readLock().unlock(); + Preconditions.checkState(lifecycleLock.isStarted()); + + 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); } - startStopLock.writeLock().lockInterruptibly(); - final LookupExtractorFactoryContainer old; - try { - assertStarted(); - old = lookupMap.put(lookupName, lookupExtractorFactoryContainer); - } finally { - startStopLock.writeLock().unlock(); - } + old = lookupMap.put(lookupName, lookupExtractorFactoryContainer); if (LOG.isDebugEnabled()) { LOG.debug("Loaded lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer); @@ -384,18 +346,11 @@ public DropNotice(String lookupName) } @Override - public void handle() throws InterruptedException + public void handle() { - startStopLock.writeLock().lockInterruptibly(); - - final LookupExtractorFactoryContainer lookupExtractorFactoryContainer; + Preconditions.checkState(lifecycleLock.isStarted()); - try { - assertStarted(); - lookupExtractorFactoryContainer = lookupMap.remove(lookupName); - } finally { - startStopLock.writeLock().unlock(); - } + final LookupExtractorFactoryContainer lookupExtractorFactoryContainer = lookupMap.remove(lookupName); if (lookupExtractorFactoryContainer != null) { takeSnapshot(); 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 6e88f0f5ef54..6931b6a8db9d 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 @@ -48,6 +48,7 @@ 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; @@ -123,12 +124,12 @@ public URL apply(HostAndPort input) private final JacksonConfigManager configManager; private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; private final AtomicReference> knownOldState = new AtomicReference<>(); - private final Object startStopSync = new Object(); + private final Object giantLock = new Object(); // Updated by config watching service private AtomicReference>> lookupMapConfigRef; - private volatile boolean started = false; + private final LifecycleLock lifecycleLock = new LifecycleLock(); private volatile ListenableScheduledFuture backgroundManagerFuture = null; private final CountDownLatch backgroundManagerExitedLatch = new CountDownLatch(1); @@ -186,7 +187,7 @@ public boolean updateLookups(final Map> priorSpec = getKnownLookups(); if (priorSpec == null && !updateSpec.isEmpty()) { // To prevent accidentally erasing configs if we haven't updated our cache of the values @@ -232,15 +233,13 @@ public boolean updateLookups(final Map> getKnownLookups() { - if (!started) { - throw new ISE("Not started"); - } + Preconditions.checkState(lifecycleLock.isStarted(), "not started"); return lookupMapConfigRef.get(); } public boolean deleteLookup(final String tier, final String lookup, AuditInfo auditInfo) { - synchronized (startStopSync) { + synchronized (giantLock) { final Map> priorSpec = getKnownLookups(); if (priorSpec == null) { LOG.warn("Requested delete lookup [%s]/[%s]. But no lookups exist!", tier, lookup); @@ -301,98 +300,109 @@ public LookupExtractorFactoryMapContainer getLookup(final String tier, final Str @LifecycleStart public void start() { - synchronized (startStopSync) { - if (started) { + synchronized (giantLock) { + if (!lifecycleLock.canStart()) { return; } - if (executorService.isShutdown()) { - throw new ISE("Cannot restart after stop!"); - } - lookupMapConfigRef = configManager.watch( - LOOKUP_CONFIG_KEY, - new TypeReference>>() - { - }, - null - ); - - // backward compatibility with 0.9.x - if (lookupMapConfigRef.get() == null) { - Map>> oldLookups = configManager.watch( - OLD_LOOKUP_CONFIG_KEY, - new TypeReference>>>() + try { + if (executorService.isShutdown()) { + throw new ISE("Cannot restart after stop!"); + } + + lookupMapConfigRef = configManager.watch( + LOOKUP_CONFIG_KEY, + new TypeReference>>() { }, null - ).get(); - - if (oldLookups != null) { - Map> converted = new HashMap<>(); - for (String tier : oldLookups.keySet()) { - Map> oldTierLookups = oldLookups.get(tier); - if (oldLookups != null && !oldLookups.isEmpty()) { - Map convertedTierLookups = new HashMap<>(); - for (Map.Entry> e : oldTierLookups.entrySet()) { - convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); + ); + + // backward compatibility with 0.9.x + if (lookupMapConfigRef.get() == null) { + Map>> oldLookups = configManager.watch( + OLD_LOOKUP_CONFIG_KEY, + new TypeReference>>>() + { + }, + null + ).get(); + + if (oldLookups != null) { + Map> converted = new HashMap<>(); + for (String tier : oldLookups.keySet()) { + Map> oldTierLookups = oldLookups.get(tier); + if (oldLookups != null && !oldLookups.isEmpty()) { + Map convertedTierLookups = new HashMap<>(); + for (Map.Entry> e : oldTierLookups.entrySet()) { + convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); + } + converted.put(tier, convertedTierLookups); } - converted.put(tier, convertedTierLookups); } + configManager.set( + LOOKUP_CONFIG_KEY, + converted, + new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") + ); } - configManager.set(LOOKUP_CONFIG_KEY, converted, new AuditInfo("autoConversion", "autoConversion", "127.0.0.1")); } - } - this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( - new Runnable() - { - @Override - public void run() + this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( + new Runnable() { - lookupMgmtLoop(); - } - }, - 0, - lookupCoordinatorManagerConfig.getPeriod(), - TimeUnit.MILLISECONDS - ); - Futures.addCallback( - backgroundManagerFuture, new FutureCallback() - { - @Override - public void onSuccess(@Nullable Object result) + @Override + public void run() + { + lookupMgmtLoop(); + } + }, + 0, + lookupCoordinatorManagerConfig.getPeriod(), + TimeUnit.MILLISECONDS + ); + Futures.addCallback( + backgroundManagerFuture, new FutureCallback() { - backgroundManagerExitedLatch.countDown(); - LOG.debug("Exited background lookup manager"); - } + @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.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(); + @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(); + } } } - } - ); - started = true; - LOG.debug("Started"); + ); + + lifecycleLock.started(); + LOG.debug("Started"); + } finally { + lifecycleLock.exitStart(); + } + } } @LifecycleStop public void stop() { - synchronized (startStopSync) { - if (!started) { + synchronized (giantLock) { + if (!lifecycleLock.canStop()) { LOG.warn("Not started, ignoring stop request"); return; } - started = false; + executorService.shutdownNow(); final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture; this.backgroundManagerFuture = null; 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 a662171e270f..d3c361f41462 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java @@ -20,16 +20,12 @@ 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.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 com.metamx.emitter.EmittingLogger; import io.druid.concurrent.Execs; import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.ISE; import io.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; import org.junit.After; @@ -40,11 +36,6 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.TimeUnit; public class LookupReferencesManagerTest { @@ -71,41 +62,41 @@ public void setUp() throws IOException mapper, true ); - Assert.assertTrue("must be closed before start call", !lookupReferencesManager.started); + Assert.assertTrue("must be closed before start call", !lookupReferencesManager.lifecycleLock.isStarted()); lookupReferencesManager.start(); - Assert.assertTrue("must start after start call", lookupReferencesManager.started); + Assert.assertTrue("must start after start call", lookupReferencesManager.lifecycleLock.isStarted()); } @After public void tearDown() { lookupReferencesManager.stop(); - Assert.assertTrue("stop call should close it", !lookupReferencesManager.started); + Assert.assertTrue("stop call should close it", !lookupReferencesManager.lifecycleLock.isStarted()); executorService.shutdownNow(); } - @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.add("test", EasyMock.createMock(LookupExtractorFactoryContainer.class)); } - @Test(expected = ISE.class) + @Test(expected = IllegalStateException.class) public void testRemoveExceptionWhenClosed() { lookupReferencesManager.stop(); lookupReferencesManager.remove("test"); } - @Test(expected = ISE.class) + @Test(expected = IllegalStateException.class) public void testGetAllLookupsStateExceptionWhenClosed() { lookupReferencesManager.stop(); @@ -289,63 +280,17 @@ public void testGetAllLookupsState() throws Exception } @Test - public void testConcurrencyStaaaaaaaaartStop() throws Exception + public void testMainThreadStartStop() { - 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(null), + mapper, + false + ); 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.assertTrue(lookupReferencesManager.mainThread.isAlive()); + lookupReferencesManager.stop(); + Assert.assertFalse(lookupReferencesManager.mainThread.isAlive()); } private void handleOneNotice(LookupReferencesManager mgr) throws Exception { From f49cabfecd89c0a61d9e84870b2d08ccb55ed2fb Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 24 Feb 2017 14:46:10 -0600 Subject: [PATCH 09/35] wip on LookupCoordinatorManager --- .../lookup/cache/LookupCoordinatorManager.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) 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 6931b6a8db9d..e75f091d83f6 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 @@ -404,6 +404,14 @@ public void stop() } executorService.shutdownNow(); + + //went in an stop error + try { + executorService.awaitTermination(1, TimeUnit.MINUTES); + } catch (InterruptedException ex) { + + } + final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture; this.backgroundManagerFuture = null; if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { @@ -417,13 +425,11 @@ public void stop() private void lookupMgmtLoop() { // Sanity check for if we are shutting down - if (Thread.currentThread().isInterrupted()) { - LOG.info("Not updating lookups because process was interrupted"); + if (Thread.currentThread().isInterrupted() || !lifecycleLock.isStarted()) { + LOG.info("Not updating lookups because process was interrupted or not started."); return; } - final Map currState = new HashMap<>(); - final Map> allLookupTiers = lookupMapConfigRef.get(); if (allLookupTiers == null) { @@ -438,6 +444,7 @@ private void lookupMgmtLoop() LOG.debug("Starting lookup sync for on all nodes."); } + final Map currState = new HashMap<>(); final Random rand = new Random(); try { From 6301b93ed022cd2e5188993e8fe3dfb3f72e7cfa Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 24 Feb 2017 14:47:24 -0600 Subject: [PATCH 10/35] lifecycle lock --- .../io/druid/concurrent/LifecycleLock.java | 246 +++--------------- 1 file changed, 31 insertions(+), 215 deletions(-) diff --git a/common/src/main/java/io/druid/concurrent/LifecycleLock.java b/common/src/main/java/io/druid/concurrent/LifecycleLock.java index 7aff1783ec49..2847fa9a9e78 100644 --- a/common/src/main/java/io/druid/concurrent/LifecycleLock.java +++ b/common/src/main/java/io/druid/concurrent/LifecycleLock.java @@ -1,192 +1,35 @@ /* - * 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. - */ +* 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.concurrent; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.AbstractQueuedSynchronizer; - /** - * A synchronization tool for lifecycled objects (see {@link io.druid.java.util.common.lifecycle.Lifecycle}, that need - * happens-before between start() and other methods and/or to check that the object was successfully started in other - * methods. - * - * Guarantees in terms of JMM: happens-before between {@link #exitStart()} and {@link #awaitStarted()}, - * exitStart() and {@link #canStop()}, if it returns {@code true}. - * - * Example: - * class ExampleLifecycledClass - * { - * final LifecycleLock lifecycleLock = new LifecycleLock(); - * - * void start() - * { - * if (!lifecycleLock.canStart()) { - * .. return or throw exception - * } - * try { - * .. do start - * lifecycleLock.started(); - * } - * finally { - * lifecycleLock.exitStart(); - * } - * } - * - * void otherMethod() - * { - * Preconditions.checkState(lifecycleLock.awaitStarted()); - * // all actions done in start() are visible here - * .. do stuff - * } - * - * void stop() - * { - * if (!lifecycleLock.canStop()) { - * .. return or throw exception - * } - * // all actions done in start() are visible here - * .. do stop - * } - * } */ -public final class LifecycleLock +public class LifecycleLock { - private static class Sync extends AbstractQueuedSynchronizer - { - private static final int NOT_STARTED = 0; - private static final int STARTING = 1; - private static final int STARTED = 2; - private static final int START_EXITED_SUCCESSFUL = 3; - private static final int START_EXITED_FAIL = 4; - private static final int STOPPING = 5; - private static final int STOPPED = 6; - - boolean canStart() - { - return compareAndSetState(NOT_STARTED, STARTING); - } - - void started() - { - if (!compareAndSetState(STARTING, STARTED)) { - throw new IllegalMonitorStateException("Called started() not in the context of start()"); - } - } - - void exitStart() - { - // see tryReleaseShared() - releaseShared(1); - } - - @Override - protected boolean tryReleaseShared(int ignore) - { - while (true) { - int state = getState(); - if (state == STARTING) { - if (compareAndSetState(STARTING, START_EXITED_FAIL)) { - return true; - } - } else if (state == STARTED) { - if (compareAndSetState(STARTED, START_EXITED_SUCCESSFUL)) { - return true; - } - } else { - throw new IllegalMonitorStateException("exitStart() called not in the end of the start() method"); - } - } - } - - boolean awaitStarted() - { - try { - // see tryAcquireShared() - acquireSharedInterruptibly(1); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - return getState() == START_EXITED_SUCCESSFUL; - } - - boolean awaitStarted(long timeNanos) - { - try { - // see tryAcquireShared() - if (!tryAcquireSharedNanos(1, timeNanos)) { - return false; - } - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - return getState() == START_EXITED_SUCCESSFUL; - } - - @Override - protected int tryAcquireShared(int ignore) - { - return getState() > STARTED ? 1 : -1; - } - - boolean canStop() - { - while (true) { - int state = getState(); - if (state == START_EXITED_FAIL || state == STOPPING) { - return false; - } else if (state == START_EXITED_SUCCESSFUL) { - if (compareAndSetState(START_EXITED_SUCCESSFUL, STOPPING)) { - return true; - } - } else { - throw new IllegalMonitorStateException("Called canStop() before start()"); - } - } - } - - void exitStop() - { - if (!compareAndSetState(STOPPING, STOPPED)) { - throw new IllegalMonitorStateException("Called exitStop() not in the context of stop()"); - } - } - - void reset() - { - if (!compareAndSetState(STOPPED, NOT_STARTED)) { - throw new IllegalMonitorStateException("Not called exitStop() before reset()"); - } - } - } - - private final Sync sync = new Sync(); - + boolean started = false; /** - * Start latch, only one canStart() call in any thread on this LifecycleLock object could return true, if {@link - * #reset()} is not called in between. + * Start latch, only one canStart() call in any thread on this LifecycleLock object could return true. */ public boolean canStart() { - return sync.canStart(); + return started == false; } /** @@ -197,7 +40,7 @@ public boolean canStart() */ public void started() { - sync.started(); + started = true; } /** @@ -207,57 +50,30 @@ public void started() */ public void exitStart() { - sync.exitStart(); } /** * Awaits until {@link #exitStart()} is called, if needed, and returns {@code true} if {@link #started()} was called * before that. */ - public boolean awaitStarted() + public boolean isStarted() { - return sync.awaitStarted(); - } - - /** - * Awaits until {@link #exitStart()} is called for at most the specified timeout, and returns {@code true} if {@link - * #started()} was called before that. Returns {@code false} if {@code started()} wasn't called before {@code - * exitStart()}, or if {@code exitStart()} isn't called on this LifecycleLock until the specified timeout expires. - */ - public boolean awaitStarted(long timeout, TimeUnit unit) - { - return sync.awaitStarted(unit.toNanos(timeout)); + return started; } /** * Stop latch, only one canStop() call in any thread on this LifecycleLock object could return {@code true}. If * {@link #started()} wasn't called on this LifecycleLock object, always returns {@code false}. * - * @throws IllegalMonitorStateException if {@link #exitStart()} are not yet called on this LifecycleLock + * @throws IllegalMonitorStateException if {@link #canStart()} and {@link #exitStart()} are not yet called on this + * LifecycleLock */ public boolean canStop() { - return sync.canStop(); - } - - /** - * If this LifecycleLock is used in a restartable object, which uses {@link #reset()}, exitStop() must be called - * before exit from stop() on this object, usually in a finally block. - * - * @throws IllegalMonitorStateException if {@link #canStop()} is not yet called on this LifecycleLock - */ - public void exitStop() - { - sync.exitStop(); - } - - /** - * Resets the LifecycleLock after {@link #exitStop()}, so that {@link #canStart()} could be called again. - * - * @throws IllegalMonitorStateException if {@link #exitStop()} is not yet called on this LifecycleLock - */ - public void reset() - { - sync.reset(); + if(started) { + started = false; + return true; + } + return false; } } From 6c7160953b1d6e15f92c30e152bef7c17cd76d56 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 28 Feb 2017 13:34:01 -0600 Subject: [PATCH 11/35] refactor thread creation into utility method --- .../java/io/druid/concurrent/Threads.java | 38 +++++++++++++++++++ .../query/lookup/LookupReferencesManager.java | 8 ++-- 2 files changed, 43 insertions(+), 3 deletions(-) create mode 100644 common/src/main/java/io/druid/concurrent/Threads.java diff --git a/common/src/main/java/io/druid/concurrent/Threads.java b/common/src/main/java/io/druid/concurrent/Threads.java new file mode 100644 index 000000000000..7b40c5f70aa5 --- /dev/null +++ b/common/src/main/java/io/druid/concurrent/Threads.java @@ -0,0 +1,38 @@ +/* +* 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.concurrent; + +import com.google.common.base.Preconditions; + +/** + */ +public class Threads +{ + public final static Thread createThread(String name, Runnable runnable, boolean isDaemon) + { + Preconditions.checkArgument(name != null && !name.isEmpty(), "name null/empty"); + Preconditions.checkNotNull(runnable, "null runnable"); + + Thread t = new Thread(runnable); + t.setName(name); + t.setDaemon(isDaemon); + return t; + } +} 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 5e3034c000a9..8cb653b12aac 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -31,6 +31,7 @@ import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import io.druid.concurrent.LifecycleLock; +import io.druid.concurrent.Threads; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Json; import io.druid.java.util.common.ISE; @@ -119,7 +120,8 @@ public void start() } if (!testMode) { - mainThread = new Thread( + mainThread = Threads.createThread( + "LookupReferencesManager-MainThread", new Runnable() { @Override @@ -147,10 +149,10 @@ public void run() LOG.info("Lookup Management loop exited, Lookup notices are not handled anymore."); } } - } + }, + true ); - mainThread.setName("LookupReferencesManager-MainThread"); mainThread.start(); } From 5ef5df38510f8e889f34f58367822cce0a860a75 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 1 Mar 2017 09:44:22 -0600 Subject: [PATCH 12/35] more review comments addressed --- .../io/druid/concurrent/LifecycleLock.java | 79 ------------------- .../query/lookup/LookupReferencesManager.java | 59 ++++++-------- .../cache/LookupCoordinatorManager.java | 41 +++++----- 3 files changed, 47 insertions(+), 132 deletions(-) delete mode 100644 common/src/main/java/io/druid/concurrent/LifecycleLock.java diff --git a/common/src/main/java/io/druid/concurrent/LifecycleLock.java b/common/src/main/java/io/druid/concurrent/LifecycleLock.java deleted file mode 100644 index 2847fa9a9e78..000000000000 --- a/common/src/main/java/io/druid/concurrent/LifecycleLock.java +++ /dev/null @@ -1,79 +0,0 @@ -/* -* 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.concurrent; - -/** - */ -public class LifecycleLock -{ - boolean started = false; - /** - * Start latch, only one canStart() call in any thread on this LifecycleLock object could return true. - */ - public boolean canStart() - { - return started == false; - } - - /** - * Announce the start was successful. - * - * @throws IllegalMonitorStateException if {@link #canStart()} is not yet called or if {@link #exitStart()} is already - * called on this LifecycleLock - */ - public void started() - { - started = true; - } - - /** - * Must be called before exit from start() on the lifecycled object, usually in a finally block. - * - * @throws IllegalMonitorStateException if {@link #canStart()} is not yet called on this LifecycleLock - */ - public void exitStart() - { - } - - /** - * Awaits until {@link #exitStart()} is called, if needed, and returns {@code true} if {@link #started()} was called - * before that. - */ - public boolean isStarted() - { - return started; - } - - /** - * Stop latch, only one canStop() call in any thread on this LifecycleLock object could return {@code true}. If - * {@link #started()} wasn't called on this LifecycleLock object, always returns {@code false}. - * - * @throws IllegalMonitorStateException if {@link #canStart()} and {@link #exitStart()} are not yet called on this - * LifecycleLock - */ - public boolean canStop() - { - if(started) { - started = false; - return true; - } - return false; - } -} 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 8cb653b12aac..3cc1b457ea56 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -22,11 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Strings; -import com.google.common.collect.Collections2; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; @@ -39,6 +36,7 @@ import io.druid.java.util.common.lifecycle.LifecycleStop; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -99,11 +97,11 @@ public LookupReferencesManager(LookupConfig lookupConfig, @Json ObjectMapper obj @LifecycleStart public void start() { - try { - if (!lifecycleLock.canStart()) { - throw new ISE("can't start."); - } + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + try { LOG.info("LookupReferencesManager is starting."); if (lookupSnapshotTaker != null) { @@ -128,12 +126,18 @@ public void start() public void run() { try { - while (lifecycleLock.isStarted() && !Thread.interrupted()) { + + if (!lifecycleLock.awaitStarted()) { + LOG.error("WTF! lifecycle not started, lookup update notices will not be handled."); + return; + } + + while (!Thread.interrupted()) { try { queue.take().handle(); } catch (InterruptedException ex) { - LOG.warn("interrupted, going down... lookups are not managed anymore"); + LOG.warn(ex, "interrupted, going down... lookups are not managed anymore"); Thread.currentThread().interrupt(); } catch (Exception ex) { @@ -145,6 +149,9 @@ public void run() } } } + 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."); } @@ -203,7 +210,7 @@ public void stop() public void add(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) { - Preconditions.checkState(lifecycleLock.isStarted()); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); try { if (!queue.offer(new LoadNotice(lookupName, lookupExtractorFactoryContainer), 1, TimeUnit.MILLISECONDS)) { @@ -216,7 +223,7 @@ public void add(String lookupName, LookupExtractorFactoryContainer lookupExtract public void remove(String lookupName) { - Preconditions.checkState(lifecycleLock.isStarted()); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); try { if (!queue.offer(new DropNotice(lookupName), 1, TimeUnit.MILLISECONDS)) { @@ -230,13 +237,13 @@ public void remove(String lookupName) @Nullable public LookupExtractorFactoryContainer get(String lookupName) { - Preconditions.checkState(lifecycleLock.isStarted()); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); return lookupMap.get(lookupName); } public LookupsState getAllLookupsState() { - Preconditions.checkState(lifecycleLock.isStarted()); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); Map lookupsToLoad = new HashMap<>(); Set lookupsToDrop = new HashSet<>(); @@ -263,24 +270,10 @@ public LookupsState getAllLookupsState() private void takeSnapshot() { if (lookupSnapshotTaker != null) { - List lookups = Lists.newArrayList( - Collections2.transform( - lookupMap.entrySet(), - new Function, LookupBean>() - { - @Nullable - @Override - public LookupBean apply( - @Nullable - Map.Entry input - ) - { - final LookupBean lookupBean = new LookupBean(input.getKey(), null, input.getValue()); - return lookupBean; - } - } - ) - ); + List lookups = new ArrayList<>(lookupMap.size()); + for (Map.Entry e : lookupMap.entrySet()) { + lookups.add(new LookupBean(e.getKey(), null, e.getValue())); + } lookupSnapshotTaker.takeSnapshot(lookups); } @@ -306,8 +299,6 @@ public LoadNotice(String lookupName, LookupExtractorFactoryContainer lookupExtra @Override public void handle() { - Preconditions.checkState(lifecycleLock.isStarted()); - LookupExtractorFactoryContainer old = lookupMap.get(lookupName); if (old != null && !lookupExtractorFactoryContainer.replaces(old)) { LOG.warn( @@ -350,8 +341,6 @@ public DropNotice(String lookupName) @Override public void handle() { - Preconditions.checkState(lifecycleLock.isStarted()); - final LookupExtractorFactoryContainer lookupExtractorFactoryContainer = lookupMap.remove(lookupName); if (lookupExtractorFactoryContainer != null) { 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 e75f091d83f6..1f61fd519c75 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 @@ -513,7 +513,12 @@ public void run() currState.put(node, updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop))); if (LOG.isDebugEnabled()) { - LOG.debug("Sent lookup updates to node [%s].", node); + LOG.debug( + "Sent lookup toAdd[%d] and toDrop[%d] updates to node [%s].", + toLoad.size(), + toDrop.size(), + node + ); } } @@ -574,7 +579,23 @@ LookupsStateWithMap updateNode( makeResponseHandler(returnCode, reasonString), lookupCoordinatorManagerConfig.getHostTimeout() ).get()) { - if (!httpStatusIsSuccess(returnCode.get())) { + if (httpStatusIsSuccess(returnCode.get())) { + try { + final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Update on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), + response + ); + } + return response; + } catch (IOException ex) { + throw new IOException( + String.format("Failed to parse update response from [%s]. response [%s]", url, result), + ex + ); + } + } else { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); try { StreamUtils.copyAndClose(result, baos); @@ -592,22 +613,6 @@ LookupsStateWithMap updateNode( StringUtils.fromUtf8(baos.toByteArray()) ) ); - } else { - try { - final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); - if (LOG.isDebugEnabled()) { - LOG.debug( - "Update on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), - response - ); - } - return response; - } catch (IOException ex) { - throw new IOException( - String.format("Failed to parse update response from [%s]. response [%s]", url, result), - ex - ); - } } } } From 7cdf7ddfcde21d75fffdf5e4d36544ca05f6bb7b Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 1 Mar 2017 09:54:29 -0600 Subject: [PATCH 13/35] support smooth roll back of lookup snapshots from 0.10.0 to 0.9.2 --- docs/content/querying/lookups.md | 2 +- .../io/druid/query/lookup/LookupBean.java | 19 +++++++++++++++++-- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 53da6aa50e9d..aba4015661c2 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -311,6 +311,6 @@ For instance you can list all the keys/values of a map based lookup by issuing a ## Druid version 0.9.2 to 0.10.0 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.9.2 to 0.10.0, then migration for all persisted metadata is handled automatically. -If you wanna downgrade from 0.10.0 to 0.9.2, then "snapshot" directory on all lookup nodes needs to be deleted and coordinator will not see any lookup updates done after 0.10.0 upgrade. +If downgrading from 0.10.0 to 0.9.2 then lookups updates done via coordinator while 0.10.0 was running, might be lost. 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 abd1c8247220..d0053aa2a698 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupBean.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupBean.java @@ -30,6 +30,11 @@ class LookupBean private final LookupExtractorFactoryContainer container; private final String name; + //to support rollback from 0.10.0 to 0.9.2 if necessary + @Deprecated + private final LookupExtractorFactory factory; + + @JsonCreator public LookupBean( @JsonProperty("name") String name, @@ -42,6 +47,7 @@ public LookupBean( this.name = name; this.container = container != null ? container : new LookupExtractorFactoryContainer(null, factory); + this.factory = factory != null ? factory : container.getLookupExtractorFactory(); } @JsonProperty @@ -56,12 +62,20 @@ public LookupExtractorFactoryContainer getContainer() return container; } + @Deprecated + @JsonProperty + public LookupExtractorFactory getFactory() + { + return factory; + } + @Override public String toString() { return "LookupBean{" + "container=" + container + ", name='" + name + '\'' + + ", factory=" + factory + '}'; } @@ -76,12 +90,13 @@ public boolean equals(Object o) } LookupBean that = (LookupBean) o; return Objects.equals(container, that.container) && - Objects.equals(name, that.name); + Objects.equals(name, that.name) && + Objects.equals(factory, that.factory); } @Override public int hashCode() { - return Objects.hash(container, name); + return Objects.hash(container, name, factory); } } From 063bc5a7451725f22fb2e802f0f53c89cbd12e65 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 1 Mar 2017 10:14:46 -0600 Subject: [PATCH 14/35] correctly use LifecycleLock in LookupCoordinatorManager and remove synchronization from start/stop --- .../cache/LookupCoordinatorManager.java | 191 +++++++++--------- 1 file changed, 93 insertions(+), 98 deletions(-) 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 1f61fd519c75..282c2addf30a 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 @@ -124,7 +124,6 @@ public URL apply(HostAndPort input) private final JacksonConfigManager configManager; private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; private final AtomicReference> knownOldState = new AtomicReference<>(); - private final Object giantLock = new Object(); // Updated by config watching service private AtomicReference>> lookupMapConfigRef; @@ -187,7 +186,7 @@ public boolean updateLookups(final Map> priorSpec = getKnownLookups(); if (priorSpec == null && !updateSpec.isEmpty()) { // To prevent accidentally erasing configs if we haven't updated our cache of the values @@ -233,13 +232,13 @@ public boolean updateLookups(final Map> getKnownLookups() { - Preconditions.checkState(lifecycleLock.isStarted(), "not started"); + Preconditions.checkState(lifecycleLock.awaitStarted(10, TimeUnit.MILLISECONDS), "not started"); return lookupMapConfigRef.get(); } public boolean deleteLookup(final String tier, final String lookup, AuditInfo auditInfo) { - synchronized (giantLock) { + synchronized (lifecycleLock) { final Map> priorSpec = getKnownLookups(); if (priorSpec == null) { LOG.warn("Requested delete lookup [%s]/[%s]. But no lookups exist!", tier, lookup); @@ -300,132 +299,128 @@ public LookupExtractorFactoryMapContainer getLookup(final String tier, final Str @LifecycleStart public void start() { - synchronized (giantLock) { - if (!lifecycleLock.canStart()) { - return; - } + if (!lifecycleLock.canStart()) { + LOG.warn("Lookup coordinator manager can not start."); + return; + } - try { - if (executorService.isShutdown()) { - throw new ISE("Cannot restart after stop!"); - } + try { + if (executorService.isShutdown()) { + throw new ISE("Cannot restart after stop!"); + } - lookupMapConfigRef = configManager.watch( - LOOKUP_CONFIG_KEY, - new TypeReference>>() + lookupMapConfigRef = configManager.watch( + LOOKUP_CONFIG_KEY, + new TypeReference>>() + { + }, + null + ); + + // backward compatibility with 0.9.x + if (lookupMapConfigRef.get() == null) { + Map>> oldLookups = configManager.watch( + OLD_LOOKUP_CONFIG_KEY, + new TypeReference>>>() { }, null - ); - - // backward compatibility with 0.9.x - if (lookupMapConfigRef.get() == null) { - Map>> oldLookups = configManager.watch( - OLD_LOOKUP_CONFIG_KEY, - new TypeReference>>>() - { - }, - null - ).get(); - - if (oldLookups != null) { - Map> converted = new HashMap<>(); - for (String tier : oldLookups.keySet()) { - Map> oldTierLookups = oldLookups.get(tier); - if (oldLookups != null && !oldLookups.isEmpty()) { - Map convertedTierLookups = new HashMap<>(); - for (Map.Entry> e : oldTierLookups.entrySet()) { - convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); - } - converted.put(tier, convertedTierLookups); + ).get(); + + if (oldLookups != null) { + Map> converted = new HashMap<>(); + for (String tier : oldLookups.keySet()) { + Map> oldTierLookups = oldLookups.get(tier); + if (oldLookups != null && !oldLookups.isEmpty()) { + Map convertedTierLookups = new HashMap<>(); + for (Map.Entry> e : oldTierLookups.entrySet()) { + convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); } + converted.put(tier, convertedTierLookups); } - configManager.set( - LOOKUP_CONFIG_KEY, - converted, - new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") - ); } + configManager.set( + LOOKUP_CONFIG_KEY, + converted, + new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") + ); } + } - this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( - new Runnable() + this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( + new Runnable() + { + @Override + public void run() { - @Override - public void run() - { - lookupMgmtLoop(); - } - }, - 0, - lookupCoordinatorManagerConfig.getPeriod(), - TimeUnit.MILLISECONDS - ); - Futures.addCallback( - backgroundManagerFuture, new FutureCallback() + lookupMgmtLoop(); + } + }, + 0, + lookupCoordinatorManagerConfig.getPeriod(), + TimeUnit.MILLISECONDS + ); + Futures.addCallback( + backgroundManagerFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) { - @Override - public void onSuccess(@Nullable Object result) - { - backgroundManagerExitedLatch.countDown(); - LOG.debug("Exited background lookup manager"); - } + backgroundManagerExitedLatch.countDown(); + LOG.debug("Exited background lookup manager"); + } - @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(); - } + @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(); } } - ); - - lifecycleLock.started(); - LOG.debug("Started"); - } finally { - lifecycleLock.exitStart(); - } + } + ); + lifecycleLock.started(); + LOG.debug("Started"); + } finally { + lifecycleLock.exitStart(); } } @LifecycleStop public void stop() { - synchronized (giantLock) { - if (!lifecycleLock.canStop()) { - LOG.warn("Not started, ignoring stop request"); - return; - } + if (!lifecycleLock.canStop()) { + LOG.warn("Not started, ignoring stop request"); + return; + } - executorService.shutdownNow(); + executorService.shutdownNow(); - //went in an stop error - try { - executorService.awaitTermination(1, TimeUnit.MINUTES); - } catch (InterruptedException ex) { + //went in an stop error + try { + executorService.awaitTermination(1, TimeUnit.MINUTES); + } catch (InterruptedException ex) { - } + } - final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture; - this.backgroundManagerFuture = null; - if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { - LOG.warn("Background lookup manager thread could not be cancelled"); - } - // NOTE: we can't un-watch the configuration key - LOG.debug("Stopped"); + final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture; + this.backgroundManagerFuture = null; + if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { + LOG.warn("Background lookup manager thread could not be cancelled"); } + // NOTE: we can't un-watch the configuration key + LOG.debug("Stopped"); } private void lookupMgmtLoop() { // Sanity check for if we are shutting down - if (Thread.currentThread().isInterrupted() || !lifecycleLock.isStarted()) { + if (Thread.currentThread().isInterrupted() || !lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { LOG.info("Not updating lookups because process was interrupted or not started."); return; } From b5160b15cf74161f754376b18dd3e37efa20c727 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 1 Mar 2017 14:20:31 -0600 Subject: [PATCH 15/35] lifecycleLock fix usage in LookupReferencesManagerTest --- .../io/druid/query/lookup/LookupReferencesManagerTest.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 d3c361f41462..183c4d3fd2f8 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java @@ -36,6 +36,7 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; +import java.util.concurrent.TimeUnit; public class LookupReferencesManagerTest { @@ -62,16 +63,16 @@ public void setUp() throws IOException mapper, true ); - Assert.assertTrue("must be closed before start call", !lookupReferencesManager.lifecycleLock.isStarted()); + Assert.assertTrue("must be closed before start call", !lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); lookupReferencesManager.start(); - Assert.assertTrue("must start after start call", lookupReferencesManager.lifecycleLock.isStarted()); + Assert.assertTrue("must start after start call", lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); } @After public void tearDown() { lookupReferencesManager.stop(); - Assert.assertTrue("stop call should close it", !lookupReferencesManager.lifecycleLock.isStarted()); + Assert.assertTrue("stop call should close it", !lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); executorService.shutdownNow(); } From 0b203152ed6393448ee360609aae337b1b7492a7 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 7 Mar 2017 12:23:56 -0600 Subject: [PATCH 16/35] add LifecycleLock back --- .../io/druid/concurrent/LifecycleLock.java | 263 ++++++++++++++++++ 1 file changed, 263 insertions(+) create mode 100644 common/src/main/java/io/druid/concurrent/LifecycleLock.java diff --git a/common/src/main/java/io/druid/concurrent/LifecycleLock.java b/common/src/main/java/io/druid/concurrent/LifecycleLock.java new file mode 100644 index 000000000000..7aff1783ec49 --- /dev/null +++ b/common/src/main/java/io/druid/concurrent/LifecycleLock.java @@ -0,0 +1,263 @@ +/* + * 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.concurrent; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.AbstractQueuedSynchronizer; + +/** + * A synchronization tool for lifecycled objects (see {@link io.druid.java.util.common.lifecycle.Lifecycle}, that need + * happens-before between start() and other methods and/or to check that the object was successfully started in other + * methods. + * + * Guarantees in terms of JMM: happens-before between {@link #exitStart()} and {@link #awaitStarted()}, + * exitStart() and {@link #canStop()}, if it returns {@code true}. + * + * Example: + * class ExampleLifecycledClass + * { + * final LifecycleLock lifecycleLock = new LifecycleLock(); + * + * void start() + * { + * if (!lifecycleLock.canStart()) { + * .. return or throw exception + * } + * try { + * .. do start + * lifecycleLock.started(); + * } + * finally { + * lifecycleLock.exitStart(); + * } + * } + * + * void otherMethod() + * { + * Preconditions.checkState(lifecycleLock.awaitStarted()); + * // all actions done in start() are visible here + * .. do stuff + * } + * + * void stop() + * { + * if (!lifecycleLock.canStop()) { + * .. return or throw exception + * } + * // all actions done in start() are visible here + * .. do stop + * } + * } + */ +public final class LifecycleLock +{ + private static class Sync extends AbstractQueuedSynchronizer + { + private static final int NOT_STARTED = 0; + private static final int STARTING = 1; + private static final int STARTED = 2; + private static final int START_EXITED_SUCCESSFUL = 3; + private static final int START_EXITED_FAIL = 4; + private static final int STOPPING = 5; + private static final int STOPPED = 6; + + boolean canStart() + { + return compareAndSetState(NOT_STARTED, STARTING); + } + + void started() + { + if (!compareAndSetState(STARTING, STARTED)) { + throw new IllegalMonitorStateException("Called started() not in the context of start()"); + } + } + + void exitStart() + { + // see tryReleaseShared() + releaseShared(1); + } + + @Override + protected boolean tryReleaseShared(int ignore) + { + while (true) { + int state = getState(); + if (state == STARTING) { + if (compareAndSetState(STARTING, START_EXITED_FAIL)) { + return true; + } + } else if (state == STARTED) { + if (compareAndSetState(STARTED, START_EXITED_SUCCESSFUL)) { + return true; + } + } else { + throw new IllegalMonitorStateException("exitStart() called not in the end of the start() method"); + } + } + } + + boolean awaitStarted() + { + try { + // see tryAcquireShared() + acquireSharedInterruptibly(1); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + return getState() == START_EXITED_SUCCESSFUL; + } + + boolean awaitStarted(long timeNanos) + { + try { + // see tryAcquireShared() + if (!tryAcquireSharedNanos(1, timeNanos)) { + return false; + } + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + return getState() == START_EXITED_SUCCESSFUL; + } + + @Override + protected int tryAcquireShared(int ignore) + { + return getState() > STARTED ? 1 : -1; + } + + boolean canStop() + { + while (true) { + int state = getState(); + if (state == START_EXITED_FAIL || state == STOPPING) { + return false; + } else if (state == START_EXITED_SUCCESSFUL) { + if (compareAndSetState(START_EXITED_SUCCESSFUL, STOPPING)) { + return true; + } + } else { + throw new IllegalMonitorStateException("Called canStop() before start()"); + } + } + } + + void exitStop() + { + if (!compareAndSetState(STOPPING, STOPPED)) { + throw new IllegalMonitorStateException("Called exitStop() not in the context of stop()"); + } + } + + void reset() + { + if (!compareAndSetState(STOPPED, NOT_STARTED)) { + throw new IllegalMonitorStateException("Not called exitStop() before reset()"); + } + } + } + + private final Sync sync = new Sync(); + + /** + * Start latch, only one canStart() call in any thread on this LifecycleLock object could return true, if {@link + * #reset()} is not called in between. + */ + public boolean canStart() + { + return sync.canStart(); + } + + /** + * Announce the start was successful. + * + * @throws IllegalMonitorStateException if {@link #canStart()} is not yet called or if {@link #exitStart()} is already + * called on this LifecycleLock + */ + public void started() + { + sync.started(); + } + + /** + * Must be called before exit from start() on the lifecycled object, usually in a finally block. + * + * @throws IllegalMonitorStateException if {@link #canStart()} is not yet called on this LifecycleLock + */ + public void exitStart() + { + sync.exitStart(); + } + + /** + * Awaits until {@link #exitStart()} is called, if needed, and returns {@code true} if {@link #started()} was called + * before that. + */ + public boolean awaitStarted() + { + return sync.awaitStarted(); + } + + /** + * Awaits until {@link #exitStart()} is called for at most the specified timeout, and returns {@code true} if {@link + * #started()} was called before that. Returns {@code false} if {@code started()} wasn't called before {@code + * exitStart()}, or if {@code exitStart()} isn't called on this LifecycleLock until the specified timeout expires. + */ + public boolean awaitStarted(long timeout, TimeUnit unit) + { + return sync.awaitStarted(unit.toNanos(timeout)); + } + + /** + * Stop latch, only one canStop() call in any thread on this LifecycleLock object could return {@code true}. If + * {@link #started()} wasn't called on this LifecycleLock object, always returns {@code false}. + * + * @throws IllegalMonitorStateException if {@link #exitStart()} are not yet called on this LifecycleLock + */ + public boolean canStop() + { + return sync.canStop(); + } + + /** + * If this LifecycleLock is used in a restartable object, which uses {@link #reset()}, exitStop() must be called + * before exit from stop() on this object, usually in a finally block. + * + * @throws IllegalMonitorStateException if {@link #canStop()} is not yet called on this LifecycleLock + */ + public void exitStop() + { + sync.exitStop(); + } + + /** + * Resets the LifecycleLock after {@link #exitStop()}, so that {@link #canStart()} could be called again. + * + * @throws IllegalMonitorStateException if {@link #exitStop()} is not yet called on this LifecycleLock + */ + public void reset() + { + sync.reset(); + } +} From 9466b190e73c0e5339cbcd89b2dcd5abb42a24be Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 13 Mar 2017 14:23:09 -0500 Subject: [PATCH 17/35] fix license hdr --- .../java/io/druid/concurrent/Threads.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/common/src/main/java/io/druid/concurrent/Threads.java b/common/src/main/java/io/druid/concurrent/Threads.java index 7b40c5f70aa5..3115f9c27f17 100644 --- a/common/src/main/java/io/druid/concurrent/Threads.java +++ b/common/src/main/java/io/druid/concurrent/Threads.java @@ -1,21 +1,21 @@ /* -* 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. -*/ + * 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.concurrent; From 1316a14896b017c214da7d4e46ae7a4999e6c2b2 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 1 Mar 2017 11:05:05 -0600 Subject: [PATCH 18/35] run lookup mgmt on leader coordinator only --- .../server/coordinator/DruidCoordinator.java | 18 ++++++++++++++---- .../main/java/io/druid/cli/CliCoordinator.java | 2 -- 2 files changed, 14 insertions(+), 6 deletions(-) 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..3bb11f96e05a 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,10 +644,12 @@ private void stopBeingLeader() serviceAnnouncer.unannounce(self); metadataRuleManager.stop(); metadataSegmentManager.stop(); - leader = false; + lookupCoordinatorManager.stop(); } catch (Exception e) { log.makeAlert(e, "Unable to stopBeingLeader").emit(); + } finally { + leader = false; } } } diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 9a6b5ac4e878..ec66d1303f97 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -162,13 +162,11 @@ public void configure(Binder binder) 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); From d562a6d85a18caa444bdb922d5c5d47b20df1ed2 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 1 Mar 2017 11:05:40 -0600 Subject: [PATCH 19/35] wip: changes to do multiple start() and stop() on LookupCoordinatorManager --- .../cache/LookupCoordinatorManager.java | 71 +++++++++++-------- 1 file changed, 41 insertions(+), 30 deletions(-) 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 282c2addf30a..045c3db42f88 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 @@ -55,8 +55,6 @@ 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.server.listener.announcer.ListenerDiscoverer; import io.druid.server.listener.resource.ListenerResource; @@ -117,7 +115,6 @@ public URL apply(HostAndPort input) } }; - private final ListeningScheduledExecutorService executorService; private final ListenerDiscoverer listenerDiscoverer; private final HttpClient httpClient; private final ObjectMapper smileMapper; @@ -129,9 +126,10 @@ public URL apply(HostAndPort input) private AtomicReference>> lookupMapConfigRef; private final LifecycleLock lifecycleLock = new LifecycleLock(); - private volatile ListenableScheduledFuture backgroundManagerFuture = null; - private final CountDownLatch backgroundManagerExitedLatch = new CountDownLatch(1); + private ListeningScheduledExecutorService executorService; + private ListenableScheduledFuture backgroundManagerFuture; + private CountDownLatch backgroundManagerExitedLatch; @Inject public LookupCoordinatorManager( @@ -147,12 +145,6 @@ public LookupCoordinatorManager( this.httpClient = httpClient; this.smileMapper = smileMapper; this.lookupCoordinatorManagerConfig = lookupCoordinatorManagerConfig; - executorService = MoreExecutors.listeningDecorator( - Executors.newScheduledThreadPool( - lookupCoordinatorManagerConfig.getThreadPoolSize(), - Execs.makeThreadFactory("LookupCoordinatorManager--%s") - ) - ); } public boolean updateLookup( @@ -296,8 +288,9 @@ public LookupExtractorFactoryMapContainer getLookup(final String tier, final Str return tierLookups.get(lookupName); } - @LifecycleStart - public void start() + // start() and stop() are synchronized so that they never run in parallel in case of ZK acting funny and + // coordinator becomes leader and drops leadership in quick succession. + public synchronized void start() { if (!lifecycleLock.canStart()) { LOG.warn("Lookup coordinator manager can not start."); @@ -305,10 +298,22 @@ public void start() } try { - if (executorService.isShutdown()) { - throw new ISE("Cannot restart after stop!"); + if (executorService != null && + !executorService.awaitTermination( + lookupCoordinatorManagerConfig.getHostTimeout().getMillis() * 10, + TimeUnit.MILLISECONDS + )) { + throw new ISE("WTF! executor from last start() hasn't finished."); } + executorService = MoreExecutors.listeningDecorator( + Executors.newScheduledThreadPool( + lookupCoordinatorManagerConfig.getThreadPoolSize(), + Execs.makeThreadFactory("LookupCoordinatorManager--%s") + ) + ); + + //Note: this call is idempotent, so multiple start() would not cause any problems. lookupMapConfigRef = configManager.watch( LOOKUP_CONFIG_KEY, new TypeReference>>() @@ -347,6 +352,7 @@ public void start() } } + this.backgroundManagerExitedLatch = new CountDownLatch(1); this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( new Runnable() { @@ -385,36 +391,37 @@ public void onFailure(Throwable t) ); lifecycleLock.started(); + LOG.debug("Started"); + } catch (Exception ex) { + LOG.makeAlert(ex, "Got Exception while start()").emit(); } finally { lifecycleLock.exitStart(); } } - @LifecycleStop - public void stop() + public synchronized void stop() { if (!lifecycleLock.canStop()) { LOG.warn("Not started, ignoring stop request"); return; } - executorService.shutdownNow(); - - //went in an stop error try { - executorService.awaitTermination(1, TimeUnit.MINUTES); - } catch (InterruptedException ex) { + if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { + LOG.warn("Background lookup manager thread could not be cancelled"); + } - } + // NOTE: we can't un-watch the configuration key - final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture; - this.backgroundManagerFuture = null; - if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { - LOG.warn("Background lookup manager thread could not be cancelled"); + executorService.shutdownNow(); + + LOG.debug("Stopped"); + } catch (Exception ex) { + LOG.makeAlert(ex, "Got Exception while stop()").emit(); + } finally { + lifecycleLock.reset(); } - // NOTE: we can't un-watch the configuration key - LOG.debug("Stopped"); } private void lookupMgmtLoop() @@ -536,7 +543,11 @@ public void run() allFuture.get(lookupCoordinatorManagerConfig.getAllHostTimeout().getMillis(), TimeUnit.MILLISECONDS); knownOldState.set(currState); } - catch (Exception ex) { + catch (InterruptedException ex) { + allFuture.cancel(true); + Thread.currentThread().interrupt(); + throw ex; + } catch (Exception ex) { allFuture.cancel(true); throw ex; } From 60dad38521df452f7846ec69d103b68b295fbc00 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 14 Mar 2017 12:04:06 -0500 Subject: [PATCH 20/35] some fixes --- .../java/io/druid/concurrent/Threads.java | 9 +- docs/content/querying/lookups.md | 6 +- .../io/druid/query/lookup/LookupBean.java | 6 +- .../query/lookup/LookupExtractorFactory.java | 4 +- .../LookupExtractorFactoryContainer.java | 2 +- .../query/lookup/LookupReferencesManager.java | 47 ++-- .../LookupExtractorFactoryContainerTest.java | 15 + .../server/coordinator/DruidCoordinator.java | 2 - .../cache/LookupCoordinatorManager.java | 262 +++++++++--------- .../lookup/LookupReferencesManagerTest.java | 69 +++-- .../coordinator/DruidCoordinatorTest.java | 4 +- .../cache/LookupCoordinatorManagerTest.java | 173 +++++++----- ...ookupExtractorFactoryMapContainerTest.java | 15 + .../java/io/druid/cli/CliCoordinator.java | 1 + .../druid/sql/calcite/util/CalciteTests.java | 74 ++--- 15 files changed, 376 insertions(+), 313 deletions(-) diff --git a/common/src/main/java/io/druid/concurrent/Threads.java b/common/src/main/java/io/druid/concurrent/Threads.java index 3115f9c27f17..1fd1f880beb0 100644 --- a/common/src/main/java/io/druid/concurrent/Threads.java +++ b/common/src/main/java/io/druid/concurrent/Threads.java @@ -20,14 +20,17 @@ package io.druid.concurrent; import com.google.common.base.Preconditions; +import com.google.common.base.Strings; /** */ -public class Threads +public final class Threads { - public final static Thread createThread(String name, Runnable runnable, boolean isDaemon) + private Threads(){} + + public static Thread createThread(String name, Runnable runnable, boolean isDaemon) { - Preconditions.checkArgument(name != null && !name.isEmpty(), "name null/empty"); + Preconditions.checkArgument(!Strings.isNullOrEmpty(name), "name null/empty"); Preconditions.checkNotNull(runnable, "null runnable"); Thread t = new Thread(runnable); diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index aba4015661c2..080d71f19c7d 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -308,9 +308,9 @@ Lookup implementations can provide some introspection capabilities by implementi 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.9.2 to 0.10.0 upgrade/downgrade +## 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.9.2 to 0.10.0, then migration for all persisted metadata is handled automatically. -If downgrading from 0.10.0 to 0.9.2 then lookups updates done via coordinator while 0.10.0 was running, might be lost. +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/processing/src/main/java/io/druid/query/lookup/LookupBean.java b/processing/src/main/java/io/druid/query/lookup/LookupBean.java index d0053aa2a698..6c7ded2405a4 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupBean.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupBean.java @@ -30,7 +30,7 @@ class LookupBean private final LookupExtractorFactoryContainer container; private final String name; - //to support rollback from 0.10.0 to 0.9.2 if necessary + //to support rollback from 0.10.1 to 0.9.0 if necessary @Deprecated private final LookupExtractorFactory factory; @@ -38,12 +38,12 @@ class LookupBean @JsonCreator public LookupBean( @JsonProperty("name") String name, - //kept for backward compatibility with druid ver <= 0.9.2 persisted snapshots + //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, "only one of factory or container should exist"); + 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); 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 6b2bea2163ac..cb881296c834 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java @@ -52,8 +52,8 @@ public interface LookupExtractorFactory extends Supplier public boolean close(); /** - * This method is deprecated and is not removed only to allow 0.9.x to 0.10.x transition. It is not used - * on a cluster that is running 0.10.x. It will be removed in the next release. + * 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 index 0edc1764421a..6d9f73d92113 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java @@ -59,7 +59,7 @@ public LookupExtractorFactory getLookupExtractorFactory() public boolean replaces(LookupExtractorFactoryContainer other) { if (version == null && other.getVersion() == null) { - return !this.lookupExtractorFactory.replaces(other.getLookupExtractorFactory()); + return this.lookupExtractorFactory.replaces(other.getLookupExtractorFactory()); } return VERSION_COMPARATOR.compare(version, other.getVersion()) > 0; 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 3cc1b457ea56..96c636cf7bbd 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -39,7 +39,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -65,7 +64,8 @@ public class LookupReferencesManager @VisibleForTesting final BlockingQueue queue = new LinkedBlockingDeque<>(10000); - private final LookupSnapshotTaker lookupSnapshotTaker; + @VisibleForTesting + final LookupSnapshotTaker lookupSnapshotTaker; @VisibleForTesting final LifecycleLock lifecycleLock = new LifecycleLock(); @@ -104,18 +104,7 @@ public void start() try { LOG.info("LookupReferencesManager is starting."); - if (lookupSnapshotTaker != null) { - final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); - for (LookupBean lookupBean : lookupBeanList) { - LookupExtractorFactoryContainer container = lookupBean.getContainer(); - - if (container.getLookupExtractorFactory().start()) { - lookupMap.put(lookupBean.getName(), container); - } else { - throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.getName(), container); - } - } - } + loadSnapshot(); if (!testMode) { mainThread = Threads.createThread( @@ -214,7 +203,7 @@ public void add(String lookupName, LookupExtractorFactoryContainer lookupExtract try { if (!queue.offer(new LoadNotice(lookupName, lookupExtractorFactoryContainer), 1, TimeUnit.MILLISECONDS)) { - throw new ISE("notice queue add timedout to add [%s] lookup drop notice", lookupName); + throw new ISE("notice queue add timedout to add [%s] lookup load notice", lookupName); } } catch (InterruptedException ex) { throw new ISE(ex, "failed to add [%s] lookup load notice", lookupName); @@ -248,9 +237,7 @@ public LookupsState getAllLookupsState() Map lookupsToLoad = new HashMap<>(); Set lookupsToDrop = new HashSet<>(); - Iterator iter = queue.iterator(); - while (iter.hasNext()) { - Notice notice = iter.next(); + for (Notice notice : queue) { if (notice instanceof LoadNotice) { LoadNotice loadNotice = (LoadNotice) notice; lookupsToLoad.put(loadNotice.lookupName, loadNotice.lookupExtractorFactoryContainer); @@ -279,6 +266,22 @@ private void takeSnapshot() } } + private void loadSnapshot() + { + if (lookupSnapshotTaker != null) { + final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); + for (LookupBean lookupBean : lookupBeanList) { + LookupExtractorFactoryContainer container = lookupBean.getContainer(); + + if (container.getLookupExtractorFactory().start()) { + lookupMap.put(lookupBean.getName(), container); + } else { + throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.getName(), container); + } + } + } + } + @VisibleForTesting interface Notice { @@ -315,9 +318,7 @@ public void handle() old = lookupMap.put(lookupName, lookupExtractorFactoryContainer); - if (LOG.isDebugEnabled()) { - LOG.debug("Loaded lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer); - } + LOG.debug("Loaded lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer); takeSnapshot(); @@ -346,9 +347,7 @@ public void handle() if (lookupExtractorFactoryContainer != null) { takeSnapshot(); - if (LOG.isDebugEnabled()) { - LOG.debug("Removed lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer); - } + 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); diff --git a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java index 2abf258e155a..58b939415a45 100644 --- a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java +++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java @@ -60,6 +60,21 @@ public void testSerde() throws Exception ); } + @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 { 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 3bb11f96e05a..ab08dbc78ac7 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -605,8 +605,6 @@ public ScheduledExecutors.Signal call() } ); } - - lookupCoordinatorManager.start(); } catch (Exception e) { log.makeAlert(e, "Unable to become leader") 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 045c3db42f88..42b01b60b586 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 @@ -25,10 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.base.Predicates; 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; @@ -125,7 +122,8 @@ public URL apply(HostAndPort input) // Updated by config watching service private AtomicReference>> lookupMapConfigRef; - private final LifecycleLock lifecycleLock = new LifecycleLock(); + @VisibleForTesting + final LifecycleLock lifecycleLock = new LifecycleLock(); private ListeningScheduledExecutorService executorService; private ListenableScheduledFuture backgroundManagerFuture; @@ -162,7 +160,9 @@ public boolean updateLookup( public boolean updateLookups(final Map> updateSpec, AuditInfo auditInfo) { - if (updateSpec == null || updateSpec.isEmpty()) { + Preconditions.checkState(lifecycleLock.awaitStarted(5, TimeUnit.SECONDS), "not started"); + + if (updateSpec.isEmpty() && lookupMapConfigRef.get() != null) { return true; } @@ -178,7 +178,7 @@ public boolean updateLookups(final Map> priorSpec = getKnownLookups(); if (priorSpec == null && !updateSpec.isEmpty()) { // To prevent accidentally erasing configs if we haven't updated our cache of the values @@ -224,13 +224,15 @@ public boolean updateLookups(final Map> getKnownLookups() { - Preconditions.checkState(lifecycleLock.awaitStarted(10, TimeUnit.MILLISECONDS), "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 (lifecycleLock) { + 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); @@ -290,137 +292,148 @@ public LookupExtractorFactoryMapContainer getLookup(final String tier, final Str // start() and stop() are synchronized so that they never run in parallel in case of ZK acting funny and // coordinator becomes leader and drops leadership in quick succession. - public synchronized void start() + public void start() { - if (!lifecycleLock.canStart()) { - LOG.warn("Lookup coordinator manager can not start."); - return; - } - - try { - if (executorService != null && - !executorService.awaitTermination( - lookupCoordinatorManagerConfig.getHostTimeout().getMillis() * 10, - TimeUnit.MILLISECONDS - )) { - throw new ISE("WTF! executor from last start() hasn't finished."); + synchronized(lifecycleLock) { + if (!lifecycleLock.canStart()) { + throw new ISE("LookupCoordinatorManager can't start."); } - executorService = MoreExecutors.listeningDecorator( - Executors.newScheduledThreadPool( - lookupCoordinatorManagerConfig.getThreadPoolSize(), - Execs.makeThreadFactory("LookupCoordinatorManager--%s") - ) - ); - - //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.9.x - if (lookupMapConfigRef.get() == null) { - Map>> oldLookups = configManager.watch( - OLD_LOOKUP_CONFIG_KEY, - new TypeReference>>>() + try { + LOG.debug("Starting."); + + 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") + ) + ); + + //Note: this call is idempotent, so multiple start() would not cause any problems. + lookupMapConfigRef = configManager.watch( + LOOKUP_CONFIG_KEY, + new TypeReference>>() { }, null - ).get(); - - if (oldLookups != null) { - Map> converted = new HashMap<>(); - for (String tier : oldLookups.keySet()) { - Map> oldTierLookups = oldLookups.get(tier); - if (oldLookups != null && !oldLookups.isEmpty()) { - Map convertedTierLookups = new HashMap<>(); - for (Map.Entry> e : oldTierLookups.entrySet()) { - convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); + ); + + // backward compatibility with 0.9.x + if (lookupMapConfigRef.get() == null) { + Map>> oldLookups = configManager.watch( + OLD_LOOKUP_CONFIG_KEY, + new TypeReference>>>() + { + }, + null + ).get(); + + if (oldLookups != null) { + Map> converted = new HashMap<>(); + for (String tier : oldLookups.keySet()) { + Map> oldTierLookups = oldLookups.get(tier); + if (oldLookups != null && !oldLookups.isEmpty()) { + Map convertedTierLookups = new HashMap<>(); + for (Map.Entry> e : oldTierLookups.entrySet()) { + convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); + } + converted.put(tier, convertedTierLookups); } - converted.put(tier, convertedTierLookups); } + configManager.set( + LOOKUP_CONFIG_KEY, + converted, + new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") + ); } - configManager.set( - LOOKUP_CONFIG_KEY, - converted, - new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") - ); } - } - this.backgroundManagerExitedLatch = new CountDownLatch(1); - this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( - new Runnable() - { - @Override - public void run() + this.backgroundManagerExitedLatch = new CountDownLatch(1); + this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( + new Runnable() { - lookupMgmtLoop(); - } - }, - 0, - lookupCoordinatorManagerConfig.getPeriod(), - TimeUnit.MILLISECONDS - ); - Futures.addCallback( - backgroundManagerFuture, new FutureCallback() - { - @Override - public void onSuccess(@Nullable Object result) + @Override + public void run() + { + lookupMgmtLoop(); + } + }, + 2000, + lookupCoordinatorManagerConfig.getPeriod(), + TimeUnit.MILLISECONDS + ); + Futures.addCallback( + backgroundManagerFuture, new FutureCallback() { - backgroundManagerExitedLatch.countDown(); - LOG.debug("Exited background lookup manager"); - } + @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.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(); + @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(); + } } } - } - ); - - lifecycleLock.started(); + ); - LOG.debug("Started"); - } catch (Exception ex) { - LOG.makeAlert(ex, "Got Exception while start()").emit(); - } finally { - lifecycleLock.exitStart(); + LOG.debug("Started"); + } + catch (Exception ex) { + LOG.makeAlert(ex, "Got Exception while start()").emit(); + } + finally { + //so that subsequent stop() would happen. + lifecycleLock.started(); + lifecycleLock.exitStart(); + } } } public synchronized void stop() { - if (!lifecycleLock.canStop()) { - LOG.warn("Not started, ignoring stop request"); - return; - } - - try { - if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { - LOG.warn("Background lookup manager thread could not be cancelled"); + synchronized (lifecycleLock) { + if (!lifecycleLock.canStop()) { + throw new ISE("LookupCoordinatorManager can't stop."); } - // NOTE: we can't un-watch the configuration key + try { + LOG.debug("Stopping"); - executorService.shutdownNow(); + if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { + LOG.warn("Background lookup manager thread could not be cancelled"); + } - LOG.debug("Stopped"); - } catch (Exception ex) { - LOG.makeAlert(ex, "Got Exception while stop()").emit(); - } finally { - lifecycleLock.reset(); + if (executorService != null) { + executorService.shutdownNow(); + } + + LOG.debug("Stopped"); + } + catch (Exception ex) { + LOG.makeAlert(ex, "Got Exception while stop()").emit(); + } + finally { + //so that next start() would happen. + lifecycleLock.exitStop(); + lifecycleLock.reset(); + } } } @@ -428,7 +441,7 @@ private void lookupMgmtLoop() { // Sanity check for if we are shutting down if (Thread.currentThread().isInterrupted() || !lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { - LOG.info("Not updating lookups because process was interrupted or not started."); + LOG.info("Not updating lookups because process was interrupted or not finished starting yet."); return; } @@ -477,7 +490,7 @@ public void run() LOG.debug("Starting lookup sync for node [%s].", node); } - LookupsStateWithMap lookupsState = knownOldState.get().get(node); + LookupsStateWithMap lookupsState = knownOldState.get() != null ? knownOldState.get().get(node) : null; if (lookupsState == null || !lookupsState.getToLoad().isEmpty() || !lookupsState.getToDrop().isEmpty() @@ -709,11 +722,6 @@ private static boolean httpStatusIsSuccess(int statusCode) return statusCode >= 200 && statusCode < 300; } - private static boolean httpStatusIsNotFound(int statusCode) - { - return statusCode == 404; - } - @VisibleForTesting boolean backgroundManagerIsRunning() { @@ -744,18 +752,4 @@ public ClientResponse handleResponse(HttpResponse response) } }; } - - @VisibleForTesting - Collection getAllHostsAnnounceEndpoint(final String tier) throws IOException - { - return ImmutableList.copyOf( - Collections2.filter( - Collections2.transform( - listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tier)), - HOST_TO_URL - ), - Predicates.notNull() - ) - ); - } } 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 183c4d3fd2f8..5cbd4d4bafd8 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java @@ -21,14 +21,10 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import com.metamx.emitter.EmittingLogger; -import io.druid.concurrent.Execs; import io.druid.jackson.DefaultObjectMapper; 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; @@ -40,17 +36,10 @@ 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 @@ -63,44 +52,50 @@ public void setUp() throws IOException mapper, true ); - Assert.assertTrue("must be closed before start call", !lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); - lookupReferencesManager.start(); - Assert.assertTrue("must start after start call", lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); } - @After - public void tearDown() + @Test + public void testStartStop() { + lookupReferencesManager = new LookupReferencesManager( + new LookupConfig(null), + mapper, + false + ); + + Assert.assertFalse(lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); + Assert.assertNull(lookupReferencesManager.mainThread); + + lookupReferencesManager.start(); + Assert.assertTrue(lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); + Assert.assertTrue(lookupReferencesManager.mainThread.isAlive()); + lookupReferencesManager.stop(); - Assert.assertTrue("stop call should close it", !lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); - executorService.shutdownNow(); + Assert.assertFalse(lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); + Assert.assertFalse(lookupReferencesManager.mainThread.isAlive()); } @Test(expected = IllegalStateException.class) public void testGetExceptionWhenClosed() { - lookupReferencesManager.stop(); lookupReferencesManager.get("test"); } @Test(expected = IllegalStateException.class) public void testAddExceptionWhenClosed() { - lookupReferencesManager.stop(); lookupReferencesManager.add("test", EasyMock.createMock(LookupExtractorFactoryContainer.class)); } @Test(expected = IllegalStateException.class) public void testRemoveExceptionWhenClosed() { - lookupReferencesManager.stop(); lookupReferencesManager.remove("test"); } @Test(expected = IllegalStateException.class) public void testGetAllLookupsStateExceptionWhenClosed() { - lookupReferencesManager.stop(); lookupReferencesManager.getAllLookupsState(); } @@ -111,9 +106,11 @@ public void testAddGetRemove() throws Exception EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); EasyMock.replay(lookupExtractorFactory); + lookupReferencesManager.start(); Assert.assertNull(lookupReferencesManager.get("test")); LookupExtractorFactoryContainer testContainer = new LookupExtractorFactoryContainer("0", lookupExtractorFactory); + lookupReferencesManager.add("test", testContainer); handleOneNotice(lookupReferencesManager); @@ -132,6 +129,7 @@ public void testCloseIsCalledAfterStopping() throws Exception EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); EasyMock.replay(lookupExtractorFactory); + lookupReferencesManager.start(); lookupReferencesManager.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); handleOneNotice(lookupReferencesManager); @@ -147,6 +145,7 @@ public void testCloseIsCalledAfterRemove() throws Exception EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); EasyMock.replay(lookupExtractorFactory); + lookupReferencesManager.start(); lookupReferencesManager.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); handleOneNotice(lookupReferencesManager); @@ -159,6 +158,7 @@ public void testCloseIsCalledAfterRemove() throws Exception @Test public void testGetNotThere() { + lookupReferencesManager.start(); Assert.assertNull(lookupReferencesManager.get("notThere")); } @@ -174,6 +174,7 @@ public void testUpdateWithHigherVersion() throws Exception EasyMock.replay(lookupExtractorFactory1, lookupExtractorFactory2); + lookupReferencesManager.start(); lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); handleOneNotice(lookupReferencesManager); @@ -193,6 +194,7 @@ public void testUpdateWithLowerVersion() throws Exception EasyMock.replay(lookupExtractorFactory1, lookupExtractorFactory2); + lookupReferencesManager.start(); lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); handleOneNotice(lookupReferencesManager); @@ -205,6 +207,7 @@ public void testUpdateWithLowerVersion() throws Exception @Test public void testRemoveNonExisting() throws Exception { + lookupReferencesManager.start(); lookupReferencesManager.remove("test"); handleOneNotice(lookupReferencesManager); } @@ -219,9 +222,16 @@ public void testBootstrapFromFile() throws Exception ), true ); LookupExtractorFactoryContainer container = new LookupExtractorFactoryContainer("v0", lookupExtractorFactory); + lookupReferencesManager.start(); lookupReferencesManager.add("testMockForBootstrap", container); handleOneNotice(lookupReferencesManager); lookupReferencesManager.stop(); + + lookupReferencesManager = new LookupReferencesManager( + new LookupConfig(lookupReferencesManager.lookupSnapshotTaker.getPersistFile().getParent()), + mapper, + true + ); lookupReferencesManager.start(); Assert.assertEquals(container, lookupReferencesManager.get("testMockForBootstrap")); } @@ -259,6 +269,7 @@ public void testGetAllLookupsState() throws Exception ) ); + lookupReferencesManager.start(); lookupReferencesManager.add("one", container1); lookupReferencesManager.add("two", container2); lookupReferencesManager.remove("one"); @@ -280,20 +291,6 @@ public void testGetAllLookupsState() throws Exception Assert.assertTrue(state.getToDrop().contains("one")); } - @Test - public void testMainThreadStartStop() - { - lookupReferencesManager = new LookupReferencesManager( - new LookupConfig(null), - mapper, - false - ); - lookupReferencesManager.start(); - Assert.assertTrue(lookupReferencesManager.mainThread.isAlive()); - lookupReferencesManager.stop(); - Assert.assertFalse(lookupReferencesManager.mainThread.isAlive()); - } - private void handleOneNotice(LookupReferencesManager mgr) throws Exception { mgr.queue.take().handle(); } 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/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 32d4e9f4e331..ff81f90ee736 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 @@ -58,6 +58,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -132,6 +133,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 @@ -550,11 +572,44 @@ public Map> getKnownLook return null; } }; + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); expectedException.expect(ISE.class); manager.updateLookups(TIERED_LOOKUP_MAP_V0, auditInfo); } + @Test + public void testUpdateLookupsInitialization() throws Exception + { + final LookupCoordinatorManager manager = new LookupCoordinatorManager( + client, + discoverer, + mapper, + configManager, + lookupCoordinatorManagerConfig + ) + { + @Override + public Map> getKnownLookups() + { + return null; + } + }; + 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(EMPTY_TIERED_LOOKUP), + EasyMock.eq(auditInfo) + )).andReturn(true).once(); + EasyMock.replay(configManager); + manager.updateLookups(EMPTY_TIERED_LOOKUP, auditInfo); + EasyMock.verify(configManager); + } + @Test public void testUpdateLookupAdds() throws Exception { @@ -572,6 +627,8 @@ public Map> getKnownLook return EMPTY_TIERED_LOOKUP; } }; + manager.start(); + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); EasyMock.reset(configManager); EasyMock.expect(configManager.set( @@ -616,6 +673,7 @@ public Map> getKnownLook ); } }; + manager.start(); final LookupExtractorFactoryMapContainer newSpec = new LookupExtractorFactoryMapContainer( "v1", ImmutableMap.of("prop", "new") @@ -676,6 +734,7 @@ public Map> getKnownLook ); } }; + manager.start(); final LookupExtractorFactoryMapContainer newSpec = new LookupExtractorFactoryMapContainer( "v1", ImmutableMap.of("prop", "new") @@ -718,7 +777,7 @@ public Map> getKnownLook return TIERED_LOOKUP_MAP_V0; } }; - + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); EasyMock.reset(configManager); EasyMock.expect(configManager.set( @@ -748,7 +807,7 @@ public Map> getKnownLook return TIERED_LOOKUP_MAP_V0; } }; - + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); try { @@ -784,6 +843,7 @@ public Map> getKnownLook ); } }; + manager.start(); final LookupExtractorFactoryMapContainer newSpec = new LookupExtractorFactoryMapContainer( "v1", ImmutableMap.of("prop", "new") @@ -838,6 +898,7 @@ public Map> getKnownLook )); } }; + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); EasyMock.reset(configManager); EasyMock.expect( @@ -882,6 +943,7 @@ public Map> getKnownLook ); } }; + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); Assert.assertFalse(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); } @@ -903,6 +965,7 @@ public Map> getKnownLook return null; } }; + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); Assert.assertFalse(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); } @@ -986,46 +1049,7 @@ public Map> getKnownLook } @Test - public void testStart() 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>>(null)).once(); - - EasyMock.replay(configManager); - - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - new LookupCoordinatorManagerConfig(){ - @Override - public long getPeriod(){ - return 1; - } - } - ); - 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 testStartStop() throws Exception { EasyMock.reset(configManager); @@ -1050,18 +1074,24 @@ 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 { EasyMock.reset(configManager); @@ -1069,13 +1099,8 @@ public void testStartTooMuch() throws Exception 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>>(null)).once(); + )).andReturn( + new AtomicReference<>(Collections.EMPTY_MAP)).anyTimes(); EasyMock.replay(configManager); @@ -1086,30 +1111,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 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 index e17fa6015de0..0e0714d8e186 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainerTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainerTest.java @@ -72,6 +72,21 @@ public void testSerde() throws Exception 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 diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index ec66d1303f97..6ef0feebbe50 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -160,6 +160,7 @@ 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(ListenerDiscoverer.class).in(ManageLifecycle.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); From a8539d0f4cc7e54dc19cc5cd635b7ab83bb434a2 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 20 Mar 2017 09:03:26 -0500 Subject: [PATCH 21/35] make LookupReferencesManager.getAllLookupsState() consistent while still being lockless --- .../query/lookup/LookupReferencesManager.java | 201 +++++++++++++----- .../lookup/LookupReferencesManagerTest.java | 66 ++++-- 2 files changed, 194 insertions(+), 73 deletions(-) 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 96c636cf7bbd..ea53c4d08202 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -24,7 +24,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Strings; -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.LifecycleLock; @@ -42,10 +43,10 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.LinkedBlockingDeque; 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. @@ -59,10 +60,8 @@ public class LookupReferencesManager { private static final EmittingLogger LOG = new EmittingLogger(LookupReferencesManager.class); - private final Map lookupMap = new ConcurrentHashMap<>(); - @VisibleForTesting - final BlockingQueue queue = new LinkedBlockingDeque<>(10000); + final AtomicReference stateRef = new AtomicReference<>(); @VisibleForTesting final LookupSnapshotTaker lookupSnapshotTaker; @@ -104,7 +103,7 @@ public void start() try { LOG.info("LookupReferencesManager is starting."); - loadSnapshot(); + loadSnapshotAndInitStateRef(); if (!testMode) { mainThread = Threads.createThread( @@ -123,18 +122,11 @@ public void run() while (!Thread.interrupted()) { try { - queue.take().handle(); - } - catch (InterruptedException ex) { - LOG.warn(ex, "interrupted, going down... lookups are not managed anymore"); - Thread.currentThread().interrupt(); - } - catch (Exception ex) { - LOG.makeAlert(ex, "Exception occured while lookup notice handling.").emit(); + handlePendingNotices(); + LockSupport.park(); } catch (Throwable t) { - LOG.makeAlert(t, "Fatal error occured while lookup notice handling.").emit(); - throw t; + LOG.makeAlert(t, "Error occured while lookup notice handling.").emit(); } } } @@ -160,6 +152,40 @@ public void run() } } + @VisibleForTesting + void handlePendingNotices() + { + if (stateRef.get().pendingNotices.isEmpty()) { + return; + } + + LookupUpdateState swappedState = atomicallyUpdateStateRef( + oldState -> { + return 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 -> { + return new LookupUpdateState(immutableLookupMap, oldState.pendingNotices, ImmutableList.of()); + } + ); + } + @LifecycleStop public void stop() { @@ -180,7 +206,7 @@ public void stop() } } - for (Map.Entry e : lookupMap.entrySet()) { + for (Map.Entry e : stateRef.get().lookupMap.entrySet()) { try { LOG.info("Closing lookup [%s]", e.getKey()); if (!e.getValue().getLookupExtractorFactory().close()) { @@ -192,52 +218,63 @@ public void stop() } } - lookupMap.clear(); - LOG.info("LookupReferencesManager is stopped."); } public void add(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - try { - if (!queue.offer(new LoadNotice(lookupName, lookupExtractorFactoryContainer), 1, TimeUnit.MILLISECONDS)) { - throw new ISE("notice queue add timedout to add [%s] lookup load notice", lookupName); - } - } catch (InterruptedException ex) { - throw new ISE(ex, "failed to add [%s] lookup load notice", lookupName); - } + addNotice(new LoadNotice(lookupName, lookupExtractorFactoryContainer)); } public void remove(String lookupName) { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + addNotice(new DropNotice(lookupName)); + } - try { - if (!queue.offer(new DropNotice(lookupName), 1, TimeUnit.MILLISECONDS)) { - throw new ISE("notice queue add timedout to add [%s] lookup drop notice", lookupName); - } - } catch (InterruptedException ex) { - throw new ISE(ex, "failed to add [%s] lookup drop notice", lookupName); - } + private void addNotice(Notice notice) + { + atomicallyUpdateStateRef( + oldState -> { + 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 lookupMap.get(lookupName); + return stateRef.get().lookupMap.get(lookupName); } public LookupsState getAllLookupsState() { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + LookupUpdateState lookupUpdateState = stateRef.get(); + Map lookupsToLoad = new HashMap<>(); Set lookupsToDrop = new HashSet<>(); - for (Notice notice : queue) { + 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); @@ -250,11 +287,9 @@ public LookupsState getAllLookupsState() throw new ISE("Unknown Notice type [%s].", notice.getClass().getName()); } } - - return new LookupsState(Maps.newHashMap(lookupMap), lookupsToLoad, lookupsToDrop); } - private void takeSnapshot() + private void takeSnapshot(Map lookupMap) { if (lookupSnapshotTaker != null) { List lookups = new ArrayList<>(lookupMap.size()); @@ -266,32 +301,49 @@ private void takeSnapshot() } } - private void loadSnapshot() + private void loadSnapshotAndInitStateRef() { if (lookupSnapshotTaker != null) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); for (LookupBean lookupBean : lookupBeanList) { LookupExtractorFactoryContainer container = lookupBean.getContainer(); if (container.getLookupExtractorFactory().start()) { - lookupMap.put(lookupBean.getName(), container); + 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())); + } + } + + private LookupUpdateState atomicallyUpdateStateRef(Function fn) + { + while(true) { + LookupUpdateState old = stateRef.get(); + LookupUpdateState newState = fn.apply(old); + if (stateRef.compareAndSet(old, newState)) { + return newState; + } } } @VisibleForTesting interface Notice { - void handle(); + void handle(Map lookupMap); } private class LoadNotice implements Notice { - String lookupName; - LookupExtractorFactoryContainer lookupExtractorFactoryContainer; + private final String lookupName; + private final LookupExtractorFactoryContainer lookupExtractorFactoryContainer; public LoadNotice(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) { @@ -300,7 +352,7 @@ public LoadNotice(String lookupName, LookupExtractorFactoryContainer lookupExtra } @Override - public void handle() + public void handle(Map lookupMap) { LookupExtractorFactoryContainer old = lookupMap.get(lookupName); if (old != null && !lookupExtractorFactoryContainer.replaces(old)) { @@ -313,26 +365,37 @@ public void handle() } if (!lookupExtractorFactoryContainer.getLookupExtractorFactory().start()) { - throw new ISE("start method returned false for lookup [%s]:[%s]", lookupName, lookupExtractorFactoryContainer); + 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); - takeSnapshot(); - if (old != null) { if (!old.getLookupExtractorFactory().close()) { throw new ISE("close method returned false for lookup [%s]:[%s]", lookupName, old); } } } + + @Override + public String toString() + { + return "LoadNotice{" + + "lookupName='" + lookupName + '\'' + + ", lookupExtractorFactoryContainer=" + lookupExtractorFactoryContainer + + '}'; + } } private class DropNotice implements Notice { - String lookupName; + private final String lookupName; public DropNotice(String lookupName) { @@ -340,19 +403,47 @@ public DropNotice(String lookupName) } @Override - public void handle() + public void handle(Map lookupMap) { final LookupExtractorFactoryContainer lookupExtractorFactoryContainer = lookupMap.remove(lookupName); if (lookupExtractorFactoryContainer != null) { - takeSnapshot(); - 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); + throw new ISE( + "close method returned false for lookup [%s]:[%s]", + lookupName, + lookupExtractorFactoryContainer + ); } } } + + @Override + public String toString() + { + return "DropNotice{" + + "lookupName='" + lookupName + '\'' + + '}'; + } + } + + private class LookupUpdateState + { + 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/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java index 5cbd4d4bafd8..8aaf8b5c86ed 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java @@ -59,16 +59,17 @@ public void testStartStop() { lookupReferencesManager = new LookupReferencesManager( new LookupConfig(null), - mapper, - false + 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.assertFalse(lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); @@ -112,12 +113,12 @@ public void testAddGetRemove() throws Exception LookupExtractorFactoryContainer testContainer = new LookupExtractorFactoryContainer("0", lookupExtractorFactory); lookupReferencesManager.add("test", testContainer); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); Assert.assertEquals(testContainer, lookupReferencesManager.get("test")); lookupReferencesManager.remove("test"); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); Assert.assertNull(lookupReferencesManager.get("test")); } @@ -131,7 +132,7 @@ public void testCloseIsCalledAfterStopping() throws Exception EasyMock.replay(lookupExtractorFactory); lookupReferencesManager.start(); lookupReferencesManager.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); lookupReferencesManager.stop(); EasyMock.verify(lookupExtractorFactory); @@ -147,10 +148,10 @@ public void testCloseIsCalledAfterRemove() throws Exception lookupReferencesManager.start(); lookupReferencesManager.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); lookupReferencesManager.remove("testMock"); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); EasyMock.verify(lookupExtractorFactory); } @@ -176,10 +177,10 @@ public void testUpdateWithHigherVersion() throws Exception lookupReferencesManager.start(); lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("2", lookupExtractorFactory2)); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); EasyMock.verify(lookupExtractorFactory1, lookupExtractorFactory2); } @@ -196,10 +197,10 @@ public void testUpdateWithLowerVersion() throws Exception lookupReferencesManager.start(); lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("0", lookupExtractorFactory2)); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); EasyMock.verify(lookupExtractorFactory1, lookupExtractorFactory2); } @@ -209,7 +210,7 @@ public void testRemoveNonExisting() throws Exception { lookupReferencesManager.start(); lookupReferencesManager.remove("test"); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); } @Test @@ -224,7 +225,7 @@ public void testBootstrapFromFile() throws Exception LookupExtractorFactoryContainer container = new LookupExtractorFactoryContainer("v0", lookupExtractorFactory); lookupReferencesManager.start(); lookupReferencesManager.add("testMockForBootstrap", container); - handleOneNotice(lookupReferencesManager); + lookupReferencesManager.handlePendingNotices(); lookupReferencesManager.stop(); lookupReferencesManager = new LookupReferencesManager( @@ -272,12 +273,10 @@ public void testGetAllLookupsState() throws Exception lookupReferencesManager.start(); lookupReferencesManager.add("one", container1); lookupReferencesManager.add("two", container2); + lookupReferencesManager.handlePendingNotices(); lookupReferencesManager.remove("one"); lookupReferencesManager.add("three", container3); - handleOneNotice(lookupReferencesManager); - handleOneNotice(lookupReferencesManager); - LookupsState state = lookupReferencesManager.getAllLookupsState(); Assert.assertEquals(2, state.getCurrent().size()); @@ -291,7 +290,38 @@ public void testGetAllLookupsState() throws Exception Assert.assertTrue(state.getToDrop().contains("one")); } - private void handleOneNotice(LookupReferencesManager mgr) throws Exception { - mgr.queue.take().handle(); + @Test (timeout = 20000) + public void testRealModeWithMainThread() throws Exception + { + 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); + } + + lookupReferencesManager.remove("test"); + + while (lookupReferencesManager.get("test") != null) { + Thread.sleep(100); + } + + lookupReferencesManager.stop(); + + Assert.assertFalse(lookupReferencesManager.mainThread.isAlive()); } } From 4a5eb721161852eb1a0d99590fa9a05df79e347f Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 24 Mar 2017 13:06:25 -0500 Subject: [PATCH 22/35] address review comments --- .../query/lookup/LookupReferencesManager.java | 18 +- .../io/druid/query/lookup/LookupModule.java | 9 +- .../resource/AbstractListenerHandler.java | 13 +- .../cache/LookupCoordinatorManager.java | 315 ++++++++---------- 4 files changed, 152 insertions(+), 203 deletions(-) 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 ea53c4d08202..f008a6004241 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -60,6 +60,10 @@ public class LookupReferencesManager { 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<>(); @@ -123,7 +127,7 @@ public void run() while (!Thread.interrupted()) { try { handlePendingNotices(); - LockSupport.park(); + LockSupport.parkNanos(LookupReferencesManager.this, TimeUnit.MINUTES.toNanos(1)); } catch (Throwable t) { LOG.makeAlert(t, "Error occured while lookup notice handling.").emit(); @@ -160,9 +164,7 @@ void handlePendingNotices() } LookupUpdateState swappedState = atomicallyUpdateStateRef( - oldState -> { - return new LookupUpdateState(oldState.lookupMap, ImmutableList.of(), oldState.pendingNotices); - } + oldState -> new LookupUpdateState(oldState.lookupMap, ImmutableList.of(), oldState.pendingNotices) ); Map lookupMap = new HashMap<>(swappedState.lookupMap); @@ -180,9 +182,7 @@ void handlePendingNotices() ImmutableMap immutableLookupMap = ImmutableMap.copyOf(lookupMap); atomicallyUpdateStateRef( - oldState -> { - return new LookupUpdateState(immutableLookupMap, oldState.pendingNotices, ImmutableList.of()); - } + oldState -> new LookupUpdateState(immutableLookupMap, oldState.pendingNotices, ImmutableList.of()) ); } @@ -237,6 +237,10 @@ 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()); + } + ImmutableList.Builder builder = ImmutableList.builder(); builder.addAll(oldState.pendingNotices); builder.add(notice); 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 051cc7a24673..ddd9155d1acd 100644 --- a/server/src/main/java/io/druid/query/lookup/LookupModule.java +++ b/server/src/main/java/io/druid/query/lookup/LookupModule.java @@ -134,13 +134,8 @@ public Response handleUpdates( } try { - for (Map.Entry e : state.getToLoad().entrySet()) { - manager.add(e.getKey(), e.getValue()); - } - - for (String lookupToDrop : state.getToDrop()) { - manager.remove(lookupToDrop); - } + state.getToLoad().forEach(manager::add); + state.getToDrop().forEach(manager::remove); return Response.status(Response.Status.ACCEPTED).entity(manager.getAllLookupsState()).build(); } 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 f0a7ab72e5bc..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; @@ -185,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 @@ -196,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 - Object getAll(); + protected abstract Object getAll(); /** * Process a POST request of the input items @@ -213,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/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index 42b01b60b586..caec3ccb7197 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 @@ -23,7 +23,6 @@ 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.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -71,8 +70,8 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; @@ -85,38 +84,22 @@ */ public class LookupCoordinatorManager { - //key used in druid-0.9.x with config manager + //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 EmittingLogger LOG = new EmittingLogger(LookupCoordinatorManager.class); - 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 ListenerDiscoverer listenerDiscoverer; private final HttpClient httpClient; private final ObjectMapper smileMapper; private final JacksonConfigManager configManager; private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; + + // 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. private final AtomicReference> knownOldState = new AtomicReference<>(); // Updated by config watching service @@ -167,13 +150,13 @@ public boolean updateLookups(final Map lookups = updateSpec.get(tier); for (Map.Entry e : lookups.entrySet()) { Preconditions.checkNotNull( e.getValue().getVersion(), - String.format("lookup [%s]:[%s] does not have version.", tier, e.getKey()) + "lookup [%s]:[%s] does not have version.", tier, e.getKey() ); } } @@ -317,61 +300,17 @@ public void start() ) ); - //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.9.x - if (lookupMapConfigRef.get() == null) { - Map>> oldLookups = configManager.watch( - OLD_LOOKUP_CONFIG_KEY, - new TypeReference>>>() - { - }, - null - ).get(); - - if (oldLookups != null) { - Map> converted = new HashMap<>(); - for (String tier : oldLookups.keySet()) { - Map> oldTierLookups = oldLookups.get(tier); - if (oldLookups != null && !oldLookups.isEmpty()) { - Map convertedTierLookups = new HashMap<>(); - for (Map.Entry> e : oldTierLookups.entrySet()) { - convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); - } - converted.put(tier, convertedTierLookups); - } - } - configManager.set( - LOOKUP_CONFIG_KEY, - converted, - new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") - ); - } - } + initializeLookupsConfigWatcher(); this.backgroundManagerExitedLatch = new CountDownLatch(1); this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( - new Runnable() - { - @Override - public void run() - { - lookupMgmtLoop(); - } - }, - 2000, + this::lookupManagementLoop, + 2000, //initial delay to start management after a little while lookupCoordinatorManagerConfig.getPeriod(), TimeUnit.MILLISECONDS ); Futures.addCallback( - backgroundManagerFuture, new FutureCallback() + backgroundManagerFuture, new FutureCallback() { @Override public void onSuccess(@Nullable Object result) @@ -399,7 +338,7 @@ public void onFailure(Throwable t) LOG.makeAlert(ex, "Got Exception while start()").emit(); } finally { - //so that subsequent stop() would happen. + //so that subsequent stop() would happen, even if start() failed with exception lifecycleLock.started(); lifecycleLock.exitStart(); } @@ -430,14 +369,56 @@ public synchronized void stop() LOG.makeAlert(ex, "Got Exception while stop()").emit(); } finally { - //so that next start() would happen. + //so that subsequent start() would happen, even if stop() failed with exception lifecycleLock.exitStop(); lifecycleLock.reset(); } } } - private void lookupMgmtLoop() + 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 + ).get(); + + if (oldLookups != null) { + Map> converted = new HashMap<>(); + for (Map.Entry>> tierEntry : oldLookups.entrySet()) { + Map> oldTierLookups = tierEntry.getValue(); + if (oldTierLookups != null && !oldTierLookups.isEmpty()) { + Map convertedTierLookups = new HashMap<>(); + for (Map.Entry> e : oldTierLookups.entrySet()) { + convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); + } + converted.put(tierEntry.getKey(), convertedTierLookups); + } + } + configManager.set( + LOOKUP_CONFIG_KEY, + converted, + new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") + ); + } + } + } + + private void lookupManagementLoop() { // Sanity check for if we are shutting down if (Thread.currentThread().isInterrupted() || !lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { @@ -449,103 +430,33 @@ private void lookupMgmtLoop() if (allLookupTiers == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not updating lookups because no data exists"); - } + LOG.debug("Not updating lookups because no data exists"); return; } - if (LOG.isDebugEnabled()) { - LOG.debug("Starting lookup sync for on all nodes."); - } + LOG.debug("Starting lookup sync for on all nodes."); - final Map currState = new HashMap<>(); - final Random rand = new Random(); + final Map currState = new ConcurrentHashMap<>(); try { List> futures = new ArrayList<>(); - for (String tier : allLookupTiers.keySet()) { + for (Map.Entry> tierEntry : allLookupTiers.entrySet()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Starting lookup mgmt for tier [%s].", tier); - } + LOG.debug("Starting lookup mgmt for tier [%s].", tierEntry.getKey()); - final Map tierLookups = allLookupTiers.get(tier); - for (final HostAndPort node : listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tier))) { + final Map tierLookups = tierEntry.getValue(); + for (final HostAndPort node : listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tierEntry.getKey()))) { - if (LOG.isDebugEnabled()) { - LOG.debug("Starting lookup mgmt for tier [%s] and host [%s:%s].", tier, node.getHostText(), node.getPort()); - } + LOG.debug( + "Starting lookup mgmt for tier [%s] and host [%s:%s].", + tierEntry.getKey(), + node.getHostText(), + node.getPort() + ); futures.add( executorService.submit( - new Runnable() - { - @Override - public void run() - { - try { - - if (LOG.isDebugEnabled()) { - LOG.debug("Starting lookup sync for node [%s].", node); - } - - LookupsStateWithMap lookupsState = knownOldState.get() != null ? knownOldState.get().get(node) : null; - if (lookupsState == null - || !lookupsState.getToLoad().isEmpty() - || !lookupsState.getToDrop().isEmpty() - || rand.nextBoolean()) { - lookupsState = getLookupStateForNode(node); - - if (LOG.isDebugEnabled()) { - LOG.debug("Received lookups state from node [%s].", node); - } - } - currState.put(node, lookupsState); - - Map toLoad = new HashMap<>(); - for (Map.Entry e : tierLookups.entrySet()) { - String name = e.getKey(); - LookupExtractorFactoryMapContainer lookupToBe = e.getValue(); - - LookupExtractorFactoryMapContainer current = lookupsState.getToLoad().get(name); - if (current == null) { - current = lookupsState.getCurrent().get(name); - } - - if (current == null || lookupToBe.replaces(current)) { - toLoad.put(name, lookupToBe); - } - } - - Set toDrop = new HashSet<>(); - toDrop.addAll(lookupsState.getCurrent().keySet()); - toDrop.addAll(lookupsState.getToLoad().keySet()); - toDrop = Sets.difference(toDrop, lookupsState.getToDrop()); - toDrop = Sets.difference(toDrop, tierLookups.keySet()); - - if (!toLoad.isEmpty() || !toDrop.isEmpty()) { - currState.put(node, updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop))); - - if (LOG.isDebugEnabled()) { - LOG.debug( - "Sent lookup toAdd[%d] and toDrop[%d] updates to node [%s].", - toLoad.size(), - toDrop.size(), - node - ); - } - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Finished lookup sync for node [%s].", node); - } - } - catch (Exception ex) { - LOG.makeAlert(ex, "Failed to manage lookups on node [%s].", node).emit(); - } - } - } + () -> this.doLookupManagementOnNode(node, tierLookups, currState) ) ); } @@ -569,8 +480,60 @@ public void run() LOG.makeAlert(ex, "Failed to finish lookup management loop.").emit(); } - if (LOG.isDebugEnabled()) { - LOG.debug("Finished lookup sync for on all nodes."); + LOG.debug("Finished lookup sync for on all nodes."); + } + + private void doLookupManagementOnNode( + HostAndPort node, + Map nodeTierLookups, + Map stateToUpdate + ) + { + try { + + LOG.debug("Starting lookup sync for node [%s].", node); + + LookupsStateWithMap lookupsState = getLookupStateForNode(node); + LOG.debug("Received lookups state from node [%s].", node); + + stateToUpdate.put(node, lookupsState); + + Map toLoad = new HashMap<>(); + for (Map.Entry e : nodeTierLookups.entrySet()) { + String name = e.getKey(); + LookupExtractorFactoryMapContainer lookupToBe = e.getValue(); + + LookupExtractorFactoryMapContainer current = lookupsState.getToLoad().get(name); + if (current == null) { + current = lookupsState.getCurrent().get(name); + } + + if (current == null || lookupToBe.replaces(current)) { + toLoad.put(name, lookupToBe); + } + } + + Set toDrop = new HashSet<>(); + toDrop.addAll(lookupsState.getCurrent().keySet()); + toDrop.addAll(lookupsState.getToLoad().keySet()); + toDrop = Sets.difference(toDrop, lookupsState.getToDrop()); + toDrop = Sets.difference(toDrop, nodeTierLookups.keySet()); + + if (!toLoad.isEmpty() || !toDrop.isEmpty()) { + stateToUpdate.put(node, updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop))); + + LOG.debug( + "Sent lookup toAdd[%d] and toDrop[%d] updates to node [%s].", + toLoad.size(), + toDrop.size(), + node + ); + } + + LOG.debug("Finished lookup sync for node [%s].", node); + } + catch (Exception ex) { + LOG.makeAlert(ex, "Failed to manage lookups on node [%s].", node).emit(); } } @@ -586,9 +549,7 @@ LookupsStateWithMap updateNode( final URL url = getLookupsUpdateURL(node); - if (LOG.isDebugEnabled()) { - LOG.debug("Sending lookups load/drop request to [%s]. Request [%s]", url, lookupsUpdate); - } + LOG.debug("Sending lookups load/drop request to [%s]. Request [%s]", url, lookupsUpdate); try (final InputStream result = httpClient.go( new Request(HttpMethod.POST, url) @@ -601,12 +562,10 @@ LookupsStateWithMap updateNode( if (httpStatusIsSuccess(returnCode.get())) { try { final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); - if (LOG.isDebugEnabled()) { - LOG.debug( - "Update on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), - response - ); - } + LOG.debug( + "Update on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), + response + ); return response; } catch (IOException ex) { throw new IOException( @@ -645,9 +604,7 @@ LookupsStateWithMap getLookupStateForNode( final AtomicInteger returnCode = new AtomicInteger(0); final AtomicReference reasonString = new AtomicReference<>(null); - if (LOG.isDebugEnabled()) { - LOG.debug("Getting lookups from [%s]", url); - } + LOG.debug("Getting lookups from [%s]", url); try (final InputStream result = httpClient.go( new Request(HttpMethod.GET, url) @@ -658,12 +615,10 @@ LookupsStateWithMap getLookupStateForNode( if (returnCode.get() == 200) { try { final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); - if (LOG.isDebugEnabled()) { - LOG.debug( - "Get on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), - response - ); - } + LOG.debug( + "Get on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), + response + ); return response; } catch(IOException ex) { throw new IOException( From 7df2ad0e13f4465ead5f941a69910b03a2e59254 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 30 Mar 2017 13:48:49 -0500 Subject: [PATCH 23/35] addressing leventov's comments --- .../cache/LookupCoordinatorManager.java | 210 +++++++++++------- .../cache/LookupCoordinatorManagerConfig.java | 5 +- 2 files changed, 130 insertions(+), 85 deletions(-) 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 caec3ccb7197..afc901a959ec 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 @@ -48,6 +48,7 @@ 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; @@ -151,12 +152,11 @@ public boolean updateLookups(final Map lookups = updateSpec.get(tier); - for (Map.Entry e : lookups.entrySet()) { + 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.", tier, e.getKey() + "lookup [%s]:[%s] does not have version.", tierEntry.getKey(), e.getKey() ); } } @@ -176,9 +176,11 @@ public boolean updateLookups(final Map(priorSpec); - for (final String tier : updateSpec.keySet()) { + for (final Map.Entry> tierEntry : updateSpec.entrySet()) { + final String tier = tierEntry.getKey(); + final Map updateTierSpec = tierEntry.getValue(); final Map priorTierSpec = priorSpec.get(tier); - final Map updateTierSpec = updateSpec.get(tier); + if (priorTierSpec == null) { // New tier updatedSpec.put(tier, updateTierSpec); @@ -273,7 +275,7 @@ public LookupExtractorFactoryMapContainer getLookup(final String tier, final Str return tierLookups.get(lookupName); } - // start() and stop() are synchronized so that they never run in parallel in case of ZK acting funny and + // start() and stop() are synchronized so that they never run in parallel in case of ZK acting funny or drui bug and // coordinator becomes leader and drops leadership in quick succession. public void start() { @@ -345,7 +347,7 @@ public void onFailure(Throwable t) } } - public synchronized void stop() + public void stop() { synchronized (lifecycleLock) { if (!lifecycleLock.canStop()) { @@ -399,16 +401,14 @@ private void initializeLookupsConfigWatcher() if (oldLookups != null) { Map> converted = new HashMap<>(); - for (Map.Entry>> tierEntry : oldLookups.entrySet()) { - Map> oldTierLookups = tierEntry.getValue(); - if (oldTierLookups != null && !oldTierLookups.isEmpty()) { - Map convertedTierLookups = new HashMap<>(); - for (Map.Entry> e : oldTierLookups.entrySet()) { - convertedTierLookups.put(e.getKey(), new LookupExtractorFactoryMapContainer(null, e.getValue())); + oldLookups.forEach( + (tier, oldTierLookups) -> { + if (oldTierLookups != null && !oldTierLookups.isEmpty()) { + converted.put(tier, convertTierLookups(oldTierLookups)); + } } - converted.put(tierEntry.getKey(), convertedTierLookups); - } - } + ); + configManager.set( LOOKUP_CONFIG_KEY, converted, @@ -418,6 +418,19 @@ private void initializeLookupsConfigWatcher() } } + private Map convertTierLookups( + Map> oldTierLookups + ) + { + Map convertedTierLookups = new HashMap<>(); + oldTierLookups.forEach( + (lookup, lookupExtractorFactory) -> { + convertedTierLookups.put(lookup, new LookupExtractorFactoryMapContainer(null, lookupExtractorFactory)); + } + ); + return convertedTierLookups; + } + private void lookupManagementLoop() { // Sanity check for if we are shutting down @@ -429,8 +442,7 @@ private void lookupManagementLoop() final Map> allLookupTiers = lookupMapConfigRef.get(); if (allLookupTiers == null) { - - LOG.debug("Not updating lookups because no data exists"); + LOG.info("Not updating lookups because no data exists"); return; } @@ -456,7 +468,19 @@ private void lookupManagementLoop() futures.add( executorService.submit( - () -> this.doLookupManagementOnNode(node, tierLookups, currState) + () -> { + try { + currState.put(node, this.doLookupManagementOnNode(node, tierLookups)); + } + catch (Exception ex) { + LOG.makeAlert( + ex, + "Failed to finish lookup management on node [%s:%s]", + node.getHostText(), + node.getPort() + ); + } + } ) ); } @@ -483,58 +507,83 @@ private void lookupManagementLoop() LOG.debug("Finished lookup sync for on all nodes."); } - private void doLookupManagementOnNode( + private LookupsStateWithMap doLookupManagementOnNode( HostAndPort node, - Map nodeTierLookups, - Map stateToUpdate - ) + Map nodeTierLookupsToBe + ) throws IOException, InterruptedException, ExecutionException { - try { + LOG.debug("Starting lookup sync for node [%s].", node); - LOG.debug("Starting lookup sync for node [%s].", node); + LookupsStateWithMap currLookupsStateOnNode = getLookupStateForNode(node); + LOG.debug("Received lookups state from node [%s].", node); - LookupsStateWithMap lookupsState = getLookupStateForNode(node); - LOG.debug("Received lookups state from node [%s].", node); - stateToUpdate.put(node, lookupsState); + // Compare currLookupsStateOnNode with nodeTierLookupsToBe to find what are the lookups + // we need to further ask node to load/drop + Map toLoad = xx(currLookupsStateOnNode, nodeTierLookupsToBe); + Set toDrop = toDrop(currLookupsStateOnNode, nodeTierLookupsToBe); - Map toLoad = new HashMap<>(); - for (Map.Entry e : nodeTierLookups.entrySet()) { - String name = e.getKey(); - LookupExtractorFactoryMapContainer lookupToBe = e.getValue(); + 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 = updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop)); - LookupExtractorFactoryMapContainer current = lookupsState.getToLoad().get(name); - if (current == null) { - current = lookupsState.getCurrent().get(name); - } + LOG.debug( + "Sent lookup toAdd[%s] and toDrop[%s] updates to node [%s].", + toLoad.keySet(), + toDrop, + node + ); + } - if (current == null || lookupToBe.replaces(current)) { - toLoad.put(name, lookupToBe); - } - } + LOG.debug("Finished lookup sync for node [%s].", node); + return currLookupsStateOnNode; + } - Set toDrop = new HashSet<>(); - toDrop.addAll(lookupsState.getCurrent().keySet()); - toDrop.addAll(lookupsState.getToLoad().keySet()); - toDrop = Sets.difference(toDrop, lookupsState.getToDrop()); - toDrop = Sets.difference(toDrop, nodeTierLookups.keySet()); + // Returns the Map that needs to be loaded by the node and it does not know about + // those already. + private Map xx( + LookupsStateWithMap currLookupsStateOnNode, + Map nodeTierLookupsToBe + ) + { + Map toLoad = new HashMap<>(); + for (Map.Entry e : nodeTierLookupsToBe.entrySet()) { + String name = e.getKey(); + LookupExtractorFactoryMapContainer lookupToBe = e.getValue(); - if (!toLoad.isEmpty() || !toDrop.isEmpty()) { - stateToUpdate.put(node, updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop))); + // get it from the current pending notices list on the node + LookupExtractorFactoryMapContainer current = currLookupsStateOnNode.getToLoad().get(name); - LOG.debug( - "Sent lookup toAdd[%d] and toDrop[%d] updates to node [%s].", - toLoad.size(), - toDrop.size(), - node - ); + if (current == null) { + //ok, not on pending list, get from currently loaded lookups on node + current = currLookupsStateOnNode.getCurrent().get(name); } - LOG.debug("Finished lookup sync for node [%s].", node); - } - catch (Exception ex) { - LOG.makeAlert(ex, "Failed to manage lookups on node [%s].", node).emit(); + if (current == null || //lookup is neither pending nor already loaded on the node OR + lookupToBe.replaces(current) //lookup is already know to node, but lookupToBe overrides that + ) { + toLoad.put(name, lookupToBe); + } } + return toLoad; + } + + // Returns Set that should be dropped from the node which has them already either in pending to load + // state or loaded + private Set toDrop( + LookupsStateWithMap 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; } @VisibleForTesting @@ -568,9 +617,8 @@ LookupsStateWithMap updateNode( ); return response; } catch (IOException ex) { - throw new IOException( - String.format("Failed to parse update response from [%s]. response [%s]", url, result), - ex + throw new IOE( + ex, "Failed to parse update response from [%s]. response [%s]", url, result ); } } else { @@ -582,14 +630,12 @@ LookupsStateWithMap updateNode( 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()) - ) + throw new IOE( + "Bad update request to [%s] : [%d] : [%s] Response: [%s]", + url, + returnCode.get(), + reasonString.get(), + StringUtils.fromUtf8(baos.toByteArray()) ); } } @@ -621,13 +667,11 @@ LookupsStateWithMap getLookupStateForNode( ); return response; } catch(IOException ex) { - throw new IOException( - String.format( - "Failed to parser GET lookups response from [%s]. response [%s].", - url, - result - ), - ex + throw new IOE( + ex, + "Failed to parser GET lookups response from [%s]. response [%s].", + url, + result ); } } else { @@ -639,14 +683,12 @@ LookupsStateWithMap getLookupStateForNode( LOG.warn(ex, "Error reading response from GET on url [%s]", url); } - throw new IOException( - String.format( - "GET request failed to [%s] : [%d] : [%s] Response: [%s]", - url, - returnCode.get(), - reasonString.get(), - StringUtils.fromUtf8(baos.toByteArray()) - ) + throw new IOE( + "GET request failed to [%s] : [%d] : [%s] Response: [%s]", + url, + returnCode.get(), + reasonString.get(), + StringUtils.fromUtf8(baos.toByteArray()) ); } } 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 bec67c44f9cc..4b05b2c046f6 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 @@ -31,11 +31,14 @@ public class LookupCoordinatorManagerConfig @JsonProperty private Duration hostTimeout = null; + @JsonProperty private Duration allHostTimeout = null; + @JsonProperty - @Min(2) + @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 = 120_000L; From d60eb8ebe1161127d8c72e03fbfac9efe010b1da Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 30 Mar 2017 14:27:52 -0500 Subject: [PATCH 24/35] address charle's comments --- .../io/druid/server/coordinator/DruidCoordinator.java | 4 ++-- .../server/lookup/cache/LookupCoordinatorManager.java | 10 ++++++++-- 2 files changed, 10 insertions(+), 4 deletions(-) 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 ab08dbc78ac7..bba45974e6c4 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -643,11 +643,11 @@ private void stopBeingLeader() metadataRuleManager.stop(); metadataSegmentManager.stop(); lookupCoordinatorManager.stop(); + + leader = false; } catch (Exception e) { log.makeAlert(e, "Unable to stopBeingLeader").emit(); - } finally { - leader = false; } } } 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 afc901a959ec..93b2440430ae 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 @@ -90,6 +90,12 @@ public class LookupCoordinatorManager 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 EmittingLogger LOG = new EmittingLogger(LookupCoordinatorManager.class); private final ListenerDiscoverer listenerDiscoverer; @@ -700,7 +706,7 @@ static URL getLookupsURL(HostAndPort druidNode) throws MalformedURLException "http", druidNode.getHostText(), druidNode.getPortOrDefault(-1), - ListenerResource.BASE_PATH + "/" + LOOKUP_LISTEN_ANNOUNCE_KEY + LOOKUP_BASE_REQUEST_PATH ); } @@ -710,7 +716,7 @@ static URL getLookupsUpdateURL(HostAndPort druidNode) throws MalformedURLExcepti "http", druidNode.getHostText(), druidNode.getPortOrDefault(-1), - ListenerResource.BASE_PATH + "/" + LOOKUP_LISTEN_ANNOUNCE_KEY + "/" + "updates" + LOOKUP_UPDATE_REQUEST_PATH ); } From bcc89b6935d58cfdb1c4fc69ba3e09d2b9c16bf0 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 30 Mar 2017 15:23:32 -0500 Subject: [PATCH 25/35] add IOE.java --- .../java/io/druid/java/util/common/IOE.java | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 java-util/src/main/java/io/druid/java/util/common/IOE.java 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); + } +} From d695df07f7cf46b87bee49858b7d68bc119bb0bc Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 31 Mar 2017 10:40:35 -0500 Subject: [PATCH 26/35] for safety in LookupReferencesManager mainThread check for lifecycle started state on each loop in addition to interrupt --- .../java/io/druid/query/lookup/LookupReferencesManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f008a6004241..b8c7c525dbd6 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -124,7 +124,7 @@ public void run() return; } - while (!Thread.interrupted()) { + while (!Thread.interrupted() && lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { try { handlePendingNotices(); LockSupport.parkNanos(LookupReferencesManager.this, TimeUnit.MINUTES.toNanos(1)); From 84d8d086f062d302fa5f717960c2841be606d4d7 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 31 Mar 2017 10:46:42 -0500 Subject: [PATCH 27/35] move thread creation utility method to Execs --- .../main/java/io/druid/concurrent/Execs.java | 14 +++++++ .../java/io/druid/concurrent/Threads.java | 41 ------------------- .../query/lookup/LookupReferencesManager.java | 4 +- 3 files changed, 16 insertions(+), 43 deletions(-) delete mode 100644 common/src/main/java/io/druid/concurrent/Threads.java 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/common/src/main/java/io/druid/concurrent/Threads.java b/common/src/main/java/io/druid/concurrent/Threads.java deleted file mode 100644 index 1fd1f880beb0..000000000000 --- a/common/src/main/java/io/druid/concurrent/Threads.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.concurrent; - -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; - -/** - */ -public final class Threads -{ - private Threads(){} - - public static Thread createThread(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; - } -} 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 b8c7c525dbd6..e0ff88ce8e25 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -28,8 +28,8 @@ 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.concurrent.Threads; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Json; import io.druid.java.util.common.ISE; @@ -110,7 +110,7 @@ public void start() loadSnapshotAndInitStateRef(); if (!testMode) { - mainThread = Threads.createThread( + mainThread = Execs.makeThread( "LookupReferencesManager-MainThread", new Runnable() { From 351f256f5897aae3359966446c00d8539f5ddf6a Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 31 Mar 2017 11:20:19 -0500 Subject: [PATCH 28/35] fix names --- .../server/lookup/cache/LookupCoordinatorManager.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) 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 93b2440430ae..1c357e8ac6e2 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 @@ -526,8 +526,11 @@ private LookupsStateWithMap doLookupManagementOnNode( // Compare currLookupsStateOnNode with nodeTierLookupsToBe to find what are the lookups // we need to further ask node to load/drop - Map toLoad = xx(currLookupsStateOnNode, nodeTierLookupsToBe); - Set toDrop = toDrop(currLookupsStateOnNode, nodeTierLookupsToBe); + 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. @@ -548,7 +551,7 @@ private LookupsStateWithMap doLookupManagementOnNode( // Returns the Map that needs to be loaded by the node and it does not know about // those already. - private Map xx( + private Map getToBeLoadedOnNode( LookupsStateWithMap currLookupsStateOnNode, Map nodeTierLookupsToBe ) @@ -577,7 +580,7 @@ private Map xx( // Returns Set that should be dropped from the node which has them already either in pending to load // state or loaded - private Set toDrop( + private Set getToBeDroppedFromNode( LookupsStateWithMap currLookupsStateOnNode, Map nodeTierLookupsToBe ) From bea763f93c64cbc1a49afb4b562e9e0c4fcd9208 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 6 Apr 2017 15:41:18 -0500 Subject: [PATCH 29/35] add tests for LookupCoordinatorManager.lookupManagementLoop() --- .../cache/LookupCoordinatorManager.java | 294 ++++++----- .../cache/LookupCoordinatorManagerConfig.java | 14 + .../cache/LookupCoordinatorManagerTest.java | 481 +++++++++++------- 3 files changed, 471 insertions(+), 318 deletions(-) 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 1c357e8ac6e2..42f0e061af01 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 @@ -99,15 +99,15 @@ public class LookupCoordinatorManager private static final EmittingLogger LOG = new EmittingLogger(LookupCoordinatorManager.class); private final ListenerDiscoverer listenerDiscoverer; - private final HttpClient httpClient; - private final ObjectMapper smileMapper; private final JacksonConfigManager configManager; private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; + 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. - private final AtomicReference> knownOldState = new AtomicReference<>(); + @VisibleForTesting + final AtomicReference> knownOldState = new AtomicReference<>(); // Updated by config watching service private AtomicReference>> lookupMapConfigRef; @@ -127,12 +127,27 @@ public LookupCoordinatorManager( final JacksonConfigManager configManager, final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig ) + { + this( + listenerDiscoverer, + configManager, + lookupCoordinatorManagerConfig, + new LookupsCommunicator(httpClient, lookupCoordinatorManagerConfig, smileMapper) + ); + } + + @VisibleForTesting + LookupCoordinatorManager( + final ListenerDiscoverer listenerDiscoverer, + final JacksonConfigManager configManager, + final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig, + final LookupsCommunicator lookupsCommunicator + ) { this.listenerDiscoverer = listenerDiscoverer; this.configManager = configManager; - this.httpClient = httpClient; - this.smileMapper = smileMapper; this.lookupCoordinatorManagerConfig = lookupCoordinatorManagerConfig; + this.lookupsCommunicator = lookupsCommunicator; } public boolean updateLookup( @@ -313,7 +328,7 @@ public void start() this.backgroundManagerExitedLatch = new CountDownLatch(1); this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( this::lookupManagementLoop, - 2000, //initial delay to start management after a little while + lookupCoordinatorManagerConfig.getInitialDelay(), lookupCoordinatorManagerConfig.getPeriod(), TimeUnit.MILLISECONDS ); @@ -437,10 +452,11 @@ private Map convertTierLookups( return convertedTierLookups; } - private void lookupManagementLoop() + @VisibleForTesting + void lookupManagementLoop() { // Sanity check for if we are shutting down - if (Thread.currentThread().isInterrupted() || !lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { + if (Thread.currentThread().isInterrupted() || !lifecycleLock.awaitStarted(15, TimeUnit.SECONDS)) { LOG.info("Not updating lookups because process was interrupted or not finished starting yet."); return; } @@ -520,7 +536,7 @@ private LookupsStateWithMap doLookupManagementOnNode( { LOG.debug("Starting lookup sync for node [%s].", node); - LookupsStateWithMap currLookupsStateOnNode = getLookupStateForNode(node); + LookupsStateWithMap currLookupsStateOnNode = lookupsCommunicator.getLookupStateForNode(node); LOG.debug("Received lookups state from node [%s].", node); @@ -535,7 +551,7 @@ private LookupsStateWithMap doLookupManagementOnNode( 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 = updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop)); + currLookupsStateOnNode = lookupsCommunicator.updateNode(node, new LookupsStateWithMap(null, toLoad, toDrop)); LOG.debug( "Sent lookup toAdd[%s] and toDrop[%s] updates to node [%s].", @@ -595,114 +611,6 @@ private Set getToBeDroppedFromNode( return toDrop; } - @VisibleForTesting - LookupsStateWithMap updateNode( - HostAndPort node, - LookupsStateWithMap 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 LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); - 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()) - ); - } - } - } - - @VisibleForTesting - LookupsStateWithMap 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() == 200) { - try { - final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); - 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()) - ); - } - } - } - static URL getLookupsURL(HostAndPort druidNode) throws MalformedURLException { return new URL( @@ -741,21 +649,145 @@ boolean waitForBackgroundTermination(long timeout) throws InterruptedException return backgroundManagerExitedLatch.await(timeout, TimeUnit.MILLISECONDS); } - @VisibleForTesting - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) + public static class LookupsCommunicator { - return new SequenceInputStreamResponseHandler() + private final HttpClient httpClient; + private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; + private final ObjectMapper smileMapper; + + public LookupsCommunicator( + HttpClient httpClient, + LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig, + ObjectMapper smileMapper + ) { - @Override - public ClientResponse handleResponse(HttpResponse response) - { - returnCode.set(response.getStatus().getCode()); - reasonString.set(response.getStatus().getReasonPhrase()); - return super.handleResponse(response); + this.httpClient = httpClient; + this.lookupCoordinatorManagerConfig = lookupCoordinatorManagerConfig; + this.smileMapper = smileMapper; + } + + public LookupsStateWithMap updateNode( + HostAndPort node, + LookupsStateWithMap 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 LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); + 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 LookupsStateWithMap 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() == 200) { + try { + final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); + 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 4b05b2c046f6..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 @@ -43,6 +43,10 @@ public class LookupCoordinatorManagerConfig @Min(1) private long period = 120_000L; + @JsonProperty + @Min(1) + private long initialDelay = 2_000L; + public Duration getHostTimeout() { return hostTimeout == null ? DEFAULT_HOST_TIMEOUT : hostTimeout; @@ -82,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/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index ff81f90ee736..c1ce61decdd6 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 @@ -23,6 +23,7 @@ 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; @@ -34,10 +35,11 @@ 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.server.listener.announcer.ListenerDiscoverer; import org.easymock.EasyMock; import org.hamcrest.BaseMatcher; @@ -169,12 +171,15 @@ public void testUpdateNodeWithSuccess() throws Exception final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream( - StringUtils.toUtf8( - mapper.writeValueAsString( - LOOKUPS_STATE - ) - ))); + future.set( + new ByteArrayInputStream( + StringUtils.toUtf8( + mapper.writeValueAsString( + LOOKUPS_STATE + ) + ) + ) + ); EasyMock.expect(client.go( EasyMock.anyObject(), EasyMock.anyObject(), @@ -183,27 +188,22 @@ public void testUpdateNodeWithSuccess() 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(Response.Status.ACCEPTED.getStatusCode()); - reasonString.set(""); - return responseHandler; - } - }; + 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; + } + }; - LookupsStateWithMap resp = manager.updateNode( + LookupsStateWithMap resp = lookupsCommunicator.updateNode( HostAndPort.fromString("localhost"), LOOKUPS_STATE ); @@ -227,28 +227,23 @@ public void testUpdateNodeRespondedWithNotOkErrorCode() 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(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode()); - reasonString.set(""); - return 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; + } + }; try { - manager.updateNode( + lookupsCommunicator.updateNode( HostAndPort.fromString("localhost"), LOOKUPS_STATE ); @@ -275,28 +270,23 @@ public void testUpdateNodeReturnsWeird() 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(Response.Status.ACCEPTED.getStatusCode()); - reasonString.set(""); - return responseHandler; - } - }; + 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.updateNode( + lookupsCommunicator.updateNode( HostAndPort.fromString("localhost"), LOOKUPS_STATE ); @@ -322,29 +312,24 @@ public void testUpdateNodeInterrupted() 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(Response.Status.ACCEPTED.getStatusCode()); - reasonString.set(""); - return responseHandler; - } - }; + 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.updateNode( + lookupsCommunicator.updateNode( HostAndPort.fromString("localhost"), LOOKUPS_STATE ); @@ -367,12 +352,15 @@ public void testGetLookupsStateNodeWithSuccess() throws Exception final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream( - StringUtils.toUtf8( - mapper.writeValueAsString( - LOOKUPS_STATE - ) - ))); + future.set( + new ByteArrayInputStream( + StringUtils.toUtf8( + mapper.writeValueAsString( + LOOKUPS_STATE + ) + ) + ) + ); EasyMock.expect(client.go( EasyMock.anyObject(), EasyMock.anyObject(), @@ -381,27 +369,22 @@ public void testGetLookupsStateNodeWithSuccess() 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(Response.Status.OK.getStatusCode()); - reasonString.set(""); - return responseHandler; - } - }; + 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; + } + }; - LookupsStateWithMap resp = manager.getLookupStateForNode( + LookupsStateWithMap resp = lookupsCommunicator.getLookupStateForNode( HostAndPort.fromString("localhost") ); @@ -425,28 +408,23 @@ public void testGetLookupsStateNodeRespondedWithNotOkErrorCode() throws Exceptio 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(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode()); - reasonString.set(""); - return 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; + } + }; try { - manager.getLookupStateForNode( + lookupsCommunicator.getLookupStateForNode( HostAndPort.fromString("localhost") ); Assert.fail(); @@ -472,28 +450,23 @@ public void testGetLookupsStateNodeReturnsWeird() 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(Response.Status.ACCEPTED.getStatusCode()); - reasonString.set(""); - return responseHandler; - } - }; + 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.getLookupStateForNode( + lookupsCommunicator.getLookupStateForNode( HostAndPort.fromString("localhost") ); Assert.fail(); @@ -518,29 +491,24 @@ public void testGetLookupsStateNodeInterrupted() 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(Response.Status.ACCEPTED.getStatusCode()); - reasonString.set(""); - return responseHandler; - } - }; + 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.getLookupStateForNode( + lookupsCommunicator.getLookupStateForNode( HostAndPort.fromString("localhost") ); Assert.fail(); @@ -1048,6 +1016,137 @@ public Map> getKnownLook Assert.assertNull(manager.getLookup(LOOKUP_TIER, "foo")); } + + @Test(timeout = 2_000) + public void testLookupManagementLoop() throws Exception + { + 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( + 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 LookupsStateWithMap( + ImmutableMap.of("lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0"))), null, null + ) + ).once(); + + LookupsStateWithMap host1UpdatedState = new LookupsStateWithMap( + lookup1, null, null + ); + + EasyMock.expect( + lookupsCommunicator.updateNode( + host1, + new LookupsStateWithMap( + null, + lookup1, + ImmutableSet.of("lookup0") + ) + ) + ).andReturn( + host1UpdatedState + ).once(); + + + EasyMock.expect( + lookupsCommunicator.getLookupStateForNode( + host2 + ) + ).andReturn( + new LookupsStateWithMap( + ImmutableMap.of("lookup3", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), + "lookup1", new LookupExtractorFactoryMapContainer("v0", ImmutableMap.of("k0", "v0"))), + null, null + ) + ).once(); + + LookupsStateWithMap host2UpdatedState = new LookupsStateWithMap( + null, lookup1, null + ); + + EasyMock.expect( + lookupsCommunicator.updateNode( + host2, + new LookupsStateWithMap( + 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 testStartStop() throws Exception { @@ -1175,20 +1274,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, @@ -1205,7 +1306,7 @@ public void describeTo(Description description) } } - //tests that lookups stored in db from 0.9.x are converted and restored. + //tests that lookups stored in db from 0.10.0 are converted and restored. @Test public void testBackwardCompatibilityMigration() throws Exception { @@ -1239,7 +1340,13 @@ public void testBackwardCompatibilityMigration() throws Exception EasyMock.eq( ImmutableMap.>of( "tier1", - ImmutableMap.of("lookup1", new LookupExtractorFactoryMapContainer(null, ImmutableMap.of("k", "v"))) + ImmutableMap.of( + "lookup1", + new LookupExtractorFactoryMapContainer( + null, + ImmutableMap.of("k", "v") + ) + ) ) ), EasyMock.anyObject(AuditInfo.class) From 50bb9304387f03ba9b260d48d7088e875a93889c Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 10 Apr 2017 09:18:57 -0500 Subject: [PATCH 30/35] add further tests for figuring out toBeLoaded and toBeDropped on LookupCoordinatorManager --- .../cache/LookupCoordinatorManager.java | 7 +- .../cache/LookupCoordinatorManagerTest.java | 67 +++++++++++++++++++ 2 files changed, 72 insertions(+), 2 deletions(-) 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 42f0e061af01..a668872c7f10 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 @@ -567,7 +567,8 @@ private LookupsStateWithMap doLookupManagementOnNode( // Returns the Map that needs to be loaded by the node and it does not know about // those already. - private Map getToBeLoadedOnNode( + @VisibleForTesting + Map getToBeLoadedOnNode( LookupsStateWithMap currLookupsStateOnNode, Map nodeTierLookupsToBe ) @@ -586,6 +587,7 @@ private Map getToBeLoadedOnNode( } 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); @@ -596,7 +598,8 @@ private Map getToBeLoadedOnNode( // Returns Set that should be dropped from the node which has them already either in pending to load // state or loaded - private Set getToBeDroppedFromNode( + @VisibleForTesting + Set getToBeDroppedFromNode( LookupsStateWithMap currLookupsStateOnNode, Map nodeTierLookupsToBe ) 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 c1ce61decdd6..9e5297fdfd0e 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 @@ -1147,6 +1147,73 @@ public int getThreadPoolSize() EasyMock.verify(discoverer, configManager, lookupsCommunicator); } + @Test + public void testGetToBeLoadedOnNode() + { + LookupCoordinatorManager manager = new LookupCoordinatorManager( + client, + discoverer, + mapper, + configManager, + lookupCoordinatorManagerConfig + ); + + LookupsStateWithMap currNodeState = new LookupsStateWithMap( + 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) + ); + } + + @Test + public void testToBeDropped() + { + LookupCoordinatorManager manager = new LookupCoordinatorManager( + client, + discoverer, + mapper, + configManager, + lookupCoordinatorManagerConfig + ); + + LookupsStateWithMap currNodeState = new LookupsStateWithMap( + 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 { From 535b738bfffce8b1d6a7c0d707329a1f4b2ea510 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 10 Apr 2017 10:01:23 -0500 Subject: [PATCH 31/35] address leventov comments --- .../query/lookup/LookupReferencesManager.java | 1 + .../cache/LookupCoordinatorManager.java | 20 ++++++++++++++----- .../lookup/cache/LookupsStateWithMap.java | 3 ++- 3 files changed, 18 insertions(+), 6 deletions(-) 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 e0ff88ce8e25..692523094c46 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -261,6 +261,7 @@ public LookupExtractorFactoryContainer get(String lookupName) 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)); 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 a668872c7f10..58dd59a81d56 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 @@ -65,12 +65,14 @@ import java.io.InputStream; 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.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; @@ -473,7 +475,7 @@ void lookupManagementLoop() final Map currState = new ConcurrentHashMap<>(); try { - List> futures = new ArrayList<>(); + List> futures = new ArrayList<>(); for (Map.Entry> tierEntry : allLookupTiers.entrySet()) { LOG.debug("Starting lookup mgmt for tier [%s].", tierEntry.getKey()); @@ -492,7 +494,7 @@ void lookupManagementLoop() executorService.submit( () -> { try { - currState.put(node, this.doLookupManagementOnNode(node, tierLookups)); + return new AbstractMap.SimpleImmutableEntry<>(node, doLookupManagementOnNode(node, tierLookups)); } catch (Exception ex) { LOG.makeAlert( @@ -501,6 +503,7 @@ void lookupManagementLoop() node.getHostText(), node.getPort() ); + return null; } } ) @@ -508,10 +511,15 @@ void lookupManagementLoop() } } - final ListenableFuture allFuture = Futures.allAsList(futures); + final ListenableFuture> allFuture = Futures.allAsList(futures); try { - allFuture.get(lookupCoordinatorManagerConfig.getAllHostTimeout().getMillis(), TimeUnit.MILLISECONDS); - knownOldState.set(currState); + 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); @@ -567,6 +575,7 @@ private LookupsStateWithMap doLookupManagementOnNode( // 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( LookupsStateWithMap currLookupsStateOnNode, @@ -598,6 +607,7 @@ Map getToBeLoadedOnNode( // 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( LookupsStateWithMap currLookupsStateOnNode, diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java index 25f9af4b7fdf..c8a3f9b92599 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java @@ -29,7 +29,8 @@ /** * Same as LookupsState except that it uses LookupExtractorFactoryMapContainer instead of - * LookupExtractorFactoryContainer to refer to lookup specs. + * LookupExtractorFactoryContainer to refer to lookup specs so that lookup extensions are not required to + * be loaded at the Coordinator. */ public class LookupsStateWithMap { From d1f8b3874ef1be902561c25f15f3bc98aec2ad47 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 18 Apr 2017 12:31:23 -0500 Subject: [PATCH 32/35] remove LookupsStateWithMap and parameterize LookupsState --- .../query/lookup/LookupReferencesManager.java | 4 +- .../io/druid/query/lookup/LookupsState.java | 16 +- .../druid/query/lookup/LookupsStateTest.java | 14 +- .../io/druid/query/lookup/LookupModule.java | 11 +- .../cache/LookupCoordinatorManager.java | 41 +++-- .../lookup/cache/LookupsStateWithMap.java | 101 ----------- .../cache/LookupCoordinatorManagerTest.java | 64 ++++--- .../lookup/cache/LookupsStateWithMapTest.java | 165 ------------------ 8 files changed, 91 insertions(+), 325 deletions(-) delete mode 100644 server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java delete mode 100644 server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java 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 692523094c46..46158ab177e4 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -262,7 +262,7 @@ public LookupExtractorFactoryContainer get(String lookupName) } // Note that this should ensure that "toLoad" and "toDrop" are disjoint. - public LookupsState getAllLookupsState() + public LookupsState getAllLookupsState() { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); @@ -274,7 +274,7 @@ public LookupsState getAllLookupsState() updateToLoadAndDrop(lookupUpdateState.noticesBeingHandled, lookupsToLoad, lookupsToDrop); updateToLoadAndDrop(lookupUpdateState.pendingNotices, lookupsToLoad, lookupsToDrop); - return new LookupsState(lookupUpdateState.lookupMap, lookupsToLoad, lookupsToDrop); + return new LookupsState<>(lookupUpdateState.lookupMap, lookupsToLoad, lookupsToDrop); } private void updateToLoadAndDrop(List notices, Map lookupsToLoad, Set lookupsToDrop) diff --git a/processing/src/main/java/io/druid/query/lookup/LookupsState.java b/processing/src/main/java/io/druid/query/lookup/LookupsState.java index fd97df653cb6..1646cbb92274 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupsState.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupsState.java @@ -29,16 +29,16 @@ /** */ -public class LookupsState +public class LookupsState { - private final Map current; - private final Map toLoad; + private final Map current; + private final Map toLoad; private final Set toDrop; @JsonCreator public LookupsState( - @JsonProperty("current") Map current, - @JsonProperty("toLoad") Map toLoad, + @JsonProperty("current") Map current, + @JsonProperty("toLoad") Map toLoad, @JsonProperty("toDrop") Set toDrop ) { @@ -48,13 +48,13 @@ public LookupsState( } @JsonProperty - public Map getCurrent() + public Map getCurrent() { return current; } @JsonProperty - public Map getToLoad() + public Map getToLoad() { return toLoad; } @@ -84,7 +84,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LookupsState that = (LookupsState) o; + LookupsState that = (LookupsState) o; return Objects.equals(current, that.current) && Objects.equals(toLoad, that.toLoad) && Objects.equals(toDrop, that.toDrop); diff --git a/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java b/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java index be8f1686268a..545f4b55df2d 100644 --- a/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java +++ b/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java @@ -19,6 +19,7 @@ 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; @@ -53,18 +54,23 @@ public void testSerde() throws Exception + " \"toDrop\": [\"l3\"]\n" + "}"; + TypeReference> typeRef = + new TypeReference>() + { + }; + final ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerSubtypes(LookupExtractorFactoryContainerTest.TestLookupExtractorFactory.class); - LookupsState actual = mapper.readValue( + LookupsState actual = mapper.readValue( mapper.writeValueAsString( - mapper.readValue(jsonStr, LookupsState.class) + mapper.readValue(jsonStr, typeRef) ), - LookupsState.class + typeRef ); Assert.assertEquals( - new LookupsState( + new LookupsState<>( ImmutableMap.of( "l1", new LookupExtractorFactoryContainer( 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 ddd9155d1acd..267b4bf13935 100644 --- a/server/src/main/java/io/druid/query/lookup/LookupModule.java +++ b/server/src/main/java/io/druid/query/lookup/LookupModule.java @@ -105,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, @@ -124,9 +129,9 @@ public Response handleUpdates( InputStream inputStream, ObjectMapper mapper ) { - final LookupsState state; + final LookupsState state; try { - state = mapper.readValue(inputStream, LookupsState.class); + state = mapper.readValue(inputStream, LOOKUPS_STATE_TYPE_REFERENCE); } catch (final IOException ex) { LOG.debug(ex, "Bad request"); @@ -169,7 +174,7 @@ public Object get(String id) } @Override - public LookupsState getAll() + public LookupsState getAll() { return manager.getAllLookupsState(); } 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 58dd59a81d56..bae5929e9963 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 @@ -53,6 +53,7 @@ import io.druid.java.util.common.StreamUtils; 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.jboss.netty.handler.codec.http.HttpHeaders; @@ -98,6 +99,11 @@ public class LookupCoordinatorManager + 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 final ListenerDiscoverer listenerDiscoverer; @@ -109,7 +115,8 @@ public class LookupCoordinatorManager // 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<>(); + final AtomicReference>> knownOldState = + new AtomicReference<>(); // Updated by config watching service private AtomicReference>> lookupMapConfigRef; @@ -472,7 +479,7 @@ void lookupManagementLoop() LOG.debug("Starting lookup sync for on all nodes."); - final Map currState = new ConcurrentHashMap<>(); + final Map> currState = new ConcurrentHashMap<>(); try { List> futures = new ArrayList<>(); @@ -513,7 +520,7 @@ void lookupManagementLoop() final ListenableFuture> allFuture = Futures.allAsList(futures); try { - ImmutableMap.Builder stateBuilder = ImmutableMap.builder(); + ImmutableMap.Builder> stateBuilder = ImmutableMap.builder(); allFuture.get(lookupCoordinatorManagerConfig.getAllHostTimeout().getMillis(), TimeUnit.MILLISECONDS) .stream() .filter(Objects::nonNull) @@ -537,14 +544,16 @@ void lookupManagementLoop() LOG.debug("Finished lookup sync for on all nodes."); } - private LookupsStateWithMap doLookupManagementOnNode( + private LookupsState doLookupManagementOnNode( HostAndPort node, Map nodeTierLookupsToBe ) throws IOException, InterruptedException, ExecutionException { LOG.debug("Starting lookup sync for node [%s].", node); - LookupsStateWithMap currLookupsStateOnNode = lookupsCommunicator.getLookupStateForNode(node); + LookupsState currLookupsStateOnNode = lookupsCommunicator.getLookupStateForNode( + node + ); LOG.debug("Received lookups state from node [%s].", node); @@ -559,7 +568,7 @@ private LookupsStateWithMap doLookupManagementOnNode( 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 LookupsStateWithMap(null, toLoad, toDrop)); + currLookupsStateOnNode = lookupsCommunicator.updateNode(node, new LookupsState<>(null, toLoad, toDrop)); LOG.debug( "Sent lookup toAdd[%s] and toDrop[%s] updates to node [%s].", @@ -578,7 +587,7 @@ private LookupsStateWithMap doLookupManagementOnNode( // It is assumed that currLookupsStateOnNode "toLoad" and "toDrop" are disjoint. @VisibleForTesting Map getToBeLoadedOnNode( - LookupsStateWithMap currLookupsStateOnNode, + LookupsState currLookupsStateOnNode, Map nodeTierLookupsToBe ) { @@ -610,7 +619,7 @@ Map getToBeLoadedOnNode( // It is assumed that currLookupsStateOnNode "toLoad" and "toDrop" are disjoint. @VisibleForTesting Set getToBeDroppedFromNode( - LookupsStateWithMap currLookupsStateOnNode, + LookupsState currLookupsStateOnNode, Map nodeTierLookupsToBe ) { @@ -679,9 +688,9 @@ public LookupsCommunicator( this.smileMapper = smileMapper; } - public LookupsStateWithMap updateNode( + public LookupsState updateNode( HostAndPort node, - LookupsStateWithMap lookupsUpdate + LookupsState lookupsUpdate ) throws IOException, InterruptedException, ExecutionException { @@ -702,7 +711,10 @@ public LookupsStateWithMap updateNode( ).get()) { if (httpStatusIsSuccess(returnCode.get())) { try { - final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); + 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 @@ -733,7 +745,7 @@ public LookupsStateWithMap updateNode( } } - public LookupsStateWithMap getLookupStateForNode( + public LookupsState getLookupStateForNode( HostAndPort node ) throws IOException, InterruptedException, ExecutionException { @@ -751,7 +763,10 @@ public LookupsStateWithMap getLookupStateForNode( ).get()) { if (returnCode.get() == 200) { try { - final LookupsStateWithMap response = smileMapper.readValue(result, LookupsStateWithMap.class); + 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 diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java b/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java deleted file mode 100644 index c8a3f9b92599..000000000000 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupsStateWithMap.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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 java.util.Collections; -import java.util.Map; -import java.util.Objects; -import java.util.Set; - -/** - * Same as LookupsState except that it uses LookupExtractorFactoryMapContainer instead of - * LookupExtractorFactoryContainer to refer to lookup specs so that lookup extensions are not required to - * be loaded at the Coordinator. - */ -public class LookupsStateWithMap -{ - private Map current; - private Map toLoad; - private Set toDrop; - - @JsonCreator - public LookupsStateWithMap( - @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; - } - LookupsStateWithMap that = (LookupsStateWithMap) 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/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 9e5297fdfd0e..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 @@ -40,6 +40,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.query.lookup.LookupModule; +import io.druid.query.lookup.LookupsState; import io.druid.server.listener.announcer.ListenerDiscoverer; import org.easymock.EasyMock; import org.hamcrest.BaseMatcher; @@ -104,7 +105,7 @@ public class LookupCoordinatorManagerTest SINGLE_LOOKUP_MAP_V1 ); private static final Map> EMPTY_TIERED_LOOKUP = ImmutableMap.of(); - private static final LookupsStateWithMap LOOKUPS_STATE = new LookupsStateWithMap( + private static final LookupsState LOOKUPS_STATE = new LookupsState<>( SINGLE_LOOKUP_MAP_V0, SINGLE_LOOKUP_MAP_V1, Collections.EMPTY_SET @@ -203,7 +204,7 @@ HttpResponseHandler makeResponseHandler( } }; - LookupsStateWithMap resp = lookupsCommunicator.updateNode( + LookupsState resp = lookupsCommunicator.updateNode( HostAndPort.fromString("localhost"), LOOKUPS_STATE ); @@ -384,7 +385,7 @@ HttpResponseHandler makeResponseHandler( } }; - LookupsStateWithMap resp = lookupsCommunicator.getLookupStateForNode( + LookupsState resp = lookupsCommunicator.getLookupStateForNode( HostAndPort.fromString("localhost") ); @@ -483,11 +484,12 @@ public void testGetLookupsStateNodeInterrupted() throws Exception final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); final SettableFuture future = SettableFuture.create(); - EasyMock.expect(client.go( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - )).andReturn(future).once(); + EasyMock.expect( + client.go( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); EasyMock.replay(client, responseHandler); @@ -661,12 +663,14 @@ public Map> getKnownLook ) ).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( + "foo2", + newSpec + ) + ), auditInfo)); EasyMock.verify(configManager); } @@ -719,12 +723,14 @@ public Map> getKnownLook ) ).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( + "foo", + newSpec + ) + ), auditInfo)); EasyMock.verify(configManager); } @@ -1055,19 +1061,19 @@ public void testLookupManagementLoop() throws Exception host1 ) ).andReturn( - new LookupsStateWithMap( + new LookupsState<>( ImmutableMap.of("lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0"))), null, null ) ).once(); - LookupsStateWithMap host1UpdatedState = new LookupsStateWithMap( + LookupsState host1UpdatedState = new LookupsState<>( lookup1, null, null ); EasyMock.expect( lookupsCommunicator.updateNode( host1, - new LookupsStateWithMap( + new LookupsState<>( null, lookup1, ImmutableSet.of("lookup0") @@ -1083,21 +1089,21 @@ public void testLookupManagementLoop() throws Exception host2 ) ).andReturn( - new LookupsStateWithMap( + new LookupsState<>( ImmutableMap.of("lookup3", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), "lookup1", new LookupExtractorFactoryMapContainer("v0", ImmutableMap.of("k0", "v0"))), null, null ) ).once(); - LookupsStateWithMap host2UpdatedState = new LookupsStateWithMap( + LookupsState host2UpdatedState = new LookupsState<>( null, lookup1, null ); EasyMock.expect( lookupsCommunicator.updateNode( host2, - new LookupsStateWithMap( + new LookupsState<>( null, lookup1, ImmutableSet.of("lookup3") @@ -1133,7 +1139,7 @@ public int getThreadPoolSize() manager.start(); - Map expectedKnownState = ImmutableMap.of( + Map> expectedKnownState = ImmutableMap.of( host1, host1UpdatedState, host2, @@ -1158,7 +1164,7 @@ public void testGetToBeLoadedOnNode() lookupCoordinatorManagerConfig ); - LookupsStateWithMap currNodeState = new LookupsStateWithMap( + LookupsState currNodeState = new LookupsState<>( ImmutableMap.of("lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), "lookup1", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k1", "v1")) ), @@ -1194,7 +1200,7 @@ public void testToBeDropped() lookupCoordinatorManagerConfig ); - LookupsStateWithMap currNodeState = new LookupsStateWithMap( + LookupsState currNodeState = new LookupsState<>( ImmutableMap.of("lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), "lookup1", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k1", "v1")) ), diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java deleted file mode 100644 index 89600f454263..000000000000 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupsStateWithMapTest.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * 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 com.google.common.collect.ImmutableSet; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.lookup.LookupExtractorFactoryContainer; -import io.druid.query.lookup.LookupsState; -import io.druid.query.lookup.MapLookupExtractorFactory; -import org.junit.Assert; -import org.junit.Test; - -/** - */ -public class LookupsStateWithMapTest -{ - private final ObjectMapper mapper; - private final String jsonStr; - private final LookupsStateWithMap testLookupsState; - - public LookupsStateWithMapTest() - { - mapper = new DefaultObjectMapper(); - mapper.registerSubtypes(MapLookupExtractorFactory.class); - - jsonStr = "{\n" - + " \"current\": {\n" - + " \"l1\": {\n" - + " \"version\": \"v1\",\n" - + " \"lookupExtractorFactory\": {\n" - + " \"type\": \"map\",\n" - + " \"map\": {\"k\": \"v\"},\n" - + " \"isOneToOne\": true\n" - + " }\n" - + " }\n" - + " },\n" - + " \"toLoad\": {\n" - + " \"l2\": {\n" - + " \"version\": \"v1\",\n" - + " \"lookupExtractorFactory\": {\n" - + " \"type\": \"map\",\n" - + " \"map\": {\"k\": \"v\"},\n" - + " \"isOneToOne\": true\n" - + " }\n" - + " }\n" - + " },\n" - + " \"toDrop\": [\"l3\"]\n" - + "}"; - - testLookupsState = new LookupsStateWithMap( - ImmutableMap.of( - "l1", - new LookupExtractorFactoryMapContainer( - "v1", - ImmutableMap.of( - "type", "map", - "map", ImmutableMap.of("k", "v"), - "isOneToOne", true - ) - ) - ), - ImmutableMap.of( - "l2", - new LookupExtractorFactoryMapContainer( - "v1", - ImmutableMap.of( - "type", "map", - "map", ImmutableMap.of("k", "v"), - "isOneToOne", true - ) - ) - ), - ImmutableSet.of("l3") - ); - } - - @Test - public void testSerde() throws Exception - { - LookupsStateWithMap actual = mapper.readValue( - mapper.writeValueAsString( - mapper.readValue(jsonStr, LookupsStateWithMap.class) - ), - LookupsStateWithMap.class - ); - Assert.assertEquals(testLookupsState, actual); - } - - - - //test interchangeability with LookupsState - //read and write as LookupsState - //then read as LookupsStateWithMap - @Test - public void testInterchangeability1() throws Exception - { - LookupsStateWithMap actual = mapper.readValue( - mapper.writeValueAsString( - mapper.readValue(jsonStr, LookupsState.class) - ), - LookupsStateWithMap.class - ); - - Assert.assertEquals(testLookupsState, actual); - } - - //test interchangeability with LookupExtractorFactoryContainer - //read and write as LookupExtractorFactoryMapContainer - //then read as LookupExtractorFactoryContainer - @Test - public void testInterchangeability2() throws Exception - { - LookupsState actual = mapper.readValue( - mapper.writeValueAsString( - mapper.readValue(jsonStr, LookupsStateWithMap.class) - ), - LookupsState.class - ); - - Assert.assertEquals(new LookupsState( - ImmutableMap.of( - "l1", - new LookupExtractorFactoryContainer( - "v1", - new MapLookupExtractorFactory( - ImmutableMap.of("k", "v"), - true - ) - ) - ), - ImmutableMap.of( - "l2", - new LookupExtractorFactoryContainer( - "v1", - new MapLookupExtractorFactory( - ImmutableMap.of("k", "v"), - true - ) - ) - ), - ImmutableSet.of("l3") - ), - actual - ); - } -} From c768e2f10215fc73483509695d847e5e652e9d67 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 18 Apr 2017 12:41:22 -0500 Subject: [PATCH 33/35] address review comments --- .../druid/server/lookup/cache/LookupCoordinatorManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 bae5929e9963..8f627f118acf 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 @@ -64,6 +64,7 @@ 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; @@ -671,6 +672,7 @@ boolean waitForBackgroundTermination(long timeout) throws InterruptedException return backgroundManagerExitedLatch.await(timeout, TimeUnit.MILLISECONDS); } + @VisibleForTesting public static class LookupsCommunicator { private final HttpClient httpClient; @@ -761,7 +763,7 @@ public LookupsState getLookupStateForNode( makeResponseHandler(returnCode, reasonString), lookupCoordinatorManagerConfig.getHostTimeout() ).get()) { - if (returnCode.get() == 200) { + if (returnCode.get() == HttpURLConnection.HTTP_OK) { try { final LookupsState response = smileMapper.readValue( result, From 18c3e3f3c8253f29f864f6d7e8b675c09965357c Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 19 Apr 2017 11:36:05 -0500 Subject: [PATCH 34/35] address more review comments --- .../lookup/cache/LookupCoordinatorManager.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) 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 8f627f118acf..904598ceac07 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 @@ -76,7 +76,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; @@ -306,7 +305,7 @@ public LookupExtractorFactoryMapContainer getLookup(final String tier, final Str return tierLookups.get(lookupName); } - // start() and stop() are synchronized so that they never run in parallel in case of ZK acting funny or drui bug and + // 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() { @@ -318,6 +317,8 @@ public void 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, @@ -392,6 +393,9 @@ public void stop() 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(); } @@ -480,8 +484,6 @@ void lookupManagementLoop() LOG.debug("Starting lookup sync for on all nodes."); - final Map> currState = new ConcurrentHashMap<>(); - try { List> futures = new ArrayList<>(); for (Map.Entry> tierEntry : allLookupTiers.entrySet()) { @@ -504,6 +506,10 @@ void lookupManagementLoop() 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, From 3ff745321ec9f740f3251ed0806aaa82488b8eba Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 21 Apr 2017 12:13:12 -0500 Subject: [PATCH 35/35] misc fixes --- .../main/java/io/druid/server/coordinator/DruidCoordinator.java | 2 ++ .../io/druid/server/lookup/cache/LookupCoordinatorManager.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) 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 bba45974e6c4..b36f0061049c 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -605,6 +605,8 @@ public ScheduledExecutors.Signal call() } ); } + + lookupCoordinatorManager.start(); } catch (Exception e) { log.makeAlert(e, "Unable to become leader") 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 904598ceac07..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 @@ -516,7 +516,7 @@ void lookupManagementLoop() "Failed to finish lookup management on node [%s:%s]", node.getHostText(), node.getPort() - ); + ).emit(); return null; } }