diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml
index 58bda817cc3e..9f1a2a8b8ae9 100644
--- a/extensions-core/kafka-extraction-namespace/pom.xml
+++ b/extensions-core/kafka-extraction-namespace/pom.xml
@@ -98,10 +98,5 @@
3.0.1
test
-
- org.easymock
- easymock
- test
-
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
new file mode 100644
index 000000000000..7276333508f2
--- /dev/null
+++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/extraction/namespace/KafkaExtractionNamespace.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.query.extraction.namespace;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+import javax.validation.constraints.NotNull;
+import java.util.Properties;
+
+/**
+ *
+ */
+@JsonTypeName("kafka")
+public class KafkaExtractionNamespace implements ExtractionNamespace
+{
+ @JsonProperty
+ private final String kafkaTopic;
+
+ @JsonProperty
+ private final Properties kafkaProperties;
+
+ @JsonCreator
+ public KafkaExtractionNamespace(
+ @NotNull @JsonProperty(value = "kafkaTopic", required = true) final String kafkaTopic,
+ @NotNull @JsonProperty(value = "kafkaProperites", required = true) final Properties kafkaProperties
+ )
+ {
+ Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required");
+ this.kafkaTopic = kafkaTopic;
+ this.kafkaProperties = kafkaProperties;
+ }
+
+ public String getKafkaTopic()
+ {
+ return kafkaTopic;
+ }
+
+ public Properties getKafkaProperties()
+ {
+ return kafkaProperties;
+ }
+
+ @Override
+ public long getPollMs()
+ {
+ return 0L;
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("KafkaExtractionNamespace = { kafkaTopic = '%s' }", kafkaTopic);
+ }
+}
diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaExtractionNamespaceModule.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaExtractionNamespaceModule.java
deleted file mode 100644
index 17159dbb32ae..000000000000
--- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaExtractionNamespaceModule.java
+++ /dev/null
@@ -1,50 +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.lookup;
-
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import io.druid.initialization.DruidModule;
-
-import java.util.List;
-
-/**
- *
- */
-public class KafkaExtractionNamespaceModule implements DruidModule
-{
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new SimpleModule("kafka-lookups").registerSubtypes(
- KafkaLookupExtractorFactory.class
- )
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
- // NOOP
- }
-}
diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java
deleted file mode 100644
index 286a8d0d538b..000000000000
--- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java
+++ /dev/null
@@ -1,423 +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.lookup;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Stopwatch;
-import com.google.common.primitives.Longs;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.common.IAE;
-import com.metamx.common.ISE;
-import com.metamx.common.StringUtils;
-import com.metamx.common.logger.Logger;
-import io.druid.concurrent.Execs;
-import io.druid.query.extraction.MapLookupExtractor;
-import io.druid.server.namespace.cache.NamespaceExtractionCacheManager;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-import javax.annotation.Nullable;
-import javax.validation.constraints.Min;
-import javax.ws.rs.GET;
-import javax.ws.rs.core.Response;
-import kafka.consumer.ConsumerConfig;
-import kafka.consumer.KafkaStream;
-import kafka.consumer.Whitelist;
-import kafka.javaapi.consumer.ConsumerConnector;
-import kafka.message.MessageAndMetadata;
-import kafka.serializer.Decoder;
-
-@JsonTypeName("kafka")
-public class KafkaLookupExtractorFactory implements LookupExtractorFactory
-{
- private static final Logger LOG = new Logger(KafkaLookupExtractorFactory.class);
- static final Decoder DEFAULT_STRING_DECODER = new Decoder()
- {
- @Override
- public String fromBytes(byte[] bytes)
- {
- return StringUtils.fromUtf8(bytes);
- }
- };
-
- private final ListeningExecutorService executorService;
- private final AtomicLong doubleEventCount = new AtomicLong(0L);
- private final NamespaceExtractionCacheManager cacheManager;
- private final String factoryId = UUID.randomUUID().toString();
- private final AtomicReference> mapRef = new AtomicReference<>(null);
- private final AtomicBoolean started = new AtomicBoolean(false);
-
- private volatile ListenableFuture> future = null;
-
- @JsonProperty
- private final String kafkaTopic;
-
- @JsonProperty
- private final Map kafkaProperties;
-
- @JsonProperty
- private final long connectTimeout;
-
- @JsonProperty
- private final boolean isOneToOne;
-
- @JsonCreator
- public KafkaLookupExtractorFactory(
- @JacksonInject NamespaceExtractionCacheManager cacheManager,
- @JsonProperty("kafkaTopic") final String kafkaTopic,
- @JsonProperty("kafkaProperties") final Map kafkaProperties,
- @JsonProperty("connectTimeout") @Min(0) long connectTimeout,
- @JsonProperty("isOneToOne") boolean isOneToOne
- )
- {
- this.kafkaTopic = Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required");
- this.kafkaProperties = Preconditions.checkNotNull(kafkaProperties, "kafkaProperties required");
- executorService = MoreExecutors.listeningDecorator(Execs.singleThreaded(
- "kafka-factory-" + kafkaTopic + "-%s",
- Thread.MIN_PRIORITY
- ));
- this.cacheManager = cacheManager;
- this.connectTimeout = connectTimeout;
- this.isOneToOne = isOneToOne;
- }
-
- public KafkaLookupExtractorFactory(
- NamespaceExtractionCacheManager cacheManager,
- String kafkaTopic,
- Map kafkaProperties
- )
- {
- this(cacheManager, kafkaTopic, kafkaProperties, 0, false);
- }
-
- public String getKafkaTopic()
- {
- return kafkaTopic;
- }
-
- public Map getKafkaProperties()
- {
- return kafkaProperties;
- }
-
- public long getConnectTimeout()
- {
- return connectTimeout;
- }
-
- public boolean isOneToOne()
- {
- return isOneToOne;
- }
-
- @Override
- public boolean start()
- {
- synchronized (started) {
- if (started.get()) {
- LOG.warn("Already started, not starting again");
- return started.get();
- }
- if (executorService.isShutdown()) {
- LOG.warn("Already shut down, not starting again");
- return false;
- }
- final Properties kafkaProperties = new Properties();
- kafkaProperties.putAll(getKafkaProperties());
- if (kafkaProperties.containsKey("group.id")) {
- throw new IAE(
- "Cannot set kafka property [group.id]. Property is randomly generated for you. Found [%s]",
- kafkaProperties.getProperty("group.id")
- );
- }
- if (kafkaProperties.containsKey("auto.offset.reset")) {
- throw new IAE(
- "Cannot set kafka property [auto.offset.reset]. Property will be forced to [smallest]. Found [%s]",
- kafkaProperties.getProperty("auto.offset.reset")
- );
- }
- Preconditions.checkNotNull(
- kafkaProperties.getProperty("zookeeper.connect"),
- "zookeeper.connect required property"
- );
-
- kafkaProperties.setProperty("group.id", factoryId);
- final String topic = getKafkaTopic();
- LOG.debug("About to listen to topic [%s] with group.id [%s]", topic, factoryId);
- final Map map = cacheManager.getCacheMap(factoryId);
- mapRef.set(map);
- // Enable publish-subscribe
- kafkaProperties.setProperty("auto.offset.reset", "smallest");
-
- final CountDownLatch startingReads = new CountDownLatch(1);
-
- final ListenableFuture> future = executorService.submit(
- new Runnable()
- {
- @Override
- public void run()
- {
- while (!executorService.isShutdown() && !Thread.currentThread().isInterrupted()) {
- final ConsumerConnector consumerConnector = buildConnector(kafkaProperties);
- try {
- final List> streams = consumerConnector.createMessageStreamsByFilter(
- new Whitelist(Pattern.quote(topic)), 1, DEFAULT_STRING_DECODER, DEFAULT_STRING_DECODER
- );
-
- if (streams == null || streams.isEmpty()) {
- throw new IAE("Topic [%s] had no streams", topic);
- }
- if (streams.size() > 1) {
- throw new ISE("Topic [%s] has %d streams! expected 1", topic, streams.size());
- }
- final KafkaStream kafkaStream = streams.get(0);
-
- startingReads.countDown();
-
- for (final MessageAndMetadata messageAndMetadata : kafkaStream) {
- final String key = messageAndMetadata.key();
- final String message = messageAndMetadata.message();
- if (key == null || message == null) {
- LOG.error("Bad key/message from topic [%s]: [%s]", topic, messageAndMetadata);
- continue;
- }
- doubleEventCount.incrementAndGet();
- map.put(key, message);
- doubleEventCount.incrementAndGet();
- LOG.trace("Placed key[%s] val[%s]", key, message);
- }
- }
- catch (Exception e) {
- LOG.error(e, "Error reading stream for topic [%s]", topic);
- }
- finally {
- consumerConnector.shutdown();
- }
- }
- }
- }
- );
- Futures.addCallback(
- future, new FutureCallback()
- {
- @Override
- public void onSuccess(Object result)
- {
- LOG.debug("Success listening to [%s]", topic);
- }
-
- @Override
- public void onFailure(Throwable t)
- {
- if (t instanceof CancellationException) {
- LOG.debug("Topic [%s] cancelled", topic);
- } else {
- LOG.error(t, "Error in listening to [%s]", topic);
- }
- }
- },
- MoreExecutors.sameThreadExecutor()
- );
- this.future = future;
- final Stopwatch stopwatch = Stopwatch.createStarted();
- try {
- while (!startingReads.await(100, TimeUnit.MILLISECONDS) && connectTimeout > 0L) {
- // Don't return until we have actually connected
- if (future.isDone()) {
- future.get();
- } else {
- if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > connectTimeout) {
- throw new TimeoutException("Failed to connect to kafka in sufficient time");
- }
- }
- }
- }
- catch (InterruptedException | ExecutionException | TimeoutException e) {
- if (!future.isDone() && !future.cancel(true) && !future.isDone()) {
- LOG.warn("Could not cancel kafka listening thread");
- }
- LOG.error(e, "Failed to start kafka extraction factory");
- cacheManager.delete(factoryId);
- return false;
- }
-
- started.set(true);
- return true;
- }
- }
-
- // Overriden in tests
- ConsumerConnector buildConnector(Properties properties)
- {
- return new kafka.javaapi.consumer.ZookeeperConsumerConnector(
- new ConsumerConfig(properties)
- );
- }
-
- @Override
- public boolean close()
- {
- synchronized (started) {
- if (!started.get() || executorService.isShutdown()) {
- LOG.info("Already shutdown, ignoring");
- return !started.get();
- }
- started.set(false);
- executorService.shutdownNow();
- final ListenableFuture> future = this.future;
- if (future != null) {
- if (!future.isDone() && !future.cancel(true) && !future.isDone()) {
- LOG.error("Error cancelling future for topic [%s]", getKafkaTopic());
- return false;
- }
- }
- if (!cacheManager.delete(factoryId)) {
- LOG.error("Error removing [%s] for topic [%s] from cache", factoryId, getKafkaTopic());
- return false;
- }
- return true;
- }
- }
-
- @Override
- public boolean replaces(@Nullable LookupExtractorFactory other)
- {
- if (this == other) {
- return false;
- }
-
- if (other == null) {
- return false;
- }
-
- if (getClass() != other.getClass()) {
- return true;
- }
-
- final KafkaLookupExtractorFactory that = (KafkaLookupExtractorFactory) other;
-
- return !(getKafkaTopic().equals(that.getKafkaTopic())
- && getKafkaProperties().equals(that.getKafkaProperties())
- && getConnectTimeout() == that.getConnectTimeout()
- && isOneToOne() == that.isOneToOne()
- );
- }
-
- @Nullable
- @Override
- public LookupIntrospectHandler getIntrospectHandler()
- {
- return new KafkaLookupExtractorIntrospectionHandler();
- }
-
- @Override
- public LookupExtractor get()
- {
- final Map map = Preconditions.checkNotNull(mapRef.get(), "Not started");
- final long startCount = doubleEventCount.get();
- return new MapLookupExtractor(map, isOneToOne())
- {
- @Override
- public byte[] getCacheKey()
- {
- final byte[] idutf8 = StringUtils.toUtf8(factoryId);
- // If the number of things added has not changed during the course of this extractor's life, we can cache it
- if (startCount == doubleEventCount.get()) {
- return ByteBuffer
- .allocate(idutf8.length + 1 + Longs.BYTES)
- .put(idutf8)
- .put((byte) 0xFF)
- .putLong(startCount)
- .array();
- } else {
- // If the number of things added HAS changed during the coruse of this extractor's life, we CANNOT cache
- final byte[] scrambler = StringUtils.toUtf8(UUID.randomUUID().toString());
- return ByteBuffer
- .allocate(idutf8.length + 1 + scrambler.length + 1)
- .put(idutf8)
- .put((byte) 0xFF)
- .put(scrambler)
- .put((byte) 0xFF)
- .array();
- }
- }
- };
- }
-
- public long getCompletedEventCount()
- {
- return doubleEventCount.get() >> 1;
- }
-
- // Used in tests
- NamespaceExtractionCacheManager getCacheManager()
- {
- return cacheManager;
- }
-
- AtomicReference> getMapRef()
- {
- return mapRef;
- }
-
- AtomicLong getDoubleEventCount()
- {
- return doubleEventCount;
- }
-
- ListenableFuture> getFuture()
- {
- return future;
- }
-
-
- class KafkaLookupExtractorIntrospectionHandler implements LookupIntrospectHandler
- {
- @GET
- public Response getActive()
- {
- final ListenableFuture> future = getFuture();
- if (future != null && !future.isDone()) {
- return Response.ok().build();
- } else {
- return Response.status(Response.Status.GONE).build();
- }
- }
- }
-}
diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java
new file mode 100644
index 000000000000..4bfd7683b0d0
--- /dev/null
+++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionManager.java
@@ -0,0 +1,237 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.server.namespace;
+
+import com.google.common.base.Throwables;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.inject.Inject;
+import com.google.inject.name.Named;
+import com.metamx.common.IAE;
+import com.metamx.common.ISE;
+import com.metamx.common.StringUtils;
+import com.metamx.common.lifecycle.LifecycleStart;
+import com.metamx.common.lifecycle.LifecycleStop;
+import com.metamx.common.logger.Logger;
+import io.druid.guice.ManageLifecycle;
+import io.druid.query.extraction.namespace.KafkaExtractionNamespace;
+import kafka.consumer.ConsumerConfig;
+import kafka.consumer.ConsumerIterator;
+import kafka.consumer.KafkaStream;
+import kafka.consumer.Whitelist;
+import kafka.javaapi.consumer.ConsumerConnector;
+import kafka.message.MessageAndMetadata;
+import kafka.serializer.Decoder;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+/**
+ *
+ */
+public class KafkaExtractionManager
+{
+ private static final Logger log = new Logger(KafkaExtractionManager.class);
+
+ private final Properties kafkaProperties = new Properties();
+ private final ConcurrentMap namespaceVersionMap = new ConcurrentHashMap<>();
+ private final ConcurrentMap topicEvents = new ConcurrentHashMap<>();
+ private final ConcurrentMap> futures = new ConcurrentHashMap<>();
+ private final ListeningExecutorService executorService = MoreExecutors.listeningDecorator(
+ Executors.newCachedThreadPool(
+ new ThreadFactoryBuilder()
+ .setNameFormat("kafka-rename-consumer-%d")
+ .setDaemon(true)
+ .setPriority(Thread.MIN_PRIORITY)
+ .build()
+ )
+ );
+ private final AtomicInteger backgroundTaskCount = new AtomicInteger(0);
+
+ // Bindings in KafkaExtractionNamespaceModule
+ @Inject
+ public KafkaExtractionManager(
+ final Properties kafkaProperties
+ )
+ {
+ if (kafkaProperties.containsKey("group.id")) {
+ throw new IAE(
+ "Cannot set kafka property [group.id]. Property is randomly generated for you. Found [%s]",
+ kafkaProperties.getProperty("group.id")
+ );
+ }
+ if (kafkaProperties.containsKey("auto.offset.reset")) {
+ throw new IAE(
+ "Cannot set kafka property [auto.offset.reset]. Property will be forced to [smallest]. Found [%s]",
+ kafkaProperties.getProperty("auto.offset.reset")
+ );
+ }
+ this.kafkaProperties.putAll(kafkaProperties);
+ if (!this.kafkaProperties.containsKey("zookeeper.connect")) {
+ this.kafkaProperties.put("zookeeper.connect", "localhost:2181/kafka");
+ }
+ // Enable publish-subscribe
+ this.kafkaProperties.setProperty("auto.offset.reset", "smallest");
+ }
+
+ boolean supports(Properties otherProperties)
+ {
+ if (otherProperties.containsKey("zookeeper.connect")) {
+ return this.kafkaProperties.getProperty("zookeeper.connect")
+ .equals(otherProperties.getProperty("zookeeper.connect"));
+ }
+ return false;
+ }
+
+ public long getBackgroundTaskCount()
+ {
+ return backgroundTaskCount.get();
+ }
+
+ private static final Decoder defaultStringDecoder = new Decoder()
+ {
+ @Override
+ public String fromBytes(byte[] bytes)
+ {
+ return StringUtils.fromUtf8(bytes);
+ }
+ };
+
+ public long getNumEvents(String namespace)
+ {
+ if(namespace == null){
+ return 0L;
+ } else {
+ final AtomicLong eventCounter = topicEvents.get(namespace);
+ if(eventCounter != null) {
+ return eventCounter.get();
+ } else {
+ return 0L;
+ }
+ }
+ }
+
+ public void addListener(final String id, final KafkaExtractionNamespace kafkaNamespace, final Map map)
+ {
+ final String topic = kafkaNamespace.getKafkaTopic();
+ final ListenableFuture> future = executorService.submit(
+ new Runnable()
+ {
+ @Override
+ public void run()
+ {
+ final Properties privateProperties = new Properties();
+ privateProperties.putAll(kafkaProperties);
+ privateProperties.setProperty("group.id", UUID.randomUUID().toString());
+ ConsumerConnector consumerConnector = new kafka.javaapi.consumer.ZookeeperConsumerConnector(
+ new ConsumerConfig(
+ privateProperties
+ )
+ );
+ List> streams = consumerConnector.createMessageStreamsByFilter(
+ new Whitelist(Pattern.quote(topic)), 1, defaultStringDecoder, defaultStringDecoder
+ );
+
+ if (streams == null || streams.isEmpty()) {
+ throw new IAE("Topic [%s] had no streams", topic);
+ }
+ if (streams.size() > 1) {
+ throw new ISE("Topic [%s] has %d streams! expected 1", topic, streams.size());
+ }
+ backgroundTaskCount.incrementAndGet();
+ final KafkaStream kafkaStream = streams.get(0);
+ final ConsumerIterator it = kafkaStream.iterator();
+ log.info("Listening to topic [%s] for namespace [%s]", topic, id);
+ AtomicLong eventCounter = topicEvents.get(id);
+ if(eventCounter == null){
+ topicEvents.putIfAbsent(id, new AtomicLong(0L));
+ eventCounter = topicEvents.get(id);
+ }
+ while (it.hasNext()) {
+ final MessageAndMetadata messageAndMetadata = it.next();
+ final String key = messageAndMetadata.key();
+ final String message = messageAndMetadata.message();
+ if (key == null || message == null) {
+ log.error("Bad key/message from topic [%s]: [%s]", topic, messageAndMetadata);
+ continue;
+ }
+ map.put(key, message);
+ namespaceVersionMap.put(id, Long.toString(eventCounter.incrementAndGet()));
+ log.debug("Placed key[%s] val[%s]", key, message);
+ }
+ }
+ }
+ );
+ futures.putIfAbsent(id, future);
+ Futures.addCallback(
+ future, new FutureCallback()
+ {
+ @Override
+ public void onSuccess(Object result)
+ {
+ topicEvents.remove(id);
+ futures.remove(id);
+ }
+
+ @Override
+ public void onFailure(Throwable t)
+ {
+ topicEvents.remove(id);
+ futures.remove(id);
+ if (t instanceof java.util.concurrent.CancellationException) {
+ log.warn("Cancelled rename task for topic [%s]", topic);
+ } else {
+ Throwables.propagate(t);
+ }
+ }
+ },
+ MoreExecutors.sameThreadExecutor()
+ );
+ }
+
+ public void removeListener(final String id)
+ {
+ Future future = futures.remove(id);
+ future.cancel(true);
+ }
+
+ public void start()
+ {
+ // NO-OP
+ // all consumers are started through KafkaExtractionNamespaceFactory.getCachePopulator
+ }
+
+ public void stop()
+ {
+ executorService.shutdown();
+ Futures.allAsList(futures.values()).cancel(true);
+ }
+}
diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java
new file mode 100644
index 000000000000..b882aa6f5c36
--- /dev/null
+++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceFactory.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.server.namespace;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.inject.Inject;
+import com.google.inject.name.Named;
+import io.druid.query.extraction.namespace.ExtractionNamespaceCacheFactory;
+import io.druid.query.extraction.namespace.KafkaExtractionNamespace;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+
+/**
+ *
+ */
+public class KafkaExtractionNamespaceFactory implements ExtractionNamespaceCacheFactory
+{
+ private final List kafkaExtractionManagers;
+ private static final String KAFKA_VERSION = "kafka versions are updated every time a new event comes in";
+
+ @Inject
+ public KafkaExtractionNamespaceFactory(
+ @JacksonInject @Named("kafkaManagers") final List kafkaExtractionManagers
+ )
+ {
+ this.kafkaExtractionManagers = kafkaExtractionManagers;
+ }
+
+ // This only fires ONCE when the namespace is first added. The version is updated externally as events come in
+ @Override
+ public Callable getCachePopulator(
+ final String id,
+ final KafkaExtractionNamespace extractionNamespace,
+ final String unused,
+ final Map cache
+ )
+ {
+ return new Callable()
+ {
+ @Override
+ public String call()
+ {
+ KafkaExtractionManager manager;
+ synchronized (kafkaExtractionManagers) {
+ manager = findAppropriate(extractionNamespace.getKafkaProperties());
+ if (manager == null) {
+ manager = new KafkaExtractionManager(extractionNamespace.getKafkaProperties());
+ kafkaExtractionManagers.add(manager);
+ }
+ }
+ manager.addListener(id, extractionNamespace, cache);
+ return KAFKA_VERSION;
+ }
+ };
+ }
+
+ private KafkaExtractionManager findAppropriate(Properties kafkaProperties)
+ {
+ for (KafkaExtractionManager manager: kafkaExtractionManagers)
+ {
+ if (manager.supports(kafkaProperties))
+ {
+ return manager;
+ }
+ }
+ return null;
+ }
+}
diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java
new file mode 100644
index 000000000000..351c9590aadc
--- /dev/null
+++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/server/namespace/KafkaExtractionNamespaceModule.java
@@ -0,0 +1,76 @@
+/*
+ * 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.core.type.TypeReference;
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.inject.Binder;
+import com.google.inject.Provides;
+import com.google.inject.name.Named;
+import io.druid.guice.LazySingleton;
+import io.druid.guice.LifecycleModule;
+import io.druid.guice.annotations.Json;
+import io.druid.initialization.DruidModule;
+import io.druid.query.extraction.namespace.KafkaExtractionNamespace;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ *
+ */
+public class KafkaExtractionNamespaceModule implements DruidModule
+{
+ private static final String PROPERTIES_KEY = "druid.query.rename.kafka.properties";
+ private List kafkaExtractionManagers = Lists.newLinkedList();
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return ImmutableList.of(
+ new SimpleModule("kafka-lookups").registerSubtypes(KafkaExtractionNamespace.class)
+ );
+ }
+
+ @Provides
+ @Named("kafkaManagers")
+ public List getManagers()
+ {
+ return kafkaExtractionManagers;
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ NamespacedExtractionModule
+ .getNamespaceFactoryMapBinder(binder)
+ .addBinding(KafkaExtractionNamespace.class)
+ .to(KafkaExtractionNamespaceFactory.class)
+ .in(LazySingleton.class);
+ }
+}
diff --git a/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule
index 48fadfc15c46..6a1d6a2e05c1 100644
--- a/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule
+++ b/extensions-core/kafka-extraction-namespace/src/main/resources/META-INF/services/io.druid.initialization.DruidModule
@@ -16,4 +16,4 @@
# specific language governing permissions and limitations
# under the License.
#
-io.druid.query.lookup.KafkaExtractionNamespaceModule
+io.druid.server.namespace.KafkaExtractionNamespaceModule
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
new file mode 100644
index 000000000000..3fb95c95b66a
--- /dev/null
+++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/KafkaExtractionNamespaceTest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.query.extraction.namespace;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.jackson.DefaultObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ *
+ */
+public class KafkaExtractionNamespaceTest
+{
+ @Test
+ public void testReflectiveSerde() throws IOException
+ {
+ ObjectMapper mapper = new DefaultObjectMapper();
+ mapper.registerSubtypes(KafkaExtractionNamespace.class);
+ final String val = "{\"type\":\"kafka\",\"kafkaTopic\":\"testTopic\",\"kafkaProperties\":{\"zookeeper.session.timeout.ms\":\"10000\",\"zookeeper.sync.time.ms\":\"200\"}}";
+ final ExtractionNamespace fn =
+ mapper.reader(ExtractionNamespace.class)
+ .readValue(
+ val
+ );
+ Assert.assertEquals(val, mapper.writeValueAsString(fn));
+ }
+
+ @Test(expected = com.fasterxml.jackson.databind.JsonMappingException.class)
+ public void testMissingTopic() throws IOException
+ {
+ ObjectMapper mapper = new DefaultObjectMapper();
+ mapper.registerSubtypes(KafkaExtractionNamespace.class);
+ final String val = "{\"type\":\"kafka\" }";
+ final ExtractionNamespace fn =
+ mapper.reader(ExtractionNamespace.class)
+ .readValue(
+ val
+ );
+ Assert.assertEquals(val, mapper.writeValueAsString(fn));
+ }
+
+}
diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java
new file mode 100644
index 000000000000..c14b9f15b340
--- /dev/null
+++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/extraction/namespace/TestKafkaExtractionCluster.java
@@ -0,0 +1,441 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.query.extraction.namespace;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Function;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import com.google.inject.Binder;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import com.google.inject.Module;
+import com.google.inject.Provider;
+import com.google.inject.TypeLiteral;
+import com.google.inject.name.Names;
+import com.metamx.common.ISE;
+import com.metamx.common.StringUtils;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.common.logger.Logger;
+import io.druid.guice.GuiceInjectors;
+import io.druid.initialization.Initialization;
+import io.druid.query.extraction.NamespaceLookupExtractorFactory;
+import io.druid.query.lookup.LookupExtractorFactory;
+import io.druid.server.namespace.KafkaExtractionManager;
+import io.druid.server.namespace.KafkaExtractionNamespaceFactory;
+import io.druid.server.namespace.KafkaExtractionNamespaceModule;
+import io.druid.server.namespace.NamespacedExtractionModule;
+import kafka.admin.AdminUtils;
+import kafka.javaapi.producer.Producer;
+import kafka.producer.KeyedMessage;
+import kafka.producer.ProducerConfig;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.Time;
+import kafka.utils.ZKStringSerializer$;
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.zookeeper.CreateMode;
+import org.joda.time.DateTime;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ *
+ */
+public class TestKafkaExtractionCluster
+{
+ private static final Logger log = new Logger(TestKafkaExtractionCluster.class);
+
+ private static final Lifecycle lifecycle = new Lifecycle();
+ private static final File tmpDir = Files.createTempDir();
+ private static final String topicName = "testTopic";
+ private static String namespace = "some_name";
+ private static final Properties kafkaProperties = new Properties();
+
+ private KafkaServer kafkaServer;
+ private KafkaConfig kafkaConfig;
+ private TestingServer zkTestServer;
+ private ZkClient zkClient;
+ private Injector injector;
+ private ObjectMapper mapper;
+ private LookupExtractorFactory factory;
+ private KafkaExtractionManager renameManager;
+
+ public static class KafkaFactoryProvider implements Provider>
+ {
+ private final List kafkaExtractionManager;
+
+ @Inject
+ public KafkaFactoryProvider(
+ KafkaExtractionManager kafkaExtractionManager
+ )
+ {
+ this.kafkaExtractionManager = Lists.newArrayList(kafkaExtractionManager);
+ }
+
+ @Override
+ public ExtractionNamespaceCacheFactory> get()
+ {
+ return new KafkaExtractionNamespaceFactory(kafkaExtractionManager);
+ }
+ }
+
+ @Before
+ public void setUp() throws Exception
+ {
+ zkTestServer = new TestingServer(-1, new File(tmpDir.getAbsolutePath() + "/zk"), true);
+ zkTestServer.start();
+
+ zkClient = new ZkClient(
+ zkTestServer.getConnectString(),
+ 10000,
+ 10000,
+ ZKStringSerializer$.MODULE$
+ );
+ if (!zkClient.exists("/kafka")) {
+ zkClient.create("/kafka", null, CreateMode.PERSISTENT);
+ }
+
+
+ log.info("---------------------------Started ZK---------------------------");
+
+
+ final Properties serverProperties = new Properties();
+ serverProperties.putAll(kafkaProperties);
+ serverProperties.put("broker.id", "0");
+ serverProperties.put("log.dir", tmpDir.getAbsolutePath() + "/log");
+ serverProperties.put("log.cleaner.enable", "true");
+ serverProperties.put("host.name", "127.0.0.1");
+ serverProperties.put("zookeeper.connect", zkTestServer.getConnectString() + "/kafka");
+ serverProperties.put("zookeeper.session.timeout.ms", "10000");
+ serverProperties.put("zookeeper.sync.time.ms", "200");
+
+ kafkaConfig = new KafkaConfig(serverProperties);
+
+ final long time = DateTime.parse("2015-01-01").getMillis();
+ kafkaServer = new KafkaServer(
+ kafkaConfig,
+ new Time()
+ {
+
+ @Override
+ public long milliseconds()
+ {
+ return time;
+ }
+
+ @Override
+ public long nanoseconds()
+ {
+ return milliseconds() * 1_000_000;
+ }
+
+ @Override
+ public void sleep(long ms)
+ {
+ try {
+ Thread.sleep(ms);
+ }
+ catch (InterruptedException e) {
+ throw Throwables.propagate(e);
+ }
+ }
+ }
+ );
+ kafkaServer.startup();
+
+ int sleepCount = 0;
+
+ while (!kafkaServer.kafkaController().isActive()) {
+ Thread.sleep(100);
+ if (++sleepCount > 10) {
+ throw new InterruptedException("Controller took to long to awaken");
+ }
+ }
+
+ log.info("---------------------------Started Kafka Server---------------------------");
+
+ ZkClient zkClient = new ZkClient(
+ zkTestServer.getConnectString() + "/kafka", 10000, 10000,
+ ZKStringSerializer$.MODULE$
+ );
+
+ try {
+ final Properties topicProperties = new Properties();
+ topicProperties.put("cleanup.policy", "compact");
+ if (!AdminUtils.topicExists(zkClient, topicName)) {
+ AdminUtils.createTopic(zkClient, topicName, 1, 1, topicProperties);
+ }
+
+ log.info("---------------------------Created topic---------------------------");
+
+ Assert.assertTrue(AdminUtils.topicExists(zkClient, topicName));
+ }
+ finally {
+ zkClient.close();
+ }
+
+ final Properties kafkaProducerProperties = makeProducerProperties();
+ Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties));
+
+ try {
+ producer.send(
+ new KeyedMessage<>(
+ topicName,
+ StringUtils.toUtf8("abcdefg"),
+ StringUtils.toUtf8("abcdefg")
+ )
+ );
+ }
+ catch (Exception e) {
+ throw Throwables.propagate(e);
+ }
+ finally {
+ producer.close();
+ }
+
+ System.setProperty("druid.extensions.searchCurrentClassloader", "false");
+
+ injector = Initialization.makeInjectorWithModules(
+ GuiceInjectors.makeStartupInjectorWithModules(
+ ImmutableList.of()
+ ),
+ ImmutableList.of(
+ new Module()
+ {
+ @Override
+ public void configure(Binder binder)
+ {
+ binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test");
+ binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
+ }
+ },
+ new NamespacedExtractionModule(),
+ new KafkaExtractionNamespaceModule()
+ )
+ );
+ mapper = injector.getInstance(ObjectMapper.class);
+
+ log.info("--------------------------- placed default item via producer ---------------------------");
+ final Properties consumerProperties = new Properties(kafkaProperties);
+ consumerProperties.put("zookeeper.connect", zkTestServer.getConnectString() + "/kafka");
+ consumerProperties.put("zookeeper.session.timeout.ms", "10000");
+ consumerProperties.put("zookeeper.sync.time.ms", "200");
+ ExtractionNamespace extractionNamespace = new KafkaExtractionNamespace(topicName, consumerProperties);
+ String namespaceString = mapper.writeValueAsString(extractionNamespace);
+ String json = String.format("{\"type\":\"namespace\", \"extractionNamespace\":%s}", namespaceString);
+ factory = mapper.readValue(json, LookupExtractorFactory.class);
+ factory.start();
+ renameManager = injector.getInstance(
+ Key.get(
+ new TypeLiteral>()
+ {
+ },
+ Names.named("kafkaManagers")
+ )
+ ).get(0);
+
+ log.info("--------------------------- started rename manager ---------------------------");
+ }
+
+ @After
+ public void tearDown() throws Exception
+ {
+
+ lifecycle.stop();
+ if (renameManager != null)
+ {
+ renameManager.stop();
+ }
+ factory.close();
+
+ if (null != kafkaServer) {
+ kafkaServer.shutdown();
+ kafkaServer.awaitShutdown();
+ }
+
+ if (null != zkClient) {
+ if (zkClient.exists("/kafka")) {
+ try {
+ zkClient.deleteRecursive("/kafka");
+ }
+ catch (org.I0Itec.zkclient.exception.ZkException ex) {
+ log.warn(ex, "error deleting /kafka zk node");
+ }
+ }
+ zkClient.close();
+ }
+ if (null != zkTestServer) {
+ zkTestServer.stop();
+ }
+ if (tmpDir.exists()) {
+ FileUtils.deleteDirectory(tmpDir);
+ }
+ }
+
+ private final Properties makeProducerProperties()
+ {
+ final Properties kafkaProducerProperties = new Properties();
+ kafkaProducerProperties.putAll(kafkaProperties);
+ kafkaProducerProperties.put(
+ "metadata.broker.list",
+ String.format("127.0.0.1:%d", kafkaServer.socketServer().port())
+ );
+ kafkaProperties.put("request.required.acks", "1");
+ return kafkaProducerProperties;
+ }
+
+ private void checkServer()
+ {
+ if (!kafkaServer.apis().controller().isActive()) {
+ throw new ISE("server is not active!");
+ }
+ }
+
+ @Ignore // Should be un-ignored after kafka rewrite to Lookups is merged
+ @Test(timeout = 60_000L)
+ public void testSimpleRename() throws InterruptedException
+ {
+ final Properties kafkaProducerProperties = makeProducerProperties();
+ final Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties));
+
+ try {
+ checkServer();
+
+ final ConcurrentMap> fnFn =
+ injector.getInstance(
+ Key.get(
+ new TypeLiteral>>()
+ {
+ },
+ Names.named("namespaceExtractionFunctionCache")
+ )
+ );
+
+ final ConcurrentMap>> reverseFn =
+ injector.getInstance(
+ Key.get(
+ new TypeLiteral>>>()
+ {
+ },
+ Names.named("namespaceReverseExtractionFunctionCache")
+ )
+ );
+
+ assertUpdated(null, namespace, "foo", fnFn);
+ assertReverseUpdated(Collections.EMPTY_LIST, namespace, "foo", reverseFn);
+
+ long events = renameManager.getNumEvents(namespace);
+
+ log.info("------------------------- Sending foo bar -------------------------------");
+ producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("foo"), StringUtils.toUtf8("bar")));
+
+ long start = System.currentTimeMillis();
+ while (events == renameManager.getNumEvents(namespace)) {
+ Thread.sleep(100);
+ if (System.currentTimeMillis() > start + 60_000) {
+ throw new ISE("Took too long to update event");
+ }
+ }
+
+ log.info("------------------------- Checking foo bar -------------------------------");
+ assertUpdated("bar", namespace, "foo", fnFn);
+ assertReverseUpdated(Arrays.asList("foo"), namespace, "bar", reverseFn);
+ assertUpdated(null, namespace, "baz", fnFn);
+
+ checkServer();
+ events = renameManager.getNumEvents(namespace);
+
+ log.info("------------------------- Sending baz bat -------------------------------");
+ producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("baz"), StringUtils.toUtf8("bat")));
+ while (events == renameManager.getNumEvents(namespace)) {
+ Thread.sleep(10);
+ if (System.currentTimeMillis() > start + 60_000) {
+ throw new ISE("Took too long to update event");
+ }
+ }
+
+ log.info("------------------------- Checking baz bat -------------------------------");
+ Assert.assertEquals("bat", fnFn.get(namespace).apply("baz"));
+ Assert.assertEquals(Arrays.asList("baz"), reverseFn.get(namespace).apply("bat"));
+ }
+ finally {
+ producer.close();
+ }
+ }
+
+ private void assertUpdated(
+ String expected,
+ String namespace,
+ String key,
+ ConcurrentMap> lookup
+ )
+ throws InterruptedException
+ {
+ Function extractionFn = lookup.get(namespace);
+
+ if (expected == null) {
+ while (extractionFn.apply(key) != null) {
+ Thread.sleep(100);
+ extractionFn = lookup.get(namespace);
+ }
+ } else {
+ while (!expected.equals(extractionFn.apply(key))) {
+ Thread.sleep(100);
+ extractionFn = lookup.get(namespace);
+ }
+ }
+
+ Assert.assertEquals("update check", expected, extractionFn.apply(key));
+ }
+
+ private void assertReverseUpdated(
+ List expected,
+ String namespace,
+ String key,
+ ConcurrentMap>> lookup
+ )
+ throws InterruptedException
+ {
+ Function> extractionFn = lookup.get(namespace);
+
+ while (!extractionFn.apply(key).equals(expected)) {
+ Thread.sleep(100);
+ extractionFn = lookup.get(namespace);
+ }
+
+ Assert.assertEquals("update check", expected, extractionFn.apply(key));
+ }
+}
diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java
deleted file mode 100644
index a8a40d0deafc..000000000000
--- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java
+++ /dev/null
@@ -1,516 +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.lookup;
-
-import com.fasterxml.jackson.databind.BeanProperty;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.InjectableValues;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.metamx.common.StringUtils;
-import io.druid.jackson.DefaultObjectMapper;
-import io.druid.server.namespace.cache.NamespaceExtractionCacheManager;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.KafkaStream;
-import kafka.consumer.TopicFilter;
-import kafka.javaapi.consumer.ConsumerConnector;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static io.druid.query.lookup.KafkaLookupExtractorFactory.DEFAULT_STRING_DECODER;
-
-public class KafkaLookupExtractorFactoryTest
-{
- private static final String TOPIC = "some_topic";
- private static final Map DEFAULT_PROPERTIES = ImmutableMap.of(
- "some.property", "some.value"
- );
- private final ObjectMapper mapper = new DefaultObjectMapper();
- final NamespaceExtractionCacheManager cacheManager = EasyMock.createStrictMock(NamespaceExtractionCacheManager.class);
-
- @Rule
- public ExpectedException expectedException = ExpectedException.none();
-
- @Before
- public void setUp()
- {
- mapper.setInjectableValues(new InjectableValues()
- {
- @Override
- public Object findInjectableValue(
- Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
- )
- {
- if ("io.druid.server.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) {
- return cacheManager;
- } else {
- return null;
- }
- }
- });
- }
-
- @Test
- public void testSimpleSerDe() throws Exception
- {
- final KafkaLookupExtractorFactory expected = new KafkaLookupExtractorFactory(null, TOPIC, DEFAULT_PROPERTIES);
- final KafkaLookupExtractorFactory result = mapper.readValue(
- mapper.writeValueAsString(expected),
- KafkaLookupExtractorFactory.class
- );
- Assert.assertEquals(expected.getKafkaTopic(), result.getKafkaTopic());
- Assert.assertEquals(expected.getKafkaProperties(), result.getKafkaProperties());
- Assert.assertEquals(cacheManager, result.getCacheManager());
- Assert.assertEquals(0, expected.getCompletedEventCount());
- Assert.assertEquals(0, result.getCompletedEventCount());
- }
-
- @Test
- public void testCacheKeyScramblesOnNewData()
- {
- final int n = 1000;
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- );
- factory.getMapRef().set(ImmutableMap.of());
- final AtomicLong events = factory.getDoubleEventCount();
-
- final LookupExtractor extractor = factory.get();
-
- final List byteArrays = new ArrayList<>(n);
- for (int i = 0; i < n; ++i) {
- final byte[] myKey = extractor.getCacheKey();
- // Not terribly efficient.. but who cares
- for (byte[] byteArray : byteArrays) {
- Assert.assertFalse(Arrays.equals(byteArray, myKey));
- }
- byteArrays.add(myKey);
- events.incrementAndGet();
- }
- Assert.assertEquals(n, byteArrays.size());
- }
-
- @Test
- public void testCacheKeyScramblesDifferentStarts()
- {
- final int n = 1000;
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- );
- factory.getMapRef().set(ImmutableMap.of());
- final AtomicLong events = factory.getDoubleEventCount();
-
- final List byteArrays = new ArrayList<>(n);
- for (int i = 0; i < n; ++i) {
- final LookupExtractor extractor = factory.get();
- final byte[] myKey = extractor.getCacheKey();
- // Not terribly efficient.. but who cares
- for (byte[] byteArray : byteArrays) {
- Assert.assertFalse(Arrays.equals(byteArray, myKey));
- }
- byteArrays.add(myKey);
- events.incrementAndGet();
- }
- Assert.assertEquals(n, byteArrays.size());
- }
-
- @Test
- public void testCacheKeySameOnNoChange()
- {
- final int n = 1000;
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- );
- factory.getMapRef().set(ImmutableMap.of());
-
- final LookupExtractor extractor = factory.get();
-
- final byte[] baseKey = extractor.getCacheKey();
- for (int i = 0; i < n; ++i) {
- Assert.assertArrayEquals(baseKey, factory.get().getCacheKey());
- }
- }
-
- @Test
- public void testCacheKeyDifferentForTopics()
- {
- final KafkaLookupExtractorFactory factory1 = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- );
- factory1.getMapRef().set(ImmutableMap.of());
- final KafkaLookupExtractorFactory factory2 = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC + "b",
- DEFAULT_PROPERTIES
- );
- factory2.getMapRef().set(ImmutableMap.of());
-
- Assert.assertFalse(Arrays.equals(factory1.get().getCacheKey(), factory2.get().getCacheKey()));
- }
-
- @Test
- public void testReplaces()
- {
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- );
- Assert.assertTrue(factory.replaces(new MapLookupExtractorFactory(ImmutableMap.of(), false)));
- Assert.assertFalse(factory.replaces(factory));
- Assert.assertFalse(factory.replaces(new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- )));
-
- Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC + "b",
- DEFAULT_PROPERTIES
- )));
-
- Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("some.property", "some.other.value")
- )));
-
- Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("some.other.property", "some.value")
- )));
-
- Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES,
- 1,
- false
- )));
-
- Assert.assertTrue(factory.replaces(new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES,
- 0,
- true
- )));
- }
-
- @Test
- public void testStopWithoutStart()
- {
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- );
- Assert.assertTrue(factory.close());
- }
-
- @Test
- public void testStartStop()
- {
- final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class);
- final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class);
- final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class);
- EasyMock.expect(consumerConnector.createMessageStreamsByFilter(
- EasyMock.anyObject(TopicFilter.class),
- EasyMock.anyInt(),
- EasyMock.eq(
- DEFAULT_STRING_DECODER),
- EasyMock.eq(DEFAULT_STRING_DECODER)
- )).andReturn(ImmutableList.of(kafkaStream)).once();
- EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes();
- EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes();
- EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString()))
- .andReturn(new ConcurrentHashMap())
- .once();
- EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once();
- consumerConnector.shutdown();
- EasyMock.expectLastCall().once();
- EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("zookeeper.connect", "localhost"),
- 10_000L,
- false
- )
- {
- @Override
- ConsumerConnector buildConnector(Properties properties)
- {
- return consumerConnector;
- }
- };
- Assert.assertTrue(factory.start());
- Assert.assertTrue(factory.close());
- Assert.assertTrue(factory.getFuture().isDone());
- EasyMock.verify(cacheManager);
- }
-
-
- @Test
- public void testStartFailsFromTimeout() throws Exception
- {
- EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString()))
- .andReturn(new ConcurrentHashMap())
- .once();
- EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once();
- EasyMock.replay(cacheManager);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("zookeeper.connect", "localhost"),
- 1,
- false
- )
- {
- @Override
- ConsumerConnector buildConnector(Properties properties)
- {
- // Lock up
- try {
- Thread.currentThread().join();
- }
- catch (InterruptedException e) {
- throw Throwables.propagate(e);
- }
- throw new RuntimeException("shouldn't make it here");
- }
- };
- Assert.assertFalse(factory.start());
- Assert.assertTrue(factory.getFuture().isDone());
- Assert.assertTrue(factory.getFuture().isCancelled());
- EasyMock.verify(cacheManager);
- }
-
- @Test
- public void testStopDeleteError()
- {
- final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class);
- final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class);
- final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class);
- EasyMock.expect(consumerConnector.createMessageStreamsByFilter(
- EasyMock.anyObject(TopicFilter.class),
- EasyMock.anyInt(),
- EasyMock.eq(
- DEFAULT_STRING_DECODER),
- EasyMock.eq(DEFAULT_STRING_DECODER)
- )).andReturn(ImmutableList.of(kafkaStream)).once();
- EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes();
- EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes();
- EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString()))
- .andReturn(new ConcurrentHashMap())
- .once();
- EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(false).once();
-
- EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("zookeeper.connect", "localhost")
- )
- {
- @Override
- ConsumerConnector buildConnector(Properties properties)
- {
- return consumerConnector;
- }
- };
- Assert.assertTrue(factory.start());
- Assert.assertFalse(factory.close());
- EasyMock.verify(cacheManager, kafkaStream, consumerConnector, consumerIterator);
- }
-
-
- @Test
- public void testStartStopStart()
- {
- final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class);
- final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class);
- final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class);
- EasyMock.expect(consumerConnector.createMessageStreamsByFilter(
- EasyMock.anyObject(TopicFilter.class),
- EasyMock.anyInt(),
- EasyMock.eq(
- DEFAULT_STRING_DECODER),
- EasyMock.eq(DEFAULT_STRING_DECODER)
- )).andReturn(ImmutableList.of(kafkaStream)).once();
- EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes();
- EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes();
- EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString()))
- .andReturn(new ConcurrentHashMap())
- .once();
- EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once();
- consumerConnector.shutdown();
- EasyMock.expectLastCall().once();
- EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("zookeeper.connect", "localhost")
- )
- {
- @Override
- ConsumerConnector buildConnector(Properties properties)
- {
- return consumerConnector;
- }
- };
- Assert.assertTrue(factory.start());
- Assert.assertTrue(factory.close());
- Assert.assertFalse(factory.start());
- EasyMock.verify(cacheManager);
- }
-
- @Test
- public void testStartStartStop()
- {
- final KafkaStream kafkaStream = EasyMock.createStrictMock(KafkaStream.class);
- final ConsumerIterator consumerIterator = EasyMock.createStrictMock(ConsumerIterator.class);
- final ConsumerConnector consumerConnector = EasyMock.createStrictMock(ConsumerConnector.class);
- EasyMock.expect(consumerConnector.createMessageStreamsByFilter(
- EasyMock.anyObject(TopicFilter.class),
- EasyMock.anyInt(),
- EasyMock.eq(
- DEFAULT_STRING_DECODER),
- EasyMock.eq(DEFAULT_STRING_DECODER)
- )).andReturn(ImmutableList.of(kafkaStream)).once();
- EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes();
- EasyMock.expect(consumerIterator.hasNext()).andReturn(false).anyTimes();
- EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString()))
- .andReturn(new ConcurrentHashMap())
- .once();
- EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once();
- consumerConnector.shutdown();
- EasyMock.expectLastCall().once();
- EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("zookeeper.connect", "localhost"),
- 10_000L,
- false
- )
- {
- @Override
- ConsumerConnector buildConnector(Properties properties)
- {
- return consumerConnector;
- }
- };
- Assert.assertTrue(factory.start());
- Assert.assertTrue(factory.start());
- Assert.assertTrue(factory.close());
- Assert.assertTrue(factory.close());
- EasyMock.verify(cacheManager);
- }
-
- @Test
- public void testStartFailsOnMissingConnect()
- {
- expectedException.expectMessage("zookeeper.connect required property");
- EasyMock.replay(cacheManager);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of()
- );
- Assert.assertTrue(factory.start());
- Assert.assertTrue(factory.close());
- EasyMock.verify(cacheManager);
- }
-
- @Test
- public void testStartFailsOnGroupID()
- {
- expectedException.expectMessage(
- "Cannot set kafka property [group.id]. Property is randomly generated for you. Found");
- EasyMock.replay(cacheManager);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("group.id", "make me fail")
- );
- Assert.assertTrue(factory.start());
- Assert.assertTrue(factory.close());
- EasyMock.verify(cacheManager);
- }
-
- @Test
- public void testStartFailsOnAutoOffset()
- {
- expectedException.expectMessage(
- "Cannot set kafka property [auto.offset.reset]. Property will be forced to [smallest]. Found ");
- EasyMock.replay(cacheManager);
- final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- ImmutableMap.of("auto.offset.reset", "make me fail")
- );
- Assert.assertTrue(factory.start());
- Assert.assertTrue(factory.close());
- EasyMock.verify(cacheManager);
- }
-
- @Test
- public void testFailsGetNotStarted()
- {
- expectedException.expectMessage("Not started");
- new KafkaLookupExtractorFactory(
- cacheManager,
- TOPIC,
- DEFAULT_PROPERTIES
- ).get();
- }
-
- @Test
- public void testDefaultDecoder()
- {
- final String str = "some string";
- Assert.assertEquals(str, DEFAULT_STRING_DECODER.fromBytes(StringUtils.toUtf8(str)));
- }
-}
diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java
deleted file mode 100644
index bcd1aebe9bb9..000000000000
--- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java
+++ /dev/null
@@ -1,400 +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.lookup;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.io.Closer;
-import com.google.inject.Binder;
-import com.google.inject.Injector;
-import com.google.inject.Module;
-import com.google.inject.name.Names;
-import com.metamx.common.ISE;
-import com.metamx.common.StringUtils;
-import com.metamx.common.logger.Logger;
-import io.druid.guice.GuiceInjectors;
-import io.druid.initialization.Initialization;
-import io.druid.server.namespace.NamespacedExtractionModule;
-import kafka.admin.AdminUtils;
-import kafka.javaapi.producer.Producer;
-import kafka.producer.KeyedMessage;
-import kafka.producer.ProducerConfig;
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-import kafka.utils.Time;
-import kafka.utils.ZKStringSerializer$;
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.curator.test.TestingServer;
-import org.apache.zookeeper.CreateMode;
-import org.joda.time.DateTime;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- *
- */
-public class TestKafkaExtractionCluster
-{
- private static final Logger log = new Logger(TestKafkaExtractionCluster.class);
- private static final String topicName = "testTopic";
- private static final Map kafkaProperties = new HashMap<>();
-
- @Rule
- public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
- private final Closer closer = Closer.create();
-
- private KafkaServer kafkaServer;
- private KafkaConfig kafkaConfig;
- private TestingServer zkTestServer;
- private ZkClient zkClient;
- private Injector injector;
- private ObjectMapper mapper;
- private KafkaLookupExtractorFactory factory;
-
- @Before
- public void setUp() throws Exception
- {
- zkTestServer = new TestingServer(-1, temporaryFolder.newFolder(), true);
- zkTestServer.start();
-
- closer.register(new Closeable()
- {
- @Override
- public void close() throws IOException
- {
- zkTestServer.stop();
- }
- });
-
- zkClient = new ZkClient(
- zkTestServer.getConnectString(),
- 10000,
- 10000,
- ZKStringSerializer$.MODULE$
- );
- closer.register(new Closeable()
- {
- @Override
- public void close() throws IOException
- {
- zkClient.close();
- }
- });
- if (!zkClient.exists("/kafka")) {
- zkClient.create("/kafka", null, CreateMode.PERSISTENT);
- }
-
- log.info("---------------------------Started ZK---------------------------");
-
- final String zkKafkaPath = "/kafka";
-
- final Properties serverProperties = new Properties();
- serverProperties.putAll(kafkaProperties);
- serverProperties.put("broker.id", "0");
- serverProperties.put("log.dir", temporaryFolder.newFolder().getAbsolutePath());
- serverProperties.put("log.cleaner.enable", "true");
- serverProperties.put("host.name", "127.0.0.1");
- serverProperties.put("zookeeper.connect", zkTestServer.getConnectString() + zkKafkaPath);
- serverProperties.put("zookeeper.session.timeout.ms", "10000");
- serverProperties.put("zookeeper.sync.time.ms", "200");
-
- kafkaConfig = new KafkaConfig(serverProperties);
-
- final long time = DateTime.parse("2015-01-01").getMillis();
- kafkaServer = new KafkaServer(
- kafkaConfig,
- new Time()
- {
-
- @Override
- public long milliseconds()
- {
- return time;
- }
-
- @Override
- public long nanoseconds()
- {
- return milliseconds() * 1_000_000;
- }
-
- @Override
- public void sleep(long ms)
- {
- try {
- Thread.sleep(ms);
- }
- catch (InterruptedException e) {
- throw Throwables.propagate(e);
- }
- }
- }
- );
- kafkaServer.startup();
- closer.register(new Closeable()
- {
- @Override
- public void close() throws IOException
- {
- kafkaServer.shutdown();
- kafkaServer.awaitShutdown();
- }
- });
-
- int sleepCount = 0;
-
- while (!kafkaServer.kafkaController().isActive()) {
- Thread.sleep(100);
- if (++sleepCount > 10) {
- throw new InterruptedException("Controller took to long to awaken");
- }
- }
-
- log.info("---------------------------Started Kafka Server---------------------------");
-
- final ZkClient zkClient = new ZkClient(
- zkTestServer.getConnectString() + zkKafkaPath, 10000, 10000,
- ZKStringSerializer$.MODULE$
- );
-
- try (final AutoCloseable autoCloseable = new AutoCloseable()
- {
- @Override
- public void close() throws Exception
- {
- if (zkClient.exists(zkKafkaPath)) {
- try {
- zkClient.deleteRecursive(zkKafkaPath);
- }
- catch (org.I0Itec.zkclient.exception.ZkException ex) {
- log.warn(ex, "error deleting %s zk node", zkKafkaPath);
- }
- }
- zkClient.close();
- }
- }) {
- final Properties topicProperties = new Properties();
- topicProperties.put("cleanup.policy", "compact");
- if (!AdminUtils.topicExists(zkClient, topicName)) {
- AdminUtils.createTopic(zkClient, topicName, 1, 1, topicProperties);
- }
-
- log.info("---------------------------Created topic---------------------------");
-
- Assert.assertTrue(AdminUtils.topicExists(zkClient, topicName));
- }
-
- final Properties kafkaProducerProperties = makeProducerProperties();
- final Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties));
- try (final AutoCloseable autoCloseable = new AutoCloseable()
- {
- @Override
- public void close() throws Exception
- {
- producer.close();
- }
- }) {
- producer.send(
- new KeyedMessage<>(
- topicName,
- StringUtils.toUtf8("abcdefg"),
- StringUtils.toUtf8("abcdefg")
- )
- );
- }
-
- System.setProperty("druid.extensions.searchCurrentClassloader", "false");
-
- injector = Initialization.makeInjectorWithModules(
- GuiceInjectors.makeStartupInjector(),
- ImmutableList.of(
- new Module()
- {
- @Override
- public void configure(Binder binder)
- {
- binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test");
- binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
- }
- },
- // These injections fail under IntelliJ but are required for maven
- new NamespacedExtractionModule(),
- new KafkaExtractionNamespaceModule()
- )
- );
- mapper = injector.getInstance(ObjectMapper.class);
-
- log.info("--------------------------- placed default item via producer ---------------------------");
- final Map consumerProperties = new HashMap<>(kafkaProperties);
- consumerProperties.put("zookeeper.connect", zkTestServer.getConnectString() + zkKafkaPath);
- consumerProperties.put("zookeeper.session.timeout.ms", "10000");
- consumerProperties.put("zookeeper.sync.time.ms", "200");
-
- final KafkaLookupExtractorFactory kafkaLookupExtractorFactory = new KafkaLookupExtractorFactory(
- null,
- topicName,
- consumerProperties
- );
-
- factory = (KafkaLookupExtractorFactory) mapper.readValue(
- mapper.writeValueAsString(kafkaLookupExtractorFactory),
- LookupExtractorFactory.class
- );
- Assert.assertEquals(kafkaLookupExtractorFactory.getKafkaTopic(), factory.getKafkaTopic());
- Assert.assertEquals(kafkaLookupExtractorFactory.getKafkaProperties(), factory.getKafkaProperties());
- factory.start();
- closer.register(new Closeable()
- {
- @Override
- public void close() throws IOException
- {
- factory.close();
- }
- });
- log.info("--------------------------- started rename manager ---------------------------");
- }
-
- @After
- public void tearDown() throws Exception
- {
- closer.close();
- }
-
- private final Properties makeProducerProperties()
- {
- final Properties kafkaProducerProperties = new Properties();
- kafkaProducerProperties.putAll(kafkaProperties);
- kafkaProducerProperties.put(
- "metadata.broker.list",
- String.format("127.0.0.1:%d", kafkaServer.socketServer().port())
- );
- kafkaProperties.put("request.required.acks", "1");
- return kafkaProducerProperties;
- }
-
- private void checkServer()
- {
- if (!kafkaServer.apis().controller().isActive()) {
- throw new ISE("server is not active!");
- }
- }
-
- @Test(timeout = 60_000L)
- public void testSimpleRename() throws InterruptedException
- {
- final Properties kafkaProducerProperties = makeProducerProperties();
- final Producer producer = new Producer<>(new ProducerConfig(kafkaProducerProperties));
- closer.register(new Closeable()
- {
- @Override
- public void close() throws IOException
- {
- producer.close();
- }
- });
- checkServer();
-
- assertUpdated(null, "foo");
- assertReverseUpdated(ImmutableList.of(), "foo");
-
- long events = factory.getCompletedEventCount();
-
- log.info("------------------------- Sending foo bar -------------------------------");
- producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("foo"), StringUtils.toUtf8("bar")));
-
- long start = System.currentTimeMillis();
- while (events == factory.getCompletedEventCount()) {
- Thread.sleep(100);
- if (System.currentTimeMillis() > start + 60_000) {
- throw new ISE("Took too long to update event");
- }
- }
-
- log.info("------------------------- Checking foo bar -------------------------------");
- assertUpdated("bar", "foo");
- assertReverseUpdated(Collections.singletonList("foo"), "bar");
- assertUpdated(null, "baz");
-
- checkServer();
- events = factory.getCompletedEventCount();
-
- log.info("------------------------- Sending baz bat -------------------------------");
- producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("baz"), StringUtils.toUtf8("bat")));
- while (events == factory.getCompletedEventCount()) {
- Thread.sleep(10);
- if (System.currentTimeMillis() > start + 60_000) {
- throw new ISE("Took too long to update event");
- }
- }
-
- log.info("------------------------- Checking baz bat -------------------------------");
- Assert.assertEquals("bat", factory.get().apply("baz"));
- Assert.assertEquals(Collections.singletonList("baz"), factory.get().unapply("bat"));
- }
-
- private void assertUpdated(
- String expected,
- String key
- )
- throws InterruptedException
- {
- final LookupExtractor extractor = factory.get();
- if (expected == null) {
- while (extractor.apply(key) != null) {
- Thread.sleep(100);
- }
- } else {
- while (!expected.equals(extractor.apply(key))) {
- Thread.sleep(100);
- }
- }
-
- Assert.assertEquals("update check", expected, extractor.apply(key));
- }
-
- private void assertReverseUpdated(
- List expected,
- String key
- )
- throws InterruptedException
- {
- final LookupExtractor extractor = factory.get();
-
- while (!expected.equals(extractor.unapply(key))) {
- Thread.sleep(100);
- }
-
- Assert.assertEquals("update check", expected, extractor.unapply(key));
- }
-}
diff --git a/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml b/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml
index 1601a94c87a8..5e1ebff74c39 100644
--- a/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml
+++ b/extensions-core/kafka-extraction-namespace/src/test/resources/log4j2.xml
@@ -28,7 +28,7 @@
-
+
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
new file mode 100644
index 000000000000..499bf41b2c57
--- /dev/null
+++ b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespaceLookupExtractorFactory.java
@@ -0,0 +1,220 @@
+/*
+ * 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.Preconditions;
+import com.metamx.common.ISE;
+import com.metamx.common.StringUtils;
+import com.metamx.common.logger.Logger;
+import io.druid.query.extraction.namespace.ExtractionNamespace;
+import io.druid.query.lookup.LookupExtractor;
+import io.druid.query.lookup.LookupExtractorFactory;
+import io.druid.query.lookup.LookupIntrospectHandler;
+import io.druid.server.namespace.cache.NamespaceExtractionCacheManager;
+
+import 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;
+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 long SCHEDULE_TIMEOUT = 60_000;
+
+ private final AtomicBoolean started = new AtomicBoolean(false);
+ private final ReadWriteLock startStopSync = new ReentrantReadWriteLock();
+ private final ExtractionNamespace extractionNamespace;
+ private final NamespaceExtractionCacheManager manager;
+ private final LookupIntrospectHandler lookupIntrospectHandler;
+
+ private final String extractorID;
+
+ @JsonCreator
+ public NamespaceLookupExtractorFactory(
+ @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace,
+ @JacksonInject NamespaceExtractionCacheManager manager
+ )
+ {
+ this.extractionNamespace = Preconditions.checkNotNull(
+ extractionNamespace,
+ "extractionNamespace should be specified"
+ );
+ 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
+ public boolean start()
+ {
+ 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();
+ }
+ }
+
+ @Override
+ public boolean close()
+ {
+ 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
+ public boolean replaces(@Nullable LookupExtractorFactory other)
+ {
+ if (other != null && other instanceof NamespaceLookupExtractorFactory) {
+ NamespaceLookupExtractorFactory that = (NamespaceLookupExtractorFactory) other;
+ return !extractionNamespace.equals(that.extractionNamespace);
+ }
+ return true;
+ }
+
+ @Override
+ public LookupIntrospectHandler getIntrospectHandler()
+ {
+ return lookupIntrospectHandler;
+ }
+
+ @JsonProperty
+ public ExtractionNamespace getExtractionNamespace()
+ {
+ return extractionNamespace;
+ }
+
+ private String buildID()
+ {
+ return UUID.randomUUID().toString();
+ }
+
+ // Grab the latest snapshot from the cache manager
+ @Override
+ public LookupExtractor get()
+ {
+ 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/query/extraction/NamespacedExtractor.java b/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java
deleted file mode 100644
index a8f8a441f8f4..000000000000
--- a/extensions-core/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java
+++ /dev/null
@@ -1,92 +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 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("dimExtractionNamespace")
- final Function> namespaces,
- @NotNull @JacksonInject @Named("dimReverseExtractionNamespace")
- 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/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/ExtractionNamespaceCacheFactory.java
similarity index 65%
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 a45ae0b0cc5c..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`
@@ -60,6 +39,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 a55d3f524675..aa522fdf6d51 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
@@ -74,8 +72,6 @@ public class URIExtractionNamespace implements ExtractionNamespace
@JsonCreator
public URIExtractionNamespace(
- @NotNull @JsonProperty(value = "namespace", required = true)
- String namespace,
@JsonProperty(value = "uri", required = false)
URI uri,
@JsonProperty(value = "uriPrefix", required = false)
@@ -91,7 +87,6 @@ public URIExtractionNamespace(
String versionRegex
)
{
- this.namespace = Preconditions.checkNotNull(namespace, "namespace");
this.uri = uri;
this.uriPrefix = uriPrefix;
if ((uri != null) == (uriPrefix != null)) {
@@ -118,12 +113,6 @@ public URIExtractionNamespace(
}
}
- @Override
- public String getNamespace()
- {
- return namespace;
- }
-
public String getFileRegex()
{
return fileRegex;
@@ -154,8 +143,7 @@ public long getPollMs()
public String toString()
{
return "URIExtractionNamespace{" +
- "namespace='" + namespace + '\'' +
- ", uri=" + uri +
+ "uri=" + uri +
", uriPrefix=" + uriPrefix +
", namespaceParseSpec=" + namespaceParseSpec +
", fileRegex='" + fileRegex + '\'' +
@@ -175,9 +163,6 @@ public boolean equals(Object o)
URIExtractionNamespace that = (URIExtractionNamespace) o;
- if (!getNamespace().equals(that.getNamespace())) {
- return false;
- }
if (getUri() != null ? !getUri().equals(that.getUri()) : that.getUri() != null) {
return false;
}
@@ -197,8 +182,7 @@ public boolean equals(Object o)
@Override
public int hashCode()
{
- int result = getNamespace().hashCode();
- result = 31 * result + (getUri() != null ? getUri().hashCode() : 0);
+ int result = getUri() != null ? getUri().hashCode() : 0;
result = 31 * result + (getUriPrefix() != null ? getUriPrefix().hashCode() : 0);
result = 31 * result + getNamespaceParseSpec().hashCode();
result = 31 * result + (getFileRegex() != null ? getFileRegex().hashCode() : 0);
@@ -338,6 +322,28 @@ public Parser getParser()
return parser;
}
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ CSVFlatDataParser that = (CSVFlatDataParser) o;
+
+ if (!getColumns().equals(that.getColumns())) {
+ return false;
+ }
+ if (!getKeyColumn().equals(that.getKeyColumn())) {
+ return false;
+ }
+
+ return getValueColumn().equals(that.getValueColumn());
+ }
+
@Override
public String toString()
{
@@ -430,6 +436,31 @@ public Parser getParser()
return parser;
}
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ TSVFlatDataParser that = (TSVFlatDataParser) o;
+
+ if (!getColumns().equals(that.getColumns())) {
+ return false;
+ }
+ if ((getDelimiter() == null) ? that.getDelimiter() == null : getDelimiter().equals(that.getDelimiter())) {
+ return false;
+ }
+ if (!getKeyColumn().equals(that.getKeyColumn())) {
+ return false;
+ }
+
+ return getValueColumn().equals(that.getValueColumn());
+ }
+
@Override
public String toString()
{
@@ -486,6 +517,25 @@ public Parser getParser()
return this.parser;
}
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ JSONFlatDataParser that = (JSONFlatDataParser) o;
+
+ if (!getKeyFieldName().equals(that.getKeyFieldName())) {
+ return false;
+ }
+
+ return getValueFieldName().equals(that.getValueFieldName());
+ }
+
@Override
public String toString()
{
@@ -545,6 +595,19 @@ public Parser getParser()
return parser;
}
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ return true;
+ }
+
@Override
public String toString()
{
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/JDBCExtractionNamespaceCacheFactory.java
similarity index 72%
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 ae2b50dc18e9..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,60 +43,22 @@
/**
*
*/
-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,
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 +74,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 +117,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 +142,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..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
@@ -24,33 +24,24 @@
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.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.NamespacedExtractor;
+import io.druid.query.extraction.NamespaceLookupExtractorFactory;
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.initialization.NamespaceLookupStaticConfig;
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;
@@ -62,29 +53,32 @@
*/
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<>();
+ private final ConcurrentMap>> reverseFnCache = new ConcurrentHashMap<>();
@Override
public List extends Module> getJacksonModules()
{
return ImmutableList.of(
new SimpleModule("DruidNamespacedExtractionModule")
- {
- @Override
- public void setupModule(SetupContext context)
- {
- context.registerSubtypes(NamespacedExtractor.class);
- context.registerSubtypes(ExtractionNamespace.class);
- }
- }
+ .registerSubtypes(
+ ExtractionNamespace.class,
+ NamespaceLookupExtractorFactory.class
+ )
);
}
- public static MapBinder, ExtractionNamespaceFunctionFactory>> getNamespaceFactoryMapBinder(
+ public static MapBinder, ExtractionNamespaceCacheFactory>> getNamespaceFactoryMapBinder(
final Binder binder
)
{
@@ -93,47 +87,15 @@ public static MapBinder, ExtractionNamespac
new TypeLiteral>()
{
},
- new TypeLiteral>()
+ new TypeLiteral>()
{
}
);
}
- @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,
@@ -150,83 +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);
-
- LifecycleModule.register(binder, NamespaceStaticConfiguration.class);
- Jerseys.addResource(binder, NamespacesCacheResource.class);
- }
-
-
- @Provides
- @Named("namespaceVersionMap")
- @LazySingleton
- public ConcurrentMap getVersionMap()
- {
- return new ConcurrentHashMap<>();
- }
-
- @Provides
- @Named("namespaceExtractionFunctionCache")
- public ConcurrentMap> getFnCache()
- {
- return fnCache;
- }
-
- @Provides
- @Named("namespaceReverseExtractionFunctionCache")
- public ConcurrentMap>> getReverseFnCache()
- {
- return reverseFnCache;
- }
-
- @Provides
- @Named("dimExtractionNamespace")
- @LazySingleton
- public Function> getFunctionMaker(
- @Named("namespaceExtractionFunctionCache")
- 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("dimReverseExtractionNamespace")
- @LazySingleton
- public Function>> getReverseFunctionMaker(
- @Named("namespaceReverseExtractionFunctionCache")
- 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 3c1011190c9a..7b377c375b8d 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,20 +29,18 @@
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.nio.file.Path;
import java.nio.file.Paths;
-import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.regex.Pattern;
@@ -55,62 +48,23 @@
/**
*
*/
-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,
final URIExtractionNamespace extractionNamespace,
final String lastVersion,
final Map cache
@@ -188,7 +142,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)
);
@@ -227,7 +181,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..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);
}
}
@@ -160,8 +145,9 @@ 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
)
{
@@ -170,8 +156,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,82 +166,88 @@ 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);
- if (priorFn != null && priorFn != namespaceDatum.fn.get()) {
- log.warn("Replaced prior function for namespace [%s]", nsName);
- }
- if (priorReverseFn != null && priorReverseFn != namespaceDatum.reverseFn.get()) {
- log.warn("Replaced prior reverse function for namespace [%s]", nsName);
- }
- namespaceDatum.fn.set(fn);
- namespaceDatum.reverseFn.set(reverseFn);
+ swapAndClearCache(id, cacheId);
}
}
};
}
- 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,28 +306,28 @@ 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)
+ final ExtractionNamespaceCacheFactory factory = (ExtractionNamespaceCacheFactory)
namespaceFunctionFactoryMap.get(namespace.getClass());
if (factory == null) {
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 ExtractionNamespaceCacheFactory 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 +348,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,20 +357,20 @@ 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);
}
- if(Thread.currentThread().isInterrupted()) {
+ if (Thread.currentThread().isInterrupted()) {
throw Throwables.propagate(t);
}
}
@@ -394,18 +385,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;
}
}
@@ -452,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);
@@ -469,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 bbe4d826232b..a2b5b8eda079 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,23 +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.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;
@@ -56,15 +53,11 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
@Inject
public OffHeapNamespaceExtractionCacheManager(
Lifecycle lifecycle,
- @Named("namespaceExtractionFunctionCache")
- ConcurrentMap> fnCache,
- @Named("namespaceReverseExtractionFunctionCache")
- 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 2d55f6489835..7f38cee2f986 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,14 @@
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.IAE;
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.query.extraction.namespace.ExtractionNamespaceCacheFactory;
-import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -46,15 +43,11 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa
@Inject
public OnHeapNamespaceExtractionCacheManager(
final Lifecycle lifecycle,
- @Named("namespaceExtractionFunctionCache")
- final ConcurrentMap> fnCache,
- @Named("namespaceReverseExtractionFunctionCache")
- 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/NamespaceLookupExtractorFactoryTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java
new file mode 100644
index 000000000000..5d4af83a8dc9
--- /dev/null
+++ b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/NamespaceLookupExtractorFactoryTest.java
@@ -0,0 +1,425 @@
+/*
+ * 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(),
+ null, null,
+ new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper),
+
+ Period.millis(0),
+ null
+ );
+ 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();
+ }
+}
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/query/extraction/namespace/URIExtractionNamespaceTest.java b/extensions-core/namespace-lookup/src/test/java/io/druid/query/extraction/namespace/URIExtractionNamespaceTest.java
index 5673bcc3ea35..9e17cc4aa801 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
@@ -338,7 +338,6 @@ public void testMatchedJson() throws IOException
namespace.getNamespaceParseSpec().getClass().getCanonicalName()
);
Assert.assertEquals("file:/foo", namespace.getUriPrefix().toString());
- Assert.assertEquals("testNamespace", namespace.getNamespace());
Assert.assertEquals("a.b.c", namespace.getFileRegex());
Assert.assertEquals(5L * 60_000L, namespace.getPollMs());
}
@@ -348,7 +347,7 @@ public void testExplicitJson() throws IOException
{
final ObjectMapper mapper = registerTypes(new DefaultObjectMapper());
URIExtractionNamespace namespace = mapper.readValue(
- "{\"type\":\"uri\", \"uri\":\"file:/foo/a.b.c\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"namespace\":\"testNamespace\"}",
+ "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\"}",
URIExtractionNamespace.class
);
@@ -356,8 +355,7 @@ public void testExplicitJson() throws IOException
URIExtractionNamespace.ObjectMapperFlatDataParser.class.getCanonicalName(),
namespace.getNamespaceParseSpec().getClass().getCanonicalName()
);
- Assert.assertEquals("file:/foo/a.b.c", namespace.getUri().toString());
- Assert.assertEquals("testNamespace", namespace.getNamespace());
+ Assert.assertEquals("file:/foo", namespace.getUri().toString());
Assert.assertEquals(5L * 60_000L, namespace.getPollMs());
}
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 3bc49241642c..0d7881dba4d9 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,11 +93,11 @@ 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";
final URIExtractionNamespace namespace = new URIExtractionNamespace(
- "ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@@ -118,37 +107,35 @@ 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(),
null, null,
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.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)
@@ -158,8 +145,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(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@@ -168,21 +155,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(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@@ -192,9 +177,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.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 77%
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 8c3aca0b80e4..c078db8e4a3b 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
@@ -32,7 +32,7 @@
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;
@@ -66,7 +66,6 @@
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@@ -82,7 +81,7 @@
*
*/
@RunWith(Parameterized.class)
-public class URIExtractionNamespaceFunctionFactoryTest
+public class URIExtractionNamespaceCacheFactoryTest
{
@Parameterized.Parameters(name = "{0}")
public static Iterable getParameters() throws NoSuchMethodException
@@ -141,15 +140,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
)
@@ -180,10 +175,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) {
@@ -219,29 +212,25 @@ public void remove()
};
}
- public URIExtractionNamespaceFunctionFactoryTest(
+ public URIExtractionNamespaceCacheFactoryTest(
String friendlyName,
String suffix,
Function outStreamSupplier,
Constructor extends NamespaceExtractionCacheManager> 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()
@@ -261,16 +250,14 @@ public URIExtractionNamespaceFunctionFactoryTest(
private NamespaceExtractionCacheManager manager;
private File tmpFile;
private File tmpFileParent;
- private URIExtractionNamespaceFunctionFactory factory;
+ private URIExtractionNamespaceCacheFactory factory;
private URIExtractionNamespace namespace;
- private ConcurrentHashMap