From 14f4c8831f70de43fce3095ad10a98059db9c961 Mon Sep 17 00:00:00 2001 From: Keuntae Park Date: Thu, 24 Mar 2016 11:19:24 +0900 Subject: [PATCH 01/42] support LookupReferencesManager registration of namespaced lookup and eliminate static configurations for lookup from namespecd lookup extensions - druid-namespace-lookup and druid-kafka-extraction-namespace are modified - However, druid-namespace-lookup still has configuration about ON/OFF HEAP cache manager selection, which is not namespace wide configuration but node wide configuration as multiple namespace shares the same cache manager --- .../namespace/KafkaExtractionNamespace.java | 16 +- .../namespace/KafkaExtractionManager.java | 52 +++-- .../KafkaExtractionNamespaceFactory.java | 32 ++- .../KafkaExtractionNamespaceModule.java | 49 +--- .../KafkaExtractionNamespaceTest.java | 17 +- .../namespace/TestKafkaExtractionCluster.java | 79 +++---- .../NamespaceLookupExtractorFactory.java | 115 +++++++++ .../query/extraction/NamespacedExtractor.java | 5 +- .../namespace/ExtractionNamespace.java | 6 - .../ExtractionNamespaceFunctionFactory.java | 3 +- .../namespace/JDBCExtractionNamespace.java | 18 +- .../namespace/URIExtractionNamespace.java | 17 +- .../NamespaceLookupStaticConfig.java | 37 --- ...DBCExtractionNamespaceFunctionFactory.java | 17 +- .../namespace/NamespacedExtractionModule.java | 85 +++---- ...URIExtractionNamespaceFunctionFactory.java | 5 +- .../NamespaceExtractionCacheManager.java | 167 +++++++------ ...ffHeapNamespaceExtractionCacheManager.java | 5 +- ...OnHeapNamespaceExtractionCacheManager.java | 5 +- .../namespace/URIExtractionNamespaceTest.java | 3 +- .../NamespaceExtractionClusterTest.java | 219 ++++++++++++++++++ .../NamespacedExtractorModuleTest.java | 38 ++- ...xtractionNamespaceFunctionFactoryTest.java | 39 ++-- .../cache/JDBCExtractionNamespaceTest.java | 26 +-- ...ceExtractionCacheManagerExecutorsTest.java | 45 ++-- 25 files changed, 668 insertions(+), 432 deletions(-) create mode 100644 extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java delete mode 100644 extensions-core/namespace-lookup/src/main/java/io/druid/server/initialization/NamespaceLookupStaticConfig.java create mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java index 2733f4a558d4..7276333508f2 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java @@ -25,6 +25,7 @@ import com.google.common.base.Preconditions; import javax.validation.constraints.NotNull; +import java.util.Properties; /** * @@ -34,19 +35,19 @@ public class KafkaExtractionNamespace implements ExtractionNamespace { @JsonProperty private final String kafkaTopic; + @JsonProperty - private final String namespace; + private final Properties kafkaProperties; @JsonCreator public KafkaExtractionNamespace( @NotNull @JsonProperty(value = "kafkaTopic", required = true) final String kafkaTopic, - @NotNull @JsonProperty(value = "namespace", required = true) final String namespace + @NotNull @JsonProperty(value = "kafkaProperites", required = true) final Properties kafkaProperties ) { Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required"); - Preconditions.checkNotNull(namespace, "namespace required"); this.kafkaTopic = kafkaTopic; - this.namespace = namespace; + this.kafkaProperties = kafkaProperties; } public String getKafkaTopic() @@ -54,10 +55,9 @@ public String getKafkaTopic() return kafkaTopic; } - @Override - public String getNamespace() + public Properties getKafkaProperties() { - return namespace; + return kafkaProperties; } @Override @@ -69,6 +69,6 @@ public long getPollMs() @Override public String toString() { - return String.format("KafkaExtractionNamespace = { kafkaTopic = '%s', namespace = '%s'", kafkaTopic, namespace); + return String.format("KafkaExtractionNamespace = { kafkaTopic = '%s' }", kafkaTopic); } } diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java index caf93162e4dd..4bfd7683b0d0 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java @@ -49,10 +49,7 @@ import java.util.Map; import java.util.Properties; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executors; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Pattern; @@ -60,15 +57,14 @@ /** * */ -@ManageLifecycle public class KafkaExtractionManager { private static final Logger log = new Logger(KafkaExtractionManager.class); private final Properties kafkaProperties = new Properties(); - private final ConcurrentMap namespaceVersionMap; + private final ConcurrentMap namespaceVersionMap = new ConcurrentHashMap<>(); private final ConcurrentMap topicEvents = new ConcurrentHashMap<>(); - private final Collection> futures = new ConcurrentLinkedQueue<>(); + private final ConcurrentMap> futures = new ConcurrentHashMap<>(); private final ListeningExecutorService executorService = MoreExecutors.listeningDecorator( Executors.newCachedThreadPool( new ThreadFactoryBuilder() @@ -83,8 +79,7 @@ public class KafkaExtractionManager // Bindings in KafkaExtractionNamespaceModule @Inject public KafkaExtractionManager( - @Named("namespaceVersionMap") final ConcurrentMap namespaceVersionMap, - @Named("renameKafkaProperties") final Properties kafkaProperties + final Properties kafkaProperties ) { if (kafkaProperties.containsKey("group.id")) { @@ -105,8 +100,15 @@ public KafkaExtractionManager( } // Enable publish-subscribe this.kafkaProperties.setProperty("auto.offset.reset", "smallest"); + } - this.namespaceVersionMap = namespaceVersionMap; + boolean supports(Properties otherProperties) + { + if (otherProperties.containsKey("zookeeper.connect")) { + return this.kafkaProperties.getProperty("zookeeper.connect") + .equals(otherProperties.getProperty("zookeeper.connect")); + } + return false; } public long getBackgroundTaskCount() @@ -137,10 +139,9 @@ public long getNumEvents(String namespace) } } - public void addListener(final KafkaExtractionNamespace kafkaNamespace, final Map map) + public void addListener(final String id, final KafkaExtractionNamespace kafkaNamespace, final Map map) { final String topic = kafkaNamespace.getKafkaTopic(); - final String namespace = kafkaNamespace.getNamespace(); final ListenableFuture future = executorService.submit( new Runnable() { @@ -168,11 +169,11 @@ public void run() backgroundTaskCount.incrementAndGet(); final KafkaStream kafkaStream = streams.get(0); final ConsumerIterator it = kafkaStream.iterator(); - log.info("Listening to topic [%s] for namespace [%s]", topic, namespace); - AtomicLong eventCounter = topicEvents.get(namespace); + log.info("Listening to topic [%s] for namespace [%s]", topic, id); + AtomicLong eventCounter = topicEvents.get(id); if(eventCounter == null){ - topicEvents.putIfAbsent(namespace, new AtomicLong(0L)); - eventCounter = topicEvents.get(namespace); + topicEvents.putIfAbsent(id, new AtomicLong(0L)); + eventCounter = topicEvents.get(id); } while (it.hasNext()) { final MessageAndMetadata messageAndMetadata = it.next(); @@ -183,25 +184,28 @@ public void run() continue; } map.put(key, message); - namespaceVersionMap.put(namespace, Long.toString(eventCounter.incrementAndGet())); + namespaceVersionMap.put(id, Long.toString(eventCounter.incrementAndGet())); log.debug("Placed key[%s] val[%s]", key, message); } } } ); + futures.putIfAbsent(id, future); Futures.addCallback( future, new FutureCallback() { @Override public void onSuccess(Object result) { - topicEvents.remove(namespace); + topicEvents.remove(id); + futures.remove(id); } @Override public void onFailure(Throwable t) { - topicEvents.remove(namespace); + topicEvents.remove(id); + futures.remove(id); if (t instanceof java.util.concurrent.CancellationException) { log.warn("Cancelled rename task for topic [%s]", topic); } else { @@ -213,17 +217,21 @@ public void onFailure(Throwable t) ); } - @LifecycleStart + public void removeListener(final String id) + { + Future future = futures.remove(id); + future.cancel(true); + } + public void start() { // NO-OP // all consumers are started through KafkaExtractionNamespaceFactory.getCachePopulator } - @LifecycleStop public void stop() { executorService.shutdown(); - Futures.allAsList(futures).cancel(true); + Futures.allAsList(futures.values()).cancel(true); } } diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java index 182afc90874b..1ecf0a3d2b0f 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java @@ -19,18 +19,21 @@ package io.druid.server.namespace; +import com.fasterxml.jackson.annotation.JacksonInject; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; +import com.google.inject.name.Named; import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; import io.druid.query.extraction.namespace.KafkaExtractionNamespace; import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.concurrent.Callable; /** @@ -38,15 +41,15 @@ */ public class KafkaExtractionNamespaceFactory implements ExtractionNamespaceFunctionFactory { - private final KafkaExtractionManager kafkaExtractionManager; + private final List kafkaExtractionManagers; private static final String KAFKA_VERSION = "kafka versions are updated every time a new event comes in"; @Inject public KafkaExtractionNamespaceFactory( - final KafkaExtractionManager kafkaExtractionManager + @JacksonInject @Named("kafkaManagers") final List kafkaExtractionManagers ) { - this.kafkaExtractionManager = kafkaExtractionManager; + this.kafkaExtractionManagers = kafkaExtractionManagers; } @@ -92,6 +95,7 @@ public List apply(@Nullable final String value) // This only fires ONCE when the namespace is first added. The version is updated externally as events come in @Override public Callable getCachePopulator( + final String id, final KafkaExtractionNamespace extractionNamespace, final String unused, final Map cache @@ -102,9 +106,29 @@ public Callable getCachePopulator( @Override public String call() { - kafkaExtractionManager.addListener(extractionNamespace, cache); + KafkaExtractionManager manager; + synchronized (kafkaExtractionManagers) { + manager = findAppropriate(extractionNamespace.getKafkaProperties()); + if (manager == null) { + manager = new KafkaExtractionManager(extractionNamespace.getKafkaProperties()); + kafkaExtractionManagers.add(manager); + } + } + manager.addListener(id, extractionNamespace, cache); return KAFKA_VERSION; } }; } + + private KafkaExtractionManager findAppropriate(Properties kafkaProperties) + { + for (KafkaExtractionManager manager: kafkaExtractionManagers) + { + if (manager.supports(kafkaProperties)) + { + return manager; + } + } + return null; + } } diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java index 4a756baf591a..351c9590aadc 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Provides; import com.google.inject.name.Named; @@ -35,6 +36,7 @@ import io.druid.query.extraction.namespace.KafkaExtractionNamespace; import java.io.IOException; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Properties; @@ -45,63 +47,26 @@ public class KafkaExtractionNamespaceModule implements DruidModule { private static final String PROPERTIES_KEY = "druid.query.rename.kafka.properties"; + private List kafkaExtractionManagers = Lists.newLinkedList(); @Override public List getJacksonModules() { return ImmutableList.of( - new SimpleModule("kafka-lookups"){ - @Override - public void setupModule(SetupContext setupContext) - { - setupContext.registerSubtypes(KafkaExtractionNamespace.class); - super.setupModule(setupContext); - } - } + new SimpleModule("kafka-lookups").registerSubtypes(KafkaExtractionNamespace.class) ); } @Provides - @Named("renameKafkaProperties") - @LazySingleton - public Properties getProperties( - @Json ObjectMapper mapper, - Properties systemProperties - ) + @Named("kafkaManagers") + public List getManagers() { - String val = systemProperties.getProperty(PROPERTIES_KEY); - if (val == null) { - return new Properties(); - } - try { - final Properties properties = new Properties(); - properties.putAll( - mapper.>readValue( - val, new TypeReference>() - { - } - ) - ); - return properties; - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - @Provides - @LazySingleton - public KafkaExtractionNamespaceFactory factoryFactory( - KafkaExtractionManager kafkaManager - ) - { - return new KafkaExtractionNamespaceFactory(kafkaManager); + return kafkaExtractionManagers; } @Override public void configure(Binder binder) { - LifecycleModule.register(binder, KafkaExtractionManager.class); NamespacedExtractionModule .getNamespaceFactoryMapBinder(binder) .addBinding(KafkaExtractionNamespace.class) diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java index d95103c6871a..79fa58299d0c 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java @@ -36,7 +36,7 @@ public void testReflectiveSerde() throws IOException { ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerSubtypes(KafkaExtractionNamespace.class); - final String val = "{\"type\":\"kafka\",\"kafkaTopic\":\"testTopic\",\"namespace\":\"testNamespace\"}"; + final String val = "{\"type\":\"kafka\",\"kafkaTopic\":\"testTopic\",\"kafkaProperties\":null}"; final ExtractionNamespace fn = mapper.reader(ExtractionNamespace.class) .readValue( @@ -50,7 +50,7 @@ public void testMissingTopic() throws IOException { ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerSubtypes(KafkaExtractionNamespace.class); - final String val = "{\"type\":\"kafka\",\"namespace\":\"testNamespace\"}"; + final String val = "{\"type\":\"kafka\" }"; final ExtractionNamespace fn = mapper.reader(ExtractionNamespace.class) .readValue( @@ -59,17 +59,4 @@ public void testMissingTopic() throws IOException Assert.assertEquals(val, mapper.writeValueAsString(fn)); } - @Test(expected = com.fasterxml.jackson.databind.JsonMappingException.class) - public void testMissingNamespace() throws IOException - { - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.registerSubtypes(KafkaExtractionNamespace.class); - final String val = "{\"type\":\"kafka\",\"kafkaTopic\":\"testTopic\"}"; - final ExtractionNamespace fn = - mapper.reader(ExtractionNamespace.class) - .readValue( - val - ); - Assert.assertEquals(val, mapper.writeValueAsString(fn)); - } } diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java index 38b9df8c2b83..4b5fbd185fe8 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.io.Files; import com.google.inject.Binder; import com.google.inject.Inject; @@ -39,6 +40,8 @@ import io.druid.guice.GuiceInjectors; import io.druid.guice.annotations.Json; import io.druid.initialization.Initialization; +import io.druid.query.extraction.NamespacedExtractor; +import io.druid.query.lookup.LookupExtractorFactory; import io.druid.server.namespace.KafkaExtractionManager; import io.druid.server.namespace.KafkaExtractionNamespaceFactory; import io.druid.server.namespace.KafkaExtractionNamespaceModule; @@ -79,27 +82,28 @@ public class TestKafkaExtractionCluster private static final Lifecycle lifecycle = new Lifecycle(); private static final File tmpDir = Files.createTempDir(); private static final String topicName = "testTopic"; - private static final String namespace = "testNamespace"; + private static String namespace; private static final Properties kafkaProperties = new Properties(); private KafkaServer kafkaServer; private KafkaConfig kafkaConfig; private TestingServer zkTestServer; private ZkClient zkClient; - private KafkaExtractionManager renameManager; - private NamespaceExtractionCacheManager extractionCacheManager; private Injector injector; + private ObjectMapper mapper; + private LookupExtractorFactory factory; + private KafkaExtractionManager renameManager; public static class KafkaFactoryProvider implements Provider> { - private final KafkaExtractionManager kafkaExtractionManager; + private final List kafkaExtractionManager; @Inject public KafkaFactoryProvider( KafkaExtractionManager kafkaExtractionManager ) { - this.kafkaExtractionManager = kafkaExtractionManager; + this.kafkaExtractionManager = Lists.newArrayList(kafkaExtractionManager); } @Override @@ -241,37 +245,30 @@ public void configure(Binder binder) }, new NamespacedExtractionModule(), new KafkaExtractionNamespaceModule() - { - @Override - public Properties getProperties( - @Json ObjectMapper mapper, - Properties systemProperties - ) - { - final Properties consumerProperties = new Properties(kafkaProperties); - consumerProperties.put("zookeeper.connect", zkTestServer.getConnectString() + "/kafka"); - consumerProperties.put("zookeeper.session.timeout.ms", "10000"); - consumerProperties.put("zookeeper.sync.time.ms", "200"); - return consumerProperties; - } - } ) ); - renameManager = injector.getInstance(KafkaExtractionManager.class); + mapper = injector.getInstance(ObjectMapper.class); log.info("--------------------------- placed default item via producer ---------------------------"); - extractionCacheManager = injector.getInstance(NamespaceExtractionCacheManager.class); - extractionCacheManager.schedule( - new KafkaExtractionNamespace(topicName, namespace) - ); + final Properties consumerProperties = new Properties(kafkaProperties); + consumerProperties.put("zookeeper.connect", zkTestServer.getConnectString() + "/kafka"); + consumerProperties.put("zookeeper.session.timeout.ms", "10000"); + consumerProperties.put("zookeeper.sync.time.ms", "200"); + ExtractionNamespace extractionNamespace = new KafkaExtractionNamespace(topicName, consumerProperties); + String namespaceString = mapper.writeValueAsString(extractionNamespace); + String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString); + factory = mapper.readValue(json, LookupExtractorFactory.class); + factory.start(); + namespace = ((NamespacedExtractor)factory.get()).getNamespace(); + renameManager = injector.getInstance( + Key.get( + new TypeLiteral>() + { + }, + Names.named("kafkaManagers") + ) + ).get(0); - long start = System.currentTimeMillis(); - while (renameManager.getBackgroundTaskCount() < 1) { - Thread.sleep(100); // wait for map populator to start up - if (System.currentTimeMillis() > start + 60_000) { - throw new ISE("renameManager took too long to start"); - } - } log.info("--------------------------- started rename manager ---------------------------"); } @@ -280,9 +277,11 @@ public void tearDown() throws Exception { lifecycle.stop(); - if (null != renameManager) { + if (renameManager != null) + { renameManager.stop(); } + factory.close(); if (null != kafkaServer) { kafkaServer.shutdown(); @@ -356,10 +355,8 @@ public void testSimpleRename() throws InterruptedException ) ); - KafkaExtractionNamespace extractionNamespace = new KafkaExtractionNamespace(topicName, namespace); - - assertUpdated(null, extractionNamespace.getNamespace(), "foo", fnFn); - assertReverseUpdated(Collections.EMPTY_LIST, extractionNamespace.getNamespace(), "foo", reverseFn); + assertUpdated(null, namespace, "foo", fnFn); + assertReverseUpdated(Collections.EMPTY_LIST, namespace, "foo", reverseFn); long events = renameManager.getNumEvents(namespace); @@ -375,9 +372,9 @@ public void testSimpleRename() throws InterruptedException } log.info("------------------------- Checking foo bar -------------------------------"); - assertUpdated("bar", extractionNamespace.getNamespace(), "foo", fnFn); - assertReverseUpdated(Arrays.asList("foo"), extractionNamespace.getNamespace(), "bar", reverseFn); - assertUpdated(null, extractionNamespace.getNamespace(), "baz", fnFn); + assertUpdated("bar", namespace, "foo", fnFn); + assertReverseUpdated(Arrays.asList("foo"), namespace, "bar", reverseFn); + assertUpdated(null, namespace, "baz", fnFn); checkServer(); events = renameManager.getNumEvents(namespace); @@ -392,8 +389,8 @@ public void testSimpleRename() throws InterruptedException } log.info("------------------------- Checking baz bat -------------------------------"); - Assert.assertEquals("bat", fnFn.get(extractionNamespace.getNamespace()).apply("baz")); - Assert.assertEquals(Arrays.asList("baz"), reverseFn.get(extractionNamespace.getNamespace()).apply("bat")); + Assert.assertEquals("bat", fnFn.get(namespace).apply("baz")); + Assert.assertEquals(Arrays.asList("baz"), reverseFn.get(namespace).apply("bat")); } finally { producer.close(); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java new file mode 100644 index 000000000000..92ab124d16f4 --- /dev/null +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -0,0 +1,115 @@ +/* + * 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.extraction; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.inject.name.Named; +import com.metamx.common.logger.Logger; +import io.druid.query.extraction.namespace.ExtractionNamespace; +import io.druid.query.lookup.LookupExtractor; +import io.druid.query.lookup.LookupExtractorFactory; +import io.druid.server.namespace.NamespacedExtractionModule; +import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +@JsonTypeName("namespace") +public class NamespaceLookupExtractorFactory implements LookupExtractorFactory +{ + private static final Logger log = new Logger(NamespaceLookupExtractorFactory.class); + + private static AtomicInteger numExtractor = new AtomicInteger(0); + private static long SCHEDULE_TIMEOUT = 60_000; + + final ExtractionNamespace extractionNamespace; + final NamespaceExtractionCacheManager manager; + final Function> fnMaker; + final Function>> reverseFnMaker; + LookupExtractor lookupExtractor; + + private final String extractorID; + + @JsonCreator + public NamespaceLookupExtractorFactory( + @JsonProperty("extractionNamespace")ExtractionNamespace extractionNamespace, + @JacksonInject @Named(NamespacedExtractionModule.EXTRACTION_CACHE_MANAGER) + NamespaceExtractionCacheManager manager, + @JacksonInject @Named(NamespacedExtractionModule.DIM_EXTRACTION_NAMESPACE) + Function> fnMaker, + @JacksonInject @Named(NamespacedExtractionModule.DIM_REVERSE_EXTRACTION_NAMESPACE) + Function>> reverseFnMaker + ) + { + this.extractionNamespace = Preconditions.checkNotNull(extractionNamespace, + "extractionNamespace should be specified"); + this.manager = manager; + this.fnMaker = fnMaker; + this.reverseFnMaker = reverseFnMaker; + this.extractorID = getID(); + } + + @Override + public boolean start() + { + if (!manager.scheduleAndWait(extractorID, extractionNamespace, SCHEDULE_TIMEOUT)) + { + return false; + } + + log.debug("NamespaceLookupExtractorFactory[%s] started", extractorID); + this.lookupExtractor = new NamespacedExtractor(fnMaker, reverseFnMaker, extractorID); + + return true; + } + + @Override + public boolean close() + { + manager.checkedDelete(extractorID); + return true; + } + + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + if (other != null && other instanceof NamespaceLookupExtractorFactory) { + NamespaceLookupExtractorFactory that = (NamespaceLookupExtractorFactory) other; + return !extractionNamespace.equals(that.extractionNamespace); + } + return true; + } + + private String getID() + { + return String.format("%d - %s", numExtractor.getAndIncrement(), extractionNamespace); + } + + @Override + public LookupExtractor get() + { + return lookupExtractor; + } +} diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java index a8f8a441f8f4..b2e73225827f 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java @@ -28,6 +28,7 @@ import com.google.inject.name.Named; import com.metamx.common.StringUtils; import io.druid.query.lookup.LookupExtractor; +import io.druid.server.namespace.NamespacedExtractionModule; import javax.validation.constraints.NotNull; import java.nio.ByteBuffer; @@ -48,9 +49,9 @@ public class NamespacedExtractor extends LookupExtractor @JsonCreator public NamespacedExtractor( - @NotNull @JacksonInject @Named("dimExtractionNamespace") + @NotNull @JacksonInject @Named(NamespacedExtractionModule.DIM_EXTRACTION_NAMESPACE) final Function> namespaces, - @NotNull @JacksonInject @Named("dimReverseExtractionNamespace") + @NotNull @JacksonInject @Named(NamespacedExtractionModule.DIM_REVERSE_EXTRACTION_NAMESPACE) final Function>> reverseNamespaces, @NotNull @JsonProperty(value = "namespace", required = true) final String namespace diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespace.java index 83d72a485903..34d8bd296c8d 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespace.java @@ -34,11 +34,5 @@ */ public interface ExtractionNamespace { - /** - * This is expected to return the namespace name. As an additional requirement, the implementation MUST supply a - * "namespace" field in the json representing the object which is equal to the return of this function - * @return The name of the namespace - */ - String getNamespace(); long getPollMs(); } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceFunctionFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceFunctionFactory.java index a45ae0b0cc5c..05032bb8740d 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceFunctionFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceFunctionFactory.java @@ -60,6 +60,7 @@ public interface ExtractionNamespaceFunctionFactory getCachePopulator(T extractionNamespace, String lastVersion, Map swap); + Callable getCachePopulator(String id, T extractionNamespace, String lastVersion, Map swap); } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/JDBCExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/JDBCExtractionNamespace.java index b03bf987d49a..a594b28f67f2 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/JDBCExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/JDBCExtractionNamespace.java @@ -47,14 +47,10 @@ public class JDBCExtractionNamespace implements ExtractionNamespace @JsonProperty private final String tsColumn; @JsonProperty - private final String namespace; - @JsonProperty private final Period pollPeriod; @JsonCreator public JDBCExtractionNamespace( - @NotNull @JsonProperty(value = "namespace", required = true) - final String namespace, @NotNull @JsonProperty(value = "connectorConfig", required = true) final MetadataStorageConnectorConfig connectorConfig, @NotNull @JsonProperty(value = "table", required = true) @@ -75,16 +71,9 @@ public JDBCExtractionNamespace( this.keyColumn = Preconditions.checkNotNull(keyColumn, "keyColumn"); this.valueColumn = Preconditions.checkNotNull(valueColumn, "valueColumn"); this.tsColumn = tsColumn; - this.namespace = Preconditions.checkNotNull(namespace, "namespace"); this.pollPeriod = pollPeriod == null ? new Period(0L) : pollPeriod; } - @Override - public String getNamespace() - { - return namespace; - } - public MetadataStorageConnectorConfig getConnectorConfig() { return connectorConfig; @@ -120,8 +109,7 @@ public long getPollMs() public String toString() { return String.format( - "JDBCExtractionNamespace = { namespace = %s, connectorConfig = { %s }, table = %s, keyColumn = %s, valueColumn = %s, tsColumn = %s, pollPeriod = %s}", - namespace, + "JDBCExtractionNamespace = { connectorConfig = { %s }, table = %s, keyColumn = %s, valueColumn = %s, tsColumn = %s, pollPeriod = %s}", connectorConfig.toString(), table, keyColumn, @@ -158,9 +146,6 @@ public boolean equals(Object o) if (tsColumn != null ? !tsColumn.equals(that.tsColumn) : that.tsColumn != null) { return false; } - if (!namespace.equals(that.namespace)) { - return false; - } return pollPeriod.equals(that.pollPeriod); } @@ -173,7 +158,6 @@ public int hashCode() result = 31 * result + keyColumn.hashCode(); result = 31 * result + valueColumn.hashCode(); result = 31 * result + (tsColumn != null ? tsColumn.hashCode() : 0); - result = 31 * result + namespace.hashCode(); result = 31 * result + pollPeriod.hashCode(); return result; } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/URIExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/URIExtractionNamespace.java index b6d2bf58bcce..d7f855f741d4 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/URIExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/URIExtractionNamespace.java @@ -59,8 +59,6 @@ @JsonTypeName("uri") public class URIExtractionNamespace implements ExtractionNamespace { - @JsonProperty - private final String namespace; @JsonProperty private final URI uri; @JsonProperty @@ -72,8 +70,6 @@ public class URIExtractionNamespace implements ExtractionNamespace @JsonCreator public URIExtractionNamespace( - @NotNull @JsonProperty(value = "namespace", required = true) - String namespace, @NotNull @JsonProperty(value = "uri", required = true) URI uri, @JsonProperty(value = "namespaceParseSpec", required = true) @@ -92,7 +88,6 @@ public URIExtractionNamespace( throw new IAE(ex, "Could not parse `versionRegex` [%s]", versionRegex); } } - this.namespace = Preconditions.checkNotNull(namespace, "namespace"); this.uri = Preconditions.checkNotNull(uri, "uri"); this.namespaceParseSpec = Preconditions.checkNotNull(namespaceParseSpec, "namespaceParseSpec"); this.pollPeriod = pollPeriod == null ? Period.ZERO : pollPeriod; @@ -100,12 +95,6 @@ public URIExtractionNamespace( this.versionRegex = versionRegex; } - @Override - public String getNamespace() - { - return namespace; - } - public String getVersionRegex() { return versionRegex; @@ -131,8 +120,7 @@ public long getPollMs() public String toString() { return String.format( - "URIExtractionNamespace = { namespace = %s, uri = %s, namespaceParseSpec = %s, pollPeriod = %s, versionRegex = %s }", - namespace, + "URIExtractionNamespace = { uri = %s, namespaceParseSpec = %s, pollPeriod = %s, versionRegex = %s }", uri.toString(), namespaceParseSpec.toString(), pollPeriod.toString(), @@ -158,8 +146,7 @@ public boolean equals(Object o) @Override public int hashCode() { - int result = namespace.hashCode(); - result = 31 * result + uri.hashCode(); + int result = uri.hashCode(); result = 31 * result + namespaceParseSpec.hashCode(); result = 31 * result + pollPeriod.hashCode(); result = 31 * result + (versionRegex != null ? versionRegex.hashCode() : 0); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/initialization/NamespaceLookupStaticConfig.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/initialization/NamespaceLookupStaticConfig.java deleted file mode 100644 index a152a61f7bd3..000000000000 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/initialization/NamespaceLookupStaticConfig.java +++ /dev/null @@ -1,37 +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.initialization; - -import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.query.extraction.namespace.ExtractionNamespace; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -public class NamespaceLookupStaticConfig -{ - @JsonProperty ExtractionNamespace[] lookups; - - public List getNamespaces() - { - return lookups == null ? Collections.emptyList() : Arrays.asList(lookups); - } -} diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceFunctionFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceFunctionFactory.java index ae2b50dc18e9..f4918155d070 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceFunctionFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceFunctionFactory.java @@ -96,13 +96,14 @@ public List apply(@Nullable final String value) @Override public Callable getCachePopulator( + final String id, final JDBCExtractionNamespace namespace, final String lastVersion, final Map cache ) { final long lastCheck = lastVersion == null ? JodaUtils.MIN_INSTANT : Long.parseLong(lastVersion); - final Long lastDBUpdate = lastUpdates(namespace); + final Long lastDBUpdate = lastUpdates(id, namespace); if (lastDBUpdate != null && lastDBUpdate <= lastCheck) { return new Callable() { @@ -118,12 +119,12 @@ public String call() throws Exception @Override public String call() { - final DBI dbi = ensureDBI(namespace); + final DBI dbi = ensureDBI(id, namespace); final String table = namespace.getTable(); final String valueColumn = namespace.getValueColumn(); final String keyColumn = namespace.getKeyColumn(); - LOG.debug("Updating [%s]", namespace.getNamespace()); + LOG.debug("Updating [%s]", id); final List> pairs = dbi.withHandle( new HandleCallback>>() { @@ -161,15 +162,15 @@ public Pair map( for (Pair pair : pairs) { cache.put(pair.lhs, pair.rhs); } - LOG.info("Finished loading %d values for namespace[%s]", cache.size(), namespace.getNamespace()); + LOG.info("Finished loading %d values for namespace[%s]", cache.size(), id); return String.format("%d", System.currentTimeMillis()); } }; } - private DBI ensureDBI(JDBCExtractionNamespace namespace) + private DBI ensureDBI(String id, JDBCExtractionNamespace namespace) { - final String key = namespace.getNamespace(); + final String key = id; DBI dbi = null; if (dbiCache.containsKey(key)) { dbi = dbiCache.get(key); @@ -186,9 +187,9 @@ private DBI ensureDBI(JDBCExtractionNamespace namespace) return dbi; } - private Long lastUpdates(JDBCExtractionNamespace namespace) + private Long lastUpdates(String id, JDBCExtractionNamespace namespace) { - final DBI dbi = ensureDBI(namespace); + final DBI dbi = ensureDBI(id, namespace); final String table = namespace.getTable(); final String tsColumn = namespace.getTsColumn(); if (tsColumn == null) { diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java index 685a89a675c7..3685adcfa457 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java @@ -30,23 +30,18 @@ import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; +import com.google.inject.name.Names; import com.metamx.common.IAE; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; import io.druid.guice.Jerseys; -import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; -import io.druid.guice.LifecycleModule; -import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; import io.druid.initialization.DruidModule; +import io.druid.query.extraction.NamespaceLookupExtractorFactory; import io.druid.query.extraction.NamespacedExtractor; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; import io.druid.query.extraction.namespace.JDBCExtractionNamespace; import io.druid.query.extraction.namespace.URIExtractionNamespace; -import io.druid.server.initialization.NamespaceLookupStaticConfig; import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager; @@ -62,9 +57,16 @@ */ public class NamespacedExtractionModule implements DruidModule { - private static final Logger log = new Logger(NamespacedExtractionModule.class); private static final String TYPE_PREFIX = "druid.query.extraction.namespace.cache.type"; - private static final String STATIC_CONFIG_PREFIX = "druid.query.extraction.namespace"; + + public static final String NAMESPACE_VERSION_MAP = "namespaceVersionMap"; + public static final String NAMESPACE_EXTRACTION_FUNCTION_CACHE = "namespaceExtractionFunctionCache"; + public static final String NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE = "namespaceReverseExtractionFunctionCache"; + public static final String DIM_EXTRACTION_NAMESPACE = "dimExtractionNamespace"; + public static final String DIM_REVERSE_EXTRACTION_NAMESPACE = "dimReverseExtractionNamespace"; + + public static final String EXTRACTION_CACHE_MANAGER = "DruidExtractionCacheManager"; + private final ConcurrentMap> fnCache = new ConcurrentHashMap<>(); private final ConcurrentMap>> reverseFnCache= new ConcurrentHashMap<>(); @@ -73,14 +75,9 @@ public List getJacksonModules() { return ImmutableList.of( new SimpleModule("DruidNamespacedExtractionModule") - { - @Override - public void setupModule(SetupContext context) - { - context.registerSubtypes(NamespacedExtractor.class); - context.registerSubtypes(ExtractionNamespace.class); - } - } + .registerSubtypes(NamespacedExtractor.class) + .registerSubtypes(ExtractionNamespace.class) + .registerSubtypes(NamespaceLookupExtractorFactory.class) ); } @@ -99,41 +96,9 @@ public static MapBinder, ExtractionNamespac ); } - @ManageLifecycle - public static class NamespaceStaticConfiguration - { - private NamespaceLookupStaticConfig configuration; - private NamespaceExtractionCacheManager manager; - - @Inject - NamespaceStaticConfiguration( - final NamespaceLookupStaticConfig configuration, - final NamespaceExtractionCacheManager manager - ) - { - this.configuration = configuration; - this.manager = manager; - } - - @LifecycleStart - public void start() - { - log.info("Loading configuration as static configuration"); - manager.scheduleOrUpdate(configuration.getNamespaces()); - log.info("Loaded %s namespace-lookup configuration", configuration.getNamespaces().size()); - } - - @LifecycleStop - public void stop() - { - //NOOP - } - } - @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, STATIC_CONFIG_PREFIX, NamespaceLookupStaticConfig.class); PolyBind.createChoiceWithDefault( binder, TYPE_PREFIX, @@ -157,13 +122,19 @@ public void configure(Binder binder) .to(URIExtractionNamespaceFunctionFactory.class) .in(LazySingleton.class); - LifecycleModule.register(binder, NamespaceStaticConfiguration.class); Jerseys.addResource(binder, NamespacesCacheResource.class); } + @Provides + @Named(EXTRACTION_CACHE_MANAGER) + @LazySingleton + public NamespaceExtractionCacheManager getCacheManager(NamespaceExtractionCacheManager manager) + { + return manager; + } @Provides - @Named("namespaceVersionMap") + @Named(NAMESPACE_VERSION_MAP) @LazySingleton public ConcurrentMap getVersionMap() { @@ -171,24 +142,24 @@ public ConcurrentMap getVersionMap() } @Provides - @Named("namespaceExtractionFunctionCache") + @Named(NAMESPACE_EXTRACTION_FUNCTION_CACHE) public ConcurrentMap> getFnCache() { return fnCache; } @Provides - @Named("namespaceReverseExtractionFunctionCache") + @Named(NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) public ConcurrentMap>> getReverseFnCache() { return reverseFnCache; } @Provides - @Named("dimExtractionNamespace") + @Named(DIM_EXTRACTION_NAMESPACE) @LazySingleton public Function> getFunctionMaker( - @Named("namespaceExtractionFunctionCache") + @Named(NAMESPACE_EXTRACTION_FUNCTION_CACHE) final ConcurrentMap> fnCache ) { @@ -208,10 +179,10 @@ public Function apply(final String namespace) } @Provides - @Named("dimReverseExtractionNamespace") + @Named(DIM_REVERSE_EXTRACTION_NAMESPACE) @LazySingleton public Function>> getReverseFunctionMaker( - @Named("namespaceReverseExtractionFunctionCache") + @Named(NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) final ConcurrentMap>> reverseFn ) { diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactory.java index dba8b90fa8dc..b600c57cde71 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactory.java @@ -109,6 +109,7 @@ public boolean apply(@Nullable String key) @Override public Callable getCachePopulator( + final String id, final URIExtractionNamespace extractionNamespace, final String lastVersion, final Map cache @@ -171,7 +172,7 @@ public String call() throws Exception log.debug( "URI [%s] for namespace [%s] was las modified [%s] but was last cached [%s]. Skipping ", uri.toString(), - extractionNamespace.getNamespace(), + id, fmt.print(lastModified), fmt.print(lastCached) ); @@ -210,7 +211,7 @@ public InputStream openStream() throws IOException log.info( "Finished loading %d lines for namespace [%s]", lineCount, - extractionNamespace.getNamespace() + id ); return version; } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java index d07f384658fb..88180e3ab9b1 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java @@ -160,6 +160,7 @@ protected boolean waitForServiceToEnd(long time, TimeUnit unit) throws Interrupt protected Runnable getPostRunnable( + final String id, final T namespace, final ExtractionNamespaceFunctionFactory factory, final String cacheId @@ -170,8 +171,7 @@ protected Runnable getPostRunnable( @Override public void run() { - final String nsName = namespace.getNamespace(); - final NamespaceImplData namespaceDatum = implData.get(nsName); + final NamespaceImplData namespaceDatum = implData.get(id); if (namespaceDatum == null) { // was removed return; @@ -181,16 +181,16 @@ public void run() // skip because it was disabled return; } - swapAndClearCache(nsName, cacheId); - final Function fn = factory.buildFn(namespace, getCacheMap(nsName)); - final Function> reverseFn = factory.buildReverseFn(namespace, getCacheMap(nsName)); - final Function priorFn = fnCache.put(nsName, fn); - final Function> priorReverseFn = reverseFnCache.put(nsName, reverseFn); + swapAndClearCache(id, cacheId); + final Function fn = factory.buildFn(namespace, getCacheMap(id)); + final Function> reverseFn = factory.buildReverseFn(namespace, getCacheMap(id)); + final Function priorFn = fnCache.put(id, fn); + final Function> priorReverseFn = reverseFnCache.put(id, reverseFn); if (priorFn != null && priorFn != namespaceDatum.fn.get()) { - log.warn("Replaced prior function for namespace [%s]", nsName); + log.warn("Replaced prior function for namespace [%s]", id); } if (priorReverseFn != null && priorReverseFn != namespaceDatum.reverseFn.get()) { - log.warn("Replaced prior reverse function for namespace [%s]", nsName); + log.warn("Replaced prior reverse function for namespace [%s]", id); } namespaceDatum.fn.set(fn); namespaceDatum.reverseFn.set(reverseFn); @@ -199,64 +199,83 @@ public void run() }; } - public void scheduleOrUpdate( - final Collection namespaces + // return value means actually delete or not + public boolean checkedDelete( + String namespaceName ) { - Set differentNamespaces = Sets.difference( - implData.keySet(), Sets.newHashSet( - Iterables.transform( - namespaces, - new Function() - { - @Nullable - @Override - public String apply( - ExtractionNamespace input - ) - { - return input.getNamespace(); - } - } - ) - ) - ); - if (log.isDebugEnabled()) { - log.debug("Deleting %d namespaces: %s", differentNamespaces.size(), differentNamespaces); + final NamespaceImplData implDatum = implData.get(namespaceName); + if (implDatum == null) { + // Delete but we don't have it? + log.wtf("Asked to delete something I just lost [%s]", namespaceName); + return false; } - for (String namespaceName : differentNamespaces) { - final NamespaceImplData implDatum = implData.get(namespaceName); - if (implDatum == null) { - // Delete but we don't have it? - log.wtf("Asked to delete something I just lost [%s]", namespaceName); - continue; - } - delete(namespaceName); + return delete(namespaceName); + } + + // return value means actually schedule or not + public boolean scheduleOrUpdate( + final String id, + ExtractionNamespace namespace + ) + { + final NamespaceImplData implDatum = implData.get(id); + if (implDatum == null) { + // New, probably + schedule(id, namespace); + return true; + } + if (!implDatum.enabled.get()) { + // Race condition. Someone else disabled it first, go ahead and reschedule + schedule(id, namespace); + return true; } - for (final ExtractionNamespace namespace : namespaces) { - final NamespaceImplData implDatum = implData.get(namespace.getNamespace()); - if (implDatum == null) { - // New, probably - schedule(namespace); - continue; - } - if (!implDatum.enabled.get()) { - // Race condition. Someone else disabled it first, go ahead and reschedule - schedule(namespace); - continue; - } - // Live one. Check if it needs updated - if (implDatum.namespace.equals(namespace)) { - // skip if no update - continue; - } - if (log.isDebugEnabled()) { - log.debug("Namespace [%s] needs updated to [%s]", implDatum.namespace, namespace); + // Live one. Check if it needs updated + if (implDatum.namespace.equals(namespace)) { + // skip if no update + return false; + } + if (log.isDebugEnabled()) { + log.debug("Namespace [%s] needs updated to [%s]", implDatum.namespace, namespace); + } + removeNamespaceLocalMetadata(implDatum); + schedule(id, namespace); + return true; + } + + public boolean scheduleAndWait( + final String id, + ExtractionNamespace namespace, + long waitForFirstRun + ) + { + String oldVersion = getVersion(id); + + if (scheduleOrUpdate(id, namespace)) + { + // wait until the namespace registration is done + String newVersion = getVersion(id); + final long startLocking = System.currentTimeMillis(); + final long timeout = startLocking + waitForFirstRun; + + while(newVersion == null || newVersion.equals(oldVersion)) + { + if (System.currentTimeMillis() > timeout) { + log.error("NamespaceLookupExtractorFactory[%s] - timeout during start", id); + return false; + } + + try { + Thread.sleep(100); + } catch (InterruptedException e) { + log.error("NamespaceLookupExtractorFactory[%s] - interrupted during start", id); + return false; + } + newVersion = getVersion(id); } - removeNamespaceLocalMetadata(implDatum); - schedule(namespace); } + return true; } private void cancelFuture(final NamespaceImplData implDatum) @@ -315,7 +334,7 @@ private boolean removeNamespaceLocalMetadata(final NamespaceImplData implDatum) } // Optimistic scheduling of updates to a namespace. - public ListenableFuture schedule(final T namespace) + public ListenableFuture schedule(final String id, final T namespace) { final ExtractionNamespaceFunctionFactory factory = (ExtractionNamespaceFunctionFactory) namespaceFunctionFactoryMap.get(namespace.getClass()); @@ -323,20 +342,20 @@ public ListenableFuture schedule(final T name throw new ISE("Cannot find factory for namespace [%s]", namespace); } final String cacheId = UUID.randomUUID().toString(); - return schedule(namespace, factory, getPostRunnable(namespace, factory, cacheId), cacheId); + return schedule(id, namespace, factory, getPostRunnable(id, namespace, factory, cacheId), cacheId); } // For testing purposes this is protected protected ListenableFuture schedule( + final String id, final T namespace, final ExtractionNamespaceFunctionFactory factory, final Runnable postRunnable, final String cacheId ) { - final String namespaceName = namespace.getNamespace(); - log.debug("Trying to update namespace [%s]", namespaceName); - final NamespaceImplData implDatum = implData.get(namespaceName); + log.debug("Trying to update namespace [%s]", id); + final NamespaceImplData implDatum = implData.get(id); if (implDatum != null) { synchronized (implDatum.enabled) { if (implDatum.enabled.get()) { @@ -357,8 +376,8 @@ public void run() startLatch.await(); // wait for "election" to leadership or cancellation if (!Thread.currentThread().isInterrupted()) { final Map cache = getCacheMap(cacheId); - final String preVersion = lastVersion.get(namespaceName); - final Callable runnable = factory.getCachePopulator(namespace, preVersion, cache); + final String preVersion = lastVersion.get(id); + final Callable runnable = factory.getCachePopulator(id, namespace, preVersion, cache); tasksStarted.incrementAndGet(); final String newVersion = runnable.call(); @@ -366,16 +385,16 @@ public void run() throw new CancellationException(String.format("Version `%s` already exists", preVersion)); } if (newVersion != null) { - lastVersion.put(namespaceName, newVersion); + lastVersion.put(id, newVersion); } postRunnable.run(); - log.debug("Namespace [%s] successfully updated", namespaceName); + log.debug("Namespace [%s] successfully updated", id); } } catch (Throwable t) { delete(cacheId); if (t instanceof CancellationException) { - log.debug(t, "Namespace [%s] cancelled", namespaceName); + log.debug(t, "Namespace [%s] cancelled", id); } else { log.error(t, "Failed update namespace [%s]", namespace); } @@ -394,18 +413,18 @@ public void run() future = listeningScheduledExecutorService.schedule(command, 0, TimeUnit.MILLISECONDS); } - final NamespaceImplData me = new NamespaceImplData(future, namespace, namespaceName); - final NamespaceImplData other = implData.putIfAbsent(namespaceName, me); + final NamespaceImplData me = new NamespaceImplData(future, namespace, id); + final NamespaceImplData other = implData.putIfAbsent(id, me); if (other != null) { if (!future.isDone() && !future.cancel(true)) { - log.warn("Unable to cancel future for namespace[%s] on race loss", namespaceName); + log.warn("Unable to cancel future for namespace[%s] on race loss", id); } throw new IAE("Namespace [%s] already exists! Leaving prior running", namespace); } else { if (!me.enabled.compareAndSet(false, true)) { log.wtf("How did someone enable this before ME?"); } - log.debug("I own namespace [%s]", namespaceName); + log.debug("I own namespace [%s]", id); return future; } } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index a56ace9c9563..361c0b4b7ecd 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -30,6 +30,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.server.namespace.NamespacedExtractionModule; import org.mapdb.DB; import org.mapdb.DBMaker; @@ -56,9 +57,9 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC @Inject public OffHeapNamespaceExtractionCacheManager( Lifecycle lifecycle, - @Named("namespaceExtractionFunctionCache") + @Named(NamespacedExtractionModule.NAMESPACE_EXTRACTION_FUNCTION_CACHE) ConcurrentMap> fnCache, - @Named("namespaceReverseExtractionFunctionCache") + @Named(NamespacedExtractionModule.NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) ConcurrentMap>> reverseFnCache, ServiceEmitter emitter, final Map, ExtractionNamespaceFunctionFactory> namespaceFunctionFactoryMap diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java index 9af6cb508415..0b846f186370 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java @@ -27,6 +27,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.server.namespace.NamespacedExtractionModule; import java.util.List; import java.util.Map; @@ -45,9 +46,9 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa @Inject public OnHeapNamespaceExtractionCacheManager( final Lifecycle lifecycle, - @Named("namespaceExtractionFunctionCache") + @Named(NamespacedExtractionModule.NAMESPACE_EXTRACTION_FUNCTION_CACHE) final ConcurrentMap> fnCache, - @Named("namespaceReverseExtractionFunctionCache") + @Named(NamespacedExtractionModule.NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) final ConcurrentMap>> reverseFnCache, final ServiceEmitter emitter, final Map, ExtractionNamespaceFunctionFactory> namespaceFunctionFactoryMap diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java index 982926a1acc5..4e79608ad0ea 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java @@ -328,7 +328,7 @@ public void testExplicitJson() throws IOException { final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); URIExtractionNamespace namespace = mapper.readValue( - "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\", \"namespace\":\"testNamespace\"}", + "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\"}", URIExtractionNamespace.class ); @@ -337,7 +337,6 @@ public void testExplicitJson() throws IOException namespace.getNamespaceParseSpec().getClass().getCanonicalName() ); Assert.assertEquals("file:/foo", namespace.getUri().toString()); - Assert.assertEquals("testNamespace", namespace.getNamespace()); Assert.assertEquals("a.b.c", namespace.getVersionRegex()); Assert.assertEquals(5L * 60_000L, namespace.getPollMs()); } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java new file mode 100644 index 000000000000..873fd7b3104c --- /dev/null +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java @@ -0,0 +1,219 @@ +/* + * 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.namespace; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.guice.GuiceInjectors; +import io.druid.initialization.Initialization; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.dimension.LookupDimensionSpec; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.extraction.NamespacedExtractor; +import io.druid.query.extraction.namespace.ExtractionNamespace; +import io.druid.query.extraction.namespace.URIExtractionNamespace; +import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; +import io.druid.query.lookup.*; +import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.OutputStreamWriter; + +public class NamespaceExtractionClusterTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public final TemporaryFolder temporaryFolder2 = new TemporaryFolder(); + + private ObjectMapper mapper; + private LookupReferencesManager lookupReferencesManager; + private Injector injector; + + @Before + public void setUp() + { + System.setProperty("druid.extensions.searchCurrentClassloader", "false"); + + injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of() + ), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + } + }, + new LookupModule(), + new NamespacedExtractionModule() + ) + ); + mapper = injector.getInstance(ObjectMapper.class); + lookupReferencesManager = injector.getInstance(LookupReferencesManager.class); + lookupReferencesManager.start(); + } + + @Test(timeout = 60_000) + public void testSimpleJson() throws IOException, InterruptedException + { + final File tempFile = temporaryFolder.newFile(); + try (OutputStreamWriter out = new FileWriter(tempFile)) { + out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); + } + final URIExtractionNamespace namespace = new URIExtractionNamespace( + tempFile.toURI(), + new URIExtractionNamespace.ObjectMapperFlatDataParser( + URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) + ), + new Period(0), + null + ); + + String namespaceString = mapper.writeValueAsString(namespace); + String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString); + LookupExtractorFactory factory = mapper.readValue(json, LookupExtractorFactory.class); + + Assert.assertNotNull(factory); + + Assert.assertTrue(factory.start()); + + LookupExtractor extractor = factory.get(); + + Assert.assertTrue(extractor instanceof NamespacedExtractor); + + NamespacedExtractor namespacedExtractor = (NamespacedExtractor)extractor; + + Assert.assertEquals("bar", namespacedExtractor.apply("foo")); + + Assert.assertTrue(factory.close()); + } + + @Test(timeout = 60_000) + public void testTwoFactories() throws IOException, InterruptedException + { + final File tempFile1 = temporaryFolder.newFile(); + try (OutputStreamWriter out = new FileWriter(tempFile1)) { + out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); + } + final URIExtractionNamespace namespace1 = new URIExtractionNamespace( + tempFile1.toURI(), + new URIExtractionNamespace.ObjectMapperFlatDataParser( + URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) + ), + new Period(0), + null + ); + String namespaceString1 = mapper.writeValueAsString(namespace1); + String json1 = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString1); + LookupExtractorFactory factory1 = mapper.readValue(json1, LookupExtractorFactory.class); + + final File tempFile2 = temporaryFolder2.newFile(); + try (OutputStreamWriter out = new FileWriter(tempFile2)) { + out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bad"))); + } + final URIExtractionNamespace namespace2 = new URIExtractionNamespace( + tempFile2.toURI(), + new URIExtractionNamespace.ObjectMapperFlatDataParser( + URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) + ), + new Period(0), + null + ); + String namespaceString2 = mapper.writeValueAsString(namespace2); + String json2 = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString2); + LookupExtractorFactory factory2 = mapper.readValue(json2, LookupExtractorFactory.class); + + Assert.assertNotNull(factory1); + Assert.assertNotNull(factory2); + + Assert.assertTrue(factory1.start()); + Assert.assertTrue(factory2.start()); + + LookupExtractor extractor1 = factory1.get(); + Assert.assertTrue(extractor1 instanceof NamespacedExtractor); + + NamespacedExtractor namespacedExtractor1 = (NamespacedExtractor)extractor1; + Assert.assertEquals("bar", namespacedExtractor1.apply("foo")); + + LookupExtractor extractor2 = factory2.get(); + Assert.assertTrue(extractor2 instanceof NamespacedExtractor); + + NamespacedExtractor namespacedExtractor2 = (NamespacedExtractor)extractor2; + Assert.assertEquals("bad", namespacedExtractor2.apply("foo")); + + Assert.assertTrue(factory1.close()); + Assert.assertTrue(factory2.close()); + } + + @Test(timeout = 60_000) + public void testReferenceManagerIntegration() throws IOException, InterruptedException + { + final File tempFile = temporaryFolder.newFile(); + try (OutputStreamWriter out = new FileWriter(tempFile)) { + out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); + } + final URIExtractionNamespace namespace = new URIExtractionNamespace( + tempFile.toURI(), + new URIExtractionNamespace.ObjectMapperFlatDataParser( + URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) + ), + new Period(0), + null + ); + + String namespaceString = mapper.writeValueAsString(namespace); + String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString); + LookupExtractorFactory factory = mapper.readValue(json, LookupExtractorFactory.class); + + lookupReferencesManager.updateIfNew("refTest", factory); + + LookupDimensionSpec lookupDimensionSpec = new LookupDimensionSpec( + "col", + "out", + null, + false, + null, + "refTest", + lookupReferencesManager, + false + ); + + ExtractionFn extractionFn = lookupDimensionSpec.getExtractionFn(); + Assert.assertEquals("bar", extractionFn.apply("foo")); + } +} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java index f38ffdf42583..c0917d08a9d3 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java @@ -107,8 +107,8 @@ public void testNewTask() throws Exception final URIExtractionNamespaceFunctionFactory factory = new URIExtractionNamespaceFunctionFactory( ImmutableMap.of("file", new LocalFileTimestampVersionFinder()) ); + final String namespaceID = "ns"; final URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) @@ -117,31 +117,29 @@ public void testNewTask() throws Exception null ); Map map = new HashMap<>(); - factory.getCachePopulator(namespace, null, map).call(); + factory.getCachePopulator(namespaceID, namespace, null, map).call(); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); + cacheManager.delete(namespaceID); } - @Test(timeout = 1_000) + @Test public void testListNamespaces() throws Exception { final File tmpFile = temporaryFolder.newFile(); try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } + final String namespaceID = "ns"; final URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser(URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper())), new Period(0), null ); - cacheManager.scheduleOrUpdate(ImmutableList.of(namespace)); - Collection strings = cacheManager.getKnownNamespaces(); - Assert.assertArrayEquals(new String[]{"ns"}, strings.toArray(new String[strings.size()])); - while (!Arrays.equals(cacheManager.getKnownNamespaces().toArray(), new Object[]{"ns"})) { - Thread.sleep(1); - } + Assert.assertTrue(cacheManager.scheduleAndWait(namespaceID, namespace, 1_000)); + Assert.assertArrayEquals(cacheManager.getKnownNamespaces().toArray(), new Object[]{namespaceID}); + Assert.assertTrue(cacheManager.delete(namespaceID)); } private static boolean noNamespaces(NamespaceExtractionCacheManager manager) @@ -156,8 +154,8 @@ public void testDeleteNamespaces() throws Exception try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } + final String namespaceID = "ns"; final URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) @@ -165,21 +163,19 @@ public void testDeleteNamespaces() throws Exception new Period(0), null ); - cacheManager.delete("ns"); - while (!noNamespaces(cacheManager)) { - Thread.sleep(1); - } + Assert.assertTrue(cacheManager.scheduleAndWait(namespaceID, namespace, 1_000)); + Assert.assertTrue(cacheManager.delete(namespaceID)); } - @Test(timeout = 10_000) + @Test public void testNewUpdate() throws Exception { final File tmpFile = temporaryFolder.newFile(); try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } + final String namespaceID = "ns"; final URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) @@ -188,9 +184,9 @@ public void testNewUpdate() throws Exception null ); Assert.assertTrue(noNamespaces(cacheManager)); - cacheManager.scheduleOrUpdate(ImmutableList.of(namespace)); - while (!Arrays.equals(cacheManager.getKnownNamespaces().toArray(), new Object[]{"ns"})) { - Thread.sleep(1); - } + Assert.assertTrue(cacheManager.scheduleAndWait(namespaceID, namespace, 10_000)); + Assert.assertArrayEquals(cacheManager.getKnownNamespaces().toArray(), new Object[]{namespaceID}); + + Assert.assertTrue(cacheManager.delete(namespaceID)); } } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactoryTest.java index e6a6009231c3..72f3641bf22c 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactoryTest.java @@ -257,6 +257,7 @@ public URIExtractionNamespaceFunctionFactoryTest( private File tmpFileParent; private URIExtractionNamespaceFunctionFactory factory; private URIExtractionNamespace namespace; + private String id; private ConcurrentHashMap> fnCache; private ConcurrentHashMap>> reverseFnCache; @@ -286,7 +287,6 @@ public void setUp() throws Exception ImmutableMap.of("file", new LocalFileTimestampVersionFinder()) ); namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -294,6 +294,7 @@ public void setUp() throws Exception new Period(0), Pattern.quote(tmpFile.getName()) ); + id = "ns"; } @After @@ -305,9 +306,9 @@ public void tearDown() @Test public void simpleTest() throws IOException, ExecutionException, InterruptedException { - Assert.assertNull(fnCache.get(namespace.getNamespace())); - NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace)); - Function fn = fnCache.get(namespace.getNamespace()); + Assert.assertNull(fnCache.get(id)); + NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace)); + Function fn = fnCache.get(id); Assert.assertNotNull(fn); Assert.assertEquals("bar", fn.apply("foo")); Assert.assertEquals(null, fn.apply("baz")); @@ -316,9 +317,9 @@ public void simpleTest() throws IOException, ExecutionException, InterruptedExce @Test public void testReverseFunction() throws InterruptedException { - Assert.assertNull(reverseFnCache.get(namespace.getNamespace())); - NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace)); - Function> reverseFn = reverseFnCache.get(namespace.getNamespace()); + Assert.assertNull(reverseFnCache.get(id)); + NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace)); + Function> reverseFn = reverseFnCache.get(id); Assert.assertNotNull(reverseFn); Assert.assertEquals(Sets.newHashSet("boo", "foo"), Sets.newHashSet(reverseFn.apply("bar"))); Assert.assertEquals(Sets.newHashSet(""), Sets.newHashSet(reverseFn.apply("MissingValue"))); @@ -332,9 +333,11 @@ public void simplePileONamespacesTest() throws InterruptedException { final int size = 128; List namespaces = new ArrayList<>(size); + List ids = new ArrayList<>(size); for (int i = 0; i < size; ++i) { + String id = String.format("%d-ns-%d", i << 10, i); + ids.add(id); URIExtractionNamespace namespace = new URIExtractionNamespace( - String.format("%d-ns-%d", i << 10, i), tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -345,35 +348,36 @@ public void simplePileONamespacesTest() throws InterruptedException namespaces.add(namespace); - Assert.assertNull(fnCache.get(namespace.getNamespace())); - NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace)); + Assert.assertNull(fnCache.get(id)); + NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace)); } for (int i = 0; i < size; ++i) { URIExtractionNamespace namespace = namespaces.get(i); - Function fn = fnCache.get(namespace.getNamespace()); + String id = ids.get(i); + Function fn = fnCache.get(id); Assert.assertNotNull(fn); Assert.assertEquals("bar", fn.apply("foo")); Assert.assertEquals(null, fn.apply("baz")); - manager.delete(namespace.getNamespace()); - Assert.assertNull(fnCache.get(namespace.getNamespace())); + manager.delete(id); + Assert.assertNull(fnCache.get(id)); } } @Test public void testLoadOnlyOnce() throws Exception { - Assert.assertNull(fnCache.get(namespace.getNamespace())); + Assert.assertNull(fnCache.get(id)); ConcurrentMap map = new ConcurrentHashMap<>(); - Callable populator = factory.getCachePopulator(namespace, null, map); + Callable populator = factory.getCachePopulator(id, namespace, null, map); String v = populator.call(); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); Assert.assertNotNull(v); - populator = factory.getCachePopulator(namespace, v, map); + populator = factory.getCachePopulator(id, namespace, v, map); String v2 = populator.call(); Assert.assertEquals(v, v2); Assert.assertEquals("bar", map.get("foo")); @@ -384,7 +388,6 @@ public void testLoadOnlyOnce() throws Exception public void testMissing() throws Exception { URIExtractionNamespace badNamespace = new URIExtractionNamespace( - namespace.getNamespace(), namespace.getUri(), namespace.getNamespaceParseSpec(), Period.millis((int) namespace.getPollMs()), @@ -393,7 +396,7 @@ public void testMissing() throws Exception ); ConcurrentMap map = new ConcurrentHashMap<>(); try { - factory.getCachePopulator(badNamespace, null, map).call(); + factory.getCachePopulator(id, badNamespace, null, map).call(); } catch (RuntimeException e) { Assert.assertNotNull(e.getCause()); diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java index cdab411e10f4..543f3513c5b7 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java @@ -203,12 +203,13 @@ public void close() throws IOException { @Override public Callable getCachePopulator( + final String id, final JDBCExtractionNamespace namespace, final String lastVersion, final Map cache ) { - final Callable cachePopulator = super.getCachePopulator(namespace, lastVersion, cache); + final Callable cachePopulator = super.getCachePopulator(id, namespace, lastVersion, cache); return new Callable() { @Override @@ -371,7 +372,6 @@ public void testMapping() InterruptedException, TimeoutException { final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - namespace, derbyConnectorRule.getMetadataConnectorConfig(), tableName, keyName, @@ -379,8 +379,8 @@ public void testMapping() tsColumn, new Period(0) ); - NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(extractionNamespace)); - Function extractionFn = fnCache.get(extractionNamespace.getNamespace()); + NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); + Function extractionFn = fnCache.get(namespace); for (Map.Entry entry : renames.entrySet()) { String key = entry.getKey(); @@ -394,7 +394,6 @@ public void testMapping() public void testReverseLookup() throws InterruptedException { final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - namespace, derbyConnectorRule.getMetadataConnectorConfig(), tableName, keyName, @@ -402,8 +401,8 @@ public void testReverseLookup() throws InterruptedException tsColumn, new Period(0) ); - NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(extractionNamespace)); - Function> reverseExtractionFn = reverseFnCache.get(extractionNamespace.getNamespace()); + NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); + Function> reverseExtractionFn = reverseFnCache.get(namespace); Assert.assertEquals( "reverse lookup should match", Sets.newHashSet("foo", "bad"), @@ -437,13 +436,13 @@ public void testSkipOld() { final JDBCExtractionNamespace extractionNamespace = ensureNamespace(); - assertUpdated(extractionNamespace.getNamespace(), "foo", "bar"); + assertUpdated(namespace, "foo", "bar"); if (tsColumn != null) { insertValues(handleRef, "foo", "baz", "1900-01-01 00:00:00"); } - assertUpdated(extractionNamespace.getNamespace(), "foo", "bar"); + assertUpdated(namespace, "foo", "bar"); } @Test(timeout = 60_000L) @@ -452,18 +451,17 @@ public void testFindNew() { final JDBCExtractionNamespace extractionNamespace = ensureNamespace(); - assertUpdated(extractionNamespace.getNamespace(), "foo", "bar"); + assertUpdated(namespace, "foo", "bar"); insertValues(handleRef, "foo", "baz", "2900-01-01 00:00:00"); - assertUpdated(extractionNamespace.getNamespace(), "foo", "baz"); + assertUpdated(namespace, "foo", "baz"); } private JDBCExtractionNamespace ensureNamespace() throws NoSuchFieldException, IllegalAccessException, InterruptedException { final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - namespace, derbyConnectorRule.getMetadataConnectorConfig(), tableName, keyName, @@ -471,14 +469,14 @@ private JDBCExtractionNamespace ensureNamespace() tsColumn, new Period(10) ); - extractionCacheManager.schedule(extractionNamespace); + extractionCacheManager.schedule(namespace, extractionNamespace); waitForUpdates(1_000L, 2L); Assert.assertEquals( "sanity check not correct", "bar", - fnCache.get(extractionNamespace.getNamespace()).apply("foo") + fnCache.get(namespace).apply("foo") ); return extractionNamespace; } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java index bde0f303b3e3..be08039a500c 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java @@ -100,6 +100,7 @@ public void setUp() throws Exception { @Override public Callable getCachePopulator( + final String id, final URIExtractionNamespace extractionNamespace, final String lastVersion, final Map cache @@ -128,14 +129,15 @@ lifecycle, fnCache, reverseFnCache, new NoopServiceEmitter(), { @Override protected Runnable getPostRunnable( + final String id, final T namespace, final ExtractionNamespaceFunctionFactory factory, final String cacheId ) { - final Runnable runnable = super.getPostRunnable(namespace, factory, cacheId); - cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object()); - final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace()); + final Runnable runnable = super.getPostRunnable(id, namespace, factory, cacheId); + cacheUpdateAlerts.putIfAbsent(id, new Object()); + final Object cacheUpdateAlerter = cacheUpdateAlerts.get(id); return new Runnable() { @Override @@ -174,8 +176,8 @@ public void tearDown() @Test(expected = IAE.class) public void testDoubleSubmission() { + final String namespaceID = "ns"; URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -183,11 +185,11 @@ public void testDoubleSubmission() new Period(0), null ); - final ListenableFuture future = manager.schedule(namespace); + final ListenableFuture future = manager.schedule(namespaceID, namespace); Assert.assertFalse(future.isDone()); Assert.assertFalse(future.isCancelled()); try { - manager.schedule(namespace).cancel(true); + manager.schedule(namespaceID, namespace).cancel(true); } finally { future.cancel(true); @@ -198,8 +200,8 @@ public void testDoubleSubmission() @Test(timeout = 60_000) public void testSimpleSubmission() throws ExecutionException, InterruptedException { + final String namespaceID = "ns"; URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -207,7 +209,7 @@ public void testSimpleSubmission() throws ExecutionException, InterruptedExcepti new Period(0), null ); - NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace)); + NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespaceID, namespace)); } @Test(timeout = 60_000) @@ -217,9 +219,9 @@ public void testRepeatSubmission() throws ExecutionException, InterruptedExcepti final long delay = 5; final long totalRunCount; final long start; + final String namespaceID = "ns"; try { final URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -227,15 +229,15 @@ public void testRepeatSubmission() throws ExecutionException, InterruptedExcepti new Period(delay), null ); - cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object()); + cacheUpdateAlerts.putIfAbsent(namespaceID, new Object()); start = System.currentTimeMillis(); - ListenableFuture future = manager.schedule(namespace); + ListenableFuture future = manager.schedule(namespaceID, namespace); Assert.assertFalse(future.isDone()); Assert.assertFalse(future.isCancelled()); final long preRunCount; - final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace()); + final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespaceID); synchronized (cacheUpdateAlerter) { preRunCount = numRuns.get(); } @@ -351,7 +353,6 @@ public void testDelete(final String ns) final long period = 1_000L;// Give it some time between attempts to update final URIExtractionNamespace namespace = new URIExtractionNamespace( - ns, tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -359,7 +360,7 @@ public void testDelete(final String ns) new Period(period), null ); - final ListenableFuture future = manager.schedule(namespace); + final ListenableFuture future = manager.schedule(ns, namespace); Assert.assertFalse(future.isCancelled()); Assert.assertFalse(future.isDone()); @@ -426,10 +427,10 @@ public void testShutdown() final long period = 5L; final ListenableFuture future; long prior = 0; + final String namespaceID = "ns"; try { final URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -437,11 +438,11 @@ public void testShutdown() new Period(period), null ); - cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object()); + cacheUpdateAlerts.putIfAbsent(namespaceID, new Object()); - future = manager.schedule(namespace); + future = manager.schedule(namespaceID, namespace); - final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace()); + final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespaceID); synchronized (cacheUpdateAlerter) { cacheUpdateAlerter.wait(); } @@ -475,9 +476,9 @@ public void testRunCount() { final long numWaits = 5; final ListenableFuture future; + final String namespaceID = "ns"; try { final URIExtractionNamespace namespace = new URIExtractionNamespace( - "ns", tmpFile.toURI(), new URIExtractionNamespace.ObjectMapperFlatDataParser( URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) @@ -486,11 +487,11 @@ public void testRunCount() null ); - cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object()); - future = manager.schedule(namespace); + cacheUpdateAlerts.putIfAbsent(namespaceID, new Object()); + future = manager.schedule(namespaceID, namespace); Assert.assertFalse(future.isDone()); - final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace()); + final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespaceID); for (int i = 0; i < numWaits; ++i) { synchronized (cacheUpdateAlerter) { cacheUpdateAlerter.wait(); From 7d75062a1c91466c51886696390c369e849bb924 Mon Sep 17 00:00:00 2001 From: Keuntae Park Date: Thu, 31 Mar 2016 10:57:44 +0900 Subject: [PATCH 02/42] update KafkaExtractionNamespaceTest to reflect argument signature changes --- .../extraction/namespace/KafkaExtractionNamespaceTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java index 79fa58299d0c..3fb95c95b66a 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java @@ -25,6 +25,7 @@ import org.junit.Test; import java.io.IOException; +import java.util.Properties; /** * @@ -36,7 +37,7 @@ public void testReflectiveSerde() throws IOException { ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerSubtypes(KafkaExtractionNamespace.class); - final String val = "{\"type\":\"kafka\",\"kafkaTopic\":\"testTopic\",\"kafkaProperties\":null}"; + final String val = "{\"type\":\"kafka\",\"kafkaTopic\":\"testTopic\",\"kafkaProperties\":{\"zookeeper.session.timeout.ms\":\"10000\",\"zookeeper.sync.time.ms\":\"200\"}}"; final ExtractionNamespace fn = mapper.reader(ExtractionNamespace.class) .readValue( From e4ae72646e6e55ac1051a2f24f3cc26f070159b9 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 7 Apr 2016 10:10:58 -0700 Subject: [PATCH 03/42] Add more synchronization functionality to NamespaceLookupExtractorFactory --- extensions-core/namespace-lookup/pom.xml | 5 + .../NamespaceLookupExtractorFactory.java | 139 ++++-- .../namespace/NamespacedExtractionModule.java | 12 +- .../NamespaceLookupExtractorFactoryTest.java | 424 ++++++++++++++++++ 4 files changed, 536 insertions(+), 44 deletions(-) create mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java diff --git a/extensions-core/namespace-lookup/pom.xml b/extensions-core/namespace-lookup/pom.xml index 8b82d3220841..6e583c4bfd81 100644 --- a/extensions-core/namespace-lookup/pom.xml +++ b/extensions-core/namespace-lookup/pom.xml @@ -77,5 +77,10 @@ 3.0.1 test + + org.easymock + easymock + test + diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index 92ab124d16f4..1fa894bf6876 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -22,74 +22,89 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.inject.name.Named; +import com.metamx.common.ISE; +import com.metamx.common.StringUtils; import com.metamx.common.logger.Logger; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractorFactory; -import io.druid.server.namespace.NamespacedExtractionModule; import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; import javax.annotation.Nullable; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; @JsonTypeName("namespace") public class NamespaceLookupExtractorFactory implements LookupExtractorFactory { - private static final Logger log = new Logger(NamespaceLookupExtractorFactory.class); + private static final Logger LOG = new Logger(NamespaceLookupExtractorFactory.class); - private static AtomicInteger numExtractor = new AtomicInteger(0); private static long SCHEDULE_TIMEOUT = 60_000; - final ExtractionNamespace extractionNamespace; - final NamespaceExtractionCacheManager manager; - final Function> fnMaker; - final Function>> reverseFnMaker; - LookupExtractor lookupExtractor; + private final AtomicBoolean started = new AtomicBoolean(false); + private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); + private final ExtractionNamespace extractionNamespace; + private final NamespaceExtractionCacheManager manager; private final String extractorID; @JsonCreator public NamespaceLookupExtractorFactory( - @JsonProperty("extractionNamespace")ExtractionNamespace extractionNamespace, - @JacksonInject @Named(NamespacedExtractionModule.EXTRACTION_CACHE_MANAGER) - NamespaceExtractionCacheManager manager, - @JacksonInject @Named(NamespacedExtractionModule.DIM_EXTRACTION_NAMESPACE) - Function> fnMaker, - @JacksonInject @Named(NamespacedExtractionModule.DIM_REVERSE_EXTRACTION_NAMESPACE) - Function>> reverseFnMaker - ) + @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace, + @JacksonInject NamespaceExtractionCacheManager manager + ) { - this.extractionNamespace = Preconditions.checkNotNull(extractionNamespace, - "extractionNamespace should be specified"); + this.extractionNamespace = Preconditions.checkNotNull( + extractionNamespace, + "extractionNamespace should be specified" + ); this.manager = manager; - this.fnMaker = fnMaker; - this.reverseFnMaker = reverseFnMaker; - this.extractorID = getID(); + this.extractorID = buildID(); } @Override public boolean start() { - if (!manager.scheduleAndWait(extractorID, extractionNamespace, SCHEDULE_TIMEOUT)) - { - return false; + final Lock writeLock = startStopSync.writeLock(); + writeLock.lock(); + try { + if (!started.compareAndSet(false, true)) { + LOG.warn("Already started!"); + return false; + } + if (!manager.scheduleAndWait(extractorID, extractionNamespace, SCHEDULE_TIMEOUT)) { + LOG.warn("Failed to schedule lookup [%s]", extractorID); + return false; + } + LOG.debug("NamespaceLookupExtractorFactory[%s] started", extractorID); + return true; + } + finally { + writeLock.unlock(); } - - log.debug("NamespaceLookupExtractorFactory[%s] started", extractorID); - this.lookupExtractor = new NamespacedExtractor(fnMaker, reverseFnMaker, extractorID); - - return true; } @Override public boolean close() { - manager.checkedDelete(extractorID); - return true; + final Lock writeLock = startStopSync.writeLock(); + writeLock.lock(); + try { + if (!started.compareAndSet(true, false)) { + LOG.warn("Not started!"); + return false; + } + return manager.checkedDelete(extractorID); + } + finally { + writeLock.unlock(); + } } @Override @@ -102,14 +117,62 @@ public boolean replaces(@Nullable LookupExtractorFactory other) return true; } - private String getID() + @JsonProperty + public ExtractionNamespace getExtractionNamespace() + { + return extractionNamespace; + } + + private String buildID() { - return String.format("%d - %s", numExtractor.getAndIncrement(), extractionNamespace); + return UUID.randomUUID().toString(); } + // Grab the latest snapshot from the cache manager @Override public LookupExtractor get() { - return lookupExtractor; + final Lock readLock = startStopSync.readLock(); + readLock.lock(); + try { + if (!started.get()) { + throw new ISE("Factory [%s] not started", extractorID); + } + String preVersion = null, postVersion = null; + Map map = null; + // Make sure we absolutely know what version of map we grabbed (for caching purposes) + do { + preVersion = manager.getVersion(extractorID); + if (preVersion == null) { + throw new ISE("Namespace vanished for [%s]", extractorID); + } + map = manager.getCacheMap(extractorID); + postVersion = manager.getVersion(extractorID); + if (postVersion == null) { + // We lost some horrible race... make sure we clean up + manager.delete(extractorID); + throw new ISE("Lookup [%s] is deleting", extractorID); + } + } while (!preVersion.equals(postVersion)); + final byte[] v = StringUtils.toUtf8(postVersion); + final byte[] id = StringUtils.toUtf8(extractorID); + return new MapLookupExtractor(map, false) + { + @Override + public byte[] getCacheKey() + { + return ByteBuffer + .allocate(id.length + 1 + v.length + 1) + .put(id) + .put((byte) 0xFF) + .put(v) + .put((byte) 0xFF) + .array(); + } + }; + } + finally { + readLock.unlock(); + } } } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java index 3685adcfa457..b0c818b93877 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java @@ -24,13 +24,11 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Key; import com.google.inject.Provides; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; -import com.google.inject.name.Names; import com.metamx.common.IAE; import io.druid.guice.Jerseys; import io.druid.guice.LazySingleton; @@ -68,16 +66,18 @@ public class NamespacedExtractionModule implements DruidModule public static final String EXTRACTION_CACHE_MANAGER = "DruidExtractionCacheManager"; private final ConcurrentMap> fnCache = new ConcurrentHashMap<>(); - private final ConcurrentMap>> reverseFnCache= new ConcurrentHashMap<>(); + private final ConcurrentMap>> reverseFnCache = new ConcurrentHashMap<>(); @Override public List getJacksonModules() { return ImmutableList.of( new SimpleModule("DruidNamespacedExtractionModule") - .registerSubtypes(NamespacedExtractor.class) - .registerSubtypes(ExtractionNamespace.class) - .registerSubtypes(NamespaceLookupExtractorFactory.class) + .registerSubtypes( + NamespacedExtractor.class, + ExtractionNamespace.class, + NamespaceLookupExtractorFactory.class + ) ); } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java new file mode 100644 index 000000000000..30f0d4c7e3d3 --- /dev/null +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java @@ -0,0 +1,424 @@ +/* + * 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.extraction; + +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.ISE; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.extraction.namespace.ExtractionNamespace; +import io.druid.query.extraction.namespace.URIExtractionNamespace; +import io.druid.query.lookup.LookupExtractor; +import io.druid.query.lookup.LookupExtractorFactory; +import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import org.easymock.EasyMock; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +import java.util.concurrent.ConcurrentHashMap; + +public class NamespaceLookupExtractorFactoryTest +{ + private final ObjectMapper mapper = new DefaultObjectMapper(); + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final NamespaceExtractionCacheManager cacheManager = EasyMock.createStrictMock(NamespaceExtractionCacheManager.class); + + @Before + public void setUp() + { + mapper.setInjectableValues( + new InjectableValues() + { + @Override + public Object findInjectableValue( + Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance + ) + { + if ("io.druid.server.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) { + return cacheManager; + } + return null; + } + } + ); + } + + @Test + public void testSimpleSerde() throws Exception + { + final URIExtractionNamespace uriExtractionNamespace = new URIExtractionNamespace( + temporaryFolder.newFolder().toURI(), + new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper), + + Period.millis(0), + "foo" + ); + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + uriExtractionNamespace, + cacheManager + ); + Assert.assertEquals( + uriExtractionNamespace, + mapper.readValue( + mapper.writeValueAsString(namespaceLookupExtractorFactory), + NamespaceLookupExtractorFactory.class + ).getExtractionNamespace() + ); + } + + @Test + public void testMissingSpec() + { + expectedException.expectMessage("extractionNamespace should be specified"); + new NamespaceLookupExtractorFactory(null, null); + } + + @Test + public void testSimpleStartStop() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(true).once(); + EasyMock.expect( + cacheManager.checkedDelete(EasyMock.anyString()) + ).andReturn(true).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); + Assert.assertTrue(namespaceLookupExtractorFactory.close()); + EasyMock.verify(cacheManager); + } + + @Test + public void testSimpleStartStopStop() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(true).once(); + EasyMock.expect( + cacheManager.checkedDelete(EasyMock.anyString()) + ).andReturn(true).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); + Assert.assertTrue(namespaceLookupExtractorFactory.close()); + Assert.assertFalse(namespaceLookupExtractorFactory.close()); + EasyMock.verify(cacheManager); + } + + @Test + public void testSimpleStartStart() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(true).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); + Assert.assertFalse(namespaceLookupExtractorFactory.start()); + EasyMock.verify(cacheManager); + } + + + @Test + public void testSimpleStartGetStop() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(true).once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); + EasyMock.expect( + cacheManager.checkedDelete(EasyMock.anyString()) + ).andReturn(true).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); + final LookupExtractor extractor = namespaceLookupExtractorFactory.get(); + Assert.assertNull(extractor.apply("foo")); + Assert.assertTrue(namespaceLookupExtractorFactory.close()); + EasyMock.verify(cacheManager); + } + + + @Test + public void testSimpleStartRacyGetDuringDelete() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(true).once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn(null).once(); + EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); + try { + namespaceLookupExtractorFactory.get(); + Assert.fail("Should have thrown ISE"); + } + catch (ISE ise) { + // NOOP + } + + EasyMock.verify(cacheManager); + } + + + @Test + public void testSimpleStartRacyGetDuringUpdate() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(true).once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap(ImmutableMap.of("foo", "bar"))) + .once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("1").once(); + + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("2").once(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("2").once(); + EasyMock.expect(cacheManager.checkedDelete(EasyMock.anyString())).andReturn(true).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); + final LookupExtractor extractor = namespaceLookupExtractorFactory.get(); + Assert.assertNull(extractor.apply("foo")); + Assert.assertNotNull(extractor.getCacheKey()); + Assert.assertTrue(namespaceLookupExtractorFactory.close()); + EasyMock.verify(cacheManager); + } + + + @Test + public void testSimpleStartRacyGetAfterDelete() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(true).once(); + EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn(null).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); + try { + namespaceLookupExtractorFactory.get(); + Assert.fail("Should have thrown ISE"); + } + catch (ISE ise) { + // NOOP + } + + EasyMock.verify(cacheManager); + } + + + @Test + public void testSartFailsToSchedule() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + EasyMock.expect(cacheManager.scheduleAndWait( + EasyMock.anyString(), + EasyMock.eq(extractionNamespace), + EasyMock.eq(60000L) + )).andReturn(false).once(); + EasyMock.expect( + cacheManager.checkedDelete(EasyMock.anyString()) + ).andReturn(false).once(); + EasyMock.replay(cacheManager); + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + Assert.assertFalse(namespaceLookupExtractorFactory.start()); + Assert.assertFalse(namespaceLookupExtractorFactory.close()); + EasyMock.verify(cacheManager); + } + + @Test + public void testReplaces() + { + final ExtractionNamespace en1 = EasyMock.createStrictMock(ExtractionNamespace.class), en2 = EasyMock.createStrictMock( + ExtractionNamespace.class); + EasyMock.replay(en1, en2); + final NamespaceLookupExtractorFactory f1 = new NamespaceLookupExtractorFactory( + en1, + cacheManager + ), f2 = new NamespaceLookupExtractorFactory(en2, cacheManager), f1b = new NamespaceLookupExtractorFactory( + en1, + cacheManager + ); + Assert.assertTrue(f1.replaces(f2)); + Assert.assertTrue(f2.replaces(f1)); + Assert.assertFalse(f1.replaces(f1b)); + Assert.assertFalse(f1b.replaces(f1)); + Assert.assertFalse(f1.replaces(f1)); + Assert.assertTrue(f1.replaces(EasyMock.createNiceMock(LookupExtractorFactory.class))); + EasyMock.verify(en1, en2); + } + + + @Test(expected = ISE.class) + public void testMustBeStarted() + { + final ExtractionNamespace extractionNamespace = new ExtractionNamespace() + { + @Override + public long getPollMs() + { + return 0; + } + }; + + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + cacheManager + ); + + namespaceLookupExtractorFactory.get(); + } +} From a525a5f81d82b838bb72afba497e39e4e0f05092 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 7 Apr 2016 16:23:16 -0700 Subject: [PATCH 04/42] Remove old way of using extraction namespaces --- .../KafkaExtractionNamespaceFactory.java | 44 +--- .../namespace/TestKafkaExtractionCluster.java | 13 +- .../query/extraction/NamespacedExtractor.java | 93 -------- ...a => ExtractionNamespaceCacheFactory.java} | 23 +- ... JDBCExtractionNamespaceCacheFactory.java} | 53 +---- .../namespace/NamespacedExtractionModule.java | 91 +------- ...> URIExtractionNamespaceCacheFactory.java} | 55 +---- .../NamespaceExtractionCacheManager.java | 61 ++--- ...ffHeapNamespaceExtractionCacheManager.java | 14 +- ...OnHeapNamespaceExtractionCacheManager.java | 14 +- .../http/NamespacesCacheResource.java | 55 ----- .../namespace/NamespacedExtractorTest.java | 201 ---------------- .../NamespaceExtractionClusterTest.java | 219 ------------------ .../NamespacedExtractorModuleTest.java | 31 +-- ...IExtractionNamespaceCacheFactoryTest.java} | 74 ++---- .../cache/JDBCExtractionNamespaceTest.java | 68 +----- ...ceExtractionCacheManagerExecutorsTest.java | 29 +-- .../NamespaceExtractionCacheManagersTest.java | 54 ++--- 18 files changed, 110 insertions(+), 1082 deletions(-) delete mode 100644 extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java rename extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/{ExtractionNamespaceFunctionFactory.java => ExtractionNamespaceCacheFactory.java} (69%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/{JDBCExtractionNamespaceFunctionFactory.java => JDBCExtractionNamespaceCacheFactory.java} (78%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/{URIExtractionNamespaceFunctionFactory.java => URIExtractionNamespaceCacheFactory.java} (80%) delete mode 100644 extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/http/NamespacesCacheResource.java delete mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/NamespacedExtractorTest.java delete mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java rename extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/{URIExtractionNamespaceFunctionFactoryTest.java => URIExtractionNamespaceCacheFactoryTest.java} (81%) diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java index 1ecf0a3d2b0f..b882aa6f5c36 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java @@ -27,7 +27,7 @@ import com.google.common.collect.Maps; import com.google.inject.Inject; import com.google.inject.name.Named; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.KafkaExtractionNamespace; import javax.annotation.Nullable; @@ -39,7 +39,7 @@ /** * */ -public class KafkaExtractionNamespaceFactory implements ExtractionNamespaceFunctionFactory +public class KafkaExtractionNamespaceFactory implements ExtractionNamespaceCacheFactory { private final List kafkaExtractionManagers; private static final String KAFKA_VERSION = "kafka versions are updated every time a new event comes in"; @@ -52,46 +52,6 @@ public KafkaExtractionNamespaceFactory( this.kafkaExtractionManagers = kafkaExtractionManagers; } - - @Override - public Function buildFn(KafkaExtractionNamespace extractionNamespace, final Map cache) - { - return new Function() - { - @Nullable - @Override - public String apply(String input) - { - if (Strings.isNullOrEmpty(input)) { - return null; - } - return Strings.emptyToNull(cache.get(input)); - } - }; - } - - @Override - public Function> buildReverseFn( - KafkaExtractionNamespace extractionNamespace, final Map cache - ) - { - return new Function>() - { - @Nullable - @Override - public List apply(@Nullable final String value) - { - return Lists.newArrayList(Maps.filterKeys(cache, new Predicate() - { - @Override public boolean apply(@Nullable String key) - { - return cache.get(key).equals(Strings.nullToEmpty(value)); - } - }).keySet()); - } - }; - } - // This only fires ONCE when the namespace is first added. The version is updated externally as events come in @Override public Callable getCachePopulator( diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java index 4b5fbd185fe8..c14b9f15b340 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java @@ -38,15 +38,13 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.guice.GuiceInjectors; -import io.druid.guice.annotations.Json; import io.druid.initialization.Initialization; -import io.druid.query.extraction.NamespacedExtractor; +import io.druid.query.extraction.NamespaceLookupExtractorFactory; import io.druid.query.lookup.LookupExtractorFactory; import io.druid.server.namespace.KafkaExtractionManager; import io.druid.server.namespace.KafkaExtractionNamespaceFactory; import io.druid.server.namespace.KafkaExtractionNamespaceModule; import io.druid.server.namespace.NamespacedExtractionModule; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; import kafka.admin.AdminUtils; import kafka.javaapi.producer.Producer; import kafka.producer.KeyedMessage; @@ -63,6 +61,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import java.io.File; @@ -82,7 +81,7 @@ public class TestKafkaExtractionCluster private static final Lifecycle lifecycle = new Lifecycle(); private static final File tmpDir = Files.createTempDir(); private static final String topicName = "testTopic"; - private static String namespace; + private static String namespace = "some_name"; private static final Properties kafkaProperties = new Properties(); private KafkaServer kafkaServer; @@ -94,7 +93,7 @@ public class TestKafkaExtractionCluster private LookupExtractorFactory factory; private KafkaExtractionManager renameManager; - public static class KafkaFactoryProvider implements Provider> + public static class KafkaFactoryProvider implements Provider> { private final List kafkaExtractionManager; @@ -107,7 +106,7 @@ public KafkaFactoryProvider( } @Override - public ExtractionNamespaceFunctionFactory get() + public ExtractionNamespaceCacheFactory get() { return new KafkaExtractionNamespaceFactory(kafkaExtractionManager); } @@ -259,7 +258,6 @@ public void configure(Binder binder) String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString); factory = mapper.readValue(json, LookupExtractorFactory.class); factory.start(); - namespace = ((NamespacedExtractor)factory.get()).getNamespace(); renameManager = injector.getInstance( Key.get( new TypeLiteral>() @@ -326,6 +324,7 @@ private void checkServer() } } + @Ignore // Should be un-ignored after kafka rewrite to Lookups is merged @Test(timeout = 60_000L) public void testSimpleRename() throws InterruptedException { diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java deleted file mode 100644 index b2e73225827f..000000000000 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java +++ /dev/null @@ -1,93 +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.query.extraction; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.inject.name.Named; -import com.metamx.common.StringUtils; -import io.druid.query.lookup.LookupExtractor; -import io.druid.server.namespace.NamespacedExtractionModule; - -import javax.validation.constraints.NotNull; -import java.nio.ByteBuffer; -import java.util.List; - -/** - * Namespaced extraction is a special case of DimExtractionFn where the actual extractor is pulled from a map of known implementations. - * In the event that an unknown namespace is passed, a simple reflective function is returned instead. - */ -@JsonTypeName("namespace") -public class NamespacedExtractor extends LookupExtractor -{ - private static final byte CACHE_TYPE_ID = 0x05; - - private final String namespace; - private final Function extractionFunction; - private final Function> reverseExtractionFunction; - - @JsonCreator - public NamespacedExtractor( - @NotNull @JacksonInject @Named(NamespacedExtractionModule.DIM_EXTRACTION_NAMESPACE) - final Function> namespaces, - @NotNull @JacksonInject @Named(NamespacedExtractionModule.DIM_REVERSE_EXTRACTION_NAMESPACE) - final Function>> reverseNamespaces, - @NotNull @JsonProperty(value = "namespace", required = true) - final String namespace - ) - { - this.namespace = Preconditions.checkNotNull(namespace, "namespace"); - this.extractionFunction = Preconditions.checkNotNull(namespaces.apply(namespace), "no namespace found"); - this.reverseExtractionFunction = Preconditions.checkNotNull( - reverseNamespaces.apply(namespace), - "can not found reverse extraction function" - ); - } - - @JsonProperty("namespace") - public String getNamespace() - { - return namespace; - } - - @Override - public byte[] getCacheKey() - { - final byte[] nsBytes = StringUtils.toUtf8(namespace); - return ByteBuffer.allocate(nsBytes.length + 1).put(CACHE_TYPE_ID).put(nsBytes).array(); - } - - @Override - public String apply(String value) - { - return extractionFunction.apply(value); - } - - @Override - public List unapply(@NotNull String value) - { - return reverseExtractionFunction.apply(value); - } - -} diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceFunctionFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceCacheFactory.java similarity index 69% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceFunctionFactory.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceCacheFactory.java index 05032bb8740d..aa5406739e0d 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceFunctionFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceCacheFactory.java @@ -28,29 +28,8 @@ /** * */ -public interface ExtractionNamespaceFunctionFactory +public interface ExtractionNamespaceCacheFactory { - - /** - * Create a function for the given namespace which will do the manipulation requested in the extractionNamespace. - * A simple implementation would simply use the cache supplied by the `NamespaceExtractionCacheManager`. - * More advanced implementations may need more than just what can be cached by `NamespaceExtractionCacheManager`. - * - * @param extractionNamespace The ExtractionNamespace for which a manipulating function is needed. - * - * @return A function which will perform an extraction in accordance with the desires of the ExtractionNamespace - */ - Function buildFn(T extractionNamespace, Map cache); - - - /** - * @param extractionNamespace The ExtractionNamespace for which a manipulating reverse function is needed. - * @param cache view of the cache containing the function mapping. - * - * @return A function that will perform reverse lookup. - */ - Function> buildReverseFn(T extractionNamespace, final Map cache); - /** * This function is called once if `ExtractionNamespace.getUpdateMs() == 0`, or every update if * `ExtractionNamespace.getUpdateMs() > 0` diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceFunctionFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceCacheFactory.java similarity index 78% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceFunctionFactory.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceCacheFactory.java index f4918155d070..b3cbd98b9b6f 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceFunctionFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceCacheFactory.java @@ -19,15 +19,10 @@ package io.druid.server.namespace; -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.JDBCExtractionNamespace; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; @@ -36,7 +31,6 @@ import org.skife.jdbi.v2.tweak.ResultSetMapper; import org.skife.jdbi.v2.util.TimestampMapper; -import javax.annotation.Nullable; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Timestamp; @@ -49,51 +43,12 @@ /** * */ -public class JDBCExtractionNamespaceFunctionFactory - implements ExtractionNamespaceFunctionFactory +public class JDBCExtractionNamespaceCacheFactory + implements ExtractionNamespaceCacheFactory { - private static final Logger LOG = new Logger(JDBCExtractionNamespaceFunctionFactory.class); + private static final Logger LOG = new Logger(JDBCExtractionNamespaceCacheFactory.class); private final ConcurrentMap dbiCache = new ConcurrentHashMap<>(); - @Override - public Function buildFn(JDBCExtractionNamespace extractionNamespace, final Map cache) - { - return new Function() - { - @Nullable - @Override - public String apply(String input) - { - if (Strings.isNullOrEmpty(input)) { - return null; - } - return Strings.emptyToNull(cache.get(input)); - } - }; - } - - @Override - public Function> buildReverseFn( - JDBCExtractionNamespace extractionNamespace, final Map cache - ) - { - return new Function>() - { - @Nullable - @Override - public List apply(@Nullable final String value) - { - return Lists.newArrayList(Maps.filterKeys(cache, new Predicate() - { - @Override public boolean apply(@Nullable String key) - { - return cache.get(key).equals(Strings.nullToEmpty(value)); - } - }).keySet()); - } - }; - } - @Override public Callable getCachePopulator( final String id, diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java index b0c818b93877..7eeb42802971 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java @@ -35,15 +35,13 @@ import io.druid.guice.PolyBind; import io.druid.initialization.DruidModule; import io.druid.query.extraction.NamespaceLookupExtractorFactory; -import io.druid.query.extraction.NamespacedExtractor; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.JDBCExtractionNamespace; import io.druid.query.extraction.namespace.URIExtractionNamespace; import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager; -import io.druid.server.namespace.http.NamespacesCacheResource; import javax.annotation.Nullable; import java.util.List; @@ -74,14 +72,13 @@ public List getJacksonModules() return ImmutableList.of( new SimpleModule("DruidNamespacedExtractionModule") .registerSubtypes( - NamespacedExtractor.class, ExtractionNamespace.class, NamespaceLookupExtractorFactory.class ) ); } - public static MapBinder, ExtractionNamespaceFunctionFactory> getNamespaceFactoryMapBinder( + public static MapBinder, ExtractionNamespaceCacheFactory> getNamespaceFactoryMapBinder( final Binder binder ) { @@ -90,7 +87,7 @@ public static MapBinder, ExtractionNamespac new TypeLiteral>() { }, - new TypeLiteral>() + new TypeLiteral>() { } ); @@ -115,89 +112,11 @@ public void configure(Binder binder) getNamespaceFactoryMapBinder(binder) .addBinding(JDBCExtractionNamespace.class) - .to(JDBCExtractionNamespaceFunctionFactory.class) + .to(JDBCExtractionNamespaceCacheFactory.class) .in(LazySingleton.class); getNamespaceFactoryMapBinder(binder) .addBinding(URIExtractionNamespace.class) - .to(URIExtractionNamespaceFunctionFactory.class) + .to(URIExtractionNamespaceCacheFactory.class) .in(LazySingleton.class); - - Jerseys.addResource(binder, NamespacesCacheResource.class); - } - - @Provides - @Named(EXTRACTION_CACHE_MANAGER) - @LazySingleton - public NamespaceExtractionCacheManager getCacheManager(NamespaceExtractionCacheManager manager) - { - return manager; - } - - @Provides - @Named(NAMESPACE_VERSION_MAP) - @LazySingleton - public ConcurrentMap getVersionMap() - { - return new ConcurrentHashMap<>(); - } - - @Provides - @Named(NAMESPACE_EXTRACTION_FUNCTION_CACHE) - public ConcurrentMap> getFnCache() - { - return fnCache; - } - - @Provides - @Named(NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) - public ConcurrentMap>> getReverseFnCache() - { - return reverseFnCache; - } - - @Provides - @Named(DIM_EXTRACTION_NAMESPACE) - @LazySingleton - public Function> getFunctionMaker( - @Named(NAMESPACE_EXTRACTION_FUNCTION_CACHE) - final ConcurrentMap> fnCache - ) - { - return new Function>() - { - @Nullable - @Override - public Function apply(final String namespace) - { - Function fn = fnCache.get(namespace); - if (fn == null) { - throw new IAE("Namespace [%s] not found", namespace); - } - return fn; - } - }; - } - - @Provides - @Named(DIM_REVERSE_EXTRACTION_NAMESPACE) - @LazySingleton - public Function>> getReverseFunctionMaker( - @Named(NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) - final ConcurrentMap>> reverseFn - ) - { - return new Function>>() - { - @Nullable - @Override - public Function> apply(final String namespace) - { - Function> fn = reverseFn.get(namespace); - if (fn == null) { - throw new IAE("Namespace reverse function [%s] not found", namespace); - } - return fn; - } - }; } } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactory.java similarity index 80% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactory.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactory.java index b600c57cde71..1f28f7e6939f 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactory.java @@ -19,12 +19,7 @@ package io.druid.server.namespace; -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Strings; import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.io.ByteSource; import com.google.inject.Inject; import com.metamx.common.CompressionUtils; @@ -34,18 +29,16 @@ import io.druid.common.utils.JodaUtils; import io.druid.data.SearchableVersionedDataFinder; import io.druid.data.input.MapPopulator; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.URIExtractionNamespace; import io.druid.segment.loading.URIDataPuller; import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; -import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.net.URI; -import java.util.List; import java.util.Map; import java.util.concurrent.Callable; import java.util.regex.Pattern; @@ -53,60 +46,20 @@ /** * */ -public class URIExtractionNamespaceFunctionFactory implements ExtractionNamespaceFunctionFactory +public class URIExtractionNamespaceCacheFactory implements ExtractionNamespaceCacheFactory { private static final int DEFAULT_NUM_RETRIES = 3; - private static final Logger log = new Logger(URIExtractionNamespaceFunctionFactory.class); + private static final Logger log = new Logger(URIExtractionNamespaceCacheFactory.class); private final Map pullers; @Inject - public URIExtractionNamespaceFunctionFactory( + public URIExtractionNamespaceCacheFactory( Map pullers ) { this.pullers = pullers; } - @Override - public Function buildFn(URIExtractionNamespace extractionNamespace, final Map cache) - { - return new Function() - { - @Nullable - @Override - public String apply(String input) - { - if (Strings.isNullOrEmpty(input)) { - return null; - } - return Strings.emptyToNull(cache.get(input)); - } - }; - } - - @Override - public Function> buildReverseFn( - URIExtractionNamespace extractionNamespace, final Map cache - ) - { - return new Function>() - { - @Nullable - @Override - public List apply(@Nullable final String value) - { - return Lists.newArrayList(Maps.filterKeys(cache, new Predicate() - { - @Override - public boolean apply(@Nullable String key) - { - return cache.get(key).equals(Strings.nullToEmpty(value)); - } - }).keySet()); - } - }; - } - @Override public Callable getCachePopulator( final String id, diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java index 88180e3ab9b1..424e262933c6 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java @@ -21,10 +21,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -39,13 +36,10 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import javax.annotation.Nullable; import java.util.Collection; -import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; @@ -56,7 +50,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; /** * @@ -85,27 +78,21 @@ public NamespaceImplData( final ExtractionNamespace namespace; final String name; final AtomicBoolean enabled = new AtomicBoolean(false); - final AtomicReference> fn = new AtomicReference<>(null); - final AtomicReference>> reverseFn = new AtomicReference<>(null); } private static final Logger log = new Logger(NamespaceExtractionCacheManager.class); private final ListeningScheduledExecutorService listeningScheduledExecutorService; - protected final ConcurrentMap> fnCache; - protected final ConcurrentMap>> reverseFnCache; protected final ConcurrentMap implData = new ConcurrentHashMap<>(); protected final AtomicLong tasksStarted = new AtomicLong(0); protected final AtomicLong dataSize = new AtomicLong(0); protected final ServiceEmitter serviceEmitter; private final ConcurrentHashMap lastVersion = new ConcurrentHashMap<>(); - private final Map, ExtractionNamespaceFunctionFactory> namespaceFunctionFactoryMap; + private final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap; public NamespaceExtractionCacheManager( Lifecycle lifecycle, - final ConcurrentMap> fnCache, - final ConcurrentMap>> reverseFnCache, final ServiceEmitter serviceEmitter, - final Map, ExtractionNamespaceFunctionFactory> namespaceFunctionFactoryMap + final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap ) { this.listeningScheduledExecutorService = MoreExecutors.listeningDecorator( @@ -120,8 +107,6 @@ public NamespaceExtractionCacheManager( ); ExecutorServices.manageLifecycle(lifecycle, listeningScheduledExecutorService); this.serviceEmitter = serviceEmitter; - this.fnCache = fnCache; - this.reverseFnCache = reverseFnCache; this.namespaceFunctionFactoryMap = namespaceFunctionFactoryMap; listeningScheduledExecutorService.scheduleAtFixedRate( new Runnable() @@ -134,15 +119,15 @@ public void run() try { final long tasks = tasksStarted.get(); serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/size", dataSize.get())); - serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/count", fnCache.size())); serviceEmitter.emit( ServiceMetricEvent.builder() .build("namespace/deltaTasksStarted", tasks - priorTasksStarted) ); priorTasksStarted = tasks; - }catch(Exception e){ + } + catch (Exception e) { log.error(e, "Error emitting namespace stats"); - if(Thread.currentThread().isInterrupted()){ + if (Thread.currentThread().isInterrupted()) { throw Throwables.propagate(e); } } @@ -162,7 +147,7 @@ protected boolean waitForServiceToEnd(long time, TimeUnit unit) throws Interrupt protected Runnable getPostRunnable( final String id, final T namespace, - final ExtractionNamespaceFunctionFactory factory, + final ExtractionNamespaceCacheFactory factory, final String cacheId ) { @@ -182,18 +167,6 @@ public void run() return; } swapAndClearCache(id, cacheId); - final Function fn = factory.buildFn(namespace, getCacheMap(id)); - final Function> reverseFn = factory.buildReverseFn(namespace, getCacheMap(id)); - final Function priorFn = fnCache.put(id, fn); - final Function> priorReverseFn = reverseFnCache.put(id, reverseFn); - if (priorFn != null && priorFn != namespaceDatum.fn.get()) { - log.warn("Replaced prior function for namespace [%s]", id); - } - if (priorReverseFn != null && priorReverseFn != namespaceDatum.reverseFn.get()) { - log.warn("Replaced prior reverse function for namespace [%s]", id); - } - namespaceDatum.fn.set(fn); - namespaceDatum.reverseFn.set(reverseFn); } } }; @@ -252,15 +225,13 @@ public boolean scheduleAndWait( { String oldVersion = getVersion(id); - if (scheduleOrUpdate(id, namespace)) - { + if (scheduleOrUpdate(id, namespace)) { // wait until the namespace registration is done String newVersion = getVersion(id); final long startLocking = System.currentTimeMillis(); final long timeout = startLocking + waitForFirstRun; - while(newVersion == null || newVersion.equals(oldVersion)) - { + while (newVersion == null || newVersion.equals(oldVersion)) { if (System.currentTimeMillis() > timeout) { log.error("NamespaceLookupExtractorFactory[%s] - timeout during start", id); return false; @@ -268,7 +239,8 @@ public boolean scheduleAndWait( try { Thread.sleep(100); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { log.error("NamespaceLookupExtractorFactory[%s] - interrupted during start", id); return false; } @@ -336,7 +308,7 @@ private boolean removeNamespaceLocalMetadata(final NamespaceImplData implDatum) // Optimistic scheduling of updates to a namespace. public ListenableFuture schedule(final String id, final T namespace) { - final ExtractionNamespaceFunctionFactory factory = (ExtractionNamespaceFunctionFactory) + final ExtractionNamespaceCacheFactory factory = (ExtractionNamespaceCacheFactory) namespaceFunctionFactoryMap.get(namespace.getClass()); if (factory == null) { throw new ISE("Cannot find factory for namespace [%s]", namespace); @@ -349,7 +321,7 @@ public ListenableFuture schedule(final String protected ListenableFuture schedule( final String id, final T namespace, - final ExtractionNamespaceFunctionFactory factory, + final ExtractionNamespaceCacheFactory factory, final Runnable postRunnable, final String cacheId ) @@ -398,7 +370,7 @@ public void run() } else { log.error(t, "Failed update namespace [%s]", namespace); } - if(Thread.currentThread().isInterrupted()) { + if (Thread.currentThread().isInterrupted()) { throw Throwables.propagate(t); } } @@ -471,7 +443,6 @@ public boolean delete(final String ns) if (deleted) { log.info("Deleting namespace [%s]", ns); lastVersion.remove(implDatum.name); - fnCache.remove(implDatum.name); return true; } else { log.debug("Did not delete namespace [%s]", ns); @@ -488,8 +459,8 @@ public String getVersion(String namespace) } } - public Collection getKnownNamespaces() + public Collection getKnownIDs() { - return fnCache.keySet(); + return implData.keySet(); } } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index 361c0b4b7ecd..ee381c184beb 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -19,24 +19,20 @@ package io.druid.server.namespace.cache; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.util.concurrent.Striped; import com.google.inject.Inject; -import com.google.inject.name.Named; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; -import io.druid.server.namespace.NamespacedExtractionModule; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import org.mapdb.DB; import org.mapdb.DBMaker; import java.io.File; import java.io.IOException; -import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -57,15 +53,11 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC @Inject public OffHeapNamespaceExtractionCacheManager( Lifecycle lifecycle, - @Named(NamespacedExtractionModule.NAMESPACE_EXTRACTION_FUNCTION_CACHE) - ConcurrentMap> fnCache, - @Named(NamespacedExtractionModule.NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) - ConcurrentMap>> reverseFnCache, ServiceEmitter emitter, - final Map, ExtractionNamespaceFunctionFactory> namespaceFunctionFactoryMap + final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap ) { - super(lifecycle, fnCache, reverseFnCache, emitter, namespaceFunctionFactoryMap); + super(lifecycle, emitter, namespaceFunctionFactoryMap); try { tmpFile = File.createTempFile("druidMapDB", getClass().getCanonicalName()); log.info("Using file [%s] for mapDB off heap namespace cache", tmpFile.getAbsolutePath()); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java index 0b846f186370..40b1cbff44bd 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java @@ -19,17 +19,13 @@ package io.druid.server.namespace.cache; -import com.google.common.base.Function; import com.google.common.util.concurrent.Striped; import com.google.inject.Inject; -import com.google.inject.name.Named; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.emitter.service.ServiceEmitter; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; -import io.druid.server.namespace.NamespacedExtractionModule; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -46,15 +42,11 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa @Inject public OnHeapNamespaceExtractionCacheManager( final Lifecycle lifecycle, - @Named(NamespacedExtractionModule.NAMESPACE_EXTRACTION_FUNCTION_CACHE) - final ConcurrentMap> fnCache, - @Named(NamespacedExtractionModule.NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE) - final ConcurrentMap>> reverseFnCache, final ServiceEmitter emitter, - final Map, ExtractionNamespaceFunctionFactory> namespaceFunctionFactoryMap + final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap ) { - super(lifecycle, fnCache, reverseFnCache,emitter, namespaceFunctionFactoryMap); + super(lifecycle, emitter, namespaceFunctionFactoryMap); } @Override diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/http/NamespacesCacheResource.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/http/NamespacesCacheResource.java deleted file mode 100644 index 2553f8e67619..000000000000 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/http/NamespacesCacheResource.java +++ /dev/null @@ -1,55 +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.namespace.http; - -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import com.google.common.base.Strings; -import com.google.inject.Inject; -import com.metamx.common.logger.Logger; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; - -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; - -@Path("/druid/v1/namespaces") -public class NamespacesCacheResource -{ - private static final Logger log = new Logger(NamespacesCacheResource.class); - private final NamespaceExtractionCacheManager namespaceExtractionCacheManager; - - @Inject - public NamespacesCacheResource(final NamespaceExtractionCacheManager namespaceExtractionCacheManager){ - this.namespaceExtractionCacheManager = namespaceExtractionCacheManager; - } - - @GET - @Produces({ MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - public Response getNamespaces(){ - try{ - return Response.ok().entity(namespaceExtractionCacheManager.getKnownNamespaces()).build(); - }catch (Exception ex){ - log.error("Can not get the list of known namespaces"); - return Response.serverError().entity(Strings.nullToEmpty(ex.getMessage())).build(); - } - } -} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/NamespacedExtractorTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/NamespacedExtractorTest.java deleted file mode 100644 index 2ebcc975cf32..000000000000 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/NamespacedExtractorTest.java +++ /dev/null @@ -1,201 +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.query.extraction.namespace; - -import com.google.common.base.Function; -import com.google.common.base.Strings; -import io.druid.query.extraction.NamespacedExtractor; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -import javax.annotation.Nullable; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -/** - * - */ -public class NamespacedExtractorTest -{ - private static final ConcurrentMap> defaultMap = new ConcurrentHashMap<>(); - private static final Function NOOP_FN = new Function() - { - @Nullable - @Override - public String apply(@Nullable String input) - { - return Strings.isNullOrEmpty(input) ? null : input; - } - }; - - private static final Function> NOOP_REVERSE_FN = new Function>() - { - @Nullable - @Override - public List apply(@Nullable String input) - { - return Strings.isNullOrEmpty(input) ? Collections.emptyList() : Arrays.asList(input); - } - }; - - private static final Function> defaultFnFinder = new Function>() - { - @Nullable - @Override - public Function apply(@Nullable String input) - { - Function fn = defaultMap.get(input); - return fn == null ? NOOP_FN : fn; - } - }; - - private static final Function>> defaultReverseFnFinder = new Function>>() - { - @Nullable - @Override - public Function> apply(@Nullable final String value) - { - return NOOP_REVERSE_FN; - } - }; - - @BeforeClass - public static void setupStatic() - { - defaultMap.put( - "noop", new Function() - { - @Nullable - @Override - public String apply(String input) - { - return input; - } - } - ); - defaultMap.put( - "null", new Function() - { - @Nullable - @Override - public String apply(@Nullable String input) - { - return null; - } - } - ); - defaultMap.put( - "turtles", new Function() - { - @Nullable - @Override - public String apply(@Nullable String input) - { - return "turtle"; - } - } - ); - defaultMap.put( - "empty", new Function() - { - @Nullable - @Override - public String apply(@Nullable String input) - { - return ""; - } - } - ); - } - - @Test - public void testSimpleNamespace() - { - NamespacedExtractor namespacedExtractor = new NamespacedExtractor(defaultFnFinder, defaultReverseFnFinder, "noop"); - for (int i = 0; i < 10; ++i) { - final String val = UUID.randomUUID().toString(); - Assert.assertEquals(val, namespacedExtractor.apply(val)); - Assert.assertEquals(Arrays.asList(val), namespacedExtractor.unapply(val)); - } - Assert.assertEquals("", namespacedExtractor.apply("")); - Assert.assertNull(namespacedExtractor.apply(null)); - Assert.assertEquals(Collections.emptyList(), namespacedExtractor.unapply(null)); - Assert.assertEquals("The awesomeness", namespacedExtractor.apply("The awesomeness")); - } - - @Test - public void testUnknownNamespace() - { - NamespacedExtractor namespacedExtractor = new NamespacedExtractor( - defaultFnFinder, - defaultReverseFnFinder, - "HFJDKSHFUINEWUINIUENFIUENFUNEWI" - ); - for (int i = 0; i < 10; ++i) { - final String val = UUID.randomUUID().toString(); - Assert.assertEquals(val, namespacedExtractor.apply(val)); - } - Assert.assertNull(namespacedExtractor.apply("")); - Assert.assertNull(namespacedExtractor.apply(null)); - } - - @Test - public void testTurtles() - { - NamespacedExtractor namespacedExtractor = new NamespacedExtractor(defaultFnFinder, defaultReverseFnFinder, "turtles"); - for (int i = 0; i < 10; ++i) { - final String val = UUID.randomUUID().toString(); - Assert.assertEquals("turtle", namespacedExtractor.apply(val)); - } - Assert.assertEquals("turtle", namespacedExtractor.apply("")); - Assert.assertEquals("turtle", namespacedExtractor.apply(null)); - } - - @Test - public void testEmpty() - { - NamespacedExtractor namespacedExtractor = new NamespacedExtractor(defaultFnFinder, defaultReverseFnFinder, "empty"); - Assert.assertEquals("", namespacedExtractor.apply("")); - Assert.assertEquals("", namespacedExtractor.apply(null)); - Assert.assertEquals("", namespacedExtractor.apply("anything")); - } - - @Test - public void testNull() - { - NamespacedExtractor namespacedExtractor = new NamespacedExtractor(defaultFnFinder, defaultReverseFnFinder, "null"); - Assert.assertNull(namespacedExtractor.apply("")); - Assert.assertNull(namespacedExtractor.apply(null)); - } - - @Test - public void testBlankMissingValueIsNull() - { - NamespacedExtractor namespacedExtractor = new NamespacedExtractor(defaultFnFinder, defaultReverseFnFinder, "null"); - Assert.assertNull(namespacedExtractor.apply("fh43u1i2")); - Assert.assertNull(namespacedExtractor.apply("")); - Assert.assertNull(namespacedExtractor.apply(null)); - } -} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java deleted file mode 100644 index 873fd7b3104c..000000000000 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespaceExtractionClusterTest.java +++ /dev/null @@ -1,219 +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.namespace; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.inject.Binder; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.name.Names; -import io.druid.guice.GuiceInjectors; -import io.druid.initialization.Initialization; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.dimension.LookupDimensionSpec; -import io.druid.query.extraction.ExtractionFn; -import io.druid.query.extraction.NamespacedExtractor; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; -import io.druid.query.lookup.*; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.io.OutputStreamWriter; - -public class NamespaceExtractionClusterTest -{ - @Rule - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - @Rule - public final TemporaryFolder temporaryFolder2 = new TemporaryFolder(); - - private ObjectMapper mapper; - private LookupReferencesManager lookupReferencesManager; - private Injector injector; - - @Before - public void setUp() - { - System.setProperty("druid.extensions.searchCurrentClassloader", "false"); - - injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjectorWithModules( - ImmutableList.of() - ), - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); - } - }, - new LookupModule(), - new NamespacedExtractionModule() - ) - ); - mapper = injector.getInstance(ObjectMapper.class); - lookupReferencesManager = injector.getInstance(LookupReferencesManager.class); - lookupReferencesManager.start(); - } - - @Test(timeout = 60_000) - public void testSimpleJson() throws IOException, InterruptedException - { - final File tempFile = temporaryFolder.newFile(); - try (OutputStreamWriter out = new FileWriter(tempFile)) { - out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); - } - final URIExtractionNamespace namespace = new URIExtractionNamespace( - tempFile.toURI(), - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) - ), - new Period(0), - null - ); - - String namespaceString = mapper.writeValueAsString(namespace); - String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString); - LookupExtractorFactory factory = mapper.readValue(json, LookupExtractorFactory.class); - - Assert.assertNotNull(factory); - - Assert.assertTrue(factory.start()); - - LookupExtractor extractor = factory.get(); - - Assert.assertTrue(extractor instanceof NamespacedExtractor); - - NamespacedExtractor namespacedExtractor = (NamespacedExtractor)extractor; - - Assert.assertEquals("bar", namespacedExtractor.apply("foo")); - - Assert.assertTrue(factory.close()); - } - - @Test(timeout = 60_000) - public void testTwoFactories() throws IOException, InterruptedException - { - final File tempFile1 = temporaryFolder.newFile(); - try (OutputStreamWriter out = new FileWriter(tempFile1)) { - out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); - } - final URIExtractionNamespace namespace1 = new URIExtractionNamespace( - tempFile1.toURI(), - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) - ), - new Period(0), - null - ); - String namespaceString1 = mapper.writeValueAsString(namespace1); - String json1 = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString1); - LookupExtractorFactory factory1 = mapper.readValue(json1, LookupExtractorFactory.class); - - final File tempFile2 = temporaryFolder2.newFile(); - try (OutputStreamWriter out = new FileWriter(tempFile2)) { - out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bad"))); - } - final URIExtractionNamespace namespace2 = new URIExtractionNamespace( - tempFile2.toURI(), - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) - ), - new Period(0), - null - ); - String namespaceString2 = mapper.writeValueAsString(namespace2); - String json2 = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString2); - LookupExtractorFactory factory2 = mapper.readValue(json2, LookupExtractorFactory.class); - - Assert.assertNotNull(factory1); - Assert.assertNotNull(factory2); - - Assert.assertTrue(factory1.start()); - Assert.assertTrue(factory2.start()); - - LookupExtractor extractor1 = factory1.get(); - Assert.assertTrue(extractor1 instanceof NamespacedExtractor); - - NamespacedExtractor namespacedExtractor1 = (NamespacedExtractor)extractor1; - Assert.assertEquals("bar", namespacedExtractor1.apply("foo")); - - LookupExtractor extractor2 = factory2.get(); - Assert.assertTrue(extractor2 instanceof NamespacedExtractor); - - NamespacedExtractor namespacedExtractor2 = (NamespacedExtractor)extractor2; - Assert.assertEquals("bad", namespacedExtractor2.apply("foo")); - - Assert.assertTrue(factory1.close()); - Assert.assertTrue(factory2.close()); - } - - @Test(timeout = 60_000) - public void testReferenceManagerIntegration() throws IOException, InterruptedException - { - final File tempFile = temporaryFolder.newFile(); - try (OutputStreamWriter out = new FileWriter(tempFile)) { - out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); - } - final URIExtractionNamespace namespace = new URIExtractionNamespace( - tempFile.toURI(), - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) - ), - new Period(0), - null - ); - - String namespaceString = mapper.writeValueAsString(namespace); - String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString); - LookupExtractorFactory factory = mapper.readValue(json, LookupExtractorFactory.class); - - lookupReferencesManager.updateIfNew("refTest", factory); - - LookupDimensionSpec lookupDimensionSpec = new LookupDimensionSpec( - "col", - "out", - null, - false, - null, - "refTest", - lookupReferencesManager, - false - ); - - ExtractionFn extractionFn = lookupDimensionSpec.getExtractionFn(); - Assert.assertEquals("bar", extractionFn.apply("foo")); - } -} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java index c0917d08a9d3..2cddef2a7294 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java @@ -21,13 +21,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.common.lifecycle.Lifecycle; import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.JDBCExtractionNamespace; import io.druid.query.extraction.namespace.URIExtractionNamespace; import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; @@ -46,10 +45,7 @@ import java.io.File; import java.io.FileWriter; import java.io.OutputStreamWriter; -import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -62,7 +58,6 @@ public class NamespacedExtractorModuleTest private static final ObjectMapper mapper = URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()); private static NamespaceExtractionCacheManager cacheManager; private static Lifecycle lifecycle; - private static ConcurrentMap> fnCache = new ConcurrentHashMap<>(); @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -70,25 +65,19 @@ public class NamespacedExtractorModuleTest @BeforeClass public static void setUpStatic() throws Exception { - final Map, ExtractionNamespaceFunctionFactory> factoryMap = - ImmutableMap., ExtractionNamespaceFunctionFactory>of( + final Map, ExtractionNamespaceCacheFactory> factoryMap = + ImmutableMap., ExtractionNamespaceCacheFactory>of( URIExtractionNamespace.class, - new URIExtractionNamespaceFunctionFactory( + new URIExtractionNamespaceCacheFactory( ImmutableMap.of( "file", new LocalFileTimestampVersionFinder() ) ), - JDBCExtractionNamespace.class, new JDBCExtractionNamespaceFunctionFactory() + JDBCExtractionNamespace.class, new JDBCExtractionNamespaceCacheFactory() ); lifecycle = new Lifecycle(); - cacheManager = new OnHeapNamespaceExtractionCacheManager( - lifecycle, - new ConcurrentHashMap>(), - new ConcurrentHashMap>>(), - new NoopServiceEmitter(), factoryMap - ); - fnCache.clear(); + cacheManager = new OnHeapNamespaceExtractionCacheManager(lifecycle, new NoopServiceEmitter(), factoryMap); } @AfterClass @@ -104,7 +93,7 @@ public void testNewTask() throws Exception try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } - final URIExtractionNamespaceFunctionFactory factory = new URIExtractionNamespaceFunctionFactory( + final URIExtractionNamespaceCacheFactory factory = new URIExtractionNamespaceCacheFactory( ImmutableMap.of("file", new LocalFileTimestampVersionFinder()) ); final String namespaceID = "ns"; @@ -138,13 +127,13 @@ public void testListNamespaces() throws Exception null ); Assert.assertTrue(cacheManager.scheduleAndWait(namespaceID, namespace, 1_000)); - Assert.assertArrayEquals(cacheManager.getKnownNamespaces().toArray(), new Object[]{namespaceID}); + Assert.assertArrayEquals(cacheManager.getKnownIDs().toArray(), new Object[]{namespaceID}); Assert.assertTrue(cacheManager.delete(namespaceID)); } private static boolean noNamespaces(NamespaceExtractionCacheManager manager) { - return manager.getKnownNamespaces().isEmpty(); + return manager.getKnownIDs().isEmpty(); } @Test//(timeout = 10_000) @@ -185,7 +174,7 @@ public void testNewUpdate() throws Exception ); Assert.assertTrue(noNamespaces(cacheManager)); Assert.assertTrue(cacheManager.scheduleAndWait(namespaceID, namespace, 10_000)); - Assert.assertArrayEquals(cacheManager.getKnownNamespaces().toArray(), new Object[]{namespaceID}); + Assert.assertArrayEquals(cacheManager.getKnownIDs().toArray(), new Object[]{namespaceID}); Assert.assertTrue(cacheManager.delete(namespaceID)); } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactoryTest.java similarity index 81% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactoryTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactoryTest.java index 72f3641bf22c..6a3515cbf7cf 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceFunctionFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactoryTest.java @@ -24,14 +24,13 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Sets; import com.metamx.common.UOE; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.emitter.service.ServiceEmitter; import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.URIExtractionNamespace; import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; @@ -61,7 +60,6 @@ import java.lang.reflect.InvocationTargetException; import java.nio.file.Files; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -78,7 +76,7 @@ * */ @RunWith(Parameterized.class) -public class URIExtractionNamespaceFunctionFactoryTest +public class URIExtractionNamespaceCacheFactoryTest { @Parameterized.Parameters(name = "{0}") public static Iterable getParameters() throws NoSuchMethodException @@ -137,15 +135,11 @@ public void close() throws IOException final List> cacheConstructors = ImmutableList.>of( OnHeapNamespaceExtractionCacheManager.class.getConstructor( Lifecycle.class, - ConcurrentMap.class, - ConcurrentMap.class, ServiceEmitter.class, Map.class ), OffHeapNamespaceExtractionCacheManager.class.getConstructor( Lifecycle.class, - ConcurrentMap.class, - ConcurrentMap.class, ServiceEmitter.class, Map.class ) @@ -176,10 +170,8 @@ public Object[] next() try { manager = constructor.newInstance( new Lifecycle(), - new ConcurrentHashMap>(), - new ConcurrentHashMap>(), new NoopServiceEmitter(), - new HashMap, ExtractionNamespaceFunctionFactory>() + new HashMap, ExtractionNamespaceCacheFactory>() ); } catch (Exception e) { @@ -215,29 +207,25 @@ public void remove() }; } - public URIExtractionNamespaceFunctionFactoryTest( + public URIExtractionNamespaceCacheFactoryTest( String friendlyName, String suffix, Function outStreamSupplier, Constructor cacheManagerConstructor ) throws IllegalAccessException, InvocationTargetException, InstantiationException { - final Map, ExtractionNamespaceFunctionFactory> namespaceFunctionFactoryMap = new HashMap<>(); + final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap = new HashMap<>(); this.suffix = suffix; this.outStreamSupplier = outStreamSupplier; this.lifecycle = new Lifecycle(); - this.fnCache = new ConcurrentHashMap<>(); - this.reverseFnCache = new ConcurrentHashMap<>(); this.manager = cacheManagerConstructor.newInstance( lifecycle, - fnCache, - reverseFnCache, new NoopServiceEmitter(), namespaceFunctionFactoryMap ); namespaceFunctionFactoryMap.put( URIExtractionNamespace.class, - new URIExtractionNamespaceFunctionFactory( + new URIExtractionNamespaceCacheFactory( ImmutableMap.of( "file", new LocalFileTimestampVersionFinder() @@ -255,17 +243,14 @@ public URIExtractionNamespaceFunctionFactoryTest( private NamespaceExtractionCacheManager manager; private File tmpFile; private File tmpFileParent; - private URIExtractionNamespaceFunctionFactory factory; + private URIExtractionNamespaceCacheFactory factory; private URIExtractionNamespace namespace; private String id; - private ConcurrentHashMap> fnCache; - private ConcurrentHashMap>> reverseFnCache; @Before public void setUp() throws Exception { lifecycle.start(); - fnCache.clear(); tmpFileParent = temporaryFolder.newFolder(); tmpFile = Files.createTempFile(tmpFileParent.toPath(), "druidTestURIExtractionNS", suffix).toFile(); final ObjectMapper mapper = new DefaultObjectMapper(); @@ -283,7 +268,7 @@ public void setUp() throws Exception ))); } } - factory = new URIExtractionNamespaceFunctionFactory( + factory = new URIExtractionNamespaceCacheFactory( ImmutableMap.of("file", new LocalFileTimestampVersionFinder()) ); namespace = new URIExtractionNamespace( @@ -306,33 +291,17 @@ public void tearDown() @Test public void simpleTest() throws IOException, ExecutionException, InterruptedException { - Assert.assertNull(fnCache.get(id)); + Assert.assertTrue(manager.getKnownIDs().isEmpty()); NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace)); - Function fn = fnCache.get(id); - Assert.assertNotNull(fn); - Assert.assertEquals("bar", fn.apply("foo")); - Assert.assertEquals(null, fn.apply("baz")); - } - - @Test - public void testReverseFunction() throws InterruptedException - { - Assert.assertNull(reverseFnCache.get(id)); - NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace)); - Function> reverseFn = reverseFnCache.get(id); - Assert.assertNotNull(reverseFn); - Assert.assertEquals(Sets.newHashSet("boo", "foo"), Sets.newHashSet(reverseFn.apply("bar"))); - Assert.assertEquals(Sets.newHashSet(""), Sets.newHashSet(reverseFn.apply("MissingValue"))); - Assert.assertEquals(Sets.newHashSet("emptyString"), Sets.newHashSet(reverseFn.apply(""))); - Assert.assertEquals(Sets.newHashSet("emptyString"), Sets.newHashSet(reverseFn.apply(null))); - Assert.assertEquals(Collections.EMPTY_LIST, reverseFn.apply("baz")); + Map map = manager.getCacheMap(id); + Assert.assertEquals("bar", map.get("foo")); + Assert.assertEquals(null, map.get("baz")); } @Test public void simplePileONamespacesTest() throws InterruptedException { final int size = 128; - List namespaces = new ArrayList<>(size); List ids = new ArrayList<>(size); for (int i = 0; i < size; ++i) { String id = String.format("%d-ns-%d", i << 10, i); @@ -346,28 +315,23 @@ public void simplePileONamespacesTest() throws InterruptedException Pattern.quote(tmpFile.getName()) ); - namespaces.add(namespace); - - Assert.assertNull(fnCache.get(id)); + Assert.assertFalse(manager.getKnownIDs().contains(id)); NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace)); } - for (int i = 0; i < size; ++i) { - URIExtractionNamespace namespace = namespaces.get(i); - String id = ids.get(i); - Function fn = fnCache.get(id); - Assert.assertNotNull(fn); - Assert.assertEquals("bar", fn.apply("foo")); - Assert.assertEquals(null, fn.apply("baz")); + for (String id : ids) { + final Map map = manager.getCacheMap(id); + Assert.assertEquals("bar", map.get("foo")); + Assert.assertEquals(null, map.get("baz")); manager.delete(id); - Assert.assertNull(fnCache.get(id)); } + Assert.assertTrue(manager.getKnownIDs().isEmpty()); } @Test public void testLoadOnlyOnce() throws Exception { - Assert.assertNull(fnCache.get(id)); + Assert.assertTrue(manager.getKnownIDs().isEmpty()); ConcurrentMap map = new ConcurrentHashMap<>(); Callable populator = factory.getCachePopulator(id, namespace, null, map); diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java index 543f3513c5b7..cc868da3f28d 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java @@ -34,10 +34,10 @@ import io.druid.concurrent.Execs; import io.druid.metadata.TestDerbyConnector; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.JDBCExtractionNamespace; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.namespace.JDBCExtractionNamespaceFunctionFactory; +import io.druid.server.namespace.JDBCExtractionNamespaceCacheFactory; import org.joda.time.Period; import org.junit.After; import org.junit.Assert; @@ -101,8 +101,6 @@ public JDBCExtractionNamespaceTest( this.tsColumn = tsColumn; } - private final ConcurrentMap> fnCache = new ConcurrentHashMap<>(); - private final ConcurrentMap>> reverseFnCache = new ConcurrentHashMap<>(); private final String tsColumn; private OnHeapNamespaceExtractionCacheManager extractionCacheManager; private final Lifecycle lifecycle = new Lifecycle(); @@ -194,12 +192,10 @@ public void close() throws IOException extractionCacheManager = new OnHeapNamespaceExtractionCacheManager( lifecycle, - fnCache, - reverseFnCache, new NoopServiceEmitter(), - ImmutableMap., ExtractionNamespaceFunctionFactory>of( + ImmutableMap., ExtractionNamespaceCacheFactory>of( JDBCExtractionNamespace.class, - new JDBCExtractionNamespaceFunctionFactory() + new JDBCExtractionNamespaceCacheFactory() { @Override public Callable getCachePopulator( @@ -380,54 +376,14 @@ public void testMapping() new Period(0) ); NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); - Function extractionFn = fnCache.get(namespace); + final Map map = extractionCacheManager.getCacheMap(namespace); for (Map.Entry entry : renames.entrySet()) { String key = entry.getKey(); String val = entry.getValue(); - Assert.assertEquals("non-null check", Strings.emptyToNull(val), extractionFn.apply(key)); + Assert.assertEquals("non-null check", Strings.emptyToNull(val), Strings.emptyToNull(map.get(key))); } - Assert.assertEquals("null check", null, extractionFn.apply("baz")); - } - - @Test(timeout = 10_000L) - public void testReverseLookup() throws InterruptedException - { - final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - derbyConnectorRule.getMetadataConnectorConfig(), - tableName, - keyName, - valName, - tsColumn, - new Period(0) - ); - NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); - Function> reverseExtractionFn = reverseFnCache.get(namespace); - Assert.assertEquals( - "reverse lookup should match", - Sets.newHashSet("foo", "bad"), - Sets.newHashSet(reverseExtractionFn.apply("bar")) - ); - Assert.assertEquals( - "reverse lookup should match", - Sets.newHashSet("how about that"), - Sets.newHashSet(reverseExtractionFn.apply("foo")) - ); - Assert.assertEquals( - "reverse lookup should match", - Sets.newHashSet("empty string"), - Sets.newHashSet(reverseExtractionFn.apply("")) - ); - Assert.assertEquals( - "null is same as empty string", - Sets.newHashSet("empty string"), - Sets.newHashSet(reverseExtractionFn.apply(null)) - ); - Assert.assertEquals( - "reverse lookup of none existing value should be empty list", - Collections.EMPTY_LIST, - reverseExtractionFn.apply("does't exist") - ); + Assert.assertEquals("null check", null, map.get("baz")); } @Test(timeout = 10_000L) @@ -476,7 +432,7 @@ private JDBCExtractionNamespace ensureNamespace() Assert.assertEquals( "sanity check not correct", "bar", - fnCache.get(namespace).apply("foo") + extractionCacheManager.getCacheMap(namespace).get("foo") ); return extractionNamespace; } @@ -512,18 +468,18 @@ private void assertUpdated(String namespace, String key, String expected) throws { waitForUpdates(1_000L, 2L); - Function extractionFn = fnCache.get(namespace); + Map map = extractionCacheManager.getCacheMap(namespace); // rely on test timeout to break out of this loop - while (!extractionFn.apply(key).equals(expected)) { + while (!expected.equals(map.get(key))) { Thread.sleep(100); - extractionFn = fnCache.get(namespace); + map = extractionCacheManager.getCacheMap(namespace); } Assert.assertEquals( "update check", expected, - extractionFn.apply(key) + map.get(key) ); } } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java index be08039a500c..a020f32b779f 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java @@ -20,7 +20,6 @@ package io.druid.server.namespace.cache; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; @@ -32,12 +31,12 @@ import io.druid.concurrent.Execs; import io.druid.data.SearchableVersionedDataFinder; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.URIExtractionNamespace; import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.namespace.URIExtractionNamespaceFunctionFactory; +import io.druid.server.namespace.URIExtractionNamespaceCacheFactory; import org.joda.time.Period; import org.junit.After; import org.junit.Assert; @@ -48,7 +47,6 @@ import java.io.File; import java.io.FileOutputStream; -import java.io.IOException; import java.io.OutputStream; import java.io.OutputStreamWriter; import java.lang.reflect.Field; @@ -56,7 +54,6 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; -import java.util.List; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; @@ -67,7 +64,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; /** * @@ -81,8 +77,6 @@ public class NamespaceExtractionCacheManagerExecutorsTest private Lifecycle lifecycle; private NamespaceExtractionCacheManager manager; private File tmpFile; - private final ConcurrentMap> fnCache = new ConcurrentHashMap<>(); - private final ConcurrentMap>> reverseFnCache = new ConcurrentHashMap<>(); private final ConcurrentMap cacheUpdateAlerts = new ConcurrentHashMap<>(); private final AtomicLong numRuns = new AtomicLong(0L); @@ -94,7 +88,7 @@ public void setUp() throws Exception lifecycle = new Lifecycle(); // Lifecycle stop is used to shut down executors. Start does nothing, so it's ok to call it here. lifecycle.start(); - final URIExtractionNamespaceFunctionFactory factory = new URIExtractionNamespaceFunctionFactory( + final URIExtractionNamespaceCacheFactory factory = new URIExtractionNamespaceCacheFactory( ImmutableMap.of("file", new LocalFileTimestampVersionFinder()) ) { @@ -120,8 +114,8 @@ public String call() throws Exception } }; manager = new OnHeapNamespaceExtractionCacheManager( - lifecycle, fnCache, reverseFnCache, new NoopServiceEmitter(), - ImmutableMap., ExtractionNamespaceFunctionFactory>of( + lifecycle, new NoopServiceEmitter(), + ImmutableMap., ExtractionNamespaceCacheFactory>of( URIExtractionNamespace.class, factory ) @@ -131,7 +125,7 @@ lifecycle, fnCache, reverseFnCache, new NoopServiceEmitter(), protected Runnable getPostRunnable( final String id, final T namespace, - final ExtractionNamespaceFunctionFactory factory, + final ExtractionNamespaceCacheFactory factory, final String cacheId ) { @@ -360,7 +354,8 @@ public void testDelete(final String ns) new Period(period), null ); - final ListenableFuture future = manager.schedule(ns, namespace); + Assert.assertTrue(manager.scheduleAndWait(ns, namespace, 10_000)); + final ListenableFuture future = manager.implData.get(ns).future; Assert.assertFalse(future.isCancelled()); Assert.assertFalse(future.isDone()); @@ -369,7 +364,7 @@ public void testDelete(final String ns) final long timeout = 45_000L; do { synchronized (cacheUpdateAlerter) { - if (!fnCache.containsKey(ns)) { + if (!manager.implData.containsKey(ns)) { cacheUpdateAlerter.wait(10_000); } } @@ -383,7 +378,7 @@ public void testDelete(final String ns) throw Throwables.propagate(e); } } - if (!fnCache.containsKey(ns) && System.currentTimeMillis() - start > timeout) { + if (!manager.implData.containsKey(ns) && System.currentTimeMillis() - start > timeout) { throw new RuntimeException( new TimeoutException( String.format( @@ -393,11 +388,10 @@ public void testDelete(final String ns) ) ); } - } while (!fnCache.containsKey(ns)); + } while (!manager.implData.containsKey(ns) || !manager.implData.get(ns).enabled.get()); Assert.assertEquals(VALUE, manager.getCacheMap(ns).get(KEY)); - Assert.assertTrue(fnCache.containsKey(ns)); Assert.assertTrue(manager.implData.containsKey(ns)); Assert.assertTrue(manager.delete(ns)); @@ -415,7 +409,6 @@ public void testDelete(final String ns) } Assert.assertFalse(manager.implData.containsKey(ns)); - Assert.assertFalse(fnCache.containsKey(ns)); Assert.assertTrue(future.isCancelled()); Assert.assertTrue(future.isDone()); } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagersTest.java index d9f80a61c005..e9a2f3aa7655 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -19,18 +19,14 @@ package io.druid.server.namespace.cache; -import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory; +import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; import org.junit.Assert; import org.junit.Before; @@ -38,14 +34,11 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -62,44 +55,34 @@ public class NamespaceExtractionCacheManagersTest public static Collection getParameters() { ArrayList params = new ArrayList<>(); - - ConcurrentMap> fnMap = new ConcurrentHashMap>(); - ConcurrentMap>> reverserFnMap = new ConcurrentHashMap>>(); params.add( new Object[]{ - new OffHeapNamespaceExtractionCacheManager( - lifecycle, - fnMap, - reverserFnMap, - new NoopServiceEmitter(), - ImmutableMap., ExtractionNamespaceFunctionFactory>of() - ), fnMap - } + new OffHeapNamespaceExtractionCacheManager( + lifecycle, + new NoopServiceEmitter(), + ImmutableMap., ExtractionNamespaceCacheFactory>of() + ) + } ); params.add( new Object[]{ new OnHeapNamespaceExtractionCacheManager( lifecycle, - fnMap, - reverserFnMap, new NoopServiceEmitter(), - ImmutableMap., ExtractionNamespaceFunctionFactory>of() - ), fnMap + ImmutableMap., ExtractionNamespaceCacheFactory>of() + ) } ); return params; } private final NamespaceExtractionCacheManager extractionCacheManager; - private final ConcurrentMap> fnMap; public NamespaceExtractionCacheManagersTest( - NamespaceExtractionCacheManager extractionCacheManager, - ConcurrentMap> fnMap + NamespaceExtractionCacheManager extractionCacheManager ) { this.extractionCacheManager = extractionCacheManager; - this.fnMap = fnMap; } private static final List nsList = ImmutableList.of("testNs", "test.ns", "//tes-tn!s"); @@ -107,21 +90,9 @@ public NamespaceExtractionCacheManagersTest( @Before public void setup() { - fnMap.clear(); // prepopulate caches for (String ns : nsList) { final ConcurrentMap map = extractionCacheManager.getCacheMap(ns); - fnMap.put( - ns, new Function() - { - @Nullable - @Override - public String apply(String input) - { - return map.get(input); - } - } - ); map.put("oldNameSeed1", "oldNameSeed2"); } } @@ -141,7 +112,10 @@ public void testSimpleCacheCreate() public void testCacheList() { List nsList = new ArrayList(NamespaceExtractionCacheManagersTest.nsList); - List retvalList = Lists.newArrayList(extractionCacheManager.getKnownNamespaces()); + for (String ns : nsList) { + extractionCacheManager.implData.put(ns, new NamespaceExtractionCacheManager.NamespaceImplData(null, null, null)); + } + List retvalList = Lists.newArrayList(extractionCacheManager.getKnownIDs()); Collections.sort(nsList); Collections.sort(retvalList); Assert.assertArrayEquals(nsList.toArray(), retvalList.toArray()); From 5b35e4262ccc244f36d37e84278ed150c64b1068 Mon Sep 17 00:00:00 2001 From: Keuntae Park Date: Wed, 27 Apr 2016 10:45:48 +0900 Subject: [PATCH 05/42] resolve compile error by supporting LookupIntrospectHandler --- .../NamespaceLookupExtractorFactory.java | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index 1fa894bf6876..499bf41b2c57 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -29,9 +29,15 @@ import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractorFactory; +import io.druid.query.lookup.LookupIntrospectHandler; import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; import javax.annotation.Nullable; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; import java.nio.ByteBuffer; import java.util.Map; import java.util.UUID; @@ -51,6 +57,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); private final ExtractionNamespace extractionNamespace; private final NamespaceExtractionCacheManager manager; + private final LookupIntrospectHandler lookupIntrospectHandler; private final String extractorID; @@ -66,6 +73,35 @@ public NamespaceLookupExtractorFactory( ); this.manager = manager; this.extractorID = buildID(); + this.lookupIntrospectHandler = new LookupIntrospectHandler() { + @GET + @Path("/keys") + @Produces(MediaType.APPLICATION_JSON) + public Response getKeys() + { + return Response.ok(getLatest().keySet().toString()).build(); + } + + @GET + @Path("/values") + @Produces(MediaType.APPLICATION_JSON) + public Response getValues() + { + return Response.ok(getLatest().values().toString()).build(); + } + + @GET + @Produces(MediaType.APPLICATION_JSON) + public Response getMap() + { + return Response.ok(getLatest()).build(); + } + + private Map getLatest() + { + return ((MapLookupExtractor)get()).getMap(); + } + }; } @Override @@ -117,6 +153,12 @@ public boolean replaces(@Nullable LookupExtractorFactory other) return true; } + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + return lookupIntrospectHandler; + } + @JsonProperty public ExtractionNamespace getExtractionNamespace() { From 00f42c1cfbd4f0e6aa7686b57c6d10906b968b2c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 5 May 2016 13:06:14 -0700 Subject: [PATCH 06/42] Remove kafka lookups --- .../kafka-extraction-namespace/pom.xml | 5 + .../KafkaExtractionNamespaceModule.java | 50 ++ .../lookup/KafkaLookupExtractorFactory.java | 423 ++++++++++++++ .../namespace/KafkaExtractionManager.java | 33 +- .../KafkaExtractionNamespaceFactory.java | 13 +- .../KafkaExtractionNamespaceModule.java | 10 - .../io.druid.initialization.DruidModule | 2 +- .../KafkaLookupExtractorFactoryTest.java | 516 ++++++++++++++++++ .../lookup/TestKafkaExtractionCluster.java | 400 ++++++++++++++ .../src/test/resources/log4j2.xml | 2 +- 10 files changed, 1413 insertions(+), 41 deletions(-) create mode 100644 extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaExtractionNamespaceModule.java create mode 100644 extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java create mode 100644 extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java create mode 100644 extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 9f1a2a8b8ae9..58bda817cc3e 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -98,5 +98,10 @@ 3.0.1 test + + org.easymock + easymock + test + diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaExtractionNamespaceModule.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaExtractionNamespaceModule.java new file mode 100644 index 000000000000..17159dbb32ae --- /dev/null +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaExtractionNamespaceModule.java @@ -0,0 +1,50 @@ +/* + * 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.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; + +import java.util.List; + +/** + * + */ +public class KafkaExtractionNamespaceModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule("kafka-lookups").registerSubtypes( + KafkaLookupExtractorFactory.class + ) + ); + } + + @Override + public void configure(Binder binder) + { + // NOOP + } +} 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 new file mode 100644 index 000000000000..286a8d0d538b --- /dev/null +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java @@ -0,0 +1,423 @@ +/* + * 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.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import com.google.common.primitives.Longs; +import com.google.common.util.concurrent.FutureCallback; +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.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.StringUtils; +import com.metamx.common.logger.Logger; +import io.druid.concurrent.Execs; +import io.druid.query.extraction.MapLookupExtractor; +import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Pattern; +import javax.annotation.Nullable; +import javax.validation.constraints.Min; +import javax.ws.rs.GET; +import javax.ws.rs.core.Response; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.KafkaStream; +import kafka.consumer.Whitelist; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.MessageAndMetadata; +import kafka.serializer.Decoder; + +@JsonTypeName("kafka") +public class KafkaLookupExtractorFactory implements LookupExtractorFactory +{ + private static final Logger LOG = new Logger(KafkaLookupExtractorFactory.class); + static final Decoder DEFAULT_STRING_DECODER = new Decoder() + { + @Override + public String fromBytes(byte[] bytes) + { + return StringUtils.fromUtf8(bytes); + } + }; + + private final ListeningExecutorService executorService; + private final AtomicLong doubleEventCount = new AtomicLong(0L); + private final NamespaceExtractionCacheManager cacheManager; + private final String factoryId = UUID.randomUUID().toString(); + private final AtomicReference> mapRef = new AtomicReference<>(null); + private final AtomicBoolean started = new AtomicBoolean(false); + + private volatile ListenableFuture future = null; + + @JsonProperty + private final String kafkaTopic; + + @JsonProperty + private final Map kafkaProperties; + + @JsonProperty + private final long connectTimeout; + + @JsonProperty + private final boolean isOneToOne; + + @JsonCreator + public KafkaLookupExtractorFactory( + @JacksonInject NamespaceExtractionCacheManager cacheManager, + @JsonProperty("kafkaTopic") final String kafkaTopic, + @JsonProperty("kafkaProperties") final Map kafkaProperties, + @JsonProperty("connectTimeout") @Min(0) long connectTimeout, + @JsonProperty("isOneToOne") boolean isOneToOne + ) + { + this.kafkaTopic = Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required"); + this.kafkaProperties = Preconditions.checkNotNull(kafkaProperties, "kafkaProperties required"); + executorService = MoreExecutors.listeningDecorator(Execs.singleThreaded( + "kafka-factory-" + kafkaTopic + "-%s", + Thread.MIN_PRIORITY + )); + this.cacheManager = cacheManager; + this.connectTimeout = connectTimeout; + this.isOneToOne = isOneToOne; + } + + public KafkaLookupExtractorFactory( + NamespaceExtractionCacheManager cacheManager, + String kafkaTopic, + Map kafkaProperties + ) + { + this(cacheManager, kafkaTopic, kafkaProperties, 0, false); + } + + public String getKafkaTopic() + { + return kafkaTopic; + } + + public Map getKafkaProperties() + { + return kafkaProperties; + } + + public long getConnectTimeout() + { + return connectTimeout; + } + + public boolean isOneToOne() + { + return isOneToOne; + } + + @Override + public boolean start() + { + synchronized (started) { + if (started.get()) { + LOG.warn("Already started, not starting again"); + return started.get(); + } + if (executorService.isShutdown()) { + LOG.warn("Already shut down, not starting again"); + return false; + } + final Properties kafkaProperties = new Properties(); + kafkaProperties.putAll(getKafkaProperties()); + if (kafkaProperties.containsKey("group.id")) { + throw new IAE( + "Cannot set kafka property [group.id]. Property is randomly generated for you. Found [%s]", + kafkaProperties.getProperty("group.id") + ); + } + if (kafkaProperties.containsKey("auto.offset.reset")) { + throw new IAE( + "Cannot set kafka property [auto.offset.reset]. Property will be forced to [smallest]. Found [%s]", + kafkaProperties.getProperty("auto.offset.reset") + ); + } + Preconditions.checkNotNull( + kafkaProperties.getProperty("zookeeper.connect"), + "zookeeper.connect required property" + ); + + kafkaProperties.setProperty("group.id", factoryId); + final String topic = getKafkaTopic(); + LOG.debug("About to listen to topic [%s] with group.id [%s]", topic, factoryId); + final Map map = cacheManager.getCacheMap(factoryId); + mapRef.set(map); + // Enable publish-subscribe + kafkaProperties.setProperty("auto.offset.reset", "smallest"); + + final CountDownLatch startingReads = new CountDownLatch(1); + + final ListenableFuture future = executorService.submit( + new Runnable() + { + @Override + public void run() + { + while (!executorService.isShutdown() && !Thread.currentThread().isInterrupted()) { + final ConsumerConnector consumerConnector = buildConnector(kafkaProperties); + try { + final List> streams = consumerConnector.createMessageStreamsByFilter( + new Whitelist(Pattern.quote(topic)), 1, DEFAULT_STRING_DECODER, DEFAULT_STRING_DECODER + ); + + if (streams == null || streams.isEmpty()) { + throw new IAE("Topic [%s] had no streams", topic); + } + if (streams.size() > 1) { + throw new ISE("Topic [%s] has %d streams! expected 1", topic, streams.size()); + } + final KafkaStream kafkaStream = streams.get(0); + + startingReads.countDown(); + + for (final MessageAndMetadata messageAndMetadata : kafkaStream) { + final String key = messageAndMetadata.key(); + final String message = messageAndMetadata.message(); + if (key == null || message == null) { + LOG.error("Bad key/message from topic [%s]: [%s]", topic, messageAndMetadata); + continue; + } + doubleEventCount.incrementAndGet(); + map.put(key, message); + doubleEventCount.incrementAndGet(); + LOG.trace("Placed key[%s] val[%s]", key, message); + } + } + catch (Exception e) { + LOG.error(e, "Error reading stream for topic [%s]", topic); + } + finally { + consumerConnector.shutdown(); + } + } + } + } + ); + Futures.addCallback( + future, new FutureCallback() + { + @Override + public void onSuccess(Object result) + { + LOG.debug("Success listening to [%s]", topic); + } + + @Override + public void onFailure(Throwable t) + { + if (t instanceof CancellationException) { + LOG.debug("Topic [%s] cancelled", topic); + } else { + LOG.error(t, "Error in listening to [%s]", topic); + } + } + }, + MoreExecutors.sameThreadExecutor() + ); + this.future = future; + final Stopwatch stopwatch = Stopwatch.createStarted(); + try { + while (!startingReads.await(100, TimeUnit.MILLISECONDS) && connectTimeout > 0L) { + // Don't return until we have actually connected + if (future.isDone()) { + future.get(); + } else { + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > connectTimeout) { + throw new TimeoutException("Failed to connect to kafka in sufficient time"); + } + } + } + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + if (!future.isDone() && !future.cancel(true) && !future.isDone()) { + LOG.warn("Could not cancel kafka listening thread"); + } + LOG.error(e, "Failed to start kafka extraction factory"); + cacheManager.delete(factoryId); + return false; + } + + started.set(true); + return true; + } + } + + // Overriden in tests + ConsumerConnector buildConnector(Properties properties) + { + return new kafka.javaapi.consumer.ZookeeperConsumerConnector( + new ConsumerConfig(properties) + ); + } + + @Override + public boolean close() + { + synchronized (started) { + if (!started.get() || executorService.isShutdown()) { + LOG.info("Already shutdown, ignoring"); + return !started.get(); + } + started.set(false); + executorService.shutdownNow(); + final ListenableFuture future = this.future; + if (future != null) { + if (!future.isDone() && !future.cancel(true) && !future.isDone()) { + LOG.error("Error cancelling future for topic [%s]", getKafkaTopic()); + return false; + } + } + if (!cacheManager.delete(factoryId)) { + LOG.error("Error removing [%s] for topic [%s] from cache", factoryId, getKafkaTopic()); + return false; + } + return true; + } + } + + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + if (this == other) { + return false; + } + + if (other == null) { + return false; + } + + if (getClass() != other.getClass()) { + return true; + } + + final KafkaLookupExtractorFactory that = (KafkaLookupExtractorFactory) other; + + return !(getKafkaTopic().equals(that.getKafkaTopic()) + && getKafkaProperties().equals(that.getKafkaProperties()) + && getConnectTimeout() == that.getConnectTimeout() + && isOneToOne() == that.isOneToOne() + ); + } + + @Nullable + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + return new KafkaLookupExtractorIntrospectionHandler(); + } + + @Override + public LookupExtractor get() + { + final Map map = Preconditions.checkNotNull(mapRef.get(), "Not started"); + final long startCount = doubleEventCount.get(); + return new MapLookupExtractor(map, isOneToOne()) + { + @Override + public byte[] getCacheKey() + { + final byte[] idutf8 = StringUtils.toUtf8(factoryId); + // If the number of things added has not changed during the course of this extractor's life, we can cache it + if (startCount == doubleEventCount.get()) { + return ByteBuffer + .allocate(idutf8.length + 1 + Longs.BYTES) + .put(idutf8) + .put((byte) 0xFF) + .putLong(startCount) + .array(); + } else { + // If the number of things added HAS changed during the coruse of this extractor's life, we CANNOT cache + final byte[] scrambler = StringUtils.toUtf8(UUID.randomUUID().toString()); + return ByteBuffer + .allocate(idutf8.length + 1 + scrambler.length + 1) + .put(idutf8) + .put((byte) 0xFF) + .put(scrambler) + .put((byte) 0xFF) + .array(); + } + } + }; + } + + public long getCompletedEventCount() + { + return doubleEventCount.get() >> 1; + } + + // Used in tests + NamespaceExtractionCacheManager getCacheManager() + { + return cacheManager; + } + + AtomicReference> getMapRef() + { + return mapRef; + } + + AtomicLong getDoubleEventCount() + { + return doubleEventCount; + } + + ListenableFuture getFuture() + { + return future; + } + + + class KafkaLookupExtractorIntrospectionHandler implements LookupIntrospectHandler + { + @GET + public Response getActive() + { + final ListenableFuture future = getFuture(); + if (future != null && !future.isDone()) { + return Response.ok().build(); + } else { + return Response.status(Response.Status.GONE).build(); + } + } + } +} diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java index 4bfd7683b0d0..748fc5d8f391 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java @@ -27,32 +27,29 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; -import com.google.inject.name.Named; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.StringUtils; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; -import io.druid.guice.ManageLifecycle; import io.druid.query.extraction.namespace.KafkaExtractionNamespace; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.ConsumerIterator; -import kafka.consumer.KafkaStream; -import kafka.consumer.Whitelist; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.MessageAndMetadata; -import kafka.serializer.Decoder; - -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.UUID; -import java.util.concurrent.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Pattern; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.ConsumerIterator; +import kafka.consumer.KafkaStream; +import kafka.consumer.Whitelist; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.MessageAndMetadata; +import kafka.serializer.Decoder; /** * @@ -106,7 +103,7 @@ boolean supports(Properties otherProperties) { if (otherProperties.containsKey("zookeeper.connect")) { return this.kafkaProperties.getProperty("zookeeper.connect") - .equals(otherProperties.getProperty("zookeeper.connect")); + .equals(otherProperties.getProperty("zookeeper.connect")); } return false; } @@ -127,11 +124,11 @@ public String fromBytes(byte[] bytes) public long getNumEvents(String namespace) { - if(namespace == null){ + if (namespace == null) { return 0L; } else { final AtomicLong eventCounter = topicEvents.get(namespace); - if(eventCounter != null) { + if (eventCounter != null) { return eventCounter.get(); } else { return 0L; @@ -171,7 +168,7 @@ public void run() final ConsumerIterator it = kafkaStream.iterator(); log.info("Listening to topic [%s] for namespace [%s]", topic, id); AtomicLong eventCounter = topicEvents.get(id); - if(eventCounter == null){ + if (eventCounter == null) { topicEvents.putIfAbsent(id, new AtomicLong(0L)); eventCounter = topicEvents.get(id); } diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java index b882aa6f5c36..990276be2337 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java @@ -20,17 +20,10 @@ package io.druid.server.namespace; import com.fasterxml.jackson.annotation.JacksonInject; -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.inject.Inject; import com.google.inject.name.Named; import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.extraction.namespace.KafkaExtractionNamespace; - -import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.Properties; @@ -82,10 +75,8 @@ public String call() private KafkaExtractionManager findAppropriate(Properties kafkaProperties) { - for (KafkaExtractionManager manager: kafkaExtractionManagers) - { - if (manager.supports(kafkaProperties)) - { + for (KafkaExtractionManager manager : kafkaExtractionManagers) { + if (manager.supports(kafkaProperties)) { return manager; } } diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java index 351c9590aadc..50617c76e324 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java @@ -19,27 +19,17 @@ package io.druid.server.namespace; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Provides; import com.google.inject.name.Named; import io.druid.guice.LazySingleton; -import io.druid.guice.LifecycleModule; -import io.druid.guice.annotations.Json; import io.druid.initialization.DruidModule; import io.druid.query.extraction.namespace.KafkaExtractionNamespace; - -import java.io.IOException; -import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.Properties; /** * diff --git a/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index 6a1d6a2e05c1..48fadfc15c46 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -16,4 +16,4 @@ # specific language governing permissions and limitations # under the License. # -io.druid.server.namespace.KafkaExtractionNamespaceModule +io.druid.query.lookup.KafkaExtractionNamespaceModule diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java new file mode 100644 index 000000000000..a8a40d0deafc --- /dev/null +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java @@ -0,0 +1,516 @@ +/* + * 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.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.StringUtils; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import kafka.consumer.ConsumerIterator; +import kafka.consumer.KafkaStream; +import kafka.consumer.TopicFilter; +import kafka.javaapi.consumer.ConsumerConnector; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import static io.druid.query.lookup.KafkaLookupExtractorFactory.DEFAULT_STRING_DECODER; + +public class KafkaLookupExtractorFactoryTest +{ + private static final String TOPIC = "some_topic"; + private static final Map DEFAULT_PROPERTIES = ImmutableMap.of( + "some.property", "some.value" + ); + private final ObjectMapper mapper = new DefaultObjectMapper(); + final NamespaceExtractionCacheManager cacheManager = EasyMock.createStrictMock(NamespaceExtractionCacheManager.class); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Before + public void setUp() + { + mapper.setInjectableValues(new InjectableValues() + { + @Override + public Object findInjectableValue( + Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance + ) + { + if ("io.druid.server.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) { + return cacheManager; + } else { + return null; + } + } + }); + } + + @Test + public void testSimpleSerDe() throws Exception + { + final KafkaLookupExtractorFactory expected = new KafkaLookupExtractorFactory(null, TOPIC, DEFAULT_PROPERTIES); + final KafkaLookupExtractorFactory result = mapper.readValue( + mapper.writeValueAsString(expected), + KafkaLookupExtractorFactory.class + ); + Assert.assertEquals(expected.getKafkaTopic(), result.getKafkaTopic()); + Assert.assertEquals(expected.getKafkaProperties(), result.getKafkaProperties()); + Assert.assertEquals(cacheManager, result.getCacheManager()); + Assert.assertEquals(0, expected.getCompletedEventCount()); + Assert.assertEquals(0, result.getCompletedEventCount()); + } + + @Test + public void testCacheKeyScramblesOnNewData() + { + final int n = 1000; + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ); + factory.getMapRef().set(ImmutableMap.of()); + final AtomicLong events = factory.getDoubleEventCount(); + + final LookupExtractor extractor = factory.get(); + + final List byteArrays = new ArrayList<>(n); + for (int i = 0; i < n; ++i) { + final byte[] myKey = extractor.getCacheKey(); + // Not terribly efficient.. but who cares + for (byte[] byteArray : byteArrays) { + Assert.assertFalse(Arrays.equals(byteArray, myKey)); + } + byteArrays.add(myKey); + events.incrementAndGet(); + } + Assert.assertEquals(n, byteArrays.size()); + } + + @Test + public void testCacheKeyScramblesDifferentStarts() + { + final int n = 1000; + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ); + factory.getMapRef().set(ImmutableMap.of()); + final AtomicLong events = factory.getDoubleEventCount(); + + final List byteArrays = new ArrayList<>(n); + for (int i = 0; i < n; ++i) { + final LookupExtractor extractor = factory.get(); + final byte[] myKey = extractor.getCacheKey(); + // Not terribly efficient.. but who cares + for (byte[] byteArray : byteArrays) { + Assert.assertFalse(Arrays.equals(byteArray, myKey)); + } + byteArrays.add(myKey); + events.incrementAndGet(); + } + Assert.assertEquals(n, byteArrays.size()); + } + + @Test + public void testCacheKeySameOnNoChange() + { + final int n = 1000; + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ); + factory.getMapRef().set(ImmutableMap.of()); + + final LookupExtractor extractor = factory.get(); + + final byte[] baseKey = extractor.getCacheKey(); + for (int i = 0; i < n; ++i) { + Assert.assertArrayEquals(baseKey, factory.get().getCacheKey()); + } + } + + @Test + public void testCacheKeyDifferentForTopics() + { + final KafkaLookupExtractorFactory factory1 = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ); + factory1.getMapRef().set(ImmutableMap.of()); + final KafkaLookupExtractorFactory factory2 = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC + "b", + DEFAULT_PROPERTIES + ); + factory2.getMapRef().set(ImmutableMap.of()); + + Assert.assertFalse(Arrays.equals(factory1.get().getCacheKey(), factory2.get().getCacheKey())); + } + + @Test + public void testReplaces() + { + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ); + Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(), false))); + Assert.assertFalse(factory.replaces(factory)); + Assert.assertFalse(factory.replaces(new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ))); + + Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory( + cacheManager, + TOPIC + "b", + DEFAULT_PROPERTIES + ))); + + Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("some.property", "some.other.value") + ))); + + Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("some.other.property", "some.value") + ))); + + Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES, + 1, + false + ))); + + Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES, + 0, + true + ))); + } + + @Test + public void testStopWithoutStart() + { + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ); + Assert.assertTrue(factory.close()); + } + + @Test + public void testStartStop() + { + final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class); + final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class); + final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class); + EasyMock.expect(consumerConnector.createMessageStreamsByFilter( + EasyMock.anyObject(TopicFilter.class), + EasyMock.anyInt(), + EasyMock.eq( + DEFAULT_STRING_DECODER), + EasyMock.eq(DEFAULT_STRING_DECODER) + )).andReturn(ImmutableList.of(kafkaStream)).once(); + EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); + EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); + consumerConnector.shutdown(); + EasyMock.expectLastCall().once(); + EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("zookeeper.connect", "localhost"), + 10_000L, + false + ) + { + @Override + ConsumerConnector buildConnector(Properties properties) + { + return consumerConnector; + } + }; + Assert.assertTrue(factory.start()); + Assert.assertTrue(factory.close()); + Assert.assertTrue(factory.getFuture().isDone()); + EasyMock.verify(cacheManager); + } + + + @Test + public void testStartFailsFromTimeout() throws Exception + { + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); + EasyMock.replay(cacheManager); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("zookeeper.connect", "localhost"), + 1, + false + ) + { + @Override + ConsumerConnector buildConnector(Properties properties) + { + // Lock up + try { + Thread.currentThread().join(); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + throw new RuntimeException("shouldn't make it here"); + } + }; + Assert.assertFalse(factory.start()); + Assert.assertTrue(factory.getFuture().isDone()); + Assert.assertTrue(factory.getFuture().isCancelled()); + EasyMock.verify(cacheManager); + } + + @Test + public void testStopDeleteError() + { + final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class); + final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class); + final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class); + EasyMock.expect(consumerConnector.createMessageStreamsByFilter( + EasyMock.anyObject(TopicFilter.class), + EasyMock.anyInt(), + EasyMock.eq( + DEFAULT_STRING_DECODER), + EasyMock.eq(DEFAULT_STRING_DECODER) + )).andReturn(ImmutableList.of(kafkaStream)).once(); + EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); + EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(false).once(); + + EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("zookeeper.connect", "localhost") + ) + { + @Override + ConsumerConnector buildConnector(Properties properties) + { + return consumerConnector; + } + }; + Assert.assertTrue(factory.start()); + Assert.assertFalse(factory.close()); + EasyMock.verify(cacheManager, kafkaStream, consumerConnector, consumerIterator); + } + + + @Test + public void testStartStopStart() + { + final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class); + final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class); + final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class); + EasyMock.expect(consumerConnector.createMessageStreamsByFilter( + EasyMock.anyObject(TopicFilter.class), + EasyMock.anyInt(), + EasyMock.eq( + DEFAULT_STRING_DECODER), + EasyMock.eq(DEFAULT_STRING_DECODER) + )).andReturn(ImmutableList.of(kafkaStream)).once(); + EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); + EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); + consumerConnector.shutdown(); + EasyMock.expectLastCall().once(); + EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("zookeeper.connect", "localhost") + ) + { + @Override + ConsumerConnector buildConnector(Properties properties) + { + return consumerConnector; + } + }; + Assert.assertTrue(factory.start()); + Assert.assertTrue(factory.close()); + Assert.assertFalse(factory.start()); + EasyMock.verify(cacheManager); + } + + @Test + public void testStartStartStop() + { + final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class); + final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class); + final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class); + EasyMock.expect(consumerConnector.createMessageStreamsByFilter( + EasyMock.anyObject(TopicFilter.class), + EasyMock.anyInt(), + EasyMock.eq( + DEFAULT_STRING_DECODER), + EasyMock.eq(DEFAULT_STRING_DECODER) + )).andReturn(ImmutableList.of(kafkaStream)).once(); + EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); + EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes(); + EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + .andReturn(new ConcurrentHashMap()) + .once(); + EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); + consumerConnector.shutdown(); + EasyMock.expectLastCall().once(); + EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("zookeeper.connect", "localhost"), + 10_000L, + false + ) + { + @Override + ConsumerConnector buildConnector(Properties properties) + { + return consumerConnector; + } + }; + Assert.assertTrue(factory.start()); + Assert.assertTrue(factory.start()); + Assert.assertTrue(factory.close()); + Assert.assertTrue(factory.close()); + EasyMock.verify(cacheManager); + } + + @Test + public void testStartFailsOnMissingConnect() + { + expectedException.expectMessage("zookeeper.connect required property"); + EasyMock.replay(cacheManager); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of() + ); + Assert.assertTrue(factory.start()); + Assert.assertTrue(factory.close()); + EasyMock.verify(cacheManager); + } + + @Test + public void testStartFailsOnGroupID() + { + expectedException.expectMessage( + "Cannot set kafka property [group.id]. Property is randomly generated for you. Found"); + EasyMock.replay(cacheManager); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("group.id", "make me fail") + ); + Assert.assertTrue(factory.start()); + Assert.assertTrue(factory.close()); + EasyMock.verify(cacheManager); + } + + @Test + public void testStartFailsOnAutoOffset() + { + expectedException.expectMessage( + "Cannot set kafka property [auto.offset.reset]. Property will be forced to [smallest]. Found "); + EasyMock.replay(cacheManager); + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + ImmutableMap.of("auto.offset.reset", "make me fail") + ); + Assert.assertTrue(factory.start()); + Assert.assertTrue(factory.close()); + EasyMock.verify(cacheManager); + } + + @Test + public void testFailsGetNotStarted() + { + expectedException.expectMessage("Not started"); + new KafkaLookupExtractorFactory( + cacheManager, + TOPIC, + DEFAULT_PROPERTIES + ).get(); + } + + @Test + public void testDefaultDecoder() + { + final String str = "some string"; + Assert.assertEquals(str, DEFAULT_STRING_DECODER.fromBytes(StringUtils.toUtf8(str))); + } +} diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java new file mode 100644 index 000000000000..bcd1aebe9bb9 --- /dev/null +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java @@ -0,0 +1,400 @@ +/* + * 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.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.io.Closer; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import com.metamx.common.ISE; +import com.metamx.common.StringUtils; +import com.metamx.common.logger.Logger; +import io.druid.guice.GuiceInjectors; +import io.druid.initialization.Initialization; +import io.druid.server.namespace.NamespacedExtractionModule; +import kafka.admin.AdminUtils; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServer; +import kafka.utils.Time; +import kafka.utils.ZKStringSerializer$; +import org.I0Itec.zkclient.ZkClient; +import org.apache.curator.test.TestingServer; +import org.apache.zookeeper.CreateMode; +import org.joda.time.DateTime; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * + */ +public class TestKafkaExtractionCluster +{ + private static final Logger log = new Logger(TestKafkaExtractionCluster.class); + private static final String topicName = "testTopic"; + private static final Map kafkaProperties = new HashMap<>(); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private final Closer closer = Closer.create(); + + private KafkaServer kafkaServer; + private KafkaConfig kafkaConfig; + private TestingServer zkTestServer; + private ZkClient zkClient; + private Injector injector; + private ObjectMapper mapper; + private KafkaLookupExtractorFactory factory; + + @Before + public void setUp() throws Exception + { + zkTestServer = new TestingServer(-1, temporaryFolder.newFolder(), true); + zkTestServer.start(); + + closer.register(new Closeable() + { + @Override + public void close() throws IOException + { + zkTestServer.stop(); + } + }); + + zkClient = new ZkClient( + zkTestServer.getConnectString(), + 10000, + 10000, + ZKStringSerializer$.MODULE$ + ); + closer.register(new Closeable() + { + @Override + public void close() throws IOException + { + zkClient.close(); + } + }); + if (!zkClient.exists("/kafka")) { + zkClient.create("/kafka", null, CreateMode.PERSISTENT); + } + + log.info("---------------------------Started ZK---------------------------"); + + final String zkKafkaPath = "/kafka"; + + final Properties serverProperties = new Properties(); + serverProperties.putAll(kafkaProperties); + serverProperties.put("broker.id", "0"); + serverProperties.put("log.dir", temporaryFolder.newFolder().getAbsolutePath()); + serverProperties.put("log.cleaner.enable", "true"); + serverProperties.put("host.name", "127.0.0.1"); + serverProperties.put("zookeeper.connect", zkTestServer.getConnectString() + zkKafkaPath); + serverProperties.put("zookeeper.session.timeout.ms", "10000"); + serverProperties.put("zookeeper.sync.time.ms", "200"); + + kafkaConfig = new KafkaConfig(serverProperties); + + final long time = DateTime.parse("2015-01-01").getMillis(); + kafkaServer = new KafkaServer( + kafkaConfig, + new Time() + { + + @Override + public long milliseconds() + { + return time; + } + + @Override + public long nanoseconds() + { + return milliseconds() * 1_000_000; + } + + @Override + public void sleep(long ms) + { + try { + Thread.sleep(ms); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + } + ); + kafkaServer.startup(); + closer.register(new Closeable() + { + @Override + public void close() throws IOException + { + kafkaServer.shutdown(); + kafkaServer.awaitShutdown(); + } + }); + + int sleepCount = 0; + + while (!kafkaServer.kafkaController().isActive()) { + Thread.sleep(100); + if (++sleepCount > 10) { + throw new InterruptedException("Controller took to long to awaken"); + } + } + + log.info("---------------------------Started Kafka Server---------------------------"); + + final ZkClient zkClient = new ZkClient( + zkTestServer.getConnectString() + zkKafkaPath, 10000, 10000, + ZKStringSerializer$.MODULE$ + ); + + try (final AutoCloseable autoCloseable = new AutoCloseable() + { + @Override + public void close() throws Exception + { + if (zkClient.exists(zkKafkaPath)) { + try { + zkClient.deleteRecursive(zkKafkaPath); + } + catch (org.I0Itec.zkclient.exception.ZkException ex) { + log.warn(ex, "error deleting %s zk node", zkKafkaPath); + } + } + zkClient.close(); + } + }) { + final Properties topicProperties = new Properties(); + topicProperties.put("cleanup.policy", "compact"); + if (!AdminUtils.topicExists(zkClient, topicName)) { + AdminUtils.createTopic(zkClient, topicName, 1, 1, topicProperties); + } + + log.info("---------------------------Created topic---------------------------"); + + Assert.assertTrue(AdminUtils.topicExists(zkClient, topicName)); + } + + final Properties kafkaProducerProperties = makeProducerProperties(); + final Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties)); + try (final AutoCloseable autoCloseable = new AutoCloseable() + { + @Override + public void close() throws Exception + { + producer.close(); + } + }) { + producer.send( + new KeyedMessage<>( + topicName, + StringUtils.toUtf8("abcdefg"), + StringUtils.toUtf8("abcdefg") + ) + ); + } + + System.setProperty("druid.extensions.searchCurrentClassloader", "false"); + + injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + } + }, + // These injections fail under IntelliJ but are required for maven + new NamespacedExtractionModule(), + new KafkaExtractionNamespaceModule() + ) + ); + mapper = injector.getInstance(ObjectMapper.class); + + log.info("--------------------------- placed default item via producer ---------------------------"); + final Map consumerProperties = new HashMap<>(kafkaProperties); + consumerProperties.put("zookeeper.connect", zkTestServer.getConnectString() + zkKafkaPath); + consumerProperties.put("zookeeper.session.timeout.ms", "10000"); + consumerProperties.put("zookeeper.sync.time.ms", "200"); + + final KafkaLookupExtractorFactory kafkaLookupExtractorFactory = new KafkaLookupExtractorFactory( + null, + topicName, + consumerProperties + ); + + factory = (KafkaLookupExtractorFactory) mapper.readValue( + mapper.writeValueAsString(kafkaLookupExtractorFactory), + LookupExtractorFactory.class + ); + Assert.assertEquals(kafkaLookupExtractorFactory.getKafkaTopic(), factory.getKafkaTopic()); + Assert.assertEquals(kafkaLookupExtractorFactory.getKafkaProperties(), factory.getKafkaProperties()); + factory.start(); + closer.register(new Closeable() + { + @Override + public void close() throws IOException + { + factory.close(); + } + }); + log.info("--------------------------- started rename manager ---------------------------"); + } + + @After + public void tearDown() throws Exception + { + closer.close(); + } + + private final Properties makeProducerProperties() + { + final Properties kafkaProducerProperties = new Properties(); + kafkaProducerProperties.putAll(kafkaProperties); + kafkaProducerProperties.put( + "metadata.broker.list", + String.format("127.0.0.1:%d", kafkaServer.socketServer().port()) + ); + kafkaProperties.put("request.required.acks", "1"); + return kafkaProducerProperties; + } + + private void checkServer() + { + if (!kafkaServer.apis().controller().isActive()) { + throw new ISE("server is not active!"); + } + } + + @Test(timeout = 60_000L) + public void testSimpleRename() throws InterruptedException + { + final Properties kafkaProducerProperties = makeProducerProperties(); + final Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties)); + closer.register(new Closeable() + { + @Override + public void close() throws IOException + { + producer.close(); + } + }); + checkServer(); + + assertUpdated(null, "foo"); + assertReverseUpdated(ImmutableList.of(), "foo"); + + long events = factory.getCompletedEventCount(); + + log.info("------------------------- Sending foo bar -------------------------------"); + producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("foo"), StringUtils.toUtf8("bar"))); + + long start = System.currentTimeMillis(); + while (events == factory.getCompletedEventCount()) { + Thread.sleep(100); + if (System.currentTimeMillis() > start + 60_000) { + throw new ISE("Took too long to update event"); + } + } + + log.info("------------------------- Checking foo bar -------------------------------"); + assertUpdated("bar", "foo"); + assertReverseUpdated(Collections.singletonList("foo"), "bar"); + assertUpdated(null, "baz"); + + checkServer(); + events = factory.getCompletedEventCount(); + + log.info("------------------------- Sending baz bat -------------------------------"); + producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("baz"), StringUtils.toUtf8("bat"))); + while (events == factory.getCompletedEventCount()) { + Thread.sleep(10); + if (System.currentTimeMillis() > start + 60_000) { + throw new ISE("Took too long to update event"); + } + } + + log.info("------------------------- Checking baz bat -------------------------------"); + Assert.assertEquals("bat", factory.get().apply("baz")); + Assert.assertEquals(Collections.singletonList("baz"), factory.get().unapply("bat")); + } + + private void assertUpdated( + String expected, + String key + ) + throws InterruptedException + { + final LookupExtractor extractor = factory.get(); + if (expected == null) { + while (extractor.apply(key) != null) { + Thread.sleep(100); + } + } else { + while (!expected.equals(extractor.apply(key))) { + Thread.sleep(100); + } + } + + Assert.assertEquals("update check", expected, extractor.apply(key)); + } + + private void assertReverseUpdated( + List expected, + String key + ) + throws InterruptedException + { + final LookupExtractor extractor = factory.get(); + + while (!expected.equals(extractor.unapply(key))) { + Thread.sleep(100); + } + + Assert.assertEquals("update check", expected, extractor.unapply(key)); + } +} diff --git a/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml b/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml index 5e1ebff74c39..1601a94c87a8 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml +++ b/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml @@ -28,7 +28,7 @@ - + From 4e91b13bcfa0413956d80938515cdaa3a783d76c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 5 May 2016 13:08:36 -0700 Subject: [PATCH 07/42] Remove unused stuff --- .../namespace/KafkaExtractionNamespace.java | 74 --- .../namespace/KafkaExtractionManager.java | 234 ---------- .../KafkaExtractionNamespaceFactory.java | 85 ---- .../KafkaExtractionNamespaceModule.java | 66 --- .../KafkaExtractionNamespaceTest.java | 63 --- .../namespace/TestKafkaExtractionCluster.java | 441 ------------------ 6 files changed, 963 deletions(-) delete mode 100644 extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java delete mode 100644 extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java delete mode 100644 extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java delete mode 100644 extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java delete mode 100644 extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java delete mode 100644 extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java deleted file mode 100644 index 7276333508f2..000000000000 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java +++ /dev/null @@ -1,74 +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.query.extraction.namespace; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Preconditions; - -import javax.validation.constraints.NotNull; -import java.util.Properties; - -/** - * - */ -@JsonTypeName("kafka") -public class KafkaExtractionNamespace implements ExtractionNamespace -{ - @JsonProperty - private final String kafkaTopic; - - @JsonProperty - private final Properties kafkaProperties; - - @JsonCreator - public KafkaExtractionNamespace( - @NotNull @JsonProperty(value = "kafkaTopic", required = true) final String kafkaTopic, - @NotNull @JsonProperty(value = "kafkaProperites", required = true) final Properties kafkaProperties - ) - { - Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required"); - this.kafkaTopic = kafkaTopic; - this.kafkaProperties = kafkaProperties; - } - - public String getKafkaTopic() - { - return kafkaTopic; - } - - public Properties getKafkaProperties() - { - return kafkaProperties; - } - - @Override - public long getPollMs() - { - return 0L; - } - - @Override - public String toString() - { - return String.format("KafkaExtractionNamespace = { kafkaTopic = '%s' }", kafkaTopic); - } -} diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java deleted file mode 100644 index 748fc5d8f391..000000000000 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java +++ /dev/null @@ -1,234 +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.namespace; - -import com.google.common.base.Throwables; -import com.google.common.util.concurrent.FutureCallback; -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.google.common.util.concurrent.ThreadFactoryBuilder; -import com.google.inject.Inject; -import com.metamx.common.IAE; -import com.metamx.common.ISE; -import com.metamx.common.StringUtils; -import com.metamx.common.logger.Logger; -import io.druid.query.extraction.namespace.KafkaExtractionNamespace; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.regex.Pattern; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.ConsumerIterator; -import kafka.consumer.KafkaStream; -import kafka.consumer.Whitelist; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.MessageAndMetadata; -import kafka.serializer.Decoder; - -/** - * - */ -public class KafkaExtractionManager -{ - private static final Logger log = new Logger(KafkaExtractionManager.class); - - private final Properties kafkaProperties = new Properties(); - private final ConcurrentMap namespaceVersionMap = new ConcurrentHashMap<>(); - private final ConcurrentMap topicEvents = new ConcurrentHashMap<>(); - private final ConcurrentMap> futures = new ConcurrentHashMap<>(); - private final ListeningExecutorService executorService = MoreExecutors.listeningDecorator( - Executors.newCachedThreadPool( - new ThreadFactoryBuilder() - .setNameFormat("kafka-rename-consumer-%d") - .setDaemon(true) - .setPriority(Thread.MIN_PRIORITY) - .build() - ) - ); - private final AtomicInteger backgroundTaskCount = new AtomicInteger(0); - - // Bindings in KafkaExtractionNamespaceModule - @Inject - public KafkaExtractionManager( - final Properties kafkaProperties - ) - { - if (kafkaProperties.containsKey("group.id")) { - throw new IAE( - "Cannot set kafka property [group.id]. Property is randomly generated for you. Found [%s]", - kafkaProperties.getProperty("group.id") - ); - } - if (kafkaProperties.containsKey("auto.offset.reset")) { - throw new IAE( - "Cannot set kafka property [auto.offset.reset]. Property will be forced to [smallest]. Found [%s]", - kafkaProperties.getProperty("auto.offset.reset") - ); - } - this.kafkaProperties.putAll(kafkaProperties); - if (!this.kafkaProperties.containsKey("zookeeper.connect")) { - this.kafkaProperties.put("zookeeper.connect", "localhost:2181/kafka"); - } - // Enable publish-subscribe - this.kafkaProperties.setProperty("auto.offset.reset", "smallest"); - } - - boolean supports(Properties otherProperties) - { - if (otherProperties.containsKey("zookeeper.connect")) { - return this.kafkaProperties.getProperty("zookeeper.connect") - .equals(otherProperties.getProperty("zookeeper.connect")); - } - return false; - } - - public long getBackgroundTaskCount() - { - return backgroundTaskCount.get(); - } - - private static final Decoder defaultStringDecoder = new Decoder() - { - @Override - public String fromBytes(byte[] bytes) - { - return StringUtils.fromUtf8(bytes); - } - }; - - public long getNumEvents(String namespace) - { - if (namespace == null) { - return 0L; - } else { - final AtomicLong eventCounter = topicEvents.get(namespace); - if (eventCounter != null) { - return eventCounter.get(); - } else { - return 0L; - } - } - } - - public void addListener(final String id, final KafkaExtractionNamespace kafkaNamespace, final Map map) - { - final String topic = kafkaNamespace.getKafkaTopic(); - final ListenableFuture future = executorService.submit( - new Runnable() - { - @Override - public void run() - { - final Properties privateProperties = new Properties(); - privateProperties.putAll(kafkaProperties); - privateProperties.setProperty("group.id", UUID.randomUUID().toString()); - ConsumerConnector consumerConnector = new kafka.javaapi.consumer.ZookeeperConsumerConnector( - new ConsumerConfig( - privateProperties - ) - ); - List> streams = consumerConnector.createMessageStreamsByFilter( - new Whitelist(Pattern.quote(topic)), 1, defaultStringDecoder, defaultStringDecoder - ); - - if (streams == null || streams.isEmpty()) { - throw new IAE("Topic [%s] had no streams", topic); - } - if (streams.size() > 1) { - throw new ISE("Topic [%s] has %d streams! expected 1", topic, streams.size()); - } - backgroundTaskCount.incrementAndGet(); - final KafkaStream kafkaStream = streams.get(0); - final ConsumerIterator it = kafkaStream.iterator(); - log.info("Listening to topic [%s] for namespace [%s]", topic, id); - AtomicLong eventCounter = topicEvents.get(id); - if (eventCounter == null) { - topicEvents.putIfAbsent(id, new AtomicLong(0L)); - eventCounter = topicEvents.get(id); - } - while (it.hasNext()) { - final MessageAndMetadata messageAndMetadata = it.next(); - final String key = messageAndMetadata.key(); - final String message = messageAndMetadata.message(); - if (key == null || message == null) { - log.error("Bad key/message from topic [%s]: [%s]", topic, messageAndMetadata); - continue; - } - map.put(key, message); - namespaceVersionMap.put(id, Long.toString(eventCounter.incrementAndGet())); - log.debug("Placed key[%s] val[%s]", key, message); - } - } - } - ); - futures.putIfAbsent(id, future); - Futures.addCallback( - future, new FutureCallback() - { - @Override - public void onSuccess(Object result) - { - topicEvents.remove(id); - futures.remove(id); - } - - @Override - public void onFailure(Throwable t) - { - topicEvents.remove(id); - futures.remove(id); - if (t instanceof java.util.concurrent.CancellationException) { - log.warn("Cancelled rename task for topic [%s]", topic); - } else { - Throwables.propagate(t); - } - } - }, - MoreExecutors.sameThreadExecutor() - ); - } - - public void removeListener(final String id) - { - Future future = futures.remove(id); - future.cancel(true); - } - - public void start() - { - // NO-OP - // all consumers are started through KafkaExtractionNamespaceFactory.getCachePopulator - } - - public void stop() - { - executorService.shutdown(); - Futures.allAsList(futures.values()).cancel(true); - } -} diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java deleted file mode 100644 index 990276be2337..000000000000 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java +++ /dev/null @@ -1,85 +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.namespace; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.google.inject.Inject; -import com.google.inject.name.Named; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.KafkaExtractionNamespace; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.Callable; - -/** - * - */ -public class KafkaExtractionNamespaceFactory implements ExtractionNamespaceCacheFactory -{ - private final List kafkaExtractionManagers; - private static final String KAFKA_VERSION = "kafka versions are updated every time a new event comes in"; - - @Inject - public KafkaExtractionNamespaceFactory( - @JacksonInject @Named("kafkaManagers") final List kafkaExtractionManagers - ) - { - this.kafkaExtractionManagers = kafkaExtractionManagers; - } - - // This only fires ONCE when the namespace is first added. The version is updated externally as events come in - @Override - public Callable getCachePopulator( - final String id, - final KafkaExtractionNamespace extractionNamespace, - final String unused, - final Map cache - ) - { - return new Callable() - { - @Override - public String call() - { - KafkaExtractionManager manager; - synchronized (kafkaExtractionManagers) { - manager = findAppropriate(extractionNamespace.getKafkaProperties()); - if (manager == null) { - manager = new KafkaExtractionManager(extractionNamespace.getKafkaProperties()); - kafkaExtractionManagers.add(manager); - } - } - manager.addListener(id, extractionNamespace, cache); - return KAFKA_VERSION; - } - }; - } - - private KafkaExtractionManager findAppropriate(Properties kafkaProperties) - { - for (KafkaExtractionManager manager : kafkaExtractionManagers) { - if (manager.supports(kafkaProperties)) { - return manager; - } - } - return null; - } -} diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java deleted file mode 100644 index 50617c76e324..000000000000 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java +++ /dev/null @@ -1,66 +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.namespace; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.inject.Binder; -import com.google.inject.Provides; -import com.google.inject.name.Named; -import io.druid.guice.LazySingleton; -import io.druid.initialization.DruidModule; -import io.druid.query.extraction.namespace.KafkaExtractionNamespace; -import java.util.List; - -/** - * - */ -public class KafkaExtractionNamespaceModule implements DruidModule -{ - private static final String PROPERTIES_KEY = "druid.query.rename.kafka.properties"; - private List kafkaExtractionManagers = Lists.newLinkedList(); - - @Override - public List getJacksonModules() - { - return ImmutableList.of( - new SimpleModule("kafka-lookups").registerSubtypes(KafkaExtractionNamespace.class) - ); - } - - @Provides - @Named("kafkaManagers") - public List getManagers() - { - return kafkaExtractionManagers; - } - - @Override - public void configure(Binder binder) - { - NamespacedExtractionModule - .getNamespaceFactoryMapBinder(binder) - .addBinding(KafkaExtractionNamespace.class) - .to(KafkaExtractionNamespaceFactory.class) - .in(LazySingleton.class); - } -} diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java deleted file mode 100644 index 3fb95c95b66a..000000000000 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java +++ /dev/null @@ -1,63 +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.query.extraction.namespace; - -import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.Properties; - -/** - * - */ -public class KafkaExtractionNamespaceTest -{ - @Test - public void testReflectiveSerde() throws IOException - { - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.registerSubtypes(KafkaExtractionNamespace.class); - final String val = "{\"type\":\"kafka\",\"kafkaTopic\":\"testTopic\",\"kafkaProperties\":{\"zookeeper.session.timeout.ms\":\"10000\",\"zookeeper.sync.time.ms\":\"200\"}}"; - final ExtractionNamespace fn = - mapper.reader(ExtractionNamespace.class) - .readValue( - val - ); - Assert.assertEquals(val, mapper.writeValueAsString(fn)); - } - - @Test(expected = com.fasterxml.jackson.databind.JsonMappingException.class) - public void testMissingTopic() throws IOException - { - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.registerSubtypes(KafkaExtractionNamespace.class); - final String val = "{\"type\":\"kafka\" }"; - final ExtractionNamespace fn = - mapper.reader(ExtractionNamespace.class) - .readValue( - val - ); - Assert.assertEquals(val, mapper.writeValueAsString(fn)); - } - -} diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java deleted file mode 100644 index c14b9f15b340..000000000000 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java +++ /dev/null @@ -1,441 +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.query.extraction.namespace; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.io.Files; -import com.google.inject.Binder; -import com.google.inject.Inject; -import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.Module; -import com.google.inject.Provider; -import com.google.inject.TypeLiteral; -import com.google.inject.name.Names; -import com.metamx.common.ISE; -import com.metamx.common.StringUtils; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import io.druid.guice.GuiceInjectors; -import io.druid.initialization.Initialization; -import io.druid.query.extraction.NamespaceLookupExtractorFactory; -import io.druid.query.lookup.LookupExtractorFactory; -import io.druid.server.namespace.KafkaExtractionManager; -import io.druid.server.namespace.KafkaExtractionNamespaceFactory; -import io.druid.server.namespace.KafkaExtractionNamespaceModule; -import io.druid.server.namespace.NamespacedExtractionModule; -import kafka.admin.AdminUtils; -import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; -import kafka.server.KafkaConfig; -import kafka.server.KafkaServer; -import kafka.utils.Time; -import kafka.utils.ZKStringSerializer$; -import org.I0Itec.zkclient.ZkClient; -import org.apache.commons.io.FileUtils; -import org.apache.curator.test.TestingServer; -import org.apache.zookeeper.CreateMode; -import org.joda.time.DateTime; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -import java.io.File; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ConcurrentMap; - -/** - * - */ -public class TestKafkaExtractionCluster -{ - private static final Logger log = new Logger(TestKafkaExtractionCluster.class); - - private static final Lifecycle lifecycle = new Lifecycle(); - private static final File tmpDir = Files.createTempDir(); - private static final String topicName = "testTopic"; - private static String namespace = "some_name"; - private static final Properties kafkaProperties = new Properties(); - - private KafkaServer kafkaServer; - private KafkaConfig kafkaConfig; - private TestingServer zkTestServer; - private ZkClient zkClient; - private Injector injector; - private ObjectMapper mapper; - private LookupExtractorFactory factory; - private KafkaExtractionManager renameManager; - - public static class KafkaFactoryProvider implements Provider> - { - private final List kafkaExtractionManager; - - @Inject - public KafkaFactoryProvider( - KafkaExtractionManager kafkaExtractionManager - ) - { - this.kafkaExtractionManager = Lists.newArrayList(kafkaExtractionManager); - } - - @Override - public ExtractionNamespaceCacheFactory get() - { - return new KafkaExtractionNamespaceFactory(kafkaExtractionManager); - } - } - - @Before - public void setUp() throws Exception - { - zkTestServer = new TestingServer(-1, new File(tmpDir.getAbsolutePath() + "/zk"), true); - zkTestServer.start(); - - zkClient = new ZkClient( - zkTestServer.getConnectString(), - 10000, - 10000, - ZKStringSerializer$.MODULE$ - ); - if (!zkClient.exists("/kafka")) { - zkClient.create("/kafka", null, CreateMode.PERSISTENT); - } - - - log.info("---------------------------Started ZK---------------------------"); - - - final Properties serverProperties = new Properties(); - serverProperties.putAll(kafkaProperties); - serverProperties.put("broker.id", "0"); - serverProperties.put("log.dir", tmpDir.getAbsolutePath() + "/log"); - serverProperties.put("log.cleaner.enable", "true"); - serverProperties.put("host.name", "127.0.0.1"); - serverProperties.put("zookeeper.connect", zkTestServer.getConnectString() + "/kafka"); - serverProperties.put("zookeeper.session.timeout.ms", "10000"); - serverProperties.put("zookeeper.sync.time.ms", "200"); - - kafkaConfig = new KafkaConfig(serverProperties); - - final long time = DateTime.parse("2015-01-01").getMillis(); - kafkaServer = new KafkaServer( - kafkaConfig, - new Time() - { - - @Override - public long milliseconds() - { - return time; - } - - @Override - public long nanoseconds() - { - return milliseconds() * 1_000_000; - } - - @Override - public void sleep(long ms) - { - try { - Thread.sleep(ms); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - } - } - ); - kafkaServer.startup(); - - int sleepCount = 0; - - while (!kafkaServer.kafkaController().isActive()) { - Thread.sleep(100); - if (++sleepCount > 10) { - throw new InterruptedException("Controller took to long to awaken"); - } - } - - log.info("---------------------------Started Kafka Server---------------------------"); - - ZkClient zkClient = new ZkClient( - zkTestServer.getConnectString() + "/kafka", 10000, 10000, - ZKStringSerializer$.MODULE$ - ); - - try { - final Properties topicProperties = new Properties(); - topicProperties.put("cleanup.policy", "compact"); - if (!AdminUtils.topicExists(zkClient, topicName)) { - AdminUtils.createTopic(zkClient, topicName, 1, 1, topicProperties); - } - - log.info("---------------------------Created topic---------------------------"); - - Assert.assertTrue(AdminUtils.topicExists(zkClient, topicName)); - } - finally { - zkClient.close(); - } - - final Properties kafkaProducerProperties = makeProducerProperties(); - Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties)); - - try { - producer.send( - new KeyedMessage<>( - topicName, - StringUtils.toUtf8("abcdefg"), - StringUtils.toUtf8("abcdefg") - ) - ); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - finally { - producer.close(); - } - - System.setProperty("druid.extensions.searchCurrentClassloader", "false"); - - injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjectorWithModules( - ImmutableList.of() - ), - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); - } - }, - new NamespacedExtractionModule(), - new KafkaExtractionNamespaceModule() - ) - ); - mapper = injector.getInstance(ObjectMapper.class); - - log.info("--------------------------- placed default item via producer ---------------------------"); - final Properties consumerProperties = new Properties(kafkaProperties); - consumerProperties.put("zookeeper.connect", zkTestServer.getConnectString() + "/kafka"); - consumerProperties.put("zookeeper.session.timeout.ms", "10000"); - consumerProperties.put("zookeeper.sync.time.ms", "200"); - ExtractionNamespace extractionNamespace = new KafkaExtractionNamespace(topicName, consumerProperties); - String namespaceString = mapper.writeValueAsString(extractionNamespace); - String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString); - factory = mapper.readValue(json, LookupExtractorFactory.class); - factory.start(); - renameManager = injector.getInstance( - Key.get( - new TypeLiteral>() - { - }, - Names.named("kafkaManagers") - ) - ).get(0); - - log.info("--------------------------- started rename manager ---------------------------"); - } - - @After - public void tearDown() throws Exception - { - - lifecycle.stop(); - if (renameManager != null) - { - renameManager.stop(); - } - factory.close(); - - if (null != kafkaServer) { - kafkaServer.shutdown(); - kafkaServer.awaitShutdown(); - } - - if (null != zkClient) { - if (zkClient.exists("/kafka")) { - try { - zkClient.deleteRecursive("/kafka"); - } - catch (org.I0Itec.zkclient.exception.ZkException ex) { - log.warn(ex, "error deleting /kafka zk node"); - } - } - zkClient.close(); - } - if (null != zkTestServer) { - zkTestServer.stop(); - } - if (tmpDir.exists()) { - FileUtils.deleteDirectory(tmpDir); - } - } - - private final Properties makeProducerProperties() - { - final Properties kafkaProducerProperties = new Properties(); - kafkaProducerProperties.putAll(kafkaProperties); - kafkaProducerProperties.put( - "metadata.broker.list", - String.format("127.0.0.1:%d", kafkaServer.socketServer().port()) - ); - kafkaProperties.put("request.required.acks", "1"); - return kafkaProducerProperties; - } - - private void checkServer() - { - if (!kafkaServer.apis().controller().isActive()) { - throw new ISE("server is not active!"); - } - } - - @Ignore // Should be un-ignored after kafka rewrite to Lookups is merged - @Test(timeout = 60_000L) - public void testSimpleRename() throws InterruptedException - { - final Properties kafkaProducerProperties = makeProducerProperties(); - final Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties)); - - try { - checkServer(); - - final ConcurrentMap> fnFn = - injector.getInstance( - Key.get( - new TypeLiteral>>() - { - }, - Names.named("namespaceExtractionFunctionCache") - ) - ); - - final ConcurrentMap>> reverseFn = - injector.getInstance( - Key.get( - new TypeLiteral>>>() - { - }, - Names.named("namespaceReverseExtractionFunctionCache") - ) - ); - - assertUpdated(null, namespace, "foo", fnFn); - assertReverseUpdated(Collections.EMPTY_LIST, namespace, "foo", reverseFn); - - long events = renameManager.getNumEvents(namespace); - - log.info("------------------------- Sending foo bar -------------------------------"); - producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("foo"), StringUtils.toUtf8("bar"))); - - long start = System.currentTimeMillis(); - while (events == renameManager.getNumEvents(namespace)) { - Thread.sleep(100); - if (System.currentTimeMillis() > start + 60_000) { - throw new ISE("Took too long to update event"); - } - } - - log.info("------------------------- Checking foo bar -------------------------------"); - assertUpdated("bar", namespace, "foo", fnFn); - assertReverseUpdated(Arrays.asList("foo"), namespace, "bar", reverseFn); - assertUpdated(null, namespace, "baz", fnFn); - - checkServer(); - events = renameManager.getNumEvents(namespace); - - log.info("------------------------- Sending baz bat -------------------------------"); - producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("baz"), StringUtils.toUtf8("bat"))); - while (events == renameManager.getNumEvents(namespace)) { - Thread.sleep(10); - if (System.currentTimeMillis() > start + 60_000) { - throw new ISE("Took too long to update event"); - } - } - - log.info("------------------------- Checking baz bat -------------------------------"); - Assert.assertEquals("bat", fnFn.get(namespace).apply("baz")); - Assert.assertEquals(Arrays.asList("baz"), reverseFn.get(namespace).apply("bat")); - } - finally { - producer.close(); - } - } - - private void assertUpdated( - String expected, - String namespace, - String key, - ConcurrentMap> lookup - ) - throws InterruptedException - { - Function extractionFn = lookup.get(namespace); - - if (expected == null) { - while (extractionFn.apply(key) != null) { - Thread.sleep(100); - extractionFn = lookup.get(namespace); - } - } else { - while (!expected.equals(extractionFn.apply(key))) { - Thread.sleep(100); - extractionFn = lookup.get(namespace); - } - } - - Assert.assertEquals("update check", expected, extractionFn.apply(key)); - } - - private void assertReverseUpdated( - List expected, - String namespace, - String key, - ConcurrentMap>> lookup - ) - throws InterruptedException - { - Function> extractionFn = lookup.get(namespace); - - while (!extractionFn.apply(key).equals(expected)) { - Thread.sleep(100); - extractionFn = lookup.get(namespace); - } - - Assert.assertEquals("update check", expected, extractionFn.apply(key)); - } -} From c23e06d139143cef1c520e5105e00ee55439a98e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 5 May 2016 14:03:48 -0700 Subject: [PATCH 08/42] Fix start and stop behavior to be consistent with new javadocs --- .../query/extraction/NamespaceLookupExtractorFactory.java | 4 ++-- .../query/extraction/NamespaceLookupExtractorFactoryTest.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index 499bf41b2c57..17a17cadde77 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -112,7 +112,7 @@ public boolean start() try { if (!started.compareAndSet(false, true)) { LOG.warn("Already started!"); - return false; + return true; } if (!manager.scheduleAndWait(extractorID, extractionNamespace, SCHEDULE_TIMEOUT)) { LOG.warn("Failed to schedule lookup [%s]", extractorID); @@ -134,7 +134,7 @@ public boolean close() try { if (!started.compareAndSet(true, false)) { LOG.warn("Not started!"); - return false; + return true; } return manager.checkedDelete(extractorID); } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java index 5d4af83a8dc9..967283cb75d3 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java @@ -160,7 +160,7 @@ public long getPollMs() ); Assert.assertTrue(namespaceLookupExtractorFactory.start()); Assert.assertTrue(namespaceLookupExtractorFactory.close()); - Assert.assertFalse(namespaceLookupExtractorFactory.close()); + Assert.assertTrue(namespaceLookupExtractorFactory.close()); EasyMock.verify(cacheManager); } @@ -187,7 +187,7 @@ public long getPollMs() cacheManager ); Assert.assertTrue(namespaceLookupExtractorFactory.start()); - Assert.assertFalse(namespaceLookupExtractorFactory.start()); + Assert.assertTrue(namespaceLookupExtractorFactory.start()); EasyMock.verify(cacheManager); } From 1b3e6cc9b0c39da685fa9e018bfad6ac1f52c4ba Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 07:48:08 -0700 Subject: [PATCH 09/42] Remove unused strings --- .../namespace/NamespacedExtractionModule.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java index 7eeb42802971..5e3185ac3ecd 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java @@ -25,12 +25,8 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Key; -import com.google.inject.Provides; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; -import com.google.inject.name.Named; -import com.metamx.common.IAE; -import io.druid.guice.Jerseys; import io.druid.guice.LazySingleton; import io.druid.guice.PolyBind; import io.druid.initialization.DruidModule; @@ -42,8 +38,6 @@ import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager; - -import javax.annotation.Nullable; import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -55,17 +49,6 @@ public class NamespacedExtractionModule implements DruidModule { private static final String TYPE_PREFIX = "druid.query.extraction.namespace.cache.type"; - public static final String NAMESPACE_VERSION_MAP = "namespaceVersionMap"; - public static final String NAMESPACE_EXTRACTION_FUNCTION_CACHE = "namespaceExtractionFunctionCache"; - public static final String NAMESPACE_REVERSE_EXTRACTION_FUNCTION_CACHE = "namespaceReverseExtractionFunctionCache"; - public static final String DIM_EXTRACTION_NAMESPACE = "dimExtractionNamespace"; - public static final String DIM_REVERSE_EXTRACTION_NAMESPACE = "dimReverseExtractionNamespace"; - - public static final String EXTRACTION_CACHE_MANAGER = "DruidExtractionCacheManager"; - - private final ConcurrentMap> fnCache = new ConcurrentHashMap<>(); - private final ConcurrentMap>> reverseFnCache = new ConcurrentHashMap<>(); - @Override public List getJacksonModules() { From 8a77bc74b76628ac750cd32e38f2eb6087ae0690 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 08:11:51 -0700 Subject: [PATCH 10/42] Add timeout option --- .../NamespaceLookupExtractorFactory.java | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index 17a17cadde77..a1247fb3e725 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -51,19 +51,21 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory { private static final Logger LOG = new Logger(NamespaceLookupExtractorFactory.class); - private static long SCHEDULE_TIMEOUT = 60_000; + private static final long DEFAULT_SCHEDULE_TIMEOUT = 60_000; private final AtomicBoolean started = new AtomicBoolean(false); private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); private final ExtractionNamespace extractionNamespace; private final NamespaceExtractionCacheManager manager; private final LookupIntrospectHandler lookupIntrospectHandler; + private final long firstCacheTimeout; private final String extractorID; @JsonCreator public NamespaceLookupExtractorFactory( @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace, + @JsonProperty("firstCacheTimeout") Long firstCacheTimeout, @JacksonInject NamespaceExtractionCacheManager manager ) { @@ -71,6 +73,8 @@ public NamespaceLookupExtractorFactory( extractionNamespace, "extractionNamespace should be specified" ); + this.firstCacheTimeout = firstCacheTimeout == null ? DEFAULT_SCHEDULE_TIMEOUT : firstCacheTimeout; + Preconditions.checkArgument(this.firstCacheTimeout >= 0); this.manager = manager; this.extractorID = buildID(); this.lookupIntrospectHandler = new LookupIntrospectHandler() { @@ -104,6 +108,13 @@ private Map getLatest() }; } + public NamespaceLookupExtractorFactory( + ExtractionNamespace extractionNamespace, + NamespaceExtractionCacheManager manager + ) { + this(extractionNamespace, null, manager); + } + @Override public boolean start() { @@ -114,7 +125,7 @@ public boolean start() LOG.warn("Already started!"); return true; } - if (!manager.scheduleAndWait(extractorID, extractionNamespace, SCHEDULE_TIMEOUT)) { + if (!manager.scheduleAndWait(extractorID, extractionNamespace, firstCacheTimeout)) { LOG.warn("Failed to schedule lookup [%s]", extractorID); return false; } @@ -165,6 +176,12 @@ public ExtractionNamespace getExtractionNamespace() return extractionNamespace; } + @JsonProperty + public long getFirstCacheTimeout() + { + return firstCacheTimeout; + } + private String buildID() { return UUID.randomUUID().toString(); From d7804681e50b24bd42d1619eb106a72f59acb75d Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 09:13:31 -0700 Subject: [PATCH 11/42] Address comments on configurations and improve docs --- .../extensions-core/namespaced-lookup.md | 33 ++++++++++++++----- .../NamespaceLookupExtractorFactory.java | 2 +- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index b48f7a88a2cc..815b6d923e72 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -13,14 +13,15 @@ Make sure to [include](../../operations/including-extensions.html) `druid-namesp ## Configuration Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size, -or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers. -Namespaced lookups can be specified as part of the runtime properties file. The property is a list of the namespaces -described as per the sections on this page. For example: +or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers, +and are small enough to reasonably populate on a node. This usually means tens to tens of thousands of entries per lookup. + +Namespaced lookups can be specified as part of the [cluster wide config for lookups](../../querying/lookups.html) as a type of `cachedNamespace` ```json - druid.query.extraction.namespace.lookups= - [ - { + { + "type": "cachedNamespace", + "extractionNamespace": { "type": "uri", "namespace": "some_uri_lookup", "uri": "file:/tmp/prefix/", @@ -33,7 +34,14 @@ described as per the sections on this page. For example: }, "pollPeriod": "PT5M" }, - { + "firstCacheTimeout": 0 + } + ``` + + ```json +{ + "type": "cachedNamespace", + "extractionNamespace": { "type": "jdbc", "namespace": "some_jdbc_lookup", "connectorConfig": { @@ -46,10 +54,17 @@ described as per the sections on this page. For example: "keyColumn": "mykeyColumn", "valueColumn": "MyValueColumn", "tsColumn": "timeColumn" - } - ] + }, + "firstCacheTimeout": 120000 +} ``` +The parameters are as follows +|Property|Description|Required|Default| +|--------|-----------|--------|-------| +|`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-| +|`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`60000` (1 minute)| + Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes: `druid-namespace-lookup` diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index a1247fb3e725..88a63ab3cf88 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -46,7 +46,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -@JsonTypeName("namespace") +@JsonTypeName("cachedNamespace") public class NamespaceLookupExtractorFactory implements LookupExtractorFactory { private static final Logger LOG = new Logger(NamespaceLookupExtractorFactory.class); From 4216aa842c66aa69ef1d0b703fcc97c5191bddc8 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 09:28:21 -0700 Subject: [PATCH 12/42] Add more options and update hash key and replaces --- .../NamespaceLookupExtractorFactory.java | 59 +++++++++++++++---- 1 file changed, 46 insertions(+), 13 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index 88a63ab3cf88..6f61f96d085b 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.metamx.common.ISE; import com.metamx.common.StringUtils; import com.metamx.common.logger.Logger; @@ -31,13 +32,8 @@ import io.druid.query.lookup.LookupExtractorFactory; import io.druid.query.lookup.LookupIntrospectHandler; import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; - -import javax.annotation.Nullable; -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Map; import java.util.UUID; @@ -45,6 +41,12 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.annotation.Nullable; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; @JsonTypeName("cachedNamespace") public class NamespaceLookupExtractorFactory implements LookupExtractorFactory @@ -52,6 +54,19 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory private static final Logger LOG = new Logger(NamespaceLookupExtractorFactory.class); private static final long DEFAULT_SCHEDULE_TIMEOUT = 60_000; + private static final byte[] CLASS_CACHE_KEY; + + static { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + baos.write(StringUtils.toUtf8(NamespaceLookupExtractorFactory.class.getCanonicalName())); + baos.write(0xFF); + CLASS_CACHE_KEY = baos.toByteArray(); + } + catch (IOException e) { + // Should never happen + throw Throwables.propagate(e); + } + } private final AtomicBoolean started = new AtomicBoolean(false); private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); @@ -59,6 +74,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory private final NamespaceExtractionCacheManager manager; private final LookupIntrospectHandler lookupIntrospectHandler; private final long firstCacheTimeout; + private final boolean oneToOne; private final String extractorID; @@ -66,6 +82,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory public NamespaceLookupExtractorFactory( @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace, @JsonProperty("firstCacheTimeout") Long firstCacheTimeout, + @JsonProperty("oneToOne") boolean oneToOne, @JacksonInject NamespaceExtractionCacheManager manager ) { @@ -75,9 +92,11 @@ public NamespaceLookupExtractorFactory( ); this.firstCacheTimeout = firstCacheTimeout == null ? DEFAULT_SCHEDULE_TIMEOUT : firstCacheTimeout; Preconditions.checkArgument(this.firstCacheTimeout >= 0); + this.oneToOne = oneToOne; this.manager = manager; this.extractorID = buildID(); - this.lookupIntrospectHandler = new LookupIntrospectHandler() { + this.lookupIntrospectHandler = new LookupIntrospectHandler() + { @GET @Path("/keys") @Produces(MediaType.APPLICATION_JSON) @@ -103,7 +122,7 @@ public Response getMap() private Map getLatest() { - return ((MapLookupExtractor)get()).getMap(); + return ((MapLookupExtractor) get()).getMap(); } }; } @@ -111,8 +130,9 @@ private Map getLatest() public NamespaceLookupExtractorFactory( ExtractionNamespace extractionNamespace, NamespaceExtractionCacheManager manager - ) { - this(extractionNamespace, null, manager); + ) + { + this(extractionNamespace, null, false, manager); } @Override @@ -159,6 +179,12 @@ public boolean replaces(@Nullable LookupExtractorFactory other) { if (other != null && other instanceof NamespaceLookupExtractorFactory) { NamespaceLookupExtractorFactory that = (NamespaceLookupExtractorFactory) other; + if (isOneToOne() != ((NamespaceLookupExtractorFactory) other).isOneToOne()) { + return true; + } + if (getFirstCacheTimeout() != ((NamespaceLookupExtractorFactory) other).getFirstCacheTimeout()) { + return true; + } return !extractionNamespace.equals(that.extractionNamespace); } return true; @@ -182,6 +208,12 @@ public long getFirstCacheTimeout() return firstCacheTimeout; } + @JsonProperty + public boolean isOneToOne() + { + return oneToOne; + } + private String buildID() { return UUID.randomUUID().toString(); @@ -215,17 +247,18 @@ public LookupExtractor get() } while (!preVersion.equals(postVersion)); final byte[] v = StringUtils.toUtf8(postVersion); final byte[] id = StringUtils.toUtf8(extractorID); - return new MapLookupExtractor(map, false) + return new MapLookupExtractor(map, isOneToOne()) { @Override public byte[] getCacheKey() { return ByteBuffer - .allocate(id.length + 1 + v.length + 1) + .allocate(CLASS_CACHE_KEY.length + id.length + 1 + v.length + 1 + 1) .put(id) .put((byte) 0xFF) .put(v) .put((byte) 0xFF) + .put(isOneToOne() ? (byte) 1 : (byte) 0) .array(); } }; From fe4418252ac04cfd9e21dc2a150e5e1003e0ee7f Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 10:27:31 -0700 Subject: [PATCH 13/42] Move monitoring to the overriding classes --- .../NamespaceExtractionCacheManager.java | 12 +++++-- ...ffHeapNamespaceExtractionCacheManager.java | 13 +++++--- ...OnHeapNamespaceExtractionCacheManager.java | 33 +++++++++++++++++-- 3 files changed, 47 insertions(+), 11 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java index 424e262933c6..b573263a3ede 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java @@ -37,7 +37,6 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; - import java.util.Collection; import java.util.Map; import java.util.UUID; @@ -84,7 +83,6 @@ public NamespaceImplData( private final ListeningScheduledExecutorService listeningScheduledExecutorService; protected final ConcurrentMap implData = new ConcurrentHashMap<>(); protected final AtomicLong tasksStarted = new AtomicLong(0); - protected final AtomicLong dataSize = new AtomicLong(0); protected final ServiceEmitter serviceEmitter; private final ConcurrentHashMap lastVersion = new ConcurrentHashMap<>(); private final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap; @@ -118,12 +116,12 @@ public void run() { try { final long tasks = tasksStarted.get(); - serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/size", dataSize.get())); serviceEmitter.emit( ServiceMetricEvent.builder() .build("namespace/deltaTasksStarted", tasks - priorTasksStarted) ); priorTasksStarted = tasks; + monitor(serviceEmitter); } catch (Exception e) { log.error(e, "Error emitting namespace stats"); @@ -138,6 +136,14 @@ public void run() ); } + /** + * Optional monitoring for overriding classes. `super.monitor` does *NOT* need to be called by overriding methods + * @param serviceEmitter The emitter to emit to + */ + protected void monitor(ServiceEmitter serviceEmitter) { + // Noop by default + } + protected boolean waitForServiceToEnd(long time, TimeUnit unit) throws InterruptedException { return listeningScheduledExecutorService.awaitTermination(time, unit); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index a2b5b8eda079..e3a0a71bae74 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -26,8 +26,10 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; +import java.util.concurrent.atomic.AtomicLong; import org.mapdb.DB; import org.mapdb.DBMaker; @@ -114,10 +116,8 @@ protected boolean swapAndClearCache(String namespaceKey, String cacheKey) if (priorCache != null) { // TODO: resolve what happens here if query is actively going on mmapDB.delete(priorCache); - dataSize.set(tmpFile.length()); return true; } else { - dataSize.set(tmpFile.length()); return false; } } @@ -129,7 +129,6 @@ protected boolean swapAndClearCache(String namespaceKey, String cacheKey) @Override public boolean delete(final String namespaceKey) { - final Lock lock = nsLocks.get(namespaceKey); lock.lock(); try { @@ -138,8 +137,7 @@ public boolean delete(final String namespaceKey) if (mmapDBkey != null) { final long pre = tmpFile.length(); mmapDB.delete(mmapDBkey); - dataSize.set(tmpFile.length()); - log.debug("MapDB file size: pre %d post %d", pre, dataSize.get()); + log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length()); return true; } else { return false; @@ -177,4 +175,9 @@ public ConcurrentMap getCacheMap(String namespaceOrCacheKey) lock.unlock(); } } + + @Override + protected void monitor(ServiceEmitter serviceEmitter) { + serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/cache/diskSize", tmpFile.length())); + } } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java index 7f38cee2f986..c46f60c8baa9 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java @@ -19,14 +19,16 @@ package io.druid.server.namespace.cache; +import com.google.common.primitives.Chars; import com.google.common.util.concurrent.Striped; import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; - import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -37,6 +39,7 @@ */ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCacheManager { + private static final Logger LOG = new Logger(OnHeapNamespaceExtractionCacheManager.class); private final ConcurrentMap> mapMap = new ConcurrentHashMap<>(); private final Striped nsLocks = Striped.lock(32); @@ -60,11 +63,9 @@ protected boolean swapAndClearCache(String namespaceKey, String cacheKey) if (cacheMap == null) { throw new IAE("Extraction Cache [%s] does not exist", cacheKey); } - dataSize.addAndGet(cacheMap.size()); ConcurrentMap prior = mapMap.put(namespaceKey, cacheMap); mapMap.remove(cacheKey); if (prior != null) { - dataSize.addAndGet(-prior.size()); // Old map will get GC'd when it is not used anymore return true; } else { @@ -100,4 +101,30 @@ public boolean delete(final String namespaceKey) lock.unlock(); } } + + @Override + protected void monitor(ServiceEmitter serviceEmitter) + { + long numEntries = 0; + long size = 0; + for (Map.Entry> entry : mapMap.entrySet()) { + final ConcurrentMap map = entry.getValue(); + if (map == null) { + LOG.debug("missing cache key for reporting [%s]", entry.getKey()); + continue; + } + numEntries += map.size(); + for (Map.Entry sEntry : map.entrySet()) { + final String key = sEntry.getKey(); + final String value = sEntry.getValue(); + if (key == null || value == null) { + LOG.debug("Missing entries for cache key [%s]", entry.getKey()); + continue; + } + size += key.length() + value.length(); + } + } + serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/cache/numEntries", numEntries)); + serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/cache/heapSizeInBytes", size * Chars.BYTES)); + } } From 4f89413760fb57c41332ce70d4c4f6c54791abca Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 10:33:27 -0700 Subject: [PATCH 14/42] Add better start/stop logging --- .../query/extraction/NamespaceLookupExtractorFactory.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index 6f61f96d085b..c12a432cc6d6 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -142,7 +142,7 @@ public boolean start() writeLock.lock(); try { if (!started.compareAndSet(false, true)) { - LOG.warn("Already started!"); + LOG.warn("Already started! [%s]", extractorID); return true; } if (!manager.scheduleAndWait(extractorID, extractionNamespace, firstCacheTimeout)) { @@ -164,7 +164,7 @@ public boolean close() writeLock.lock(); try { if (!started.compareAndSet(true, false)) { - LOG.warn("Not started!"); + LOG.warn("Not started! [%s]", extractorID); return true; } return manager.checkedDelete(extractorID); From 99e8ac28cc82b21dac1a02ea5b000942007419cf Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 16:17:47 -0700 Subject: [PATCH 15/42] Remove old docs about namespace names --- docs/content/development/extensions-core/namespaced-lookup.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index 815b6d923e72..71d02d59101e 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -91,7 +91,6 @@ The remapping values for each namespaced lookup can be specified by a json objec ```json { "type":"uri", - "namespace":"some_lookup", "uri": "s3://bucket/some/key/prefix/renames-0003.gz", "namespaceParseSpec":{ "format":"csv", @@ -104,7 +103,6 @@ The remapping values for each namespaced lookup can be specified by a json objec ```json { "type":"uri", - "namespace":"some_lookup", "uriPrefix": "s3://bucket/some/key/prefix/", "fileRegex":"renames-[0-9]*\\.gz", "namespaceParseSpec":{ From 4313a7937d5ec31450006e5b0f20c7d9e555bb76 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 May 2016 16:20:22 -0700 Subject: [PATCH 16/42] Fix bad comma --- docs/content/development/extensions-core/namespaced-lookup.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index 71d02d59101e..09ea6a15459b 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -96,7 +96,7 @@ The remapping values for each namespaced lookup can be specified by a json objec "format":"csv", "columns":["key","value"] }, - "pollPeriod":"PT5M", + "pollPeriod":"PT5M" } ``` @@ -109,7 +109,7 @@ The remapping values for each namespaced lookup can be specified by a json objec "format":"csv", "columns":["key","value"] }, - "pollPeriod":"PT5M", + "pollPeriod":"PT5M" } ``` |Property|Description|Required|Default| From a7b35ce60662acb45c10ea20575003a8f48525c0 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 10 May 2016 07:38:18 -0700 Subject: [PATCH 17/42] Add `@JsonIgnore` to lookup factory --- .../NamespaceLookupExtractorFactory.java | 7 +++- .../NamespaceLookupExtractorFactoryTest.java | 39 ++++++++++++++++++- 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java index c12a432cc6d6..1acf8bc4c584 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; @@ -68,11 +69,15 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory } } + @JsonIgnore private final AtomicBoolean started = new AtomicBoolean(false); + @JsonIgnore private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); - private final ExtractionNamespace extractionNamespace; + @JsonIgnore private final NamespaceExtractionCacheManager manager; + @JsonIgnore private final LookupIntrospectHandler lookupIntrospectHandler; + private final ExtractionNamespace extractionNamespace; private final long firstCacheTimeout; private final boolean oneToOne; diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java index 967283cb75d3..aed4224027d0 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java @@ -23,13 +23,25 @@ import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; import com.metamx.common.ISE; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Self; +import io.druid.guice.annotations.Smile; +import io.druid.initialization.Initialization; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.extraction.namespace.ExtractionNamespace; import io.druid.query.extraction.namespace.URIExtractionNamespace; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractorFactory; +import io.druid.server.DruidNode; import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; import org.easymock.EasyMock; import org.joda.time.Period; @@ -402,7 +414,6 @@ public void testReplaces() EasyMock.verify(en1, en2); } - @Test(expected = ISE.class) public void testMustBeStarted() { @@ -422,4 +433,30 @@ public long getPollMs() namespaceLookupExtractorFactory.get(); } + + // Note this does NOT catch problems with returning factories as failed in error messages. + @Test + public void testSerDe() throws Exception + { + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null) + ); + } + } + ) + ); + final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + mapper.registerSubtypes(NamespaceLookupExtractorFactory.class); + final LookupExtractorFactory factory = mapper.readValue("{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }", LookupExtractorFactory.class); + Assert.assertTrue(factory instanceof NamespaceLookupExtractorFactory); + Assert.assertNotNull(mapper.writeValueAsString(factory)); + } } From b0379b93e2fd49ede32ed01bb7eeae1f6f5eb82c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 11 May 2016 08:38:35 -0700 Subject: [PATCH 18/42] Address code review comments --- .../extensions-core/namespaced-lookup.md | 17 ++++++++++++--- .../lookup/KafkaLookupExtractorFactory.java | 2 +- .../KafkaLookupExtractorFactoryTest.java | 4 ++-- .../lookup/TestKafkaExtractionCluster.java | 2 +- .../NamespaceLookupExtractorFactory.java | 13 ++++++------ .../namespace/ExtractionNamespace.java | 2 +- .../ExtractionNamespaceCacheFactory.java | 5 +---- .../namespace/JDBCExtractionNamespace.java | 2 +- .../namespace/URIExtractionNamespace.java | 2 +- .../JDBCExtractionNamespaceCacheFactory.java | 6 +++--- .../namespace/NamespacedExtractionModule.java | 21 ++++++++----------- .../URIExtractionNamespaceCacheFactory.java | 6 +++--- .../NamespaceExtractionCacheManager.java | 6 +++--- ...ffHeapNamespaceExtractionCacheManager.java | 15 +++++++------ ...OnHeapNamespaceExtractionCacheManager.java | 6 +++--- .../io.druid.initialization.DruidModule | 2 +- .../NamespaceLookupExtractorFactoryTest.java | 10 ++++----- .../namespace/URIExtractionNamespaceTest.java | 2 +- .../NamespacedExtractorModuleTest.java | 19 +++++++---------- ...RIExtractionNamespaceCacheFactoryTest.java | 18 ++++++++-------- .../cache/JDBCExtractionNamespaceTest.java | 16 +++++--------- ...ceExtractionCacheManagerExecutorsTest.java | 12 +++++------ .../NamespaceExtractionCacheManagersTest.java | 6 +++--- 23 files changed, 95 insertions(+), 99 deletions(-) rename extensions-core/namespace-lookup/src/main/java/io/druid/query/{extraction => lookup}/NamespaceLookupExtractorFactory.java (96%) rename extensions-core/namespace-lookup/src/main/java/io/druid/query/{extraction => lookup}/namespace/ExtractionNamespace.java (97%) rename extensions-core/namespace-lookup/src/main/java/io/druid/query/{extraction => lookup}/namespace/ExtractionNamespaceCacheFactory.java (95%) rename extensions-core/namespace-lookup/src/main/java/io/druid/query/{extraction => lookup}/namespace/JDBCExtractionNamespace.java (99%) rename extensions-core/namespace-lookup/src/main/java/io/druid/query/{extraction => lookup}/namespace/URIExtractionNamespace.java (99%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/{ => lookup}/namespace/JDBCExtractionNamespaceCacheFactory.java (96%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/{ => lookup}/namespace/NamespacedExtractionModule.java (80%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/{ => lookup}/namespace/URIExtractionNamespaceCacheFactory.java (97%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/{ => lookup}/namespace/cache/NamespaceExtractionCacheManager.java (98%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/{ => lookup}/namespace/cache/OffHeapNamespaceExtractionCacheManager.java (94%) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/{ => lookup}/namespace/cache/OnHeapNamespaceExtractionCacheManager.java (95%) rename extensions-core/namespace-lookup/src/test/java/io/druid/query/{extraction => lookup}/namespace/URIExtractionNamespaceTest.java (99%) rename extensions-core/namespace-lookup/src/test/java/io/druid/server/{ => lookup}/namespace/NamespacedExtractorModuleTest.java (90%) rename extensions-core/namespace-lookup/src/test/java/io/druid/server/{ => lookup}/namespace/URIExtractionNamespaceCacheFactoryTest.java (96%) rename extensions-core/namespace-lookup/src/test/java/io/druid/server/{ => lookup}/namespace/cache/JDBCExtractionNamespaceTest.java (96%) rename extensions-core/namespace-lookup/src/test/java/io/druid/server/{ => lookup}/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java (97%) rename extensions-core/namespace-lookup/src/test/java/io/druid/server/{ => lookup}/namespace/cache/NamespaceExtractionCacheManagersTest.java (96%) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index 98caec5ed90d..d8a4e8c16699 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -11,12 +11,17 @@ Lookups are an experimental featu Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` as an extension. ## Configuration +
+Static configuration is no longer supported. Only cluster wide configuration is supported +
-Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size, +Cached namespace lookups are appropriate for lookups which are not possible to pass at query time due to their size, or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers, and are small enough to reasonably populate on a node. This usually means tens to tens of thousands of entries per lookup. -Namespaced lookups can be specified as part of the [cluster wide config for lookups](../../querying/lookups.html) as a type of `cachedNamespace` +Cached namespace lookups all draw from the same cache pool, allowing each node to have a fixed cache pool that can be used by namespace lookups. + +Cached namespace lookups can be specified as part of the [cluster wide config for lookups](../../querying/lookups.html) as a type of `cachedNamespace` ```json { @@ -55,7 +60,8 @@ Namespaced lookups can be specified as part of the [cluster wide config for look "valueColumn": "MyValueColumn", "tsColumn": "timeColumn" }, - "firstCacheTimeout": 120000 + "firstCacheTimeout": 120000, + "oneToOne":true } ``` @@ -64,6 +70,7 @@ The parameters are as follows |--------|-----------|--------|-------| |`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-| |`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`60000` (1 minute)| +|`oneToOne`|If the underlying map is injective (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`| Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes: `druid-namespace-lookup` @@ -263,3 +270,7 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol "pollPeriod":600000 } ``` + +# Introspection + +Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. 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 286a8d0d538b..f40d234b9a64 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 @@ -37,7 +37,7 @@ import com.metamx.common.logger.Logger; import io.druid.concurrent.Execs; import io.druid.query.extraction.MapLookupExtractor; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import java.nio.ByteBuffer; import java.util.List; import java.util.Map; diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java index a8a40d0deafc..3f4ab3327993 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java @@ -28,7 +28,7 @@ import com.google.common.collect.ImmutableMap; import com.metamx.common.StringUtils; import io.druid.jackson.DefaultObjectMapper; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import kafka.consumer.ConsumerIterator; import kafka.consumer.KafkaStream; import kafka.consumer.TopicFilter; @@ -72,7 +72,7 @@ public Object findInjectableValue( Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance ) { - if ("io.druid.server.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) { + if ("io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) { return cacheManager; } else { return null; diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java index bcd1aebe9bb9..f99abc8e4b25 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java @@ -32,7 +32,7 @@ import com.metamx.common.logger.Logger; import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; -import io.druid.server.namespace.NamespacedExtractionModule; +import io.druid.server.lookup.namespace.NamespacedExtractionModule; import kafka.admin.AdminUtils; import kafka.javaapi.producer.Producer; import kafka.producer.KeyedMessage; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java similarity index 96% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index 1acf8bc4c584..824f90aec101 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -16,23 +16,23 @@ * specific language governing permissions and limitations * under the License. */ -package io.druid.query.extraction; + +package io.druid.query.lookup; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.metamx.common.ISE; import com.metamx.common.StringUtils; import com.metamx.common.logger.Logger; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.lookup.LookupExtractor; -import io.druid.query.lookup.LookupExtractorFactory; -import io.druid.query.lookup.LookupIntrospectHandler; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import io.druid.query.extraction.MapLookupExtractor; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -132,6 +132,7 @@ private Map getLatest() }; } + @VisibleForTesting public NamespaceLookupExtractorFactory( ExtractionNamespace extractionNamespace, NamespaceExtractionCacheManager manager diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java similarity index 97% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespace.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java index 34d8bd296c8d..2573d4fc420c 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.extraction.namespace; +package io.druid.query.lookup.namespace; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java similarity index 95% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceCacheFactory.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java index aa5406739e0d..c23aace46eba 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/ExtractionNamespaceCacheFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java @@ -17,11 +17,8 @@ * under the License. */ -package io.druid.query.extraction.namespace; +package io.druid.query.lookup.namespace; -import com.google.common.base.Function; - -import java.util.List; import java.util.Map; import java.util.concurrent.Callable; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/JDBCExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java similarity index 99% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/JDBCExtractionNamespace.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java index a594b28f67f2..9fb9c3c5de8c 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/JDBCExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.extraction.namespace; +package io.druid.query.lookup.namespace; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/URIExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java similarity index 99% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/URIExtractionNamespace.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java index b06a3a98b522..705c3ea614fe 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/namespace/URIExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.extraction.namespace; +package io.druid.query.lookup.namespace; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java similarity index 96% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceCacheFactory.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java index b3cbd98b9b6f..7a7ad0abba82 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/JDBCExtractionNamespaceCacheFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java @@ -17,13 +17,13 @@ * under the License. */ -package io.druid.server.namespace; +package io.druid.server.lookup.namespace; import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.JDBCExtractionNamespace; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.StatementContext; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java similarity index 80% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java index 5e3185ac3ecd..fba575cd684b 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java @@ -17,11 +17,10 @@ * under the License. */ -package io.druid.server.namespace; +package io.druid.server.lookup.namespace; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Key; @@ -30,17 +29,15 @@ import io.druid.guice.LazySingleton; import io.druid.guice.PolyBind; import io.druid.initialization.DruidModule; -import io.druid.query.extraction.NamespaceLookupExtractorFactory; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.JDBCExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespace; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; -import io.druid.server.namespace.cache.OffHeapNamespaceExtractionCacheManager; -import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager; +import io.druid.query.lookup.NamespaceLookupExtractorFactory; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java similarity index 97% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactory.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java index 92518178d672..2b7b244ff793 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.server.namespace; +package io.druid.server.lookup.namespace; import com.google.common.base.Throwables; import com.google.common.io.ByteSource; @@ -29,8 +29,8 @@ import io.druid.common.utils.JodaUtils; import io.druid.data.SearchableVersionedDataFinder; import io.druid.data.input.MapPopulator; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.URIExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.segment.loading.URIDataPuller; import java.io.FileNotFoundException; import java.io.IOException; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java similarity index 98% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java index b573263a3ede..608269ceb15b 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.server.namespace.cache; +package io.druid.server.lookup.namespace.cache; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; @@ -35,8 +35,8 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import java.util.Collection; import java.util.Map; import java.util.UUID; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java similarity index 94% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index e3a0a71bae74..1c9a232dc5c6 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.server.namespace.cache; +package io.druid.server.lookup.namespace.cache; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -27,12 +27,8 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import java.util.concurrent.atomic.AtomicLong; -import org.mapdb.DB; -import org.mapdb.DBMaker; - +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import java.io.File; import java.io.IOException; import java.util.Map; @@ -40,6 +36,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.Lock; +import org.mapdb.DB; +import org.mapdb.DBMaker; /** * @@ -177,7 +175,8 @@ public ConcurrentMap getCacheMap(String namespaceOrCacheKey) } @Override - protected void monitor(ServiceEmitter serviceEmitter) { + protected void monitor(ServiceEmitter serviceEmitter) + { serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/cache/diskSize", tmpFile.length())); } } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java similarity index 95% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java index c46f60c8baa9..fb5bb28e03ac 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/namespace/cache/OnHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.server.namespace.cache; +package io.druid.server.lookup.namespace.cache; import com.google.common.primitives.Chars; import com.google.common.util.concurrent.Striped; @@ -27,8 +27,8 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; diff --git a/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index 6466ed7e1535..cc999cac090f 100644 --- a/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -17,4 +17,4 @@ # under the License. # -io.druid.server.namespace.NamespacedExtractionModule +io.druid.server.lookup.namespace.NamespacedExtractionModule diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java index aed4224027d0..1eaa205134d5 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java @@ -34,15 +34,15 @@ import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Self; -import io.druid.guice.annotations.Smile; import io.druid.initialization.Initialization; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespace; +import io.druid.query.lookup.NamespaceLookupExtractorFactory; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractorFactory; import io.druid.server.DruidNode; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import org.easymock.EasyMock; import org.joda.time.Period; import org.junit.Assert; @@ -75,7 +75,7 @@ public Object findInjectableValue( Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance ) { - if ("io.druid.server.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) { + if ("io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) { return cacheManager; } return null; diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java similarity index 99% rename from extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java index daf5d788f82b..6d620b73e0b1 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.extraction.namespace; +package io.druid.query.lookup.namespace; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java similarity index 90% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java index 0d7881dba4d9..165b0e123ba4 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/NamespacedExtractorModuleTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java @@ -17,23 +17,22 @@ * under the License. */ -package io.druid.server.namespace; +package io.druid.server.lookup.namespace; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.metamx.common.lifecycle.Lifecycle; import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.JDBCExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; -import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; import org.joda.time.Period; import org.junit.AfterClass; import org.junit.Assert; @@ -47,8 +46,6 @@ import java.io.OutputStreamWriter; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java similarity index 96% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactoryTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java index 3b7c492a77e9..60f4c1b98b98 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/URIExtractionNamespaceCacheFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.server.namespace; +package io.druid.server.lookup.namespace; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; @@ -30,16 +30,16 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.URIExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManager; -import io.druid.server.namespace.cache.NamespaceExtractionCacheManagersTest; -import io.druid.server.namespace.cache.OffHeapNamespaceExtractionCacheManager; -import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManagersTest; +import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; +import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java similarity index 96% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java index cc868da3f28d..004370390d37 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/JDBCExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java @@ -17,14 +17,12 @@ * under the License. */ -package io.druid.server.namespace.cache; +package io.druid.server.lookup.namespace.cache; -import com.google.common.base.Function; import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Sets; import com.google.common.io.Closer; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -33,11 +31,11 @@ import com.metamx.common.logger.Logger; import io.druid.concurrent.Execs; import io.druid.metadata.TestDerbyConnector; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.JDBCExtractionNamespace; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.namespace.JDBCExtractionNamespaceCacheFactory; +import io.druid.server.lookup.namespace.JDBCExtractionNamespaceCacheFactory; import org.joda.time.Period; import org.junit.After; import org.junit.Assert; @@ -51,12 +49,8 @@ import java.io.Closeable; import java.io.IOException; import java.util.Collection; -import java.util.Collections; -import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java similarity index 97% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java index 593f4048a704..265dde10e4a2 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.server.namespace.cache; +package io.druid.server.lookup.namespace.cache; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; @@ -30,13 +30,13 @@ import com.metamx.common.lifecycle.Lifecycle; import io.druid.concurrent.Execs; import io.druid.data.SearchableVersionedDataFinder; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.extraction.namespace.URIExtractionNamespace; -import io.druid.query.extraction.namespace.URIExtractionNamespaceTest; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.namespace.URIExtractionNamespaceCacheFactory; +import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory; import org.joda.time.Period; import org.junit.After; import org.junit.Assert; diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java similarity index 96% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagersTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java index 9102af8ee901..e90c86fdea99 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.server.namespace.cache; +package io.druid.server.lookup.namespace.cache; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -25,8 +25,8 @@ import com.google.common.collect.Lists; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import io.druid.query.extraction.namespace.ExtractionNamespace; -import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; import org.junit.Assert; import org.junit.Before; From 15dc879da47633c3b6d2a447688cd1aee5827a7f Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 11 May 2016 09:09:17 -0700 Subject: [PATCH 19/42] Remove ExtractionNamespace from module json registration --- .../server/lookup/namespace/NamespacedExtractionModule.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java index fba575cd684b..f8578fb746fa 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java @@ -52,7 +52,6 @@ public List getJacksonModules() return ImmutableList.of( new SimpleModule("DruidNamespacedExtractionModule") .registerSubtypes( - ExtractionNamespace.class, NamespaceLookupExtractorFactory.class ) ); From ab2230caf1c4a6c67c5deb901bb66e6fea83a6f6 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 11 May 2016 19:38:58 -0700 Subject: [PATCH 20/42] Fix problems with naming and initialization. Add tests --- .../namespace/NamespacedExtractionModule.java | 6 +- .../NamespaceExtractionCacheManager.java | 7 --- ...ffHeapNamespaceExtractionCacheManager.java | 37 ++++++----- .../NamespaceLookupExtractorFactoryTest.java | 13 ++-- ...apNamespaceExtractionCacheManagerTest.java | 62 ++++++++++++++++++ ...apNamespaceExtractionCacheManagerTest.java | 63 +++++++++++++++++++ 6 files changed, 155 insertions(+), 33 deletions(-) rename extensions-core/namespace-lookup/src/test/java/io/druid/query/{extraction => lookup}/NamespaceLookupExtractorFactoryTest.java (96%) create mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java create mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java index f8578fb746fa..0e9332fc9135 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java @@ -44,7 +44,7 @@ */ public class NamespacedExtractionModule implements DruidModule { - private static final String TYPE_PREFIX = "druid.query.extraction.namespace.cache.type"; + public static final String TYPE_PREFIX = "druid.query.extraction.namespace.cache.type"; @Override public List getJacksonModules() @@ -80,12 +80,12 @@ public void configure(Binder binder) TYPE_PREFIX, Key.get(NamespaceExtractionCacheManager.class), Key.get(OnHeapNamespaceExtractionCacheManager.class), - "onheap" + "onHeap" ).in(LazySingleton.class); PolyBind .optionBinder(binder, Key.get(NamespaceExtractionCacheManager.class)) - .addBinding("offheap") + .addBinding("offHeap") .to(OffHeapNamespaceExtractionCacheManager.class) .in(LazySingleton.class); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java index 608269ceb15b..293e623215ed 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java @@ -19,8 +19,6 @@ package io.druid.server.lookup.namespace.cache; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Throwables; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -53,11 +51,6 @@ /** * */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = OnHeapNamespaceExtractionCacheManager.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "offHeap", value = OffHeapNamespaceExtractionCacheManager.class), - @JsonSubTypes.Type(name = "onHeap", value = OnHeapNamespaceExtractionCacheManager.class) -}) public abstract class NamespaceExtractionCacheManager { protected static class NamespaceImplData diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index 1c9a232dc5c6..0603f6998234 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -76,27 +76,32 @@ public OffHeapNamespaceExtractionCacheManager( .commitFileSyncDisable() .cacheSize(10_000_000) .make(); - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception + try { + lifecycle.addMaybeStartHandler( + new Lifecycle.Handler() { - // NOOP - } + @Override + public void start() throws Exception + { + // NOOP + } - @Override - public void stop() - { - if (!mmapDB.isClosed()) { - mmapDB.close(); - if (!tmpFile.delete()) { - log.warn("Unable to delete file at [%s]", tmpFile.getAbsolutePath()); + @Override + public void stop() + { + if (!mmapDB.isClosed()) { + mmapDB.close(); + if (!tmpFile.delete()) { + log.warn("Unable to delete file at [%s]", tmpFile.getAbsolutePath()); + } } } } - } - ); + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } } @Override diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java similarity index 96% rename from extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java rename to extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java index 1eaa205134d5..1f6b2a1a6af9 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.extraction; +package io.druid.query.lookup; import com.fasterxml.jackson.databind.BeanProperty; import com.fasterxml.jackson.databind.DeserializationContext; @@ -36,13 +36,11 @@ import io.druid.guice.annotations.Self; import io.druid.initialization.Initialization; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.lookup.NamespaceLookupExtractorFactory; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespace; -import io.druid.query.lookup.LookupExtractor; -import io.druid.query.lookup.LookupExtractorFactory; import io.druid.server.DruidNode; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; +import java.util.concurrent.ConcurrentHashMap; import org.easymock.EasyMock; import org.joda.time.Period; import org.junit.Assert; @@ -52,8 +50,6 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; -import java.util.concurrent.ConcurrentHashMap; - public class NamespaceLookupExtractorFactoryTest { private final ObjectMapper mapper = new DefaultObjectMapper(); @@ -455,7 +451,10 @@ public void configure(Binder binder) ); final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); mapper.registerSubtypes(NamespaceLookupExtractorFactory.class); - final LookupExtractorFactory factory = mapper.readValue("{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }", LookupExtractorFactory.class); + final LookupExtractorFactory factory = mapper.readValue( + "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }", + LookupExtractorFactory.class + ); Assert.assertTrue(factory instanceof NamespaceLookupExtractorFactory); Assert.assertNotNull(mapper.writeValueAsString(factory)); } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java new file mode 100644 index 000000000000..3418b7e7d0c6 --- /dev/null +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java @@ -0,0 +1,62 @@ +/* + * 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.namespace.cache; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; +import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; +import io.druid.server.lookup.namespace.NamespacedExtractionModule; +import java.util.Properties; +import org.junit.Assert; +import org.junit.Test; + +public class OffHeapNamespaceExtractionCacheManagerTest +{ + @Test + public void testInjection() + { + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module(){ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null) + ); + } + } + ) + ); + final Properties properties = injector.getInstance(Properties.class); + properties.clear(); + properties.put(NamespacedExtractionModule.TYPE_PREFIX, "offHeap"); + final NamespaceExtractionCacheManager manager = injector.getInstance(NamespaceExtractionCacheManager.class); + Assert.assertEquals(OffHeapNamespaceExtractionCacheManager.class, manager.getClass()); + } +} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java new file mode 100644 index 000000000000..373fac8b3ca8 --- /dev/null +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java @@ -0,0 +1,63 @@ +/* + * 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.namespace.cache; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; +import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; +import io.druid.server.lookup.namespace.NamespacedExtractionModule; +import java.util.Properties; +import org.junit.Assert; +import org.junit.Test; + +public class OnHeapNamespaceExtractionCacheManagerTest +{ + @Test + public void testInjection() + { + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null) + ); + } + } + ) + ); + final Properties properties = injector.getInstance(Properties.class); + properties.clear(); + properties.put(NamespacedExtractionModule.TYPE_PREFIX, "onHeap"); + final NamespaceExtractionCacheManager manager = injector.getInstance(NamespaceExtractionCacheManager.class); + Assert.assertEquals(OnHeapNamespaceExtractionCacheManager.class, manager.getClass()); + } +} From f33ed53881c33737d3bc36e98fbb15a0cab73a36 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 11 May 2016 19:41:57 -0700 Subject: [PATCH 21/42] Optimize imports / reformat --- .../io/druid/data/input/MapPopulator.java | 1 - .../namespace/JDBCExtractionNamespace.java | 3 +- .../namespace/URIExtractionNamespace.java | 9 +++--- .../JDBCExtractionNamespaceCacheFactory.java | 13 ++++----- .../NamespaceExtractionCacheManager.java | 4 ++- .../namespace/URIExtractionNamespaceTest.java | 5 ++-- .../NamespacedExtractorModuleTest.java | 13 ++++----- ...RIExtractionNamespaceCacheFactoryTest.java | 2 +- .../cache/JDBCExtractionNamespaceTest.java | 28 +++++++++++-------- ...ceExtractionCacheManagerExecutorsTest.java | 17 ++++++----- .../NamespaceExtractionCacheManagersTest.java | 11 ++++---- ...apNamespaceExtractionCacheManagerTest.java | 3 +- 12 files changed, 54 insertions(+), 55 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java b/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java index 509d9016b016..58f82d339576 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java @@ -23,7 +23,6 @@ import com.google.common.io.ByteSource; import com.google.common.io.LineProcessor; import com.metamx.common.parsers.Parser; - import java.io.IOException; import java.util.Map; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java index 9fb9c3c5de8c..eba8896aa78a 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java @@ -24,11 +24,10 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import io.druid.metadata.MetadataStorageConnectorConfig; -import org.joda.time.Period; - import javax.annotation.Nullable; import javax.validation.constraints.Min; import javax.validation.constraints.NotNull; +import org.joda.time.Period; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java index 705c3ea614fe..4f080a44198c 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java @@ -40,11 +40,6 @@ import com.metamx.common.parsers.JSONParser; import com.metamx.common.parsers.Parser; import io.druid.guice.annotations.Json; -import org.joda.time.Period; - -import javax.annotation.Nullable; -import javax.validation.constraints.Min; -import javax.validation.constraints.NotNull; import java.io.IOException; import java.net.URI; import java.util.Arrays; @@ -52,6 +47,10 @@ import java.util.Map; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; +import javax.annotation.Nullable; +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; +import org.joda.time.Period; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java index 7a7ad0abba82..ec0a701b3b47 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java @@ -24,13 +24,6 @@ import io.druid.common.utils.JodaUtils; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.lookup.namespace.JDBCExtractionNamespace; -import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.tweak.HandleCallback; -import org.skife.jdbi.v2.tweak.ResultSetMapper; -import org.skife.jdbi.v2.util.TimestampMapper; - import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Timestamp; @@ -39,6 +32,12 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; +import org.skife.jdbi.v2.tweak.ResultSetMapper; +import org.skife.jdbi.v2.util.TimestampMapper; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java index 293e623215ed..3063ee5668a6 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java @@ -131,9 +131,11 @@ public void run() /** * Optional monitoring for overriding classes. `super.monitor` does *NOT* need to be called by overriding methods + * * @param serviceEmitter The emitter to emit to */ - protected void monitor(ServiceEmitter serviceEmitter) { + protected void monitor(ServiceEmitter serviceEmitter) + { // Noop by default } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java index 6d620b73e0b1..c4957315a368 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java @@ -31,11 +31,10 @@ import io.druid.guice.GuiceInjectableValues; import io.druid.guice.annotations.Json; import io.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Test; - import java.io.IOException; import java.util.Map; +import org.junit.Assert; +import org.junit.Test; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java index 165b0e123ba4..6598229cddac 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java @@ -30,9 +30,14 @@ import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; -import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; +import io.druid.server.metrics.NoopServiceEmitter; +import java.io.File; +import java.io.FileWriter; +import java.io.OutputStreamWriter; +import java.util.HashMap; +import java.util.Map; import org.joda.time.Period; import org.junit.AfterClass; import org.junit.Assert; @@ -41,12 +46,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.FileWriter; -import java.io.OutputStreamWriter; -import java.util.HashMap; -import java.util.Map; - /** * */ diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java index 60f4c1b98b98..e8b9ba94eb19 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java @@ -35,11 +35,11 @@ import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; -import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManagersTest; import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; +import io.druid.server.metrics.NoopServiceEmitter; import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java index 004370390d37..0082e88554dd 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java @@ -34,18 +34,8 @@ import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.lookup.namespace.JDBCExtractionNamespace; -import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.lookup.namespace.JDBCExtractionNamespaceCacheFactory; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.skife.jdbi.v2.Handle; - +import io.druid.server.metrics.NoopServiceEmitter; import java.io.Closeable; import java.io.IOException; import java.util.Collection; @@ -57,6 +47,15 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.skife.jdbi.v2.Handle; /** * @@ -199,7 +198,12 @@ public Callable getCachePopulator( final Map cache ) { - final Callable cachePopulator = super.getCachePopulator(id, namespace, lastVersion, cache); + final Callable cachePopulator = super.getCachePopulator( + id, + namespace, + lastVersion, + cache + ); return new Callable() { @Override diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java index 265dde10e4a2..d24b0cfb7467 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java @@ -35,16 +35,8 @@ import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; -import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - +import io.druid.server.metrics.NoopServiceEmitter; import java.io.File; import java.io.FileOutputStream; import java.io.OutputStream; @@ -64,6 +56,13 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java index e90c86fdea99..c2a49e1b140c 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -28,12 +28,6 @@ import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -41,6 +35,11 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java index 3418b7e7d0c6..2cc875fea7c1 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java @@ -42,7 +42,8 @@ public void testInjection() final Injector injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), ImmutableList.of( - new Module(){ + new Module() + { @Override public void configure(Binder binder) { From 7d5f6810ace3200eb36f035d7b16cc063d4e17cb Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 May 2016 10:18:13 -0700 Subject: [PATCH 22/42] Fix future not being properly cancelled on failed initial scheduling --- .../NamespaceExtractionCacheManager.java | 2 + .../namespace/JSONFlatDataParserTest.java | 103 ++++++++++++++++++ .../NamespaceExtractionCacheManagersTest.java | 45 +++++++- 3 files changed, 148 insertions(+), 2 deletions(-) create mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java index 3063ee5668a6..e37a6e5a6d81 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java @@ -235,6 +235,7 @@ public boolean scheduleAndWait( while (newVersion == null || newVersion.equals(oldVersion)) { if (System.currentTimeMillis() > timeout) { log.error("NamespaceLookupExtractorFactory[%s] - timeout during start", id); + delete(id); return false; } @@ -243,6 +244,7 @@ public boolean scheduleAndWait( } catch (InterruptedException e) { log.error("NamespaceLookupExtractorFactory[%s] - interrupted during start", id); + delete(id); return false; } newVersion = getVersion(id); diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java new file mode 100644 index 000000000000..ed6d2732d89f --- /dev/null +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java @@ -0,0 +1,103 @@ +/* + * 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.namespace; + + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; +import com.google.common.io.CharSink; +import com.google.common.io.Files; +import io.druid.data.input.MapPopulator; +import io.druid.jackson.DefaultObjectMapper; +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +public class JSONFlatDataParserTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private static final String KEY = "foo"; + private static final String VAL = "bar"; + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + private File tmpFile; + + @Before + public void setUp() throws Exception + { + tmpFile = temporaryFolder.newFile("lookup.json"); + final CharSink sink = Files.asByteSink(tmpFile).asCharSink(Charsets.UTF_8); + sink.write("{\"key\":\"" + KEY + "\",\"val\":\"" + VAL + "\"}"); + } + + @Test + public void testSimpleParse() throws Exception + { + final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + MAPPER, + "key", + "val" + ); + final Map map = new HashMap<>(); + new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map); + Assert.assertEquals(VAL, map.get(KEY)); + } + + @Test + public void testFailParse() throws Exception + { + expectedException.expect(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + if (!(o instanceof NullPointerException)) { + return false; + } + final NullPointerException npe = (NullPointerException) o; + return npe.getMessage().startsWith("Key column [keyWHOOPS] missing data in line"); + } + + @Override + public void describeTo(Description description) + { + + } + }); + final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + MAPPER, + "keyWHOOPS", + "val" + ); + final Map map = new HashMap<>(); + new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map); + Assert.assertEquals(VAL, map.get(KEY)); + } +} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java index c2a49e1b140c..7e83a65f17c2 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -22,19 +22,29 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; +import io.druid.data.SearchableVersionedDataFinder; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.segment.loading.LocalFileTimestampVersionFinder; +import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; +import java.net.URI; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import org.joda.time.Period; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -49,6 +59,13 @@ public class NamespaceExtractionCacheManagersTest { private static final Logger log = new Logger(NamespaceExtractionCacheManagersTest.class); private static final Lifecycle lifecycle = new Lifecycle(); + private static final Map PULLERS = ImmutableMap.of( + "file", + new LocalFileTimestampVersionFinder() + ); + private static final Map, ExtractionNamespaceCacheFactory> CACHE_FACTORIES = ImmutableMap., ExtractionNamespaceCacheFactory>of( + URIExtractionNamespace.class, new URIExtractionNamespaceCacheFactory(PULLERS) + ); @Parameterized.Parameters(name = "{0}") public static Collection getParameters() @@ -59,7 +76,7 @@ public static Collection getParameters() new OffHeapNamespaceExtractionCacheManager( lifecycle, new NoopServiceEmitter(), - ImmutableMap., ExtractionNamespaceCacheFactory>of() + CACHE_FACTORIES ) } ); @@ -68,7 +85,7 @@ public static Collection getParameters() new OnHeapNamespaceExtractionCacheManager( lifecycle, new NoopServiceEmitter(), - ImmutableMap., ExtractionNamespaceCacheFactory>of() + CACHE_FACTORIES ) } ); @@ -156,6 +173,30 @@ public void testNoDeleteNonexistant() Assert.assertFalse(extractionCacheManager.delete("I don't exist")); } + @Test + public void testDeleteOnScheduleFail() throws Exception + { + final String id = "SOME_ID"; + Assert.assertFalse(extractionCacheManager.scheduleAndWait( + id, + new URIExtractionNamespace( + new URI("file://tmp/I_DONT_REALLY_EXIST" + + UUID.randomUUID().toString()), + null, + null, + new URIExtractionNamespace.JSONFlatDataParser( + new DefaultObjectMapper(), + "key", + "val" + ), + Period.millis(10000), + null + ), + 500 + )); + Assert.assertEquals(ImmutableSet.of(), extractionCacheManager.getKnownIDs()); + } + public static void waitFor(Future future) throws InterruptedException { while (!future.isDone()) { From e061eb6c870cf63c46fdab6d19ad9be34fadd2a8 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 May 2016 16:26:08 -0700 Subject: [PATCH 23/42] Fix delete returns --- .../OffHeapNamespaceExtractionCacheManager.java | 17 ++++++++++------- .../OnHeapNamespaceExtractionCacheManager.java | 3 +-- .../NamespaceExtractionCacheManagersTest.java | 2 +- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index 0603f6998234..3bd658e936fb 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -135,13 +135,16 @@ public boolean delete(final String namespaceKey) final Lock lock = nsLocks.get(namespaceKey); lock.lock(); try { - super.delete(namespaceKey); - final String mmapDBkey = currentNamespaceCache.get(namespaceKey); - if (mmapDBkey != null) { - final long pre = tmpFile.length(); - mmapDB.delete(mmapDBkey); - log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length()); - return true; + if(super.delete(namespaceKey)) { + final String mmapDBkey = currentNamespaceCache.get(namespaceKey); + if (mmapDBkey != null) { + final long pre = tmpFile.length(); + mmapDB.delete(mmapDBkey); + log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length()); + return true; + } else { + return false; + } } else { return false; } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java index fb5bb28e03ac..f3cdc9fc2873 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java @@ -94,8 +94,7 @@ public boolean delete(final String namespaceKey) final Lock lock = nsLocks.get(namespaceKey); lock.lock(); try { - super.delete(namespaceKey); - return mapMap.remove(namespaceKey) != null; + return super.delete(namespaceKey) && mapMap.remove(namespaceKey) != null; } finally { lock.unlock(); diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java index 7e83a65f17c2..194a6a193db7 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -194,7 +194,7 @@ public void testDeleteOnScheduleFail() throws Exception ), 500 )); - Assert.assertEquals(ImmutableSet.of(), extractionCacheManager.getKnownIDs()); + Assert.assertEquals(ImmutableSet.copyOf(nsList), extractionCacheManager.getKnownIDs()); } public static void waitFor(Future future) throws InterruptedException From 423e39232a2708c52e2590f471d7acb447b8a86d Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 16 May 2016 13:15:11 -0700 Subject: [PATCH 24/42] Add more docs about whole introspection --- docs/content/development/extensions-core/namespaced-lookup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index d8a4e8c16699..98dd450751ec 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -273,4 +273,4 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol # Introspection -Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. +Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. Introspection to `/` returns the entire map. From 25083a32c4b2164f833f590bebf418e8d1f0a49b Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 17 May 2016 16:33:11 -0700 Subject: [PATCH 25/42] Add `/version` introspection point for lookups --- .../extensions-core/namespaced-lookup.md | 2 +- .../NamespaceLookupExtractorFactory.java | 30 ++++++++++++++----- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index 98dd450751ec..313bfdab3c19 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -273,4 +273,4 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol # Introspection -Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. Introspection to `/` returns the entire map. +Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. Introspection to `/` returns the entire map. Introspection to `/version` returns the version indicator for the lookup, or a 404 on a race condition during a delete. diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index 824f90aec101..ee53c2ebf480 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -27,12 +27,20 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; import com.metamx.common.ISE; import com.metamx.common.StringUtils; import com.metamx.common.logger.Logger; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; + +import javax.annotation.Nullable; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -42,12 +50,6 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import javax.annotation.Nullable; -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; @JsonTypeName("cachedNamespace") public class NamespaceLookupExtractorFactory implements LookupExtractorFactory @@ -88,7 +90,7 @@ public NamespaceLookupExtractorFactory( @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace, @JsonProperty("firstCacheTimeout") Long firstCacheTimeout, @JsonProperty("oneToOne") boolean oneToOne, - @JacksonInject NamespaceExtractionCacheManager manager + @JacksonInject final NamespaceExtractionCacheManager manager ) { this.extractionNamespace = Preconditions.checkNotNull( @@ -118,6 +120,20 @@ public Response getValues() return Response.ok(getLatest().values().toString()).build(); } + @GET + @Path("/version") + @Produces(MediaType.APPLICATION_JSON) + public Response getVersion() + { + final String version = manager.getVersion(extractorID); + if (null == version) { + // Handle race between delete and this method being called + return Response.status(Response.Status.NOT_FOUND).build(); + } else { + return Response.ok(ImmutableMap.of("version", version)).build(); + } + } + @GET @Produces(MediaType.APPLICATION_JSON) public Response getMap() From 42bb4b24062c6853e3fc5fcbbd737fa11509807c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 18 May 2016 08:14:06 -0700 Subject: [PATCH 26/42] Add more tests and address comments --- .../namespace/NamespacedExtractionModule.java | 2 +- ...ffHeapNamespaceExtractionCacheManager.java | 2 +- .../NamespaceLookupExtractorFactoryTest.java | 30 +++++++++++++++---- 3 files changed, 27 insertions(+), 7 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java index 0e9332fc9135..edd3df638ba0 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java @@ -50,7 +50,7 @@ public class NamespacedExtractionModule implements DruidModule public List getJacksonModules() { return ImmutableList.of( - new SimpleModule("DruidNamespacedExtractionModule") + new SimpleModule("DruidNamespacedCachedExtractionModule") .registerSubtypes( NamespaceLookupExtractorFactory.class ) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index 3bd658e936fb..40661efaf81b 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -87,7 +87,7 @@ public void start() throws Exception } @Override - public void stop() + public synchronized void stop() { if (!mmapDB.isClosed()) { mmapDB.close(); diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java index 1f6b2a1a6af9..572e1a0e5010 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java @@ -19,6 +19,7 @@ package io.druid.query.lookup; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.BeanProperty; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.InjectableValues; @@ -40,7 +41,6 @@ import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.server.DruidNode; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; -import java.util.concurrent.ConcurrentHashMap; import org.easymock.EasyMock; import org.joda.time.Period; import org.junit.Assert; @@ -50,6 +50,10 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + public class NamespaceLookupExtractorFactoryTest { private final ObjectMapper mapper = new DefaultObjectMapper(); @@ -451,11 +455,27 @@ public void configure(Binder binder) ); final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); mapper.registerSubtypes(NamespaceLookupExtractorFactory.class); - final LookupExtractorFactory factory = mapper.readValue( - "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }", - LookupExtractorFactory.class - ); + final String str = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }"; + final LookupExtractorFactory factory = mapper.readValue(str, LookupExtractorFactory.class); Assert.assertTrue(factory instanceof NamespaceLookupExtractorFactory); + final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = (NamespaceLookupExtractorFactory) factory; Assert.assertNotNull(mapper.writeValueAsString(factory)); + Assert.assertFalse(factory.replaces(mapper.readValue( + mapper.writeValueAsString(factory), + LookupExtractorFactory.class + ))); + Assert.assertEquals( + URIExtractionNamespace.class, + namespaceLookupExtractorFactory.getExtractionNamespace().getClass() + ); + Assert.assertFalse(namespaceLookupExtractorFactory.replaces(mapper.readValue(str, LookupExtractorFactory.class))); + final Map map = new HashMap<>(mapper.>readValue( + str, + new TypeReference>() + { + } + )); + map.put("firstCacheTimeout", "1"); + Assert.assertTrue(namespaceLookupExtractorFactory.replaces(mapper.convertValue(map, LookupExtractorFactory.class))); } } From ef0fab24421769c434e23343e41e1298dcee72a1 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 18 May 2016 09:21:25 -0700 Subject: [PATCH 27/42] Add StaticMap extraction namespace for testing. Also add a bunch of tests --- .../NamespaceLookupExtractorFactory.java | 31 +++-- .../lookup/namespace/ExtractionNamespace.java | 3 +- .../StaticMapExtractionNamespace.java | 86 ++++++++++++ .../namespace/NamespacedExtractionModule.java | 5 + ...ticMapExtractionNamespaceCacheFactory.java | 61 +++++++++ .../NamespaceLookupExtractorFactoryTest.java | 123 ++++++++++++++++++ .../StaticMapExtractionNamespaceTest.java | 57 ++++++++ ...apExtractionNamespaceCacheFactoryTest.java | 44 +++++++ 8 files changed, 401 insertions(+), 9 deletions(-) create mode 100644 extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java create mode 100644 extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java create mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java create mode 100644 extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index ee53c2ebf480..adde5f3c0685 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -31,6 +31,7 @@ import com.metamx.common.ISE; import com.metamx.common.StringUtils; import com.metamx.common.logger.Logger; +import io.druid.common.utils.ServletResourceUtils; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; @@ -109,7 +110,12 @@ public NamespaceLookupExtractorFactory( @Produces(MediaType.APPLICATION_JSON) public Response getKeys() { - return Response.ok(getLatest().keySet().toString()).build(); + try { + return Response.ok(getLatest().keySet()).build(); + } + catch (ISE e) { + return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build(); + } } @GET @@ -117,7 +123,12 @@ public Response getKeys() @Produces(MediaType.APPLICATION_JSON) public Response getValues() { - return Response.ok(getLatest().values().toString()).build(); + try { + return Response.ok(getLatest().values()).build(); + } + catch (ISE e) { + return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build(); + } } @GET @@ -138,7 +149,12 @@ public Response getVersion() @Produces(MediaType.APPLICATION_JSON) public Response getMap() { - return Response.ok(getLatest()).build(); + try { + return Response.ok(getLatest()).build(); + } + catch (ISE e) { + return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build(); + } } private Map getLatest() @@ -275,11 +291,10 @@ public LookupExtractor get() public byte[] getCacheKey() { return ByteBuffer - .allocate(CLASS_CACHE_KEY.length + id.length + 1 + v.length + 1 + 1) - .put(id) - .put((byte) 0xFF) - .put(v) - .put((byte) 0xFF) + .allocate(CLASS_CACHE_KEY.length + 1 + id.length + 1 + v.length + 1 + 1) + .put(CLASS_CACHE_KEY).put((byte) 0xFF) + .put(id).put((byte) 0xFF) + .put(v).put((byte) 0xFF) .put(isOneToOne() ? (byte) 1 : (byte) 0) .array(); } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java index 2573d4fc420c..8eb26137295e 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java @@ -25,7 +25,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "jdbc", value = JDBCExtractionNamespace.class), - @JsonSubTypes.Type(name = "uri", value = URIExtractionNamespace.class) + @JsonSubTypes.Type(name = "uri", value = URIExtractionNamespace.class), + @JsonSubTypes.Type(name = StaticMapExtractionNamespace.TYPE_NAME, value = StaticMapExtractionNamespace.class) }) /** * The ExtractionNamespace is a simple object for extracting namespaceLookup values from a source of data. diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java new file mode 100644 index 000000000000..374083b6d35f --- /dev/null +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java @@ -0,0 +1,86 @@ +/* + * 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.namespace; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; + +import java.util.Map; + +@JsonTypeName(StaticMapExtractionNamespace.TYPE_NAME) +public class StaticMapExtractionNamespace implements ExtractionNamespace +{ + static final String TYPE_NAME = "staticMap"; + private final Map map; + private final long pollMs; + + @JsonCreator + public StaticMapExtractionNamespace( + @JsonProperty("pollMs") long pollMs, + @JsonProperty("map") Map map + ) + { + this.pollMs = pollMs; + this.map = Preconditions.checkNotNull(map, "`map` required"); + } + + @JsonProperty + public Map getMap() + { + return map; + } + + @Override + @JsonProperty + public long getPollMs() + { + return pollMs; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StaticMapExtractionNamespace that = (StaticMapExtractionNamespace) o; + + if (getPollMs() != that.getPollMs()) { + return false; + } + return getMap().equals(that.getMap()); + + } + + @Override + public int hashCode() + { + int result = getMap().hashCode(); + result = 31 * result + (int) (getPollMs() ^ (getPollMs() >>> 32)); + return result; + } +} diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java index edd3df638ba0..f9cb5d94c2d5 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java @@ -33,6 +33,7 @@ import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.StaticMapExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; @@ -97,5 +98,9 @@ public void configure(Binder binder) .addBinding(URIExtractionNamespace.class) .to(URIExtractionNamespaceCacheFactory.class) .in(LazySingleton.class); + getNamespaceFactoryMapBinder(binder) + .addBinding(StaticMapExtractionNamespace.class) + .to(StaticMapExtractionNamespaceCacheFactory.class) + .in(LazySingleton.class); } } diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java new file mode 100644 index 000000000000..3edaf8906963 --- /dev/null +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java @@ -0,0 +1,61 @@ +/* + * 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.namespace; + +import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.StaticMapExtractionNamespace; + +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.Callable; + +public class StaticMapExtractionNamespaceCacheFactory + implements ExtractionNamespaceCacheFactory +{ + private final String VERSION = UUID.randomUUID().toString(); + + @Override + public Callable getCachePopulator( + final String id, + final StaticMapExtractionNamespace extractionNamespace, + final String lastVersion, + final Map swap + ) + { + return new Callable() + { + @Override + public String call() throws Exception + { + if (VERSION.equals(lastVersion)) { + return null; + } else { + swap.putAll(extractionNamespace.getMap()); + return VERSION; + } + } + }; + } + + String getVersion() + { + return VERSION; + } +} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java index 572e1a0e5010..7c31d33168e8 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java @@ -26,6 +26,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.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; @@ -50,6 +51,7 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import javax.ws.rs.core.Response; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -478,4 +480,125 @@ public void configure(Binder binder) map.put("firstCacheTimeout", "1"); Assert.assertTrue(namespaceLookupExtractorFactory.replaces(mapper.convertValue(map, LookupExtractorFactory.class))); } + + @Test + public void testSimpleIntrospectionHandler() throws Exception + { + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null) + ); + } + } + ) + ); + final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + mapper.registerSubtypes(NamespaceLookupExtractorFactory.class); + final String str = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"staticMap\", \"map\": {\"foo\":\"bar\"} } }"; + final LookupExtractorFactory lookupExtractorFactory = mapper.readValue(str, LookupExtractorFactory.class); + Assert.assertTrue(lookupExtractorFactory.start()); + try { + final LookupIntrospectHandler handler = lookupExtractorFactory.getIntrospectHandler(); + Assert.assertNotNull(handler); + final Class clazz = handler.getClass(); + Assert.assertNotNull(clazz.getMethod("getVersion").invoke(handler)); + Assert.assertEquals(ImmutableSet.of("foo"), ((Response) clazz.getMethod("getKeys").invoke(handler)).getEntity()); + Assert.assertEquals( + ImmutableSet.of("bar"), + ((Response) clazz.getMethod("getValues").invoke(handler)).getEntity() + ); + Assert.assertEquals( + ImmutableMap.builder().put("foo", "bar").build(), + ((Response) clazz.getMethod("getMap").invoke(handler)).getEntity() + ); + } + finally { + Assert.assertTrue(lookupExtractorFactory.close()); + } + } + + @Test + public void testExceptionalIntrospectionHandler() throws Exception + { + final NamespaceExtractionCacheManager manager = EasyMock.createStrictMock(NamespaceExtractionCacheManager.class); + final ExtractionNamespace extractionNamespace = EasyMock.createStrictMock(ExtractionNamespace.class); + EasyMock.expect(manager.scheduleAndWait(EasyMock.anyString(), EasyMock.eq(extractionNamespace), EasyMock.anyLong())) + .andReturn(true) + .once(); + EasyMock.replay(manager); + final LookupExtractorFactory lookupExtractorFactory = new NamespaceLookupExtractorFactory( + extractionNamespace, + manager + ); + Assert.assertTrue(lookupExtractorFactory.start()); + + final LookupIntrospectHandler handler = lookupExtractorFactory.getIntrospectHandler(); + Assert.assertNotNull(handler); + final Class clazz = handler.getClass(); + + synchronized (manager) { + EasyMock.verify(manager); + EasyMock.reset(manager); + EasyMock.expect(manager.getVersion(EasyMock.anyString())).andReturn(null).once(); + EasyMock.replay(manager); + } + final Response response = (Response) clazz.getMethod("getVersion").invoke(handler); + Assert.assertEquals(404, response.getStatus()); + + + validateCode( + new ISE("some exception"), + 404, + "getKeys", + handler, + manager, + clazz + ); + + validateCode( + new ISE("some exception"), + 404, + "getValues", + handler, + manager, + clazz + ); + + validateCode( + new ISE("some exception"), + 404, + "getMap", + handler, + manager, + clazz + ); + + EasyMock.verify(manager); + } + + void validateCode( + Throwable thrown, + int expectedCode, + String method, + LookupIntrospectHandler handler, + NamespaceExtractionCacheManager manager, + Class clazz + ) throws Exception + { + synchronized (manager) { + EasyMock.verify(manager); + EasyMock.reset(manager); + EasyMock.expect(manager.getVersion(EasyMock.anyString())).andThrow(thrown).once(); + EasyMock.replay(manager); + } + final Response response = (Response) clazz.getMethod(method).invoke(handler); + Assert.assertEquals(expectedCode, response.getStatus()); + } } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java new file mode 100644 index 000000000000..f2bc5f52f964 --- /dev/null +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java @@ -0,0 +1,57 @@ +/* + * 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.namespace; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Map; + +public class StaticMapExtractionNamespaceTest +{ + private static final Map MAP = ImmutableMap.builder().put("foo", "bar").build(); + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private static String MAP_STRING; + + @BeforeClass + public static void setUpStatic() throws Exception + { + MAP_STRING = MAPPER.writeValueAsString(MAP); + } + + @Test + public void testSimpleSerDe() throws Exception + { + final String str = "{\"type\":\"staticMap\", \"map\":" + MAP_STRING + ",\"pollMs\":3}"; + final StaticMapExtractionNamespace extractionNamespace = MAPPER.readValue(str, StaticMapExtractionNamespace.class); + Assert.assertEquals(MAP, extractionNamespace.getMap()); + Assert.assertEquals(3L, extractionNamespace.getPollMs()); + Assert.assertEquals(extractionNamespace, MAPPER.readValue(str, StaticMapExtractionNamespace.class)); + Assert.assertNotEquals(extractionNamespace, new StaticMapExtractionNamespace(0, MAP)); + Assert.assertNotEquals( + extractionNamespace, + new StaticMapExtractionNamespace(3, ImmutableMap.of("foo", "not_bar")) + ); + } +} diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java new file mode 100644 index 000000000000..067b2c342687 --- /dev/null +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java @@ -0,0 +1,44 @@ +/* + * 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.namespace; + +import com.google.common.collect.ImmutableMap; +import io.druid.query.lookup.namespace.StaticMapExtractionNamespace; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class StaticMapExtractionNamespaceCacheFactoryTest +{ + private static final Map MAP = ImmutableMap.builder().put("foo", "bar").build(); + + @Test + public void testSimplePopulator() throws Exception + { + final StaticMapExtractionNamespaceCacheFactory factory = new StaticMapExtractionNamespaceCacheFactory(); + final StaticMapExtractionNamespace namespace = new StaticMapExtractionNamespace(0, MAP); + final Map cache = new HashMap<>(); + Assert.assertEquals(factory.getVersion(), factory.getCachePopulator(null, namespace, null, cache).call()); + Assert.assertEquals(MAP, cache); + Assert.assertNull(factory.getCachePopulator(null, namespace, factory.getVersion(), cache).call()); + } +} From e772c5cbe611f75c4746c25d5ce0e069eec2d66a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 18 May 2016 09:23:55 -0700 Subject: [PATCH 28/42] Move cache system property to `druid.lookup.namespace.cache.type` --- docs/content/development/extensions-core/namespaced-lookup.md | 2 +- .../server/lookup/namespace/NamespacedExtractionModule.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index 313bfdab3c19..43677cf9401e 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -82,7 +82,7 @@ setting namespaces (broker, peon, historical) |Property|Description|Default| |--------|-----------|-------| -|`druid.query.extraction.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`| +|`druid.lookup.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`| The cache is populated in different ways depending on the settings below. In general, most namespaces employ a `pollPeriod` at the end of which time they poll the remote resource of interest for updates. diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java index f9cb5d94c2d5..e84c8e99e2af 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java @@ -45,7 +45,7 @@ */ public class NamespacedExtractionModule implements DruidModule { - public static final String TYPE_PREFIX = "druid.query.extraction.namespace.cache.type"; + public static final String TYPE_PREFIX = "druid.lookup.namespace.cache.type"; @Override public List getJacksonModules() From b2c7f960cd56c40e36c0742eb89b6f62a81ef06b Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 18 May 2016 13:03:14 -0700 Subject: [PATCH 29/42] Make VERSION lower case --- .../StaticMapExtractionNamespaceCacheFactory.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java index 3edaf8906963..4b13e4d44aa8 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java @@ -29,7 +29,7 @@ public class StaticMapExtractionNamespaceCacheFactory implements ExtractionNamespaceCacheFactory { - private final String VERSION = UUID.randomUUID().toString(); + private final String version = UUID.randomUUID().toString(); @Override public Callable getCachePopulator( @@ -44,11 +44,11 @@ public Callable getCachePopulator( @Override public String call() throws Exception { - if (VERSION.equals(lastVersion)) { + if (version.equals(lastVersion)) { return null; } else { swap.putAll(extractionNamespace.getMap()); - return VERSION; + return version; } } }; @@ -56,6 +56,6 @@ public String call() throws Exception String getVersion() { - return VERSION; + return version; } } From bcccf12b695fc094ef554c3cdffae525535c9bd6 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 18 May 2016 13:26:20 -0700 Subject: [PATCH 30/42] Change poll period to 0ms for StaticMap --- .../namespace/StaticMapExtractionNamespace.java | 7 ++----- .../namespace/StaticMapExtractionNamespaceTest.java | 11 +++++++---- .../StaticMapExtractionNamespaceCacheFactoryTest.java | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java index 374083b6d35f..3a56b4c5918c 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java @@ -32,15 +32,12 @@ public class StaticMapExtractionNamespace implements ExtractionNamespace { static final String TYPE_NAME = "staticMap"; private final Map map; - private final long pollMs; @JsonCreator public StaticMapExtractionNamespace( - @JsonProperty("pollMs") long pollMs, @JsonProperty("map") Map map ) { - this.pollMs = pollMs; this.map = Preconditions.checkNotNull(map, "`map` required"); } @@ -51,10 +48,10 @@ public Map getMap() } @Override - @JsonProperty public long getPollMs() { - return pollMs; + // Load once and forget it + return 0; } @Override diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java index f2bc5f52f964..efb8c1735dbd 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java @@ -43,15 +43,18 @@ public static void setUpStatic() throws Exception @Test public void testSimpleSerDe() throws Exception { - final String str = "{\"type\":\"staticMap\", \"map\":" + MAP_STRING + ",\"pollMs\":3}"; + final String str = "{\"type\":\"staticMap\", \"map\":" + MAP_STRING + "}"; final StaticMapExtractionNamespace extractionNamespace = MAPPER.readValue(str, StaticMapExtractionNamespace.class); Assert.assertEquals(MAP, extractionNamespace.getMap()); - Assert.assertEquals(3L, extractionNamespace.getPollMs()); + Assert.assertEquals(0L, extractionNamespace.getPollMs()); Assert.assertEquals(extractionNamespace, MAPPER.readValue(str, StaticMapExtractionNamespace.class)); - Assert.assertNotEquals(extractionNamespace, new StaticMapExtractionNamespace(0, MAP)); Assert.assertNotEquals( extractionNamespace, - new StaticMapExtractionNamespace(3, ImmutableMap.of("foo", "not_bar")) + new StaticMapExtractionNamespace(ImmutableMap.of("foo", "not_bar")) + ); + Assert.assertNotEquals( + extractionNamespace, + new StaticMapExtractionNamespace(ImmutableMap.of("not_foo", "bar")) ); } } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java index 067b2c342687..b3fcaa6daf3e 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java @@ -35,7 +35,7 @@ public class StaticMapExtractionNamespaceCacheFactoryTest public void testSimplePopulator() throws Exception { final StaticMapExtractionNamespaceCacheFactory factory = new StaticMapExtractionNamespaceCacheFactory(); - final StaticMapExtractionNamespace namespace = new StaticMapExtractionNamespace(0, MAP); + final StaticMapExtractionNamespace namespace = new StaticMapExtractionNamespace(MAP); final Map cache = new HashMap<>(); Assert.assertEquals(factory.getVersion(), factory.getCachePopulator(null, namespace, null, cache).call()); Assert.assertEquals(MAP, cache); From db45e445af0b1fa02b693c60bdedbda20e7a328e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 19 May 2016 09:03:09 -0700 Subject: [PATCH 31/42] Move cache key to bytebuffer --- .../lookup/NamespaceLookupExtractorFactory.java | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index adde5f3c0685..b9bab1be0422 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -26,7 +26,6 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.metamx.common.ISE; import com.metamx.common.StringUtils; @@ -42,8 +41,6 @@ import javax.ws.rs.Produces; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.Map; import java.util.UUID; @@ -61,15 +58,8 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory private static final byte[] CLASS_CACHE_KEY; static { - try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - baos.write(StringUtils.toUtf8(NamespaceLookupExtractorFactory.class.getCanonicalName())); - baos.write(0xFF); - CLASS_CACHE_KEY = baos.toByteArray(); - } - catch (IOException e) { - // Should never happen - throw Throwables.propagate(e); - } + final byte[] keyUtf8 = StringUtils.toUtf8(NamespaceLookupExtractorFactory.class.getCanonicalName()); + CLASS_CACHE_KEY = ByteBuffer.allocate(keyUtf8.length + 1).put(keyUtf8).put((byte) 0xFF).array(); } @JsonIgnore From 552114a35433de90d5c4970ac96f5f26fa89814b Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 19 May 2016 09:09:28 -0700 Subject: [PATCH 32/42] Change hashCode and equals on static map extraction fn --- .../lookup/namespace/StaticMapExtractionNamespace.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java index 3a56b4c5918c..81d5abedabc0 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java @@ -66,9 +66,6 @@ public boolean equals(Object o) StaticMapExtractionNamespace that = (StaticMapExtractionNamespace) o; - if (getPollMs() != that.getPollMs()) { - return false; - } return getMap().equals(that.getMap()); } @@ -76,8 +73,6 @@ public boolean equals(Object o) @Override public int hashCode() { - int result = getMap().hashCode(); - result = 31 * result + (int) (getPollMs() ^ (getPollMs() >>> 32)); - return result; + return getMap().hashCode(); } } From 365d8f1f3200bcd742b133e2205cdb1695ee12ea Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 19 May 2016 09:14:49 -0700 Subject: [PATCH 33/42] Add more comments on StaticMap --- .../query/lookup/namespace/StaticMapExtractionNamespace.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java index 81d5abedabc0..ae05d5c931c3 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java @@ -27,6 +27,11 @@ import java.util.Map; +/** + * This class is intended to be used in general cluster testing, and not as a serious lookup. + * Any desire to use a static map in a lookup in *general* should use `io.druid.query.extraction.MapLookupExtractor` + * Any desire to test the *caching mechanisms in this extension* can use this class. + */ @JsonTypeName(StaticMapExtractionNamespace.TYPE_NAME) public class StaticMapExtractionNamespace implements ExtractionNamespace { From df6dfc44e1c12a082e204341500cb02fe64894e2 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 20 May 2016 14:30:29 -0700 Subject: [PATCH 34/42] Address comments --- .../extensions-core/namespaced-lookup.md | 8 ++++++-- .../query/lookup/KafkaLookupExtractorFactory.java | 2 +- .../lookup/NamespaceLookupExtractorFactory.java | 14 +++++++------- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index 43677cf9401e..f98295d82a44 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -61,7 +61,7 @@ Cached namespace lookups can be specified as part of the [cluster wide config fo "tsColumn": "timeColumn" }, "firstCacheTimeout": 120000, - "oneToOne":true + "injective":true } ``` @@ -70,7 +70,7 @@ The parameters are as follows |--------|-----------|--------|-------| |`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-| |`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`60000` (1 minute)| -|`oneToOne`|If the underlying map is injective (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`| +|`injective`|If the underlying map is injective (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`| Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes: `druid-namespace-lookup` @@ -87,6 +87,10 @@ setting namespaces (broker, peon, historical) The cache is populated in different ways depending on the settings below. In general, most namespaces employ a `pollPeriod` at the end of which time they poll the remote resource of interest for updates. +`onHeap` uses `ConcurrentMap`s in the java heap, and thus affects garbage collection and heap sizing. +`offHeap` uses a 10MB on-heap buffer and MapDB using memory-mapped files in the java temporary directory. +So if total `cachedNamespace` lookup size is in excess of 10MB, the extra will be kept in memory as page cache, and paged in and out by general OS tunings. + # Supported Lookups For additional lookups, please see our [extensions list](../extensions.html). 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 f40d234b9a64..341ef046148f 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 @@ -103,7 +103,7 @@ public KafkaLookupExtractorFactory( @JsonProperty("kafkaTopic") final String kafkaTopic, @JsonProperty("kafkaProperties") final Map kafkaProperties, @JsonProperty("connectTimeout") @Min(0) long connectTimeout, - @JsonProperty("isOneToOne") boolean isOneToOne + @JsonProperty("isInjective") boolean isOneToOne ) { this.kafkaTopic = Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required"); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index b9bab1be0422..3c8a0e9eac89 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -72,7 +72,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory private final LookupIntrospectHandler lookupIntrospectHandler; private final ExtractionNamespace extractionNamespace; private final long firstCacheTimeout; - private final boolean oneToOne; + private final boolean injective; private final String extractorID; @@ -80,7 +80,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory public NamespaceLookupExtractorFactory( @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace, @JsonProperty("firstCacheTimeout") Long firstCacheTimeout, - @JsonProperty("oneToOne") boolean oneToOne, + @JsonProperty("injective") boolean injective, @JacksonInject final NamespaceExtractionCacheManager manager ) { @@ -90,7 +90,7 @@ public NamespaceLookupExtractorFactory( ); this.firstCacheTimeout = firstCacheTimeout == null ? DEFAULT_SCHEDULE_TIMEOUT : firstCacheTimeout; Preconditions.checkArgument(this.firstCacheTimeout >= 0); - this.oneToOne = oneToOne; + this.injective = injective; this.manager = manager; this.extractorID = buildID(); this.lookupIntrospectHandler = new LookupIntrospectHandler() @@ -207,7 +207,7 @@ public boolean replaces(@Nullable LookupExtractorFactory other) { if (other != null && other instanceof NamespaceLookupExtractorFactory) { NamespaceLookupExtractorFactory that = (NamespaceLookupExtractorFactory) other; - if (isOneToOne() != ((NamespaceLookupExtractorFactory) other).isOneToOne()) { + if (isInjective() != ((NamespaceLookupExtractorFactory) other).isInjective()) { return true; } if (getFirstCacheTimeout() != ((NamespaceLookupExtractorFactory) other).getFirstCacheTimeout()) { @@ -237,9 +237,9 @@ public long getFirstCacheTimeout() } @JsonProperty - public boolean isOneToOne() + public boolean isInjective() { - return oneToOne; + return injective; } private String buildID() @@ -275,7 +275,7 @@ public LookupExtractor get() } while (!preVersion.equals(postVersion)); final byte[] v = StringUtils.toUtf8(postVersion); final byte[] id = StringUtils.toUtf8(extractorID); - return new MapLookupExtractor(map, isOneToOne()) + return new MapLookupExtractor(map, isInjective()) { @Override public byte[] getCacheKey() From e430113cf791d7953c0de92cc3e4bd9448583254 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 20 May 2016 14:57:43 -0700 Subject: [PATCH 35/42] Make scheduleAndWait use a latch --- .../NamespaceExtractionCacheManager.java | 54 +++++++++---------- 1 file changed, 26 insertions(+), 28 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java index e37a6e5a6d81..8476bac63521 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java @@ -35,6 +35,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; + import java.util.Collection; import java.util.Map; import java.util.UUID; @@ -70,6 +71,7 @@ public NamespaceImplData( final ExtractionNamespace namespace; final String name; final AtomicBoolean enabled = new AtomicBoolean(false); + final CountDownLatch firstRun = new CountDownLatch(1); } private static final Logger log = new Logger(NamespaceExtractionCacheManager.class); @@ -163,11 +165,16 @@ public void run() return; } synchronized (namespaceDatum.enabled) { - if (!namespaceDatum.enabled.get()) { - // skip because it was disabled - return; + try { + if (!namespaceDatum.enabled.get()) { + // skip because it was disabled + return; + } + swapAndClearCache(id, cacheId); + } + finally { + namespaceDatum.firstRun.countDown(); } - swapAndClearCache(id, cacheId); } } }; @@ -224,33 +231,24 @@ public boolean scheduleAndWait( long waitForFirstRun ) { - String oldVersion = getVersion(id); - if (scheduleOrUpdate(id, namespace)) { - // wait until the namespace registration is done - String newVersion = getVersion(id); - final long startLocking = System.currentTimeMillis(); - final long timeout = startLocking + waitForFirstRun; - - while (newVersion == null || newVersion.equals(oldVersion)) { - if (System.currentTimeMillis() > timeout) { - log.error("NamespaceLookupExtractorFactory[%s] - timeout during start", id); - delete(id); - return false; - } + log.debug("Scheduled new namespace [%s]: %s", id, namespace); + } else { + log.debug("Namespace [%s] already running: %s", id, namespace); + } - try { - Thread.sleep(100); - } - catch (InterruptedException e) { - log.error("NamespaceLookupExtractorFactory[%s] - interrupted during start", id); - delete(id); - return false; - } - newVersion = getVersion(id); - } + final NamespaceImplData namespaceImplData = implData.get(id); + boolean success = false; + try { + success = namespaceImplData.firstRun.await(waitForFirstRun, TimeUnit.MILLISECONDS); } - return true; + catch (InterruptedException e) { + log.error(e, "NamespaceLookupExtractorFactory[%s] - interrupted during start", id); + } + if (!success) { + delete(id); + } + return success; } private void cancelFuture(final NamespaceImplData implDatum) From 6283dc236081269ddbb2b19f3c8dddcbe20d945c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 20 May 2016 16:29:44 -0700 Subject: [PATCH 36/42] Sanity renames and fix imports --- .../lookup/TestKafkaExtractionCluster.java | 4 +-- .../io/druid/data/input/MapPopulator.java | 3 ++- .../namespace/JDBCExtractionNamespace.java | 3 ++- .../namespace/URIExtractionNamespace.java | 9 ++++--- .../JDBCExtractionNamespaceCacheFactory.java | 13 ++++----- ...le.java => NamespaceExtractionModule.java} | 3 ++- .../URIExtractionNamespaceCacheFactory.java | 5 ++-- ...ffHeapNamespaceExtractionCacheManager.java | 7 ++--- ...OnHeapNamespaceExtractionCacheManager.java | 1 + .../io.druid.initialization.DruidModule | 2 +- .../namespace/JSONFlatDataParserTest.java | 7 ++--- .../namespace/URIExtractionNamespaceTest.java | 5 ++-- .../NamespacedExtractorModuleTest.java | 11 ++++---- ...RIExtractionNamespaceCacheFactoryTest.java | 27 ++++++++++--------- .../cache/JDBCExtractionNamespaceTest.java | 19 ++++++------- ...ceExtractionCacheManagerExecutorsTest.java | 15 ++++++----- .../NamespaceExtractionCacheManagersTest.java | 13 ++++----- ...apNamespaceExtractionCacheManagerTest.java | 7 ++--- ...apNamespaceExtractionCacheManagerTest.java | 7 ++--- 19 files changed, 89 insertions(+), 72 deletions(-) rename extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/{NamespacedExtractionModule.java => NamespaceExtractionModule.java} (98%) diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java index f99abc8e4b25..66c3e2307407 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java @@ -32,7 +32,7 @@ import com.metamx.common.logger.Logger; import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; -import io.druid.server.lookup.namespace.NamespacedExtractionModule; +import io.druid.server.lookup.namespace.NamespaceExtractionModule; import kafka.admin.AdminUtils; import kafka.javaapi.producer.Producer; import kafka.producer.KeyedMessage; @@ -249,7 +249,7 @@ public void configure(Binder binder) } }, // These injections fail under IntelliJ but are required for maven - new NamespacedExtractionModule(), + new NamespaceExtractionModule(), new KafkaExtractionNamespaceModule() ) ); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java b/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java index 58f82d339576..971370c9f7bb 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java @@ -23,13 +23,14 @@ import com.google.common.io.ByteSource; import com.google.common.io.LineProcessor; import com.metamx.common.parsers.Parser; + import java.io.IOException; import java.util.Map; /** * Simple class that takes a `ByteSource` and uses a `Parser` to populate a `Map` * The `ByteSource` must be UTF-8 encoded - * + *

* If this is handy for other use cases pleaes move this class into a common module */ public class MapPopulator diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java index eba8896aa78a..9fb9c3c5de8c 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java @@ -24,10 +24,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import io.druid.metadata.MetadataStorageConnectorConfig; +import org.joda.time.Period; + import javax.annotation.Nullable; import javax.validation.constraints.Min; import javax.validation.constraints.NotNull; -import org.joda.time.Period; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java index 4f080a44198c..705c3ea614fe 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java @@ -40,6 +40,11 @@ import com.metamx.common.parsers.JSONParser; import com.metamx.common.parsers.Parser; import io.druid.guice.annotations.Json; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; import java.io.IOException; import java.net.URI; import java.util.Arrays; @@ -47,10 +52,6 @@ import java.util.Map; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; -import javax.annotation.Nullable; -import javax.validation.constraints.Min; -import javax.validation.constraints.NotNull; -import org.joda.time.Period; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java index ec0a701b3b47..7a7ad0abba82 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java @@ -24,6 +24,13 @@ import io.druid.common.utils.JodaUtils; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; +import org.skife.jdbi.v2.tweak.ResultSetMapper; +import org.skife.jdbi.v2.util.TimestampMapper; + import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Timestamp; @@ -32,12 +39,6 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.tweak.HandleCallback; -import org.skife.jdbi.v2.tweak.ResultSetMapper; -import org.skife.jdbi.v2.util.TimestampMapper; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java similarity index 98% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java rename to extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java index e84c8e99e2af..84dc64149984 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespacedExtractionModule.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java @@ -38,12 +38,13 @@ import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; + import java.util.List; /** * */ -public class NamespacedExtractionModule implements DruidModule +public class NamespaceExtractionModule implements DruidModule { public static final String TYPE_PREFIX = "druid.lookup.namespace.cache.type"; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java index 2b7b244ff793..11ba66002d47 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java @@ -32,6 +32,9 @@ import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.segment.loading.URIDataPuller; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; + import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; @@ -39,8 +42,6 @@ import java.util.Map; import java.util.concurrent.Callable; import java.util.regex.Pattern; -import org.joda.time.format.DateTimeFormatter; -import org.joda.time.format.ISODateTimeFormat; /** * diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index 40661efaf81b..f33f25949f9c 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -29,6 +29,9 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import org.mapdb.DB; +import org.mapdb.DBMaker; + import java.io.File; import java.io.IOException; import java.util.Map; @@ -36,8 +39,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.Lock; -import org.mapdb.DB; -import org.mapdb.DBMaker; /** * @@ -135,7 +136,7 @@ public boolean delete(final String namespaceKey) final Lock lock = nsLocks.get(namespaceKey); lock.lock(); try { - if(super.delete(namespaceKey)) { + if (super.delete(namespaceKey)) { final String mmapDBkey = currentNamespaceCache.get(namespaceKey); if (mmapDBkey != null) { final long pre = tmpFile.length(); diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java index f3cdc9fc2873..ef78312bb82b 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java @@ -29,6 +29,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; + import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; diff --git a/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index cc999cac090f..0903ef4dee6d 100644 --- a/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -17,4 +17,4 @@ # under the License. # -io.druid.server.lookup.namespace.NamespacedExtractionModule +io.druid.server.lookup.namespace.NamespaceExtractionModule diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java index ed6d2732d89f..b6dfe65ed4c3 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java @@ -26,9 +26,6 @@ import com.google.common.io.Files; import io.druid.data.input.MapPopulator; import io.druid.jackson.DefaultObjectMapper; -import java.io.File; -import java.util.HashMap; -import java.util.Map; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.junit.Assert; @@ -38,6 +35,10 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.util.HashMap; +import java.util.Map; + public class JSONFlatDataParserTest { private static final ObjectMapper MAPPER = new DefaultObjectMapper(); diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java index c4957315a368..6d620b73e0b1 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java @@ -31,11 +31,12 @@ import io.druid.guice.GuiceInjectableValues; import io.druid.guice.annotations.Json; import io.druid.jackson.DefaultObjectMapper; -import java.io.IOException; -import java.util.Map; import org.junit.Assert; import org.junit.Test; +import java.io.IOException; +import java.util.Map; + /** * */ diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java index 6598229cddac..452dff37c8cd 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java @@ -33,11 +33,6 @@ import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; import io.druid.server.metrics.NoopServiceEmitter; -import java.io.File; -import java.io.FileWriter; -import java.io.OutputStreamWriter; -import java.util.HashMap; -import java.util.Map; import org.joda.time.Period; import org.junit.AfterClass; import org.junit.Assert; @@ -46,6 +41,12 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.io.FileWriter; +import java.io.OutputStreamWriter; +import java.util.HashMap; +import java.util.Map; + /** * */ diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java index e8b9ba94eb19..2665d8a0fabf 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java @@ -40,6 +40,20 @@ import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; import io.druid.server.metrics.NoopServiceEmitter; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; @@ -64,19 +78,6 @@ import java.util.concurrent.ExecutionException; import java.util.regex.Pattern; import java.util.zip.GZIPOutputStream; -import javax.annotation.Nullable; -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java index 0082e88554dd..140f0f658b35 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java @@ -36,6 +36,16 @@ import io.druid.query.lookup.namespace.JDBCExtractionNamespace; import io.druid.server.lookup.namespace.JDBCExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.skife.jdbi.v2.Handle; + import java.io.Closeable; import java.io.IOException; import java.util.Collection; @@ -47,15 +57,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.skife.jdbi.v2.Handle; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java index d24b0cfb7467..8fbcc9ab5de0 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java @@ -37,6 +37,14 @@ import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + import java.io.File; import java.io.FileOutputStream; import java.io.OutputStream; @@ -56,13 +64,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java index 194a6a193db7..019fea5c916d 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -34,6 +34,13 @@ import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.net.URI; import java.util.ArrayList; import java.util.Collection; @@ -44,12 +51,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; /** * diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java index 2cc875fea7c1..0af60778d9f0 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java @@ -29,11 +29,12 @@ import io.druid.guice.annotations.Self; import io.druid.initialization.Initialization; import io.druid.server.DruidNode; -import io.druid.server.lookup.namespace.NamespacedExtractionModule; -import java.util.Properties; +import io.druid.server.lookup.namespace.NamespaceExtractionModule; import org.junit.Assert; import org.junit.Test; +import java.util.Properties; + public class OffHeapNamespaceExtractionCacheManagerTest { @Test @@ -56,7 +57,7 @@ public void configure(Binder binder) ); final Properties properties = injector.getInstance(Properties.class); properties.clear(); - properties.put(NamespacedExtractionModule.TYPE_PREFIX, "offHeap"); + properties.put(NamespaceExtractionModule.TYPE_PREFIX, "offHeap"); final NamespaceExtractionCacheManager manager = injector.getInstance(NamespaceExtractionCacheManager.class); Assert.assertEquals(OffHeapNamespaceExtractionCacheManager.class, manager.getClass()); } diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java index 373fac8b3ca8..8dae04f0d93d 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java @@ -29,11 +29,12 @@ import io.druid.guice.annotations.Self; import io.druid.initialization.Initialization; import io.druid.server.DruidNode; -import io.druid.server.lookup.namespace.NamespacedExtractionModule; -import java.util.Properties; +import io.druid.server.lookup.namespace.NamespaceExtractionModule; import org.junit.Assert; import org.junit.Test; +import java.util.Properties; + public class OnHeapNamespaceExtractionCacheManagerTest { @Test @@ -56,7 +57,7 @@ public void configure(Binder binder) ); final Properties properties = injector.getInstance(Properties.class); properties.clear(); - properties.put(NamespacedExtractionModule.TYPE_PREFIX, "onHeap"); + properties.put(NamespaceExtractionModule.TYPE_PREFIX, "onHeap"); final NamespaceExtractionCacheManager manager = injector.getInstance(NamespaceExtractionCacheManager.class); Assert.assertEquals(OnHeapNamespaceExtractionCacheManager.class, manager.getClass()); } From c9db080ebe8b56f1146f4114225c4a2b780bbb0a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 20 May 2016 16:38:14 -0700 Subject: [PATCH 37/42] Remove extra info in docs --- docs/content/development/extensions-core/namespaced-lookup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/namespaced-lookup.md index f98295d82a44..d378e55121a0 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/namespaced-lookup.md @@ -277,4 +277,4 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol # Introspection -Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. Introspection to `/` returns the entire map. Introspection to `/version` returns the version indicator for the lookup, or a 404 on a race condition during a delete. +Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. Introspection to `/` returns the entire map. Introspection to `/version` returns the version indicator for the lookup. From fa1c0c19b8492b276bfed341c1815c4002bacc10 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 20 May 2016 16:43:01 -0700 Subject: [PATCH 38/42] Fix review comments --- .../query/lookup/NamespaceLookupExtractorFactory.java | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index 3c8a0e9eac89..4cf095e94779 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.annotations.VisibleForTesting; @@ -62,13 +61,9 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory CLASS_CACHE_KEY = ByteBuffer.allocate(keyUtf8.length + 1).put(keyUtf8).put((byte) 0xFF).array(); } - @JsonIgnore private final AtomicBoolean started = new AtomicBoolean(false); - @JsonIgnore private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); - @JsonIgnore private final NamespaceExtractionCacheManager manager; - @JsonIgnore private final LookupIntrospectHandler lookupIntrospectHandler; private final ExtractionNamespace extractionNamespace; private final long firstCacheTimeout; @@ -281,8 +276,8 @@ public LookupExtractor get() public byte[] getCacheKey() { return ByteBuffer - .allocate(CLASS_CACHE_KEY.length + 1 + id.length + 1 + v.length + 1 + 1) - .put(CLASS_CACHE_KEY).put((byte) 0xFF) + .allocate(CLASS_CACHE_KEY.length + id.length + 1 + v.length + 1 + 1) + .put(CLASS_CACHE_KEY) .put(id).put((byte) 0xFF) .put(v).put((byte) 0xFF) .put(isOneToOne() ? (byte) 1 : (byte) 0) From 38ca68e4299bc5afbb9469a2362eab86a0d76daa Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 20 May 2016 16:47:34 -0700 Subject: [PATCH 39/42] Strengthen failure on start from warn to error --- .../io/druid/query/lookup/NamespaceLookupExtractorFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index 4cf095e94779..e3c9472efe4a 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -169,7 +169,7 @@ public boolean start() return true; } if (!manager.scheduleAndWait(extractorID, extractionNamespace, firstCacheTimeout)) { - LOG.warn("Failed to schedule lookup [%s]", extractorID); + LOG.error("Failed to schedule lookup [%s]", extractorID); return false; } LOG.debug("NamespaceLookupExtractorFactory[%s] started", extractorID); From 6762c912538ee44b9e47c5c99d68ec68a4e486c1 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 20 May 2016 17:01:06 -0700 Subject: [PATCH 40/42] Address comments --- .../query/lookup/NamespaceLookupExtractorFactory.java | 11 ++++++----- .../cache/NamespaceExtractionCacheManager.java | 5 +++++ .../lookup/NamespaceLookupExtractorFactoryTest.java | 6 ++---- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index e3c9472efe4a..9dd55f1c1523 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -43,7 +43,6 @@ import java.nio.ByteBuffer; import java.util.Map; import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -61,7 +60,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory CLASS_CACHE_KEY = ByteBuffer.allocate(keyUtf8.length + 1).put(keyUtf8).put((byte) 0xFF).array(); } - private final AtomicBoolean started = new AtomicBoolean(false); + private volatile boolean started = false; private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); private final NamespaceExtractionCacheManager manager; private final LookupIntrospectHandler lookupIntrospectHandler; @@ -164,7 +163,7 @@ public boolean start() final Lock writeLock = startStopSync.writeLock(); writeLock.lock(); try { - if (!started.compareAndSet(false, true)) { + if (started) { LOG.warn("Already started! [%s]", extractorID); return true; } @@ -173,6 +172,7 @@ public boolean start() return false; } LOG.debug("NamespaceLookupExtractorFactory[%s] started", extractorID); + started = true; return true; } finally { @@ -186,10 +186,11 @@ public boolean close() final Lock writeLock = startStopSync.writeLock(); writeLock.lock(); try { - if (!started.compareAndSet(true, false)) { + if (!started) { LOG.warn("Not started! [%s]", extractorID); return true; } + started = false; return manager.checkedDelete(extractorID); } finally { @@ -249,7 +250,7 @@ public LookupExtractor get() final Lock readLock = startStopSync.readLock(); readLock.lock(); try { - if (!started.get()) { + if (!started) { throw new ISE("Factory [%s] not started", extractorID); } String preVersion = null, postVersion = null; diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java index 8476bac63521..d9badf8a1c74 100644 --- a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java @@ -238,6 +238,11 @@ public boolean scheduleAndWait( } final NamespaceImplData namespaceImplData = implData.get(id); + if (namespaceImplData == null) { + log.warn("NamespaceLookupExtractorFactory[%s] - deleted during start", id); + return false; + } + boolean success = false; try { success = namespaceImplData.firstRun.await(waitForFirstRun, TimeUnit.MILLISECONDS); diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java index 7c31d33168e8..0bb9984ee91b 100644 --- a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java @@ -380,9 +380,6 @@ public long getPollMs() EasyMock.eq(extractionNamespace), EasyMock.eq(60000L) )).andReturn(false).once(); - EasyMock.expect( - cacheManager.checkedDelete(EasyMock.anyString()) - ).andReturn(false).once(); EasyMock.replay(cacheManager); final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory( @@ -390,7 +387,8 @@ public long getPollMs() cacheManager ); Assert.assertFalse(namespaceLookupExtractorFactory.start()); - Assert.assertFalse(namespaceLookupExtractorFactory.close()); + // true because it never fully started + Assert.assertTrue(namespaceLookupExtractorFactory.close()); EasyMock.verify(cacheManager); } From 15363e05c6d0d8bd9df0b8df35b127baae96772f Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 23 May 2016 08:40:51 -0700 Subject: [PATCH 41/42] Rename namespace-lookup to lookups-cached-global --- distribution/pom.xml | 2 +- .../development/extensions-core/kafka-extraction-namespace.md | 2 +- .../{namespaced-lookup.md => lookups-cached-global.md} | 4 ++-- docs/content/development/extensions.md | 2 +- examples/conf/druid/_common/common.runtime.properties | 2 +- extensions-core/kafka-extraction-namespace/pom.xml | 2 +- .../{namespace-lookup => lookups-cached-global}/pom.xml | 4 ++-- .../src/main/java/io/druid/data/input/MapPopulator.java | 0 .../druid/query/lookup/NamespaceLookupExtractorFactory.java | 0 .../io/druid/query/lookup/namespace/ExtractionNamespace.java | 0 .../lookup/namespace/ExtractionNamespaceCacheFactory.java | 0 .../druid/query/lookup/namespace/JDBCExtractionNamespace.java | 0 .../query/lookup/namespace/StaticMapExtractionNamespace.java | 0 .../druid/query/lookup/namespace/URIExtractionNamespace.java | 0 .../lookup/namespace/JDBCExtractionNamespaceCacheFactory.java | 0 .../server/lookup/namespace/NamespaceExtractionModule.java | 0 .../namespace/StaticMapExtractionNamespaceCacheFactory.java | 0 .../lookup/namespace/URIExtractionNamespaceCacheFactory.java | 0 .../namespace/cache/NamespaceExtractionCacheManager.java | 0 .../cache/OffHeapNamespaceExtractionCacheManager.java | 0 .../cache/OnHeapNamespaceExtractionCacheManager.java | 0 .../META-INF/services/io.druid.initialization.DruidModule | 0 .../query/lookup/NamespaceLookupExtractorFactoryTest.java | 0 .../druid/query/lookup/namespace/JSONFlatDataParserTest.java | 0 .../lookup/namespace/StaticMapExtractionNamespaceTest.java | 0 .../query/lookup/namespace/URIExtractionNamespaceTest.java | 0 .../lookup/namespace/NamespacedExtractorModuleTest.java | 0 .../StaticMapExtractionNamespaceCacheFactoryTest.java | 0 .../namespace/URIExtractionNamespaceCacheFactoryTest.java | 0 .../lookup/namespace/cache/JDBCExtractionNamespaceTest.java | 0 .../cache/NamespaceExtractionCacheManagerExecutorsTest.java | 0 .../namespace/cache/NamespaceExtractionCacheManagersTest.java | 0 .../cache/OffHeapNamespaceExtractionCacheManagerTest.java | 0 .../cache/OnHeapNamespaceExtractionCacheManagerTest.java | 0 pom.xml | 2 +- 35 files changed, 10 insertions(+), 10 deletions(-) rename docs/content/development/extensions-core/{namespaced-lookup.md => lookups-cached-global.md} (99%) rename extensions-core/{namespace-lookup => lookups-cached-global}/pom.xml (96%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/data/input/MapPopulator.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/main/resources/META-INF/services/io.druid.initialization.DruidModule (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java (100%) rename extensions-core/{namespace-lookup => lookups-cached-global}/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java (100%) diff --git a/distribution/pom.xml b/distribution/pom.xml index 9d08b006fb59..310018f4a625 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -85,7 +85,7 @@ -c io.druid.extensions:mysql-metadata-storage -c - io.druid.extensions:druid-namespace-lookup + io.druid.extensions:druid-lookups-cached-global -c io.druid.extensions:postgresql-metadata-storage -c diff --git a/docs/content/development/extensions-core/kafka-extraction-namespace.md b/docs/content/development/extensions-core/kafka-extraction-namespace.md index a9b4fe0547f3..f100e6de9886 100644 --- a/docs/content/development/extensions-core/kafka-extraction-namespace.md +++ b/docs/content/development/extensions-core/kafka-extraction-namespace.md @@ -8,7 +8,7 @@ layout: doc_page Lookups are an experimental feature. -Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` and `druid-kafka-extraction-namespace` as an extension. +Make sure to [include](../../operations/including-extensions.html) `druid-lookups-cached-global` and `druid-kafka-extraction-namespace` as an extension. If you need updates to populate as promptly as possible, it is possible to plug into a kafka topic whose key is the old value and message is the desired new value (both in UTF-8) as a LookupExtractorFactory. diff --git a/docs/content/development/extensions-core/namespaced-lookup.md b/docs/content/development/extensions-core/lookups-cached-global.md similarity index 99% rename from docs/content/development/extensions-core/namespaced-lookup.md rename to docs/content/development/extensions-core/lookups-cached-global.md index 1ac1257d2804..9f9eb7bb1580 100644 --- a/docs/content/development/extensions-core/namespaced-lookup.md +++ b/docs/content/development/extensions-core/lookups-cached-global.md @@ -8,7 +8,7 @@ layout: doc_page Lookups are an experimental feature. -Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` as an extension. +Make sure to [include](../../operations/including-extensions.html) `druid-lookups-cached-global` as an extension. ## Configuration

@@ -73,7 +73,7 @@ The parameters are as follows |`injective`|If the underlying map is injective (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`| Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes: -`druid-namespace-lookup` +`druid-lookups-cached-global` ## Cache Settings diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index aaeac3aa7db1..0b61fc3902f0 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -27,7 +27,7 @@ Core extensions are maintained by Druid committers. |druid-histogram|Approximate histograms and quantiles aggregator.|[link](../development/extensions-core/approximate-histograms.html)| |druid-kafka-eight|Kafka ingest firehose (high level consumer).|[link](../development/extensions-core/kafka-eight-firehose.html)| |druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)| -|druid-namespace-lookup|Required module for [lookups](../querying/lookups.html).|[link](../development/extensions-core/namespaced-lookup.html)| +|druid-lookups-cached-global|Required module for [lookups](../querying/lookups.html).|[link](../development/extensions-core/lookups-cached-global.html)| |druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)| |mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)| |postgresql-metadata-storage|PostgreSQL metadata store.|[link](../development/extensions-core/postgresql.html)| diff --git a/examples/conf/druid/_common/common.runtime.properties b/examples/conf/druid/_common/common.runtime.properties index 19b005257369..641ef03c6151 100644 --- a/examples/conf/druid/_common/common.runtime.properties +++ b/examples/conf/druid/_common/common.runtime.properties @@ -23,7 +23,7 @@ # This is not the full list of Druid extensions, but common ones that people often use. You may need to change this list # based on your particular setup. -druid.extensions.loadList=["druid-kafka-eight", "druid-s3-extensions", "druid-histogram", "druid-datasketches", "druid-namespace-lookup", "mysql-metadata-storage"] +druid.extensions.loadList=["druid-kafka-eight", "druid-s3-extensions", "druid-histogram", "druid-datasketches", "druid-lookups-cached-global", "mysql-metadata-storage"] # If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory # and uncomment the line below to point to your directory. diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 58bda817cc3e..81e7b74e2962 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -48,7 +48,7 @@ io.druid.extensions - druid-namespace-lookup + druid-lookups-cached-global ${project.parent.version} diff --git a/extensions-core/namespace-lookup/pom.xml b/extensions-core/lookups-cached-global/pom.xml similarity index 96% rename from extensions-core/namespace-lookup/pom.xml rename to extensions-core/lookups-cached-global/pom.xml index 6e583c4bfd81..4225f7cd22be 100644 --- a/extensions-core/namespace-lookup/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -22,8 +22,8 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 io.druid.extensions - druid-namespace-lookup - druid-namespace-lookup + druid-lookups-cached-global + druid-lookups-cached-global Extension to rename Druid dimension values using namespaces diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/data/input/MapPopulator.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java diff --git a/extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java similarity index 100% rename from extensions-core/namespace-lookup/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java diff --git a/extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/lookups-cached-global/src/main/resources/META-INF/services/io.druid.initialization.DruidModule similarity index 100% rename from extensions-core/namespace-lookup/src/main/resources/META-INF/services/io.druid.initialization.DruidModule rename to extensions-core/lookups-cached-global/src/main/resources/META-INF/services/io.druid.initialization.DruidModule diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespaceTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java diff --git a/extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java similarity index 100% rename from extensions-core/namespace-lookup/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManagerTest.java diff --git a/pom.xml b/pom.xml index ebca25b8b2df..8049228605eb 100644 --- a/pom.xml +++ b/pom.xml @@ -91,7 +91,7 @@ extensions-core/kafka-indexing-service extensions-core/mysql-metadata-storage extensions-core/postgresql-metadata-storage - extensions-core/namespace-lookup + extensions-core/lookups-cached-global extensions-core/s3-extensions extensions-contrib/azure-extensions From 9900d9935884538970dfa599d679b506d3583df2 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 23 May 2016 21:18:47 -0700 Subject: [PATCH 42/42] Fix injective mis-naming * Also add serde test --- .../lookup/KafkaLookupExtractorFactory.java | 35 ++++++++++--------- .../KafkaLookupExtractorFactoryTest.java | 25 +++++++++++++ 2 files changed, 43 insertions(+), 17 deletions(-) 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 341ef046148f..834efea0c97d 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 @@ -38,6 +38,17 @@ import io.druid.concurrent.Execs; import io.druid.query.extraction.MapLookupExtractor; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.KafkaStream; +import kafka.consumer.Whitelist; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.MessageAndMetadata; +import kafka.serializer.Decoder; + +import javax.annotation.Nullable; +import javax.validation.constraints.Min; +import javax.ws.rs.GET; +import javax.ws.rs.core.Response; import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -52,16 +63,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; -import javax.annotation.Nullable; -import javax.validation.constraints.Min; -import javax.ws.rs.GET; -import javax.ws.rs.core.Response; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.KafkaStream; -import kafka.consumer.Whitelist; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.MessageAndMetadata; -import kafka.serializer.Decoder; @JsonTypeName("kafka") public class KafkaLookupExtractorFactory implements LookupExtractorFactory @@ -95,7 +96,7 @@ public String fromBytes(byte[] bytes) private final long connectTimeout; @JsonProperty - private final boolean isOneToOne; + private final boolean injective; @JsonCreator public KafkaLookupExtractorFactory( @@ -103,7 +104,7 @@ public KafkaLookupExtractorFactory( @JsonProperty("kafkaTopic") final String kafkaTopic, @JsonProperty("kafkaProperties") final Map kafkaProperties, @JsonProperty("connectTimeout") @Min(0) long connectTimeout, - @JsonProperty("isInjective") boolean isOneToOne + @JsonProperty("injective") boolean injective ) { this.kafkaTopic = Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required"); @@ -114,7 +115,7 @@ public KafkaLookupExtractorFactory( )); this.cacheManager = cacheManager; this.connectTimeout = connectTimeout; - this.isOneToOne = isOneToOne; + this.injective = injective; } public KafkaLookupExtractorFactory( @@ -141,9 +142,9 @@ public long getConnectTimeout() return connectTimeout; } - public boolean isOneToOne() + public boolean isInjective() { - return isOneToOne; + return injective; } @Override @@ -335,7 +336,7 @@ public boolean replaces(@Nullable LookupExtractorFactory other) return !(getKafkaTopic().equals(that.getKafkaTopic()) && getKafkaProperties().equals(that.getKafkaProperties()) && getConnectTimeout() == that.getConnectTimeout() - && isOneToOne() == that.isOneToOne() + && isInjective() == that.isInjective() ); } @@ -351,7 +352,7 @@ public LookupExtractor get() { final Map map = Preconditions.checkNotNull(mapRef.get(), "Not started"); final long startCount = doubleEventCount.get(); - return new MapLookupExtractor(map, isOneToOne()) + return new MapLookupExtractor(map, isInjective()) { @Override public byte[] getCacheKey() diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java index 3f4ab3327993..d62d412fb041 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java @@ -507,6 +507,31 @@ public void testFailsGetNotStarted() ).get(); } + @Test + public void testSerDe() throws Exception + { + final NamespaceExtractionCacheManager cacheManager = EasyMock.createStrictMock(NamespaceExtractionCacheManager.class); + final String kafkaTopic = "some_topic"; + final Map kafkaProperties = ImmutableMap.of("some_key", "some_value"); + final long connectTimeout = 999; + final boolean injective = true; + final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory( + cacheManager, + kafkaTopic, + kafkaProperties, + connectTimeout, + injective + ); + final KafkaLookupExtractorFactory otherFactory = mapper.readValue( + mapper.writeValueAsString(factory), + KafkaLookupExtractorFactory.class + ); + Assert.assertEquals(kafkaTopic, otherFactory.getKafkaTopic()); + Assert.assertEquals(kafkaProperties, otherFactory.getKafkaProperties()); + Assert.assertEquals(connectTimeout, otherFactory.getConnectTimeout()); + Assert.assertEquals(injective, otherFactory.isInjective()); + } + @Test public void testDefaultDecoder() {