From 14f4c8831f70de43fce3095ad10a98059db9c961 Mon Sep 17 00:00:00 2001 From: Keuntae Park Date: Thu, 24 Mar 2016 11:19:24 +0900 Subject: [PATCH 1/5] 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 2/5] 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 3/5] 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 4/5] 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 5/5] 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() {