[QTL] Implement LookupExtractorFactory of namespaced lookup (#2926)

* support LookupReferencesManager registration of namespaced lookup and eliminate static configurations for lookup from namespecd lookup extensions

- druid-namespace-lookup and druid-kafka-extraction-namespace are modified
- However, druid-namespace-lookup still has configuration about ON/OFF
  HEAP cache manager selection, which is not namespace wide
  configuration but node wide configuration as multiple namespace shares
  the same cache manager

* update KafkaExtractionNamespaceTest to reflect argument signature changes

* Add more synchronization functionality to NamespaceLookupExtractorFactory

* Remove old way of using extraction namespaces

* resolve compile error by supporting LookupIntrospectHandler

* Remove kafka lookups

* Remove unused stuff

* Fix start and stop behavior to be consistent with new javadocs

* Remove unused strings

* Add timeout option

* Address comments on configurations and improve docs

* Add more options and update hash key and replaces

* Move monitoring to the overriding classes

* Add better start/stop logging

* Remove old docs about namespace names

* Fix bad comma

* Add `@JsonIgnore` to lookup factory

* Address code review comments

* Remove ExtractionNamespace from module json registration

* Fix problems with naming and initialization. Add tests

* Optimize imports / reformat

* Fix future not being properly cancelled on failed initial scheduling

* Fix delete returns

* Add more docs about whole introspection

* Add `/version` introspection point for lookups

* Add more tests and address comments

* Add StaticMap extraction namespace for testing. Also add a bunch of tests

* Move cache system property to `druid.lookup.namespace.cache.type`

* Make VERSION lower case

* Change poll period to 0ms  for StaticMap

* Move cache key to bytebuffer

* Change hashCode and equals on static map extraction fn

* Add more comments on StaticMap

* Address comments

* Make scheduleAndWait use a latch

* Sanity renames and fix imports

* Remove extra info in docs

* Fix review comments

* Strengthen failure on start from warn to error

* Address comments

* Rename namespace-lookup to lookups-cached-global

* Fix injective mis-naming
* Also add serde test
This commit is contained in:
Charles Allen 2016-05-24 10:56:40 -07:00 committed by Fangjin Yang
parent 0ac1b27d53
commit 8024b915e2
43 changed files with 2130 additions and 1357 deletions

View File

@ -85,7 +85,7 @@
<argument>-c</argument>
<argument>io.druid.extensions:mysql-metadata-storage</argument>
<argument>-c</argument>
<argument>io.druid.extensions:druid-namespace-lookup</argument>
<argument>io.druid.extensions:druid-lookups-cached-global</argument>
<argument>-c</argument>
<argument>io.druid.extensions:postgresql-metadata-storage</argument>
<argument>-c</argument>

View File

@ -8,7 +8,7 @@ layout: doc_page
Lookups are an <a href="../experimental.html">experimental</a> feature.
</div>
Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` and `druid-kafka-extraction-namespace` as an extension.
Make sure to [include](../../operations/including-extensions.html) `druid-lookups-cached-global` and `druid-kafka-extraction-namespace` as an extension.
If you need updates to populate as promptly as possible, it is possible to plug into a kafka topic whose key is the old value and message is the desired new value (both in UTF-8) as a LookupExtractorFactory.

View File

@ -8,19 +8,25 @@ layout: doc_page
Lookups are an <a href="../experimental.html">experimental</a> feature.
</div>
Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` as an extension.
Make sure to [include](../../operations/including-extensions.html) `druid-lookups-cached-global` as an extension.
## Configuration
<div class="note caution">
Static configuration is no longer supported. Only cluster wide configuration is supported
</div>
Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size,
or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers.
Namespaced lookups can be specified as part of the runtime properties file. The property is a list of the namespaces
described as per the sections on this page. For example:
Cached namespace lookups are appropriate for lookups which are not possible to pass at query time due to their size,
or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers,
and are small enough to reasonably populate on a node. This usually means tens to tens of thousands of entries per lookup.
Cached namespace lookups all draw from the same cache pool, allowing each node to have a fixed cache pool that can be used by namespace lookups.
Cached namespace lookups can be specified as part of the [cluster wide config for lookups](../../querying/lookups.html) as a type of `cachedNamespace`
```json
druid.query.extraction.namespace.lookups=
[
{
{
"type": "cachedNamespace",
"extractionNamespace": {
"type": "uri",
"namespace": "some_uri_lookup",
"uri": "file:/tmp/prefix/",
@ -33,7 +39,14 @@ described as per the sections on this page. For example:
},
"pollPeriod": "PT5M"
},
{
"firstCacheTimeout": 0
}
```
```json
{
"type": "cachedNamespace",
"extractionNamespace": {
"type": "jdbc",
"namespace": "some_jdbc_lookup",
"connectorConfig": {
@ -46,12 +59,21 @@ described as per the sections on this page. For example:
"keyColumn": "mykeyColumn",
"valueColumn": "MyValueColumn",
"tsColumn": "timeColumn"
}
]
},
"firstCacheTimeout": 120000,
"injective":true
}
```
The parameters are as follows
|Property|Description|Required|Default|
|--------|-----------|--------|-------|
|`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-|
|`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`60000` (1 minute)|
|`injective`|If the underlying map is injective (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`|
Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes:
`druid-namespace-lookup`
`druid-lookups-cached-global`
## Cache Settings
@ -60,11 +82,15 @@ setting namespaces (broker, peon, historical)
|Property|Description|Default|
|--------|-----------|-------|
|`druid.query.extraction.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`|
|`druid.lookup.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`|
The cache is populated in different ways depending on the settings below. In general, most namespaces employ
a `pollPeriod` at the end of which time they poll the remote resource of interest for updates.
`onHeap` uses `ConcurrentMap`s in the java heap, and thus affects garbage collection and heap sizing.
`offHeap` uses a 10MB on-heap buffer and MapDB using memory-mapped files in the java temporary directory.
So if total `cachedNamespace` lookup size is in excess of 10MB, the extra will be kept in memory as page cache, and paged in and out by general OS tunings.
# Supported Lookups
For additional lookups, please see our [extensions list](../extensions.html).
@ -76,27 +102,25 @@ The remapping values for each namespaced lookup can be specified by a json objec
```json
{
"type":"uri",
"namespace":"some_lookup",
"uri": "s3://bucket/some/key/prefix/renames-0003.gz",
"namespaceParseSpec":{
"format":"csv",
"columns":["key","value"]
},
"pollPeriod":"PT5M",
"pollPeriod":"PT5M"
}
```
```json
{
"type":"uri",
"namespace":"some_lookup",
"uriPrefix": "s3://bucket/some/key/prefix/",
"fileRegex":"renames-[0-9]*\\.gz",
"namespaceParseSpec":{
"format":"csv",
"columns":["key","value"]
},
"pollPeriod":"PT5M",
"pollPeriod":"PT5M"
}
```
|Property|Description|Required|Default|
@ -250,3 +274,7 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol
"pollPeriod":600000
}
```
# Introspection
Cached namespace lookups have introspection points at `/keys` and `/values` which return a complete set of the keys and values (respectively) in the lookup. Introspection to `/` returns the entire map. Introspection to `/version` returns the version indicator for the lookup.

View File

@ -27,7 +27,7 @@ Core extensions are maintained by Druid committers.
|druid-histogram|Approximate histograms and quantiles aggregator.|[link](../development/extensions-core/approximate-histograms.html)|
|druid-kafka-eight|Kafka ingest firehose (high level consumer).|[link](../development/extensions-core/kafka-eight-firehose.html)|
|druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)|
|druid-namespace-lookup|Required module for [lookups](../querying/lookups.html).|[link](../development/extensions-core/namespaced-lookup.html)|
|druid-lookups-cached-global|Required module for [lookups](../querying/lookups.html).|[link](../development/extensions-core/lookups-cached-global.html)|
|druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)|
|mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)|
|postgresql-metadata-storage|PostgreSQL metadata store.|[link](../development/extensions-core/postgresql.html)|

View File

@ -23,7 +23,7 @@
# This is not the full list of Druid extensions, but common ones that people often use. You may need to change this list
# based on your particular setup.
druid.extensions.loadList=["druid-kafka-eight", "druid-s3-extensions", "druid-histogram", "druid-datasketches", "druid-namespace-lookup", "mysql-metadata-storage"]
druid.extensions.loadList=["druid-kafka-eight", "druid-s3-extensions", "druid-histogram", "druid-datasketches", "druid-lookups-cached-global", "mysql-metadata-storage"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -48,7 +48,7 @@
</dependency>
<dependency>
<groupId>io.druid.extensions</groupId>
<artifactId>druid-namespace-lookup</artifactId>
<artifactId>druid-lookups-cached-global</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>

View File

@ -37,7 +37,18 @@ 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 io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager;
import kafka.consumer.ConsumerConfig;
import kafka.consumer.KafkaStream;
import kafka.consumer.Whitelist;
import kafka.javaapi.consumer.ConsumerConnector;
import kafka.message.MessageAndMetadata;
import kafka.serializer.Decoder;
import javax.annotation.Nullable;
import javax.validation.constraints.Min;
import javax.ws.rs.GET;
import javax.ws.rs.core.Response;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
@ -52,16 +63,6 @@ import java.util.concurrent.atomic.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
@ -95,7 +96,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
private final long connectTimeout;
@JsonProperty
private final boolean isOneToOne;
private final boolean injective;
@JsonCreator
public KafkaLookupExtractorFactory(
@ -103,7 +104,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
@JsonProperty("kafkaTopic") final String kafkaTopic,
@JsonProperty("kafkaProperties") final Map<String, String> kafkaProperties,
@JsonProperty("connectTimeout") @Min(0) long connectTimeout,
@JsonProperty("isOneToOne") boolean isOneToOne
@JsonProperty("injective") boolean injective
)
{
this.kafkaTopic = Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required");
@ -114,7 +115,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
));
this.cacheManager = cacheManager;
this.connectTimeout = connectTimeout;
this.isOneToOne = isOneToOne;
this.injective = injective;
}
public KafkaLookupExtractorFactory(
@ -141,9 +142,9 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
return connectTimeout;
}
public boolean isOneToOne()
public boolean isInjective()
{
return isOneToOne;
return injective;
}
@Override
@ -335,7 +336,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
return !(getKafkaTopic().equals(that.getKafkaTopic())
&& getKafkaProperties().equals(that.getKafkaProperties())
&& getConnectTimeout() == that.getConnectTimeout()
&& isOneToOne() == that.isOneToOne()
&& isInjective() == that.isInjective()
);
}
@ -351,7 +352,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
{
final Map<String, String> map = Preconditions.checkNotNull(mapRef.get(), "Not started");
final long startCount = doubleEventCount.get();
return new MapLookupExtractor(map, isOneToOne())
return new MapLookupExtractor(map, isInjective())
{
@Override
public byte[] getCacheKey()

View File

@ -28,7 +28,7 @@ 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 io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream;
import kafka.consumer.TopicFilter;
@ -72,7 +72,7 @@ public class KafkaLookupExtractorFactoryTest
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
)
{
if ("io.druid.server.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) {
if ("io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) {
return cacheManager;
} else {
return null;
@ -507,6 +507,31 @@ public class KafkaLookupExtractorFactoryTest
).get();
}
@Test
public void testSerDe() throws Exception
{
final NamespaceExtractionCacheManager cacheManager = EasyMock.createStrictMock(NamespaceExtractionCacheManager.class);
final String kafkaTopic = "some_topic";
final Map<String, String> kafkaProperties = ImmutableMap.of("some_key", "some_value");
final long connectTimeout = 999;
final boolean injective = true;
final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
cacheManager,
kafkaTopic,
kafkaProperties,
connectTimeout,
injective
);
final KafkaLookupExtractorFactory otherFactory = mapper.readValue(
mapper.writeValueAsString(factory),
KafkaLookupExtractorFactory.class
);
Assert.assertEquals(kafkaTopic, otherFactory.getKafkaTopic());
Assert.assertEquals(kafkaProperties, otherFactory.getKafkaProperties());
Assert.assertEquals(connectTimeout, otherFactory.getConnectTimeout());
Assert.assertEquals(injective, otherFactory.isInjective());
}
@Test
public void testDefaultDecoder()
{

View File

@ -32,7 +32,7 @@ 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 io.druid.server.lookup.namespace.NamespaceExtractionModule;
import kafka.admin.AdminUtils;
import kafka.javaapi.producer.Producer;
import kafka.producer.KeyedMessage;
@ -249,7 +249,7 @@ public class TestKafkaExtractionCluster
}
},
// These injections fail under IntelliJ but are required for maven
new NamespacedExtractionModule(),
new NamespaceExtractionModule(),
new KafkaExtractionNamespaceModule()
)
);

View File

@ -22,8 +22,8 @@
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>io.druid.extensions</groupId>
<artifactId>druid-namespace-lookup</artifactId>
<name>druid-namespace-lookup</name>
<artifactId>druid-lookups-cached-global</artifactId>
<name>druid-lookups-cached-global</name>
<description>Extension to rename Druid dimension values using namespaces</description>
<parent>
@ -77,5 +77,10 @@
<version>3.0.1</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -30,7 +30,7 @@ import java.util.Map;
/**
* Simple class that takes a `ByteSource` and uses a `Parser<K, V>` to populate a `Map<K, V>`
* The `ByteSource` must be UTF-8 encoded
*
* <p>
* If this is handy for other use cases pleaes move this class into a common module
*/
public class MapPopulator<K, V>

View File

@ -0,0 +1,293 @@
/*
* 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.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.ISE;
import com.metamx.common.StringUtils;
import com.metamx.common.logger.Logger;
import io.druid.common.utils.ServletResourceUtils;
import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager;
import javax.annotation.Nullable;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@JsonTypeName("cachedNamespace")
public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
{
private static final Logger LOG = new Logger(NamespaceLookupExtractorFactory.class);
private static final long DEFAULT_SCHEDULE_TIMEOUT = 60_000;
private static final byte[] CLASS_CACHE_KEY;
static {
final byte[] keyUtf8 = StringUtils.toUtf8(NamespaceLookupExtractorFactory.class.getCanonicalName());
CLASS_CACHE_KEY = ByteBuffer.allocate(keyUtf8.length + 1).put(keyUtf8).put((byte) 0xFF).array();
}
private volatile boolean started = false;
private final ReadWriteLock startStopSync = new ReentrantReadWriteLock();
private final NamespaceExtractionCacheManager manager;
private final LookupIntrospectHandler lookupIntrospectHandler;
private final ExtractionNamespace extractionNamespace;
private final long firstCacheTimeout;
private final boolean injective;
private final String extractorID;
@JsonCreator
public NamespaceLookupExtractorFactory(
@JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace,
@JsonProperty("firstCacheTimeout") Long firstCacheTimeout,
@JsonProperty("injective") boolean injective,
@JacksonInject final NamespaceExtractionCacheManager manager
)
{
this.extractionNamespace = Preconditions.checkNotNull(
extractionNamespace,
"extractionNamespace should be specified"
);
this.firstCacheTimeout = firstCacheTimeout == null ? DEFAULT_SCHEDULE_TIMEOUT : firstCacheTimeout;
Preconditions.checkArgument(this.firstCacheTimeout >= 0);
this.injective = injective;
this.manager = manager;
this.extractorID = buildID();
this.lookupIntrospectHandler = new LookupIntrospectHandler()
{
@GET
@Path("/keys")
@Produces(MediaType.APPLICATION_JSON)
public Response getKeys()
{
try {
return Response.ok(getLatest().keySet()).build();
}
catch (ISE e) {
return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build();
}
}
@GET
@Path("/values")
@Produces(MediaType.APPLICATION_JSON)
public Response getValues()
{
try {
return Response.ok(getLatest().values()).build();
}
catch (ISE e) {
return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build();
}
}
@GET
@Path("/version")
@Produces(MediaType.APPLICATION_JSON)
public Response getVersion()
{
final String version = manager.getVersion(extractorID);
if (null == version) {
// Handle race between delete and this method being called
return Response.status(Response.Status.NOT_FOUND).build();
} else {
return Response.ok(ImmutableMap.of("version", version)).build();
}
}
@GET
@Produces(MediaType.APPLICATION_JSON)
public Response getMap()
{
try {
return Response.ok(getLatest()).build();
}
catch (ISE e) {
return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build();
}
}
private Map<String, String> getLatest()
{
return ((MapLookupExtractor) get()).getMap();
}
};
}
@VisibleForTesting
public NamespaceLookupExtractorFactory(
ExtractionNamespace extractionNamespace,
NamespaceExtractionCacheManager manager
)
{
this(extractionNamespace, null, false, manager);
}
@Override
public boolean start()
{
final Lock writeLock = startStopSync.writeLock();
writeLock.lock();
try {
if (started) {
LOG.warn("Already started! [%s]", extractorID);
return true;
}
if (!manager.scheduleAndWait(extractorID, extractionNamespace, firstCacheTimeout)) {
LOG.error("Failed to schedule lookup [%s]", extractorID);
return false;
}
LOG.debug("NamespaceLookupExtractorFactory[%s] started", extractorID);
started = true;
return true;
}
finally {
writeLock.unlock();
}
}
@Override
public boolean close()
{
final Lock writeLock = startStopSync.writeLock();
writeLock.lock();
try {
if (!started) {
LOG.warn("Not started! [%s]", extractorID);
return true;
}
started = 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;
if (isInjective() != ((NamespaceLookupExtractorFactory) other).isInjective()) {
return true;
}
if (getFirstCacheTimeout() != ((NamespaceLookupExtractorFactory) other).getFirstCacheTimeout()) {
return true;
}
return !extractionNamespace.equals(that.extractionNamespace);
}
return true;
}
@Override
public LookupIntrospectHandler getIntrospectHandler()
{
return lookupIntrospectHandler;
}
@JsonProperty
public ExtractionNamespace getExtractionNamespace()
{
return extractionNamespace;
}
@JsonProperty
public long getFirstCacheTimeout()
{
return firstCacheTimeout;
}
@JsonProperty
public boolean isInjective()
{
return injective;
}
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) {
throw new ISE("Factory [%s] not started", extractorID);
}
String preVersion = null, postVersion = null;
Map<String, String> 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, isInjective())
{
@Override
public byte[] getCacheKey()
{
return ByteBuffer
.allocate(CLASS_CACHE_KEY.length + id.length + 1 + v.length + 1 + 1)
.put(CLASS_CACHE_KEY)
.put(id).put((byte) 0xFF)
.put(v).put((byte) 0xFF)
.put(isOneToOne() ? (byte) 1 : (byte) 0)
.array();
}
};
}
finally {
readLock.unlock();
}
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package io.druid.query.extraction.namespace;
package io.druid.query.lookup.namespace;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
@ -25,7 +25,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "jdbc", value = JDBCExtractionNamespace.class),
@JsonSubTypes.Type(name = "uri", value = URIExtractionNamespace.class)
@JsonSubTypes.Type(name = "uri", value = URIExtractionNamespace.class),
@JsonSubTypes.Type(name = StaticMapExtractionNamespace.TYPE_NAME, value = StaticMapExtractionNamespace.class)
})
/**
* The ExtractionNamespace is a simple object for extracting namespaceLookup values from a source of data.
@ -34,11 +35,5 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
*/
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();
}

View File

@ -17,40 +17,16 @@
* under the License.
*/
package io.druid.query.extraction.namespace;
package io.druid.query.lookup.namespace;
import com.google.common.base.Function;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
/**
*
*/
public interface ExtractionNamespaceFunctionFactory<T extends ExtractionNamespace>
public interface ExtractionNamespaceCacheFactory<T extends ExtractionNamespace>
{
/**
* 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<String, String> buildFn(T extractionNamespace, Map<String, String> 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<String, List<String>> buildReverseFn(T extractionNamespace, final Map<String, String> cache);
/**
* This function is called once if `ExtractionNamespace.getUpdateMs() == 0`, or every update if
* `ExtractionNamespace.getUpdateMs() > 0`
@ -60,6 +36,7 @@ public interface ExtractionNamespaceFunctionFactory<T extends ExtractionNamespac
* initialize resources.
* If the result of the Callable is the same as what is passed in as lastVersion, then no swap takes place, and the swap is discarded.
*
* @param id The ID of ExtractionNamespace
* @param extractionNamespace The ExtractionNamespace for which to populate data.
* @param lastVersion The version which was last cached
* @param swap The temporary Map into which data may be placed and will be "swapped" with the proper
@ -70,5 +47,5 @@ public interface ExtractionNamespaceFunctionFactory<T extends ExtractionNamespac
* @return A callable that will be used to refresh resources of the namespace and return the version string used in
* the populating
*/
Callable<String> getCachePopulator(T extractionNamespace, String lastVersion, Map<String, String> swap);
Callable<String> getCachePopulator(String id, T extractionNamespace, String lastVersion, Map<String, String> swap);
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package io.druid.query.extraction.namespace;
package io.druid.query.lookup.namespace;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
@ -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 class JDBCExtractionNamespace implements ExtractionNamespace
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 class JDBCExtractionNamespace implements ExtractionNamespace
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 class JDBCExtractionNamespace implements ExtractionNamespace
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 class JDBCExtractionNamespace implements ExtractionNamespace
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;
}

View File

@ -0,0 +1,83 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.lookup.namespace;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import java.util.Map;
/**
* This class is intended to be used in general cluster testing, and not as a serious lookup.
* Any desire to use a static map in a lookup in *general* should use `io.druid.query.extraction.MapLookupExtractor`
* Any desire to test the *caching mechanisms in this extension* can use this class.
*/
@JsonTypeName(StaticMapExtractionNamespace.TYPE_NAME)
public class StaticMapExtractionNamespace implements ExtractionNamespace
{
static final String TYPE_NAME = "staticMap";
private final Map<String, String> map;
@JsonCreator
public StaticMapExtractionNamespace(
@JsonProperty("map") Map<String, String> map
)
{
this.map = Preconditions.checkNotNull(map, "`map` required");
}
@JsonProperty
public Map<String, String> getMap()
{
return map;
}
@Override
public long getPollMs()
{
// Load once and forget it
return 0;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
StaticMapExtractionNamespace that = (StaticMapExtractionNamespace) o;
return getMap().equals(that.getMap());
}
@Override
public int hashCode()
{
return getMap().hashCode();
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package io.druid.query.extraction.namespace;
package io.druid.query.lookup.namespace;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
@ -59,8 +59,6 @@ import java.util.regex.PatternSyntaxException;
@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 class URIExtractionNamespace implements ExtractionNamespace
String versionRegex
)
{
this.namespace = Preconditions.checkNotNull(namespace, "namespace");
this.uri = uri;
this.uriPrefix = uriPrefix;
if ((uri != null) == (uriPrefix != null)) {
@ -118,12 +113,6 @@ public class URIExtractionNamespace implements ExtractionNamespace
}
}
@Override
public String getNamespace()
{
return namespace;
}
public String getFileRegex()
{
return fileRegex;
@ -154,8 +143,7 @@ public class URIExtractionNamespace implements ExtractionNamespace
public String toString()
{
return "URIExtractionNamespace{" +
"namespace='" + namespace + '\'' +
", uri=" + uri +
"uri=" + uri +
", uriPrefix=" + uriPrefix +
", namespaceParseSpec=" + namespaceParseSpec +
", fileRegex='" + fileRegex + '\'' +
@ -175,9 +163,6 @@ public class URIExtractionNamespace implements ExtractionNamespace
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 class URIExtractionNamespace implements ExtractionNamespace
@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 class URIExtractionNamespace implements ExtractionNamespace
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()
{
@ -439,6 +445,31 @@ public class URIExtractionNamespace implements ExtractionNamespace
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()
{
@ -496,6 +527,25 @@ public class URIExtractionNamespace implements ExtractionNamespace
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()
{
@ -555,6 +605,19 @@ public class URIExtractionNamespace implements ExtractionNamespace
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()
{

View File

@ -17,18 +17,13 @@
* under the License.
*/
package io.druid.server.namespace;
package io.druid.server.lookup.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.JDBCExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.JDBCExtractionNamespace;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.StatementContext;
@ -36,7 +31,6 @@ import org.skife.jdbi.v2.tweak.HandleCallback;
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 @@ import java.util.concurrent.ConcurrentMap;
/**
*
*/
public class JDBCExtractionNamespaceFunctionFactory
implements ExtractionNamespaceFunctionFactory<JDBCExtractionNamespace>
public class JDBCExtractionNamespaceCacheFactory
implements ExtractionNamespaceCacheFactory<JDBCExtractionNamespace>
{
private static final Logger LOG = new Logger(JDBCExtractionNamespaceFunctionFactory.class);
private static final Logger LOG = new Logger(JDBCExtractionNamespaceCacheFactory.class);
private final ConcurrentMap<String, DBI> dbiCache = new ConcurrentHashMap<>();
@Override
public Function<String, String> buildFn(JDBCExtractionNamespace extractionNamespace, final Map<String, String> cache)
{
return new Function<String, String>()
{
@Nullable
@Override
public String apply(String input)
{
if (Strings.isNullOrEmpty(input)) {
return null;
}
return Strings.emptyToNull(cache.get(input));
}
};
}
@Override
public Function<String, List<String>> buildReverseFn(
JDBCExtractionNamespace extractionNamespace, final Map<String, String> cache
)
{
return new Function<String, List<String>>()
{
@Nullable
@Override
public List<String> apply(@Nullable final String value)
{
return Lists.newArrayList(Maps.filterKeys(cache, new Predicate<String>()
{
@Override public boolean apply(@Nullable String key)
{
return cache.get(key).equals(Strings.nullToEmpty(value));
}
}).keySet());
}
};
}
@Override
public Callable<String> getCachePopulator(
final String id,
final JDBCExtractionNamespace namespace,
final String lastVersion,
final Map<String, String> 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<String>()
{
@ -118,12 +74,12 @@ public class JDBCExtractionNamespaceFunctionFactory
@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<Pair<String, String>> pairs = dbi.withHandle(
new HandleCallback<List<Pair<String, String>>>()
{
@ -161,15 +117,15 @@ public class JDBCExtractionNamespaceFunctionFactory
for (Pair<String, String> 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 @@ public class JDBCExtractionNamespaceFunctionFactory
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) {

View File

@ -0,0 +1,107 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.lookup.namespace;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Key;
import com.google.inject.TypeLiteral;
import com.google.inject.multibindings.MapBinder;
import io.druid.guice.LazySingleton;
import io.druid.guice.PolyBind;
import io.druid.initialization.DruidModule;
import io.druid.query.lookup.NamespaceLookupExtractorFactory;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.JDBCExtractionNamespace;
import io.druid.query.lookup.namespace.StaticMapExtractionNamespace;
import io.druid.query.lookup.namespace.URIExtractionNamespace;
import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager;
import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager;
import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager;
import java.util.List;
/**
*
*/
public class NamespaceExtractionModule implements DruidModule
{
public static final String TYPE_PREFIX = "druid.lookup.namespace.cache.type";
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.<Module>of(
new SimpleModule("DruidNamespacedCachedExtractionModule")
.registerSubtypes(
NamespaceLookupExtractorFactory.class
)
);
}
public static MapBinder<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> getNamespaceFactoryMapBinder(
final Binder binder
)
{
return MapBinder.newMapBinder(
binder,
new TypeLiteral<Class<? extends ExtractionNamespace>>()
{
},
new TypeLiteral<ExtractionNamespaceCacheFactory<?>>()
{
}
);
}
@Override
public void configure(Binder binder)
{
PolyBind.createChoiceWithDefault(
binder,
TYPE_PREFIX,
Key.get(NamespaceExtractionCacheManager.class),
Key.get(OnHeapNamespaceExtractionCacheManager.class),
"onHeap"
).in(LazySingleton.class);
PolyBind
.optionBinder(binder, Key.get(NamespaceExtractionCacheManager.class))
.addBinding("offHeap")
.to(OffHeapNamespaceExtractionCacheManager.class)
.in(LazySingleton.class);
getNamespaceFactoryMapBinder(binder)
.addBinding(JDBCExtractionNamespace.class)
.to(JDBCExtractionNamespaceCacheFactory.class)
.in(LazySingleton.class);
getNamespaceFactoryMapBinder(binder)
.addBinding(URIExtractionNamespace.class)
.to(URIExtractionNamespaceCacheFactory.class)
.in(LazySingleton.class);
getNamespaceFactoryMapBinder(binder)
.addBinding(StaticMapExtractionNamespace.class)
.to(StaticMapExtractionNamespaceCacheFactory.class)
.in(LazySingleton.class);
}
}

View File

@ -0,0 +1,61 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.lookup.namespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.StaticMapExtractionNamespace;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.Callable;
public class StaticMapExtractionNamespaceCacheFactory
implements ExtractionNamespaceCacheFactory<StaticMapExtractionNamespace>
{
private final String version = UUID.randomUUID().toString();
@Override
public Callable<String> getCachePopulator(
final String id,
final StaticMapExtractionNamespace extractionNamespace,
final String lastVersion,
final Map<String, String> swap
)
{
return new Callable<String>()
{
@Override
public String call() throws Exception
{
if (version.equals(lastVersion)) {
return null;
} else {
swap.putAll(extractionNamespace.getMap());
return version;
}
}
};
}
String getVersion()
{
return version;
}
}

View File

@ -17,14 +17,9 @@
* under the License.
*/
package io.druid.server.namespace;
package io.druid.server.lookup.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,80 +29,40 @@ import com.metamx.common.logger.Logger;
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.URIExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.URIExtractionNamespace;
import io.druid.segment.loading.URIDataPuller;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.regex.Pattern;
import javax.annotation.Nullable;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
/**
*
*/
public class URIExtractionNamespaceFunctionFactory implements ExtractionNamespaceFunctionFactory<URIExtractionNamespace>
public class URIExtractionNamespaceCacheFactory implements ExtractionNamespaceCacheFactory<URIExtractionNamespace>
{
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<String, SearchableVersionedDataFinder> pullers;
@Inject
public URIExtractionNamespaceFunctionFactory(
public URIExtractionNamespaceCacheFactory(
Map<String, SearchableVersionedDataFinder> pullers
)
{
this.pullers = pullers;
}
@Override
public Function<String, String> buildFn(URIExtractionNamespace extractionNamespace, final Map<String, String> cache)
{
return new Function<String, String>()
{
@Nullable
@Override
public String apply(String input)
{
if (Strings.isNullOrEmpty(input)) {
return null;
}
return Strings.emptyToNull(cache.get(input));
}
};
}
@Override
public Function<String, List<String>> buildReverseFn(
URIExtractionNamespace extractionNamespace, final Map<String, String> cache
)
{
return new Function<String, List<String>>()
{
@Nullable
@Override
public List<String> apply(@Nullable final String value)
{
return Lists.newArrayList(Maps.filterKeys(cache, new Predicate<String>()
{
@Override
public boolean apply(@Nullable String key)
{
return cache.get(key).equals(Strings.nullToEmpty(value));
}
}).keySet());
}
};
}
@Override
public Callable<String> getCachePopulator(
final String id,
final URIExtractionNamespace extractionNamespace,
final String lastVersion,
final Map<String, String> cache
@ -184,7 +139,7 @@ public class URIExtractionNamespaceFunctionFactory implements ExtractionNamespac
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)
);
@ -223,7 +178,7 @@ public class URIExtractionNamespaceFunctionFactory implements ExtractionNamespac
log.info(
"Finished loading %d lines for namespace [%s]",
lineCount,
extractionNamespace.getNamespace()
id
);
return version;
}

View File

@ -17,14 +17,9 @@
* under the License.
*/
package io.druid.server.namespace.cache;
package io.druid.server.lookup.namespace.cache;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
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;
@ -38,14 +33,11 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.query.extraction.namespace.ExtractionNamespace;
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.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,16 +48,10 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
/**
*
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = OnHeapNamespaceExtractionCacheManager.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "offHeap", value = OffHeapNamespaceExtractionCacheManager.class),
@JsonSubTypes.Type(name = "onHeap", value = OnHeapNamespaceExtractionCacheManager.class)
})
public abstract class NamespaceExtractionCacheManager
{
protected static class NamespaceImplData
@ -85,27 +71,21 @@ public abstract class NamespaceExtractionCacheManager
final ExtractionNamespace namespace;
final String name;
final AtomicBoolean enabled = new AtomicBoolean(false);
final AtomicReference<Function<String, String>> fn = new AtomicReference<>(null);
final AtomicReference<Function<String, List<String>>> reverseFn = new AtomicReference<>(null);
final CountDownLatch firstRun = new CountDownLatch(1);
}
private static final Logger log = new Logger(NamespaceExtractionCacheManager.class);
private final ListeningScheduledExecutorService listeningScheduledExecutorService;
protected final ConcurrentMap<String, Function<String, String>> fnCache;
protected final ConcurrentMap<String, Function<String, List<String>>> reverseFnCache;
protected final ConcurrentMap<String, NamespaceImplData> implData = new ConcurrentHashMap<>();
protected final AtomicLong tasksStarted = new AtomicLong(0);
protected final AtomicLong dataSize = new AtomicLong(0);
protected final ServiceEmitter serviceEmitter;
private final ConcurrentHashMap<String, String> lastVersion = new ConcurrentHashMap<>();
private final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>> namespaceFunctionFactoryMap;
private final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> namespaceFunctionFactoryMap;
public NamespaceExtractionCacheManager(
Lifecycle lifecycle,
final ConcurrentMap<String, Function<String, String>> fnCache,
final ConcurrentMap<String, Function<String, List<String>>> reverseFnCache,
final ServiceEmitter serviceEmitter,
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>> namespaceFunctionFactoryMap
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> namespaceFunctionFactoryMap
)
{
this.listeningScheduledExecutorService = MoreExecutors.listeningDecorator(
@ -120,8 +100,6 @@ public abstract class NamespaceExtractionCacheManager
);
ExecutorServices.manageLifecycle(lifecycle, listeningScheduledExecutorService);
this.serviceEmitter = serviceEmitter;
this.fnCache = fnCache;
this.reverseFnCache = reverseFnCache;
this.namespaceFunctionFactoryMap = namespaceFunctionFactoryMap;
listeningScheduledExecutorService.scheduleAtFixedRate(
new Runnable()
@ -133,16 +111,16 @@ public abstract class NamespaceExtractionCacheManager
{
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){
monitor(serviceEmitter);
}
catch (Exception e) {
log.error(e, "Error emitting namespace stats");
if(Thread.currentThread().isInterrupted()){
if (Thread.currentThread().isInterrupted()) {
throw Throwables.propagate(e);
}
}
@ -153,6 +131,16 @@ public abstract class NamespaceExtractionCacheManager
);
}
/**
* Optional monitoring for overriding classes. `super.monitor` does *NOT* need to be called by overriding methods
*
* @param serviceEmitter The emitter to emit to
*/
protected void monitor(ServiceEmitter serviceEmitter)
{
// Noop by default
}
protected boolean waitForServiceToEnd(long time, TimeUnit unit) throws InterruptedException
{
return listeningScheduledExecutorService.awaitTermination(time, unit);
@ -160,8 +148,9 @@ public abstract class NamespaceExtractionCacheManager
protected <T extends ExtractionNamespace> Runnable getPostRunnable(
final String id,
final T namespace,
final ExtractionNamespaceFunctionFactory<T> factory,
final ExtractionNamespaceCacheFactory<T> factory,
final String cacheId
)
{
@ -170,93 +159,101 @@ public abstract class NamespaceExtractionCacheManager
@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;
}
synchronized (namespaceDatum.enabled) {
if (!namespaceDatum.enabled.get()) {
// skip because it was disabled
return;
try {
if (!namespaceDatum.enabled.get()) {
// skip because it was disabled
return;
}
swapAndClearCache(id, cacheId);
}
swapAndClearCache(nsName, cacheId);
final Function<String, String> fn = factory.buildFn(namespace, getCacheMap(nsName));
final Function<String, List<String>> reverseFn = factory.buildReverseFn(namespace, getCacheMap(nsName));
final Function<String, String> priorFn = fnCache.put(nsName, fn);
final Function<String, List<String>> priorReverseFn = reverseFnCache.put(nsName, reverseFn);
if (priorFn != null && priorFn != namespaceDatum.fn.get()) {
log.warn("Replaced prior function for namespace [%s]", nsName);
finally {
namespaceDatum.firstRun.countDown();
}
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);
}
}
};
}
public void scheduleOrUpdate(
final Collection<ExtractionNamespace> namespaces
// return value means actually delete or not
public boolean checkedDelete(
String namespaceName
)
{
Set<String> differentNamespaces = Sets.difference(
implData.keySet(), Sets.newHashSet(
Iterables.transform(
namespaces,
new Function<ExtractionNamespace, String>()
{
@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);
}
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;
}
return delete(namespaceName);
}
// 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);
}
removeNamespaceLocalMetadata(implDatum);
schedule(namespace);
// 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;
}
// 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
)
{
if (scheduleOrUpdate(id, namespace)) {
log.debug("Scheduled new namespace [%s]: %s", id, namespace);
} else {
log.debug("Namespace [%s] already running: %s", id, namespace);
}
final NamespaceImplData namespaceImplData = implData.get(id);
if (namespaceImplData == null) {
log.warn("NamespaceLookupExtractorFactory[%s] - deleted during start", id);
return false;
}
boolean success = false;
try {
success = namespaceImplData.firstRun.await(waitForFirstRun, TimeUnit.MILLISECONDS);
}
catch (InterruptedException e) {
log.error(e, "NamespaceLookupExtractorFactory[%s] - interrupted during start", id);
}
if (!success) {
delete(id);
}
return success;
}
private void cancelFuture(final NamespaceImplData implDatum)
@ -315,28 +312,28 @@ public abstract class NamespaceExtractionCacheManager
}
// Optimistic scheduling of updates to a namespace.
public <T extends ExtractionNamespace> ListenableFuture<?> schedule(final T namespace)
public <T extends ExtractionNamespace> ListenableFuture<?> schedule(final String id, final T namespace)
{
final ExtractionNamespaceFunctionFactory<T> factory = (ExtractionNamespaceFunctionFactory<T>)
final ExtractionNamespaceCacheFactory<T> factory = (ExtractionNamespaceCacheFactory<T>)
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 <T extends ExtractionNamespace> ListenableFuture<?> schedule(
final String id,
final T namespace,
final ExtractionNamespaceFunctionFactory<T> factory,
final ExtractionNamespaceCacheFactory<T> 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 +354,8 @@ public abstract class NamespaceExtractionCacheManager
startLatch.await(); // wait for "election" to leadership or cancellation
if (!Thread.currentThread().isInterrupted()) {
final Map<String, String> cache = getCacheMap(cacheId);
final String preVersion = lastVersion.get(namespaceName);
final Callable<String> runnable = factory.getCachePopulator(namespace, preVersion, cache);
final String preVersion = lastVersion.get(id);
final Callable<String> runnable = factory.getCachePopulator(id, namespace, preVersion, cache);
tasksStarted.incrementAndGet();
final String newVersion = runnable.call();
@ -366,20 +363,20 @@ public abstract class NamespaceExtractionCacheManager
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 +391,18 @@ public abstract class NamespaceExtractionCacheManager
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 +449,6 @@ public abstract class NamespaceExtractionCacheManager
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 +465,8 @@ public abstract class NamespaceExtractionCacheManager
}
}
public Collection<String> getKnownNamespaces()
public Collection<String> getKnownIDs()
{
return fnCache.keySet();
return implData.keySet();
}
}

View File

@ -17,25 +17,23 @@
* under the License.
*/
package io.druid.server.namespace.cache;
package io.druid.server.lookup.namespace.cache;
import com.google.common.base.Function;
import com.google.common.base.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 com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import org.mapdb.DB;
import org.mapdb.DBMaker;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
@ -56,15 +54,11 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
@Inject
public OffHeapNamespaceExtractionCacheManager(
Lifecycle lifecycle,
@Named("namespaceExtractionFunctionCache")
ConcurrentMap<String, Function<String, String>> fnCache,
@Named("namespaceReverseExtractionFunctionCache")
ConcurrentMap<String, Function<String, List<String>>> reverseFnCache,
ServiceEmitter emitter,
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>> namespaceFunctionFactoryMap
final Map<Class<? extends ExtractionNamespace>, 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());
@ -83,27 +77,32 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
.commitFileSyncDisable()
.cacheSize(10_000_000)
.make();
lifecycle.addHandler(
new Lifecycle.Handler()
{
@Override
public void start() throws Exception
try {
lifecycle.addMaybeStartHandler(
new Lifecycle.Handler()
{
// NOOP
}
@Override
public void start() throws Exception
{
// NOOP
}
@Override
public void stop()
{
if (!mmapDB.isClosed()) {
mmapDB.close();
if (!tmpFile.delete()) {
log.warn("Unable to delete file at [%s]", tmpFile.getAbsolutePath());
@Override
public synchronized void stop()
{
if (!mmapDB.isClosed()) {
mmapDB.close();
if (!tmpFile.delete()) {
log.warn("Unable to delete file at [%s]", tmpFile.getAbsolutePath());
}
}
}
}
}
);
);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
@ -121,10 +120,8 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
if (priorCache != null) {
// TODO: resolve what happens here if query is actively going on
mmapDB.delete(priorCache);
dataSize.set(tmpFile.length());
return true;
} else {
dataSize.set(tmpFile.length());
return false;
}
}
@ -136,18 +133,19 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
@Override
public boolean delete(final String namespaceKey)
{
final Lock lock = nsLocks.get(namespaceKey);
lock.lock();
try {
super.delete(namespaceKey);
final String mmapDBkey = currentNamespaceCache.get(namespaceKey);
if (mmapDBkey != null) {
final long pre = tmpFile.length();
mmapDB.delete(mmapDBkey);
dataSize.set(tmpFile.length());
log.debug("MapDB file size: pre %d post %d", pre, dataSize.get());
return true;
if (super.delete(namespaceKey)) {
final String mmapDBkey = currentNamespaceCache.get(namespaceKey);
if (mmapDBkey != null) {
final long pre = tmpFile.length();
mmapDB.delete(mmapDBkey);
log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length());
return true;
} else {
return false;
}
} else {
return false;
}
@ -184,4 +182,10 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
lock.unlock();
}
}
@Override
protected void monitor(ServiceEmitter serviceEmitter)
{
serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/cache/diskSize", tmpFile.length()));
}
}

View File

@ -17,19 +17,19 @@
* under the License.
*/
package io.druid.server.namespace.cache;
package io.druid.server.lookup.namespace.cache;
import com.google.common.base.Function;
import com.google.common.primitives.Chars;
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.common.logger.Logger;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.query.extraction.namespace.ExtractionNamespace;
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -40,21 +40,18 @@ import java.util.concurrent.locks.Lock;
*/
public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCacheManager
{
private static final Logger LOG = new Logger(OnHeapNamespaceExtractionCacheManager.class);
private final ConcurrentMap<String, ConcurrentMap<String, String>> mapMap = new ConcurrentHashMap<>();
private final Striped<Lock> nsLocks = Striped.lock(32);
@Inject
public OnHeapNamespaceExtractionCacheManager(
final Lifecycle lifecycle,
@Named("namespaceExtractionFunctionCache")
final ConcurrentMap<String, Function<String, String>> fnCache,
@Named("namespaceReverseExtractionFunctionCache")
final ConcurrentMap<String, Function<String, List<String>>> reverseFnCache,
final ServiceEmitter emitter,
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>> namespaceFunctionFactoryMap
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> namespaceFunctionFactoryMap
)
{
super(lifecycle, fnCache, reverseFnCache, emitter, namespaceFunctionFactoryMap);
super(lifecycle, emitter, namespaceFunctionFactoryMap);
}
@Override
@ -67,11 +64,9 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa
if (cacheMap == null) {
throw new IAE("Extraction Cache [%s] does not exist", cacheKey);
}
dataSize.addAndGet(cacheMap.size());
ConcurrentMap<String, String> prior = mapMap.put(namespaceKey, cacheMap);
mapMap.remove(cacheKey);
if (prior != null) {
dataSize.addAndGet(-prior.size());
// Old map will get GC'd when it is not used anymore
return true;
} else {
@ -100,11 +95,36 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa
final Lock lock = nsLocks.get(namespaceKey);
lock.lock();
try {
super.delete(namespaceKey);
return mapMap.remove(namespaceKey) != null;
return super.delete(namespaceKey) && mapMap.remove(namespaceKey) != null;
}
finally {
lock.unlock();
}
}
@Override
protected void monitor(ServiceEmitter serviceEmitter)
{
long numEntries = 0;
long size = 0;
for (Map.Entry<String, ConcurrentMap<String, String>> entry : mapMap.entrySet()) {
final ConcurrentMap<String, String> map = entry.getValue();
if (map == null) {
LOG.debug("missing cache key for reporting [%s]", entry.getKey());
continue;
}
numEntries += map.size();
for (Map.Entry<String, String> sEntry : map.entrySet()) {
final String key = sEntry.getKey();
final String value = sEntry.getValue();
if (key == null || value == null) {
LOG.debug("Missing entries for cache key [%s]", entry.getKey());
continue;
}
size += key.length() + value.length();
}
}
serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/cache/numEntries", numEntries));
serviceEmitter.emit(ServiceMetricEvent.builder().build("namespace/cache/heapSizeInBytes", size * Chars.BYTES));
}
}

View File

@ -17,4 +17,4 @@
# under the License.
#
io.druid.server.namespace.NamespacedExtractionModule
io.druid.server.lookup.namespace.NamespaceExtractionModule

View File

@ -0,0 +1,602 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.lookup;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import com.metamx.common.ISE;
import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.URIExtractionNamespace;
import io.druid.server.DruidNode;
import io.druid.server.lookup.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 javax.ws.rs.core.Response;
import java.util.HashMap;
import java.util.Map;
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.lookup.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.assertTrue(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.assertTrue(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<String, String>())
.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<String, String>())
.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<String, String>(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<String, String>())
.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.replay(cacheManager);
final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory(
extractionNamespace,
cacheManager
);
Assert.assertFalse(namespaceLookupExtractorFactory.start());
// true because it never fully started
Assert.assertTrue(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();
}
// Note this does NOT catch problems with returning factories as failed in error messages.
@Test
public void testSerDe() throws Exception
{
final Injector injector = Initialization.makeInjectorWithModules(
GuiceInjectors.makeStartupInjector(),
ImmutableList.of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null)
);
}
}
)
);
final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class));
mapper.registerSubtypes(NamespaceLookupExtractorFactory.class);
final String str = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }";
final LookupExtractorFactory factory = mapper.readValue(str, LookupExtractorFactory.class);
Assert.assertTrue(factory instanceof NamespaceLookupExtractorFactory);
final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = (NamespaceLookupExtractorFactory) factory;
Assert.assertNotNull(mapper.writeValueAsString(factory));
Assert.assertFalse(factory.replaces(mapper.readValue(
mapper.writeValueAsString(factory),
LookupExtractorFactory.class
)));
Assert.assertEquals(
URIExtractionNamespace.class,
namespaceLookupExtractorFactory.getExtractionNamespace().getClass()
);
Assert.assertFalse(namespaceLookupExtractorFactory.replaces(mapper.readValue(str, LookupExtractorFactory.class)));
final Map<String, Object> map = new HashMap<>(mapper.<Map<String, Object>>readValue(
str,
new TypeReference<Map<String, Object>>()
{
}
));
map.put("firstCacheTimeout", "1");
Assert.assertTrue(namespaceLookupExtractorFactory.replaces(mapper.convertValue(map, LookupExtractorFactory.class)));
}
@Test
public void testSimpleIntrospectionHandler() throws Exception
{
final Injector injector = Initialization.makeInjectorWithModules(
GuiceInjectors.makeStartupInjector(),
ImmutableList.of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null)
);
}
}
)
);
final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class));
mapper.registerSubtypes(NamespaceLookupExtractorFactory.class);
final String str = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"staticMap\", \"map\": {\"foo\":\"bar\"} } }";
final LookupExtractorFactory lookupExtractorFactory = mapper.readValue(str, LookupExtractorFactory.class);
Assert.assertTrue(lookupExtractorFactory.start());
try {
final LookupIntrospectHandler handler = lookupExtractorFactory.getIntrospectHandler();
Assert.assertNotNull(handler);
final Class<? extends LookupIntrospectHandler> clazz = handler.getClass();
Assert.assertNotNull(clazz.getMethod("getVersion").invoke(handler));
Assert.assertEquals(ImmutableSet.of("foo"), ((Response) clazz.getMethod("getKeys").invoke(handler)).getEntity());
Assert.assertEquals(
ImmutableSet.of("bar"),
((Response) clazz.getMethod("getValues").invoke(handler)).getEntity()
);
Assert.assertEquals(
ImmutableMap.builder().put("foo", "bar").build(),
((Response) clazz.getMethod("getMap").invoke(handler)).getEntity()
);
}
finally {
Assert.assertTrue(lookupExtractorFactory.close());
}
}
@Test
public void testExceptionalIntrospectionHandler() throws Exception
{
final NamespaceExtractionCacheManager manager = EasyMock.createStrictMock(NamespaceExtractionCacheManager.class);
final ExtractionNamespace extractionNamespace = EasyMock.createStrictMock(ExtractionNamespace.class);
EasyMock.expect(manager.scheduleAndWait(EasyMock.anyString(), EasyMock.eq(extractionNamespace), EasyMock.anyLong()))
.andReturn(true)
.once();
EasyMock.replay(manager);
final LookupExtractorFactory lookupExtractorFactory = new NamespaceLookupExtractorFactory(
extractionNamespace,
manager
);
Assert.assertTrue(lookupExtractorFactory.start());
final LookupIntrospectHandler handler = lookupExtractorFactory.getIntrospectHandler();
Assert.assertNotNull(handler);
final Class<? extends LookupIntrospectHandler> clazz = handler.getClass();
synchronized (manager) {
EasyMock.verify(manager);
EasyMock.reset(manager);
EasyMock.expect(manager.getVersion(EasyMock.anyString())).andReturn(null).once();
EasyMock.replay(manager);
}
final Response response = (Response) clazz.getMethod("getVersion").invoke(handler);
Assert.assertEquals(404, response.getStatus());
validateCode(
new ISE("some exception"),
404,
"getKeys",
handler,
manager,
clazz
);
validateCode(
new ISE("some exception"),
404,
"getValues",
handler,
manager,
clazz
);
validateCode(
new ISE("some exception"),
404,
"getMap",
handler,
manager,
clazz
);
EasyMock.verify(manager);
}
void validateCode(
Throwable thrown,
int expectedCode,
String method,
LookupIntrospectHandler handler,
NamespaceExtractionCacheManager manager,
Class<? extends LookupIntrospectHandler> clazz
) throws Exception
{
synchronized (manager) {
EasyMock.verify(manager);
EasyMock.reset(manager);
EasyMock.expect(manager.getVersion(EasyMock.anyString())).andThrow(thrown).once();
EasyMock.replay(manager);
}
final Response response = (Response) clazz.getMethod(method).invoke(handler);
Assert.assertEquals(expectedCode, response.getStatus());
}
}

View File

@ -0,0 +1,104 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.lookup.namespace;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.io.CharSink;
import com.google.common.io.Files;
import io.druid.data.input.MapPopulator;
import io.druid.jackson.DefaultObjectMapper;
import org.hamcrest.BaseMatcher;
import org.hamcrest.Description;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.util.HashMap;
import java.util.Map;
public class JSONFlatDataParserTest
{
private static final ObjectMapper MAPPER = new DefaultObjectMapper();
private static final String KEY = "foo";
private static final String VAL = "bar";
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule
public ExpectedException expectedException = ExpectedException.none();
private File tmpFile;
@Before
public void setUp() throws Exception
{
tmpFile = temporaryFolder.newFile("lookup.json");
final CharSink sink = Files.asByteSink(tmpFile).asCharSink(Charsets.UTF_8);
sink.write("{\"key\":\"" + KEY + "\",\"val\":\"" + VAL + "\"}");
}
@Test
public void testSimpleParse() throws Exception
{
final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser(
MAPPER,
"key",
"val"
);
final Map<String, String> map = new HashMap<>();
new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map);
Assert.assertEquals(VAL, map.get(KEY));
}
@Test
public void testFailParse() throws Exception
{
expectedException.expect(new BaseMatcher<Object>()
{
@Override
public boolean matches(Object o)
{
if (!(o instanceof NullPointerException)) {
return false;
}
final NullPointerException npe = (NullPointerException) o;
return npe.getMessage().startsWith("Key column [keyWHOOPS] missing data in line");
}
@Override
public void describeTo(Description description)
{
}
});
final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser(
MAPPER,
"keyWHOOPS",
"val"
);
final Map<String, String> map = new HashMap<>();
new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map);
Assert.assertEquals(VAL, map.get(KEY));
}
}

View File

@ -0,0 +1,60 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.lookup.namespace;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.Map;
public class StaticMapExtractionNamespaceTest
{
private static final Map<String, String> MAP = ImmutableMap.<String, String>builder().put("foo", "bar").build();
private static final ObjectMapper MAPPER = new DefaultObjectMapper();
private static String MAP_STRING;
@BeforeClass
public static void setUpStatic() throws Exception
{
MAP_STRING = MAPPER.writeValueAsString(MAP);
}
@Test
public void testSimpleSerDe() throws Exception
{
final String str = "{\"type\":\"staticMap\", \"map\":" + MAP_STRING + "}";
final StaticMapExtractionNamespace extractionNamespace = MAPPER.readValue(str, StaticMapExtractionNamespace.class);
Assert.assertEquals(MAP, extractionNamespace.getMap());
Assert.assertEquals(0L, extractionNamespace.getPollMs());
Assert.assertEquals(extractionNamespace, MAPPER.readValue(str, StaticMapExtractionNamespace.class));
Assert.assertNotEquals(
extractionNamespace,
new StaticMapExtractionNamespace(ImmutableMap.<String, String>of("foo", "not_bar"))
);
Assert.assertNotEquals(
extractionNamespace,
new StaticMapExtractionNamespace(ImmutableMap.<String, String>of("not_foo", "bar"))
);
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package io.druid.query.extraction.namespace;
package io.druid.query.lookup.namespace;
import com.fasterxml.jackson.databind.JsonMappingException;
import com.fasterxml.jackson.databind.ObjectMapper;
@ -350,7 +350,6 @@ public class URIExtractionNamespaceTest
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());
}
@ -360,7 +359,7 @@ public class URIExtractionNamespaceTest
{
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
);
@ -368,8 +367,7 @@ public class URIExtractionNamespaceTest
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());
}

View File

@ -17,24 +17,22 @@
* under the License.
*/
package io.druid.server.namespace;
package io.druid.server.lookup.namespace;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.collect.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.JDBCExtractionNamespace;
import io.druid.query.extraction.namespace.URIExtractionNamespace;
import io.druid.query.extraction.namespace.URIExtractionNamespaceTest;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.JDBCExtractionNamespace;
import io.druid.query.lookup.namespace.URIExtractionNamespace;
import io.druid.query.lookup.namespace.URIExtractionNamespaceTest;
import io.druid.segment.loading.LocalFileTimestampVersionFinder;
import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager;
import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager;
import io.druid.server.metrics.NoopServiceEmitter;
import io.druid.server.namespace.cache.NamespaceExtractionCacheManager;
import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager;
import org.joda.time.Period;
import org.junit.AfterClass;
import org.junit.Assert;
@ -46,13 +44,8 @@ import org.junit.rules.TemporaryFolder;
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 +55,6 @@ public class NamespacedExtractorModuleTest
private static final ObjectMapper mapper = URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper());
private static NamespaceExtractionCacheManager cacheManager;
private static Lifecycle lifecycle;
private static ConcurrentMap<String, Function<String, String>> fnCache = new ConcurrentHashMap<>();
@Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder();
@ -70,25 +62,19 @@ public class NamespacedExtractorModuleTest
@BeforeClass
public static void setUpStatic() throws Exception
{
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>> factoryMap =
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of(
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> factoryMap =
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>>of(
URIExtractionNamespace.class,
new URIExtractionNamespaceFunctionFactory(
new URIExtractionNamespaceCacheFactory(
ImmutableMap.<String, SearchableVersionedDataFinder>of(
"file",
new LocalFileTimestampVersionFinder()
)
),
JDBCExtractionNamespace.class, new JDBCExtractionNamespaceFunctionFactory()
JDBCExtractionNamespace.class, new JDBCExtractionNamespaceCacheFactory()
);
lifecycle = new Lifecycle();
cacheManager = new OnHeapNamespaceExtractionCacheManager(
lifecycle,
new ConcurrentHashMap<String, Function<String, String>>(),
new ConcurrentHashMap<String, Function<String, List<String>>>(),
new NoopServiceEmitter(), factoryMap
);
fnCache.clear();
cacheManager = new OnHeapNamespaceExtractionCacheManager(lifecycle, new NoopServiceEmitter(), factoryMap);
}
@AfterClass
@ -104,11 +90,11 @@ public class NamespacedExtractorModuleTest
try (OutputStreamWriter out = new FileWriter(tmpFile)) {
out.write(mapper.writeValueAsString(ImmutableMap.<String, String>of("foo", "bar")));
}
final URIExtractionNamespaceFunctionFactory factory = new URIExtractionNamespaceFunctionFactory(
final URIExtractionNamespaceCacheFactory factory = new URIExtractionNamespaceCacheFactory(
ImmutableMap.<String, SearchableVersionedDataFinder>of("file", new LocalFileTimestampVersionFinder())
);
final String namespaceID = "ns";
final URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -118,37 +104,35 @@ public class NamespacedExtractorModuleTest
null
);
Map<String, String> 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.<String, String>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.<ExtractionNamespace>of(namespace));
Collection<String> 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 +142,8 @@ public class NamespacedExtractorModuleTest
try (OutputStreamWriter out = new FileWriter(tmpFile)) {
out.write(mapper.writeValueAsString(ImmutableMap.<String, String>of("foo", "bar")));
}
final String namespaceID = "ns";
final URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -168,21 +152,19 @@ public class NamespacedExtractorModuleTest
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.<String, String>of("foo", "bar")));
}
final String namespaceID = "ns";
final URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -192,9 +174,9 @@ public class NamespacedExtractorModuleTest
null
);
Assert.assertTrue(noNamespaces(cacheManager));
cacheManager.scheduleOrUpdate(ImmutableList.<ExtractionNamespace>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));
}
}

View File

@ -0,0 +1,44 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.lookup.namespace;
import com.google.common.collect.ImmutableMap;
import io.druid.query.lookup.namespace.StaticMapExtractionNamespace;
import org.junit.Assert;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
public class StaticMapExtractionNamespaceCacheFactoryTest
{
private static final Map<String, String> MAP = ImmutableMap.<String, String>builder().put("foo", "bar").build();
@Test
public void testSimplePopulator() throws Exception
{
final StaticMapExtractionNamespaceCacheFactory factory = new StaticMapExtractionNamespaceCacheFactory();
final StaticMapExtractionNamespace namespace = new StaticMapExtractionNamespace(MAP);
final Map<String, String> cache = new HashMap<>();
Assert.assertEquals(factory.getVersion(), factory.getCachePopulator(null, namespace, null, cache).call());
Assert.assertEquals(MAP, cache);
Assert.assertNull(factory.getCachePopulator(null, namespace, factory.getVersion(), cache).call());
}
}

View File

@ -17,30 +17,43 @@
* under the License.
*/
package io.druid.server.namespace;
package io.druid.server.lookup.namespace;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import com.metamx.common.IAE;
import com.metamx.common.UOE;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.data.SearchableVersionedDataFinder;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.extraction.namespace.ExtractionNamespace;
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
import io.druid.query.extraction.namespace.URIExtractionNamespace;
import io.druid.query.extraction.namespace.URIExtractionNamespaceTest;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.URIExtractionNamespace;
import io.druid.query.lookup.namespace.URIExtractionNamespaceTest;
import io.druid.segment.loading.LocalFileTimestampVersionFinder;
import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager;
import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManagersTest;
import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager;
import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager;
import io.druid.server.metrics.NoopServiceEmitter;
import io.druid.server.namespace.cache.NamespaceExtractionCacheManager;
import io.druid.server.namespace.cache.NamespaceExtractionCacheManagersTest;
import io.druid.server.namespace.cache.OffHeapNamespaceExtractionCacheManager;
import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager;
import org.hamcrest.BaseMatcher;
import org.hamcrest.Description;
import org.joda.time.Period;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
@ -55,7 +68,6 @@ import java.net.URISyntaxException;
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;
@ -66,25 +78,12 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.regex.Pattern;
import java.util.zip.GZIPOutputStream;
import javax.annotation.Nullable;
import org.hamcrest.BaseMatcher;
import org.hamcrest.Description;
import org.joda.time.Period;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
/**
*
*/
@RunWith(Parameterized.class)
public class URIExtractionNamespaceFunctionFactoryTest
public class URIExtractionNamespaceCacheFactoryTest
{
private static final String FAKE_SCHEME = "wabblywoo";
private static final Map<String, SearchableVersionedDataFinder> FINDERS = ImmutableMap.<String, SearchableVersionedDataFinder>of(
@ -184,15 +183,11 @@ public class URIExtractionNamespaceFunctionFactoryTest
final List<Constructor<? extends NamespaceExtractionCacheManager>> cacheConstructors = ImmutableList.<Constructor<? extends NamespaceExtractionCacheManager>>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
)
@ -223,10 +218,8 @@ public class URIExtractionNamespaceFunctionFactoryTest
try {
manager = constructor.newInstance(
new Lifecycle(),
new ConcurrentHashMap<String, Function<String, String>>(),
new ConcurrentHashMap<String, Function<String, String>>(),
new NoopServiceEmitter(),
new HashMap<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>()
new HashMap<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>>()
);
}
catch (Exception e) {
@ -262,29 +255,26 @@ public class URIExtractionNamespaceFunctionFactoryTest
};
}
public URIExtractionNamespaceFunctionFactoryTest(
public URIExtractionNamespaceCacheFactoryTest(
String friendlyName,
String suffix,
Function<File, OutputStream> outStreamSupplier,
Constructor<? extends NamespaceExtractionCacheManager> cacheManagerConstructor
) throws IllegalAccessException, InvocationTargetException, InstantiationException
{
final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>> namespaceFunctionFactoryMap = new HashMap<>();
final Map<Class<? extends ExtractionNamespace>, 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(FINDERS)
new URIExtractionNamespaceCacheFactory(FINDERS)
);
}
@ -299,16 +289,14 @@ public class URIExtractionNamespaceFunctionFactoryTest
private NamespaceExtractionCacheManager manager;
private File tmpFile;
private File tmpFileParent;
private URIExtractionNamespaceFunctionFactory factory;
private URIExtractionNamespaceCacheFactory factory;
private URIExtractionNamespace namespace;
private ConcurrentHashMap<String, Function<String, String>> fnCache;
private ConcurrentHashMap<String, Function<String, List<String>>> reverseFnCache;
private String id;
@Before
public void setUp() throws Exception
{
lifecycle.start();
fnCache.clear();
tmpFileParent = new File(temporaryFolder.newFolder(), "");
Assert.assertTrue(tmpFileParent.mkdir());
Assert.assertTrue(tmpFileParent.isDirectory());
@ -328,9 +316,8 @@ public class URIExtractionNamespaceFunctionFactoryTest
)));
}
}
factory = new URIExtractionNamespaceFunctionFactory(FINDERS);
factory = new URIExtractionNamespaceCacheFactory(FINDERS);
namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -339,6 +326,7 @@ public class URIExtractionNamespaceFunctionFactoryTest
new Period(0),
null
);
id = "ns";
}
@After
@ -350,19 +338,18 @@ public class URIExtractionNamespaceFunctionFactoryTest
@Test
public void simpleTest() throws IOException, ExecutionException, InterruptedException
{
Assert.assertNull(fnCache.get(namespace.getNamespace()));
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
Function<String, String> fn = fnCache.get(namespace.getNamespace());
Assert.assertNotNull(fn);
Assert.assertEquals("bar", fn.apply("foo"));
Assert.assertEquals(null, fn.apply("baz"));
Assert.assertTrue(manager.getKnownIDs().isEmpty());
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace));
Map<String, String> map = manager.getCacheMap(id);
Assert.assertEquals("bar", map.get("foo"));
Assert.assertEquals(null, map.get("baz"));
}
@Test
public void simpleTestRegex() throws IOException, ExecutionException, InterruptedException
{
String regexID = "regex";
final URIExtractionNamespace namespace = new URIExtractionNamespace(
this.namespace.getNamespace(),
null,
Paths.get(this.namespace.getUri()).getParent().toUri(),
Pattern.quote(Paths.get(this.namespace.getUri()).getFileName().toString()),
@ -370,36 +357,23 @@ public class URIExtractionNamespaceFunctionFactoryTest
Period.millis((int) this.namespace.getPollMs()),
null
);
Assert.assertNull(fnCache.get(namespace.getNamespace()));
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
Function<String, String> fn = fnCache.get(namespace.getNamespace());
Assert.assertNotNull(fn);
Assert.assertEquals("bar", fn.apply("foo"));
Assert.assertEquals(null, fn.apply("baz"));
}
@Test
public void testReverseFunction() throws InterruptedException
{
Assert.assertNull(reverseFnCache.get(namespace.getNamespace()));
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
Function<String, List<String>> reverseFn = reverseFnCache.get(namespace.getNamespace());
Assert.assertNotNull(reverseFn);
Assert.assertEquals(Sets.newHashSet("boo", "foo"), Sets.newHashSet(reverseFn.apply("bar")));
Assert.assertEquals(Sets.newHashSet(""), Sets.newHashSet(reverseFn.apply("MissingValue")));
Assert.assertEquals(Sets.newHashSet("emptyString"), Sets.newHashSet(reverseFn.apply("")));
Assert.assertEquals(Sets.newHashSet("emptyString"), Sets.newHashSet(reverseFn.apply(null)));
Assert.assertEquals(Collections.EMPTY_LIST, reverseFn.apply("baz"));
Assert.assertTrue(!manager.getKnownIDs().contains(regexID));
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(regexID, namespace));
Map<String, String> map = manager.getCacheMap(regexID);
Assert.assertNotNull(map);
Assert.assertEquals("bar", map.get("foo"));
Assert.assertEquals(null, map.get("baz"));
}
@Test
public void simplePileONamespacesTest() throws InterruptedException
{
final int size = 128;
List<URIExtractionNamespace> namespaces = new ArrayList<>(size);
List<String> ids = new ArrayList<>(size);
for (int i = 0; i < size; ++i) {
String id = String.format("%d-ns-%d", i << 10, i);
ids.add(id);
URIExtractionNamespace namespace = new URIExtractionNamespace(
String.format("%d-ns-%d", i << 10, i),
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -409,37 +383,33 @@ public class URIExtractionNamespaceFunctionFactoryTest
null
);
namespaces.add(namespace);
Assert.assertNull(fnCache.get(namespace.getNamespace()));
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
Assert.assertFalse(manager.getKnownIDs().contains(id));
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace));
}
for (int i = 0; i < size; ++i) {
URIExtractionNamespace namespace = namespaces.get(i);
Function<String, String> fn = fnCache.get(namespace.getNamespace());
Assert.assertNotNull(fn);
Assert.assertEquals("bar", fn.apply("foo"));
Assert.assertEquals(null, fn.apply("baz"));
manager.delete(namespace.getNamespace());
Assert.assertNull(fnCache.get(namespace.getNamespace()));
for (String id : ids) {
final Map<String, String> map = manager.getCacheMap(id);
Assert.assertEquals("bar", map.get("foo"));
Assert.assertEquals(null, map.get("baz"));
manager.delete(id);
}
Assert.assertTrue(manager.getKnownIDs().isEmpty());
}
@Test
public void testLoadOnlyOnce() throws Exception
{
Assert.assertNull(fnCache.get(namespace.getNamespace()));
Assert.assertTrue(manager.getKnownIDs().isEmpty());
ConcurrentMap<String, String> map = new ConcurrentHashMap<>();
Callable<String> populator = factory.getCachePopulator(namespace, null, map);
Callable<String> populator = factory.getCachePopulator(id, namespace, null, map);
String v = populator.call();
Assert.assertEquals("bar", map.get("foo"));
Assert.assertEquals(null, map.get("baz"));
Assert.assertNotNull(v);
populator = factory.getCachePopulator(namespace, v, map);
populator = factory.getCachePopulator(id, namespace, v, map);
String v2 = populator.call();
Assert.assertEquals(v, v2);
Assert.assertEquals("bar", map.get("foo"));
@ -450,7 +420,6 @@ public class URIExtractionNamespaceFunctionFactoryTest
public void testMissing() throws Exception
{
URIExtractionNamespace badNamespace = new URIExtractionNamespace(
namespace.getNamespace(),
namespace.getUri(),
null, null,
namespace.getNamespaceParseSpec(),
@ -459,32 +428,22 @@ public class URIExtractionNamespaceFunctionFactoryTest
);
Assert.assertTrue(new File(namespace.getUri()).delete());
ConcurrentMap<String, String> map = new ConcurrentHashMap<>();
expectedException.expect(new BaseMatcher<Throwable>()
{
@Override
public void describeTo(Description description)
{
}
@Override
public boolean matches(Object o)
{
if (!(o instanceof Throwable)) {
return false;
}
final Throwable t = (Throwable) o;
return t.getCause() != null && t.getCause() instanceof FileNotFoundException;
}
});
factory.getCachePopulator(badNamespace, null, map).call();
try {
factory.getCachePopulator(id, badNamespace, null, map).call();
}
catch (RuntimeException e) {
Assert.assertNotNull(e.getCause());
Assert.assertEquals(FileNotFoundException.class, e.getCause().getClass());
return;
}
Assert.fail("Did not have exception");
}
@Test
public void testMissingRegex() throws Exception
{
String badId = "bad";
URIExtractionNamespace badNamespace = new URIExtractionNamespace(
namespace.getNamespace(),
null,
Paths.get(namespace.getUri()).getParent().toUri(),
Pattern.quote(Paths.get(namespace.getUri()).getFileName().toString()),
@ -512,14 +471,13 @@ public class URIExtractionNamespaceFunctionFactoryTest
return t.getCause() != null && t.getCause() instanceof FileNotFoundException;
}
});
factory.getCachePopulator(badNamespace, null, map).call();
factory.getCachePopulator(badId, badNamespace, null, map).call();
}
@Test(expected = IAE.class)
public void testExceptionalCreationDoubleURI()
{
new URIExtractionNamespace(
namespace.getNamespace(),
namespace.getUri(),
namespace.getUri(),
null,
@ -533,7 +491,6 @@ public class URIExtractionNamespaceFunctionFactoryTest
public void testExceptionalCreationURIWithPattern()
{
new URIExtractionNamespace(
namespace.getNamespace(),
namespace.getUri(),
null,
"",
@ -547,7 +504,6 @@ public class URIExtractionNamespaceFunctionFactoryTest
public void testExceptionalCreationURIWithLegacyPattern()
{
new URIExtractionNamespace(
namespace.getNamespace(),
namespace.getUri(),
null,
null,
@ -561,7 +517,6 @@ public class URIExtractionNamespaceFunctionFactoryTest
public void testLegacyMix()
{
new URIExtractionNamespace(
namespace.getNamespace(),
null,
namespace.getUri(),
"",
@ -576,7 +531,6 @@ public class URIExtractionNamespaceFunctionFactoryTest
public void testBadPattern()
{
new URIExtractionNamespace(
namespace.getNamespace(),
null,
namespace.getUri(),
"[",
@ -590,7 +544,6 @@ public class URIExtractionNamespaceFunctionFactoryTest
public void testWeirdSchemaOnExactURI() throws Exception
{
final URIExtractionNamespace extractionNamespace = new URIExtractionNamespace(
namespace.getNamespace(),
new URI(
FAKE_SCHEME,
namespace.getUri().getUserInfo(),
@ -607,6 +560,6 @@ public class URIExtractionNamespaceFunctionFactoryTest
null
);
final Map<String, String> map = new HashMap<>();
Assert.assertNotNull(factory.getCachePopulator(extractionNamespace, null, map).call());
Assert.assertNotNull(factory.getCachePopulator(id, extractionNamespace, null, map).call());
}
}

View File

@ -17,14 +17,12 @@
* under the License.
*/
package io.druid.server.namespace.cache;
package io.druid.server.lookup.namespace.cache;
import com.google.common.base.Function;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import com.google.common.io.Closer;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
@ -33,11 +31,11 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import io.druid.concurrent.Execs;
import io.druid.metadata.TestDerbyConnector;
import io.druid.query.extraction.namespace.ExtractionNamespace;
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
import io.druid.query.extraction.namespace.JDBCExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.JDBCExtractionNamespace;
import io.druid.server.lookup.namespace.JDBCExtractionNamespaceCacheFactory;
import io.druid.server.metrics.NoopServiceEmitter;
import io.druid.server.namespace.JDBCExtractionNamespaceFunctionFactory;
import org.joda.time.Period;
import org.junit.After;
import org.junit.Assert;
@ -51,12 +49,8 @@ import org.skife.jdbi.v2.Handle;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@ -101,8 +95,6 @@ public class JDBCExtractionNamespaceTest
this.tsColumn = tsColumn;
}
private final ConcurrentMap<String, Function<String, String>> fnCache = new ConcurrentHashMap<>();
private final ConcurrentMap<String, Function<String, List<String>>> reverseFnCache = new ConcurrentHashMap<>();
private final String tsColumn;
private OnHeapNamespaceExtractionCacheManager extractionCacheManager;
private final Lifecycle lifecycle = new Lifecycle();
@ -194,21 +186,25 @@ public class JDBCExtractionNamespaceTest
extractionCacheManager = new OnHeapNamespaceExtractionCacheManager(
lifecycle,
fnCache,
reverseFnCache,
new NoopServiceEmitter(),
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of(
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>>of(
JDBCExtractionNamespace.class,
new JDBCExtractionNamespaceFunctionFactory()
new JDBCExtractionNamespaceCacheFactory()
{
@Override
public Callable<String> getCachePopulator(
final String id,
final JDBCExtractionNamespace namespace,
final String lastVersion,
final Map<String, String> cache
)
{
final Callable<String> cachePopulator = super.getCachePopulator(namespace, lastVersion, cache);
final Callable<String> cachePopulator = super.getCachePopulator(
id,
namespace,
lastVersion,
cache
);
return new Callable<String>()
{
@Override
@ -371,7 +367,6 @@ public class JDBCExtractionNamespaceTest
InterruptedException, TimeoutException
{
final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace(
namespace,
derbyConnectorRule.getMetadataConnectorConfig(),
tableName,
keyName,
@ -379,56 +374,15 @@ public class JDBCExtractionNamespaceTest
tsColumn,
new Period(0)
);
NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(extractionNamespace));
Function<String, String> extractionFn = fnCache.get(extractionNamespace.getNamespace());
NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace));
final Map<String, String> map = extractionCacheManager.getCacheMap(namespace);
for (Map.Entry<String, String> entry : renames.entrySet()) {
String key = entry.getKey();
String val = entry.getValue();
Assert.assertEquals("non-null check", Strings.emptyToNull(val), extractionFn.apply(key));
Assert.assertEquals("non-null check", Strings.emptyToNull(val), Strings.emptyToNull(map.get(key)));
}
Assert.assertEquals("null check", null, extractionFn.apply("baz"));
}
@Test(timeout = 10_000L)
public void testReverseLookup() throws InterruptedException
{
final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace(
namespace,
derbyConnectorRule.getMetadataConnectorConfig(),
tableName,
keyName,
valName,
tsColumn,
new Period(0)
);
NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(extractionNamespace));
Function<String, List<String>> reverseExtractionFn = reverseFnCache.get(extractionNamespace.getNamespace());
Assert.assertEquals(
"reverse lookup should match",
Sets.newHashSet("foo", "bad"),
Sets.newHashSet(reverseExtractionFn.apply("bar"))
);
Assert.assertEquals(
"reverse lookup should match",
Sets.newHashSet("how about that"),
Sets.newHashSet(reverseExtractionFn.apply("foo"))
);
Assert.assertEquals(
"reverse lookup should match",
Sets.newHashSet("empty string"),
Sets.newHashSet(reverseExtractionFn.apply(""))
);
Assert.assertEquals(
"null is same as empty string",
Sets.newHashSet("empty string"),
Sets.newHashSet(reverseExtractionFn.apply(null))
);
Assert.assertEquals(
"reverse lookup of none existing value should be empty list",
Collections.EMPTY_LIST,
reverseExtractionFn.apply("does't exist")
);
Assert.assertEquals("null check", null, map.get("baz"));
}
@Test(timeout = 10_000L)
@ -437,13 +391,13 @@ public class JDBCExtractionNamespaceTest
{
final JDBCExtractionNamespace extractionNamespace = ensureNamespace();
assertUpdated(extractionNamespace.getNamespace(), "foo", "bar");
assertUpdated(namespace, "foo", "bar");
if (tsColumn != null) {
insertValues(handleRef, "foo", "baz", "1900-01-01 00:00:00");
}
assertUpdated(extractionNamespace.getNamespace(), "foo", "bar");
assertUpdated(namespace, "foo", "bar");
}
@Test(timeout = 60_000L)
@ -452,18 +406,17 @@ public class JDBCExtractionNamespaceTest
{
final JDBCExtractionNamespace extractionNamespace = ensureNamespace();
assertUpdated(extractionNamespace.getNamespace(), "foo", "bar");
assertUpdated(namespace, "foo", "bar");
insertValues(handleRef, "foo", "baz", "2900-01-01 00:00:00");
assertUpdated(extractionNamespace.getNamespace(), "foo", "baz");
assertUpdated(namespace, "foo", "baz");
}
private JDBCExtractionNamespace ensureNamespace()
throws NoSuchFieldException, IllegalAccessException, InterruptedException
{
final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace(
namespace,
derbyConnectorRule.getMetadataConnectorConfig(),
tableName,
keyName,
@ -471,14 +424,14 @@ public class JDBCExtractionNamespaceTest
tsColumn,
new Period(10)
);
extractionCacheManager.schedule(extractionNamespace);
extractionCacheManager.schedule(namespace, extractionNamespace);
waitForUpdates(1_000L, 2L);
Assert.assertEquals(
"sanity check not correct",
"bar",
fnCache.get(extractionNamespace.getNamespace()).apply("foo")
extractionCacheManager.getCacheMap(namespace).get("foo")
);
return extractionNamespace;
}
@ -514,18 +467,18 @@ public class JDBCExtractionNamespaceTest
{
waitForUpdates(1_000L, 2L);
Function<String, String> extractionFn = fnCache.get(namespace);
Map<String, String> map = extractionCacheManager.getCacheMap(namespace);
// rely on test timeout to break out of this loop
while (!extractionFn.apply(key).equals(expected)) {
while (!expected.equals(map.get(key))) {
Thread.sleep(100);
extractionFn = fnCache.get(namespace);
map = extractionCacheManager.getCacheMap(namespace);
}
Assert.assertEquals(
"update check",
expected,
extractionFn.apply(key)
map.get(key)
);
}
}

View File

@ -17,10 +17,9 @@
* under the License.
*/
package io.druid.server.namespace.cache;
package io.druid.server.lookup.namespace.cache;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.ListenableFuture;
@ -31,13 +30,13 @@ import com.metamx.common.IAE;
import com.metamx.common.lifecycle.Lifecycle;
import io.druid.concurrent.Execs;
import io.druid.data.SearchableVersionedDataFinder;
import io.druid.query.extraction.namespace.ExtractionNamespace;
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
import io.druid.query.extraction.namespace.URIExtractionNamespace;
import io.druid.query.extraction.namespace.URIExtractionNamespaceTest;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.URIExtractionNamespace;
import io.druid.query.lookup.namespace.URIExtractionNamespaceTest;
import io.druid.segment.loading.LocalFileTimestampVersionFinder;
import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory;
import io.druid.server.metrics.NoopServiceEmitter;
import io.druid.server.namespace.URIExtractionNamespaceFunctionFactory;
import org.joda.time.Period;
import org.junit.After;
import org.junit.Assert;
@ -48,7 +47,6 @@ import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.lang.reflect.Field;
@ -56,7 +54,6 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException;
@ -67,7 +64,6 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
/**
*
@ -81,8 +77,6 @@ public class NamespaceExtractionCacheManagerExecutorsTest
private Lifecycle lifecycle;
private NamespaceExtractionCacheManager manager;
private File tmpFile;
private final ConcurrentMap<String, Function<String, String>> fnCache = new ConcurrentHashMap<>();
private final ConcurrentMap<String, Function<String, List<String>>> reverseFnCache = new ConcurrentHashMap<>();
private final ConcurrentMap<String, Object> cacheUpdateAlerts = new ConcurrentHashMap<>();
private final AtomicLong numRuns = new AtomicLong(0L);
@ -94,12 +88,13 @@ public class NamespaceExtractionCacheManagerExecutorsTest
lifecycle = new Lifecycle();
// Lifecycle stop is used to shut down executors. Start does nothing, so it's ok to call it here.
lifecycle.start();
final URIExtractionNamespaceFunctionFactory factory = new URIExtractionNamespaceFunctionFactory(
final URIExtractionNamespaceCacheFactory factory = new URIExtractionNamespaceCacheFactory(
ImmutableMap.<String, SearchableVersionedDataFinder>of("file", new LocalFileTimestampVersionFinder())
)
{
@Override
public Callable<String> getCachePopulator(
final String id,
final URIExtractionNamespace extractionNamespace,
final String lastVersion,
final Map<String, String> cache
@ -119,8 +114,8 @@ public class NamespaceExtractionCacheManagerExecutorsTest
}
};
manager = new OnHeapNamespaceExtractionCacheManager(
lifecycle, fnCache, reverseFnCache, new NoopServiceEmitter(),
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of(
lifecycle, new NoopServiceEmitter(),
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>>of(
URIExtractionNamespace.class,
factory
)
@ -128,14 +123,15 @@ public class NamespaceExtractionCacheManagerExecutorsTest
{
@Override
protected <T extends ExtractionNamespace> Runnable getPostRunnable(
final String id,
final T namespace,
final ExtractionNamespaceFunctionFactory<T> factory,
final ExtractionNamespaceCacheFactory<T> factory,
final String cacheId
)
{
final Runnable runnable = super.getPostRunnable(namespace, factory, cacheId);
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
final Runnable runnable = super.getPostRunnable(id, namespace, factory, cacheId);
cacheUpdateAlerts.putIfAbsent(id, new Object());
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(id);
return new Runnable()
{
@Override
@ -174,8 +170,8 @@ public class NamespaceExtractionCacheManagerExecutorsTest
@Test(expected = IAE.class)
public void testDoubleSubmission()
{
final String namespaceID = "ns";
URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -184,11 +180,11 @@ public class NamespaceExtractionCacheManagerExecutorsTest
new Period(0),
null
);
final ListenableFuture<?> future = manager.schedule(namespace);
final ListenableFuture<?> future = manager.schedule(namespaceID, namespace);
Assert.assertFalse(future.isDone());
Assert.assertFalse(future.isCancelled());
try {
manager.schedule(namespace).cancel(true);
manager.schedule(namespaceID, namespace).cancel(true);
}
finally {
future.cancel(true);
@ -199,8 +195,8 @@ public class NamespaceExtractionCacheManagerExecutorsTest
@Test(timeout = 60_000)
public void testSimpleSubmission() throws ExecutionException, InterruptedException
{
final String namespaceID = "ns";
URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -209,7 +205,7 @@ public class NamespaceExtractionCacheManagerExecutorsTest
new Period(0),
null
);
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespaceID, namespace));
}
@Test(timeout = 60_000)
@ -219,9 +215,9 @@ public class NamespaceExtractionCacheManagerExecutorsTest
final long delay = 5;
final long totalRunCount;
final long start;
final String namespaceID = "ns";
try {
final URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -230,15 +226,15 @@ public class NamespaceExtractionCacheManagerExecutorsTest
new Period(delay),
null
);
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
cacheUpdateAlerts.putIfAbsent(namespaceID, new Object());
start = System.currentTimeMillis();
ListenableFuture<?> future = manager.schedule(namespace);
ListenableFuture<?> future = manager.schedule(namespaceID, namespace);
Assert.assertFalse(future.isDone());
Assert.assertFalse(future.isCancelled());
final long preRunCount;
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespaceID);
synchronized (cacheUpdateAlerter) {
preRunCount = numRuns.get();
}
@ -354,7 +350,6 @@ public class NamespaceExtractionCacheManagerExecutorsTest
final long period = 1_000L;// Give it some time between attempts to update
final URIExtractionNamespace namespace = new URIExtractionNamespace(
ns,
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -363,7 +358,8 @@ public class NamespaceExtractionCacheManagerExecutorsTest
new Period(period),
null
);
final ListenableFuture<?> future = manager.schedule(namespace);
Assert.assertTrue(manager.scheduleAndWait(ns, namespace, 10_000));
final ListenableFuture<?> future = manager.implData.get(ns).future;
Assert.assertFalse(future.isCancelled());
Assert.assertFalse(future.isDone());
@ -372,7 +368,7 @@ public class NamespaceExtractionCacheManagerExecutorsTest
final long timeout = 45_000L;
do {
synchronized (cacheUpdateAlerter) {
if (!fnCache.containsKey(ns)) {
if (!manager.implData.containsKey(ns)) {
cacheUpdateAlerter.wait(10_000);
}
}
@ -386,7 +382,7 @@ public class NamespaceExtractionCacheManagerExecutorsTest
throw Throwables.propagate(e);
}
}
if (!fnCache.containsKey(ns) && System.currentTimeMillis() - start > timeout) {
if (!manager.implData.containsKey(ns) && System.currentTimeMillis() - start > timeout) {
throw new RuntimeException(
new TimeoutException(
String.format(
@ -396,11 +392,10 @@ public class NamespaceExtractionCacheManagerExecutorsTest
)
);
}
} while (!fnCache.containsKey(ns));
} while (!manager.implData.containsKey(ns) || !manager.implData.get(ns).enabled.get());
Assert.assertEquals(VALUE, manager.getCacheMap(ns).get(KEY));
Assert.assertTrue(fnCache.containsKey(ns));
Assert.assertTrue(manager.implData.containsKey(ns));
Assert.assertTrue(manager.delete(ns));
@ -418,7 +413,6 @@ public class NamespaceExtractionCacheManagerExecutorsTest
}
Assert.assertFalse(manager.implData.containsKey(ns));
Assert.assertFalse(fnCache.containsKey(ns));
Assert.assertTrue(future.isCancelled());
Assert.assertTrue(future.isDone());
}
@ -430,10 +424,10 @@ public class NamespaceExtractionCacheManagerExecutorsTest
final long period = 5L;
final ListenableFuture future;
long prior = 0;
final String namespaceID = "ns";
try {
final URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -442,11 +436,11 @@ public class NamespaceExtractionCacheManagerExecutorsTest
new Period(period),
null
);
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
cacheUpdateAlerts.putIfAbsent(namespaceID, new Object());
future = manager.schedule(namespace);
future = manager.schedule(namespaceID, namespace);
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespaceID);
synchronized (cacheUpdateAlerter) {
cacheUpdateAlerter.wait();
}
@ -480,9 +474,9 @@ public class NamespaceExtractionCacheManagerExecutorsTest
{
final long numWaits = 5;
final ListenableFuture<?> future;
final String namespaceID = "ns";
try {
final URIExtractionNamespace namespace = new URIExtractionNamespace(
"ns",
tmpFile.toURI(),
null, null,
new URIExtractionNamespace.ObjectMapperFlatDataParser(
@ -492,11 +486,11 @@ public class NamespaceExtractionCacheManagerExecutorsTest
null
);
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
future = manager.schedule(namespace);
cacheUpdateAlerts.putIfAbsent(namespaceID, new Object());
future = manager.schedule(namespaceID, namespace);
Assert.assertFalse(future.isDone());
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespaceID);
for (int i = 0; i < numWaits; ++i) {
synchronized (cacheUpdateAlerter) {
cacheUpdateAlerter.wait();

View File

@ -17,30 +17,37 @@
* under the License.
*/
package io.druid.server.namespace.cache;
package io.druid.server.lookup.namespace.cache;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import io.druid.query.extraction.namespace.ExtractionNamespace;
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
import io.druid.data.SearchableVersionedDataFinder;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import io.druid.query.lookup.namespace.URIExtractionNamespace;
import io.druid.segment.loading.LocalFileTimestampVersionFinder;
import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory;
import io.druid.server.metrics.NoopServiceEmitter;
import org.joda.time.Period;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
@ -53,49 +60,46 @@ public class NamespaceExtractionCacheManagersTest
{
private static final Logger log = new Logger(NamespaceExtractionCacheManagersTest.class);
private static final Lifecycle lifecycle = new Lifecycle();
private static final Map<String, SearchableVersionedDataFinder> PULLERS = ImmutableMap.<String, SearchableVersionedDataFinder>of(
"file",
new LocalFileTimestampVersionFinder()
);
private static final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> CACHE_FACTORIES = ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>>of(
URIExtractionNamespace.class, new URIExtractionNamespaceCacheFactory(PULLERS)
);
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> getParameters()
{
ArrayList<Object[]> params = new ArrayList<>();
ConcurrentMap<String, Function<String, String>> fnMap = new ConcurrentHashMap<String, Function<String, String>>();
ConcurrentMap<String, Function<String, List<String>>> reverserFnMap = new ConcurrentHashMap<String, Function<String, List<String>>>();
params.add(
new Object[]{
new OffHeapNamespaceExtractionCacheManager(
lifecycle,
fnMap,
reverserFnMap,
new NoopServiceEmitter(),
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of()
), fnMap
CACHE_FACTORIES
)
}
);
params.add(
new Object[]{
new OnHeapNamespaceExtractionCacheManager(
lifecycle,
fnMap,
reverserFnMap,
new NoopServiceEmitter(),
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of()
), fnMap
CACHE_FACTORIES
)
}
);
return params;
}
private final NamespaceExtractionCacheManager extractionCacheManager;
private final ConcurrentMap<String, Function<String, String>> fnMap;
public NamespaceExtractionCacheManagersTest(
NamespaceExtractionCacheManager extractionCacheManager,
ConcurrentMap<String, Function<String, String>> fnMap
NamespaceExtractionCacheManager extractionCacheManager
)
{
this.extractionCacheManager = extractionCacheManager;
this.fnMap = fnMap;
}
private static final List<String> nsList = ImmutableList.<String>of("testNs", "test.ns", "//tes-tn!s");
@ -103,21 +107,9 @@ public class NamespaceExtractionCacheManagersTest
@Before
public void setup()
{
fnMap.clear();
// prepopulate caches
for (String ns : nsList) {
final ConcurrentMap<String, String> map = extractionCacheManager.getCacheMap(ns);
fnMap.put(
ns, new Function<String, String>()
{
@Nullable
@Override
public String apply(String input)
{
return map.get(input);
}
}
);
map.put("oldNameSeed1", "oldNameSeed2");
}
}
@ -167,7 +159,10 @@ public class NamespaceExtractionCacheManagersTest
public void testCacheList()
{
List<String> nsList = new ArrayList<String>(NamespaceExtractionCacheManagersTest.nsList);
List<String> retvalList = Lists.newArrayList(extractionCacheManager.getKnownNamespaces());
for (String ns : nsList) {
extractionCacheManager.implData.put(ns, new NamespaceExtractionCacheManager.NamespaceImplData(null, null, null));
}
List<String> retvalList = Lists.newArrayList(extractionCacheManager.getKnownIDs());
Collections.sort(nsList);
Collections.sort(retvalList);
Assert.assertArrayEquals(nsList.toArray(), retvalList.toArray());
@ -179,6 +174,30 @@ public class NamespaceExtractionCacheManagersTest
Assert.assertFalse(extractionCacheManager.delete("I don't exist"));
}
@Test
public void testDeleteOnScheduleFail() throws Exception
{
final String id = "SOME_ID";
Assert.assertFalse(extractionCacheManager.scheduleAndWait(
id,
new URIExtractionNamespace(
new URI("file://tmp/I_DONT_REALLY_EXIST" +
UUID.randomUUID().toString()),
null,
null,
new URIExtractionNamespace.JSONFlatDataParser(
new DefaultObjectMapper(),
"key",
"val"
),
Period.millis(10000),
null
),
500
));
Assert.assertEquals(ImmutableSet.copyOf(nsList), extractionCacheManager.getKnownIDs());
}
public static void waitFor(Future<?> future) throws InterruptedException
{
while (!future.isDone()) {

View File

@ -0,0 +1,64 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.lookup.namespace.cache;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.server.DruidNode;
import io.druid.server.lookup.namespace.NamespaceExtractionModule;
import org.junit.Assert;
import org.junit.Test;
import java.util.Properties;
public class OffHeapNamespaceExtractionCacheManagerTest
{
@Test
public void testInjection()
{
final Injector injector = Initialization.makeInjectorWithModules(
GuiceInjectors.makeStartupInjector(),
ImmutableList.of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null)
);
}
}
)
);
final Properties properties = injector.getInstance(Properties.class);
properties.clear();
properties.put(NamespaceExtractionModule.TYPE_PREFIX, "offHeap");
final NamespaceExtractionCacheManager manager = injector.getInstance(NamespaceExtractionCacheManager.class);
Assert.assertEquals(OffHeapNamespaceExtractionCacheManager.class, manager.getClass());
}
}

View File

@ -0,0 +1,64 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.lookup.namespace.cache;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.server.DruidNode;
import io.druid.server.lookup.namespace.NamespaceExtractionModule;
import org.junit.Assert;
import org.junit.Test;
import java.util.Properties;
public class OnHeapNamespaceExtractionCacheManagerTest
{
@Test
public void testInjection()
{
final Injector injector = Initialization.makeInjectorWithModules(
GuiceInjectors.makeStartupInjector(),
ImmutableList.of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null)
);
}
}
)
);
final Properties properties = injector.getInstance(Properties.class);
properties.clear();
properties.put(NamespaceExtractionModule.TYPE_PREFIX, "onHeap");
final NamespaceExtractionCacheManager manager = injector.getInstance(NamespaceExtractionCacheManager.class);
Assert.assertEquals(OnHeapNamespaceExtractionCacheManager.class, manager.getClass());
}
}

View File

@ -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<String, String> extractionFunction;
private final Function<String, List<String>> reverseExtractionFunction;
@JsonCreator
public NamespacedExtractor(
@NotNull @JacksonInject @Named("dimExtractionNamespace")
final Function<String, Function<String, String>> namespaces,
@NotNull @JacksonInject @Named("dimReverseExtractionNamespace")
final Function<String, Function<String, List<String>>> 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<String> unapply(@NotNull String value)
{
return reverseExtractionFunction.apply(value);
}
}

View File

@ -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<ExtractionNamespace> getNamespaces()
{
return lookups == null ? Collections.<ExtractionNamespace>emptyList() : Arrays.asList(lookups);
}
}

View File

@ -1,232 +0,0 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.namespace;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.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.namespace.ExtractionNamespace;
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
import io.druid.query.extraction.namespace.JDBCExtractionNamespace;
import io.druid.query.extraction.namespace.URIExtractionNamespace;
import io.druid.server.initialization.NamespaceLookupStaticConfig;
import io.druid.server.namespace.cache.NamespaceExtractionCacheManager;
import io.druid.server.namespace.cache.OffHeapNamespaceExtractionCacheManager;
import io.druid.server.namespace.cache.OnHeapNamespaceExtractionCacheManager;
import io.druid.server.namespace.http.NamespacesCacheResource;
import javax.annotation.Nullable;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
*
*/
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";
private final ConcurrentMap<String, Function<String, String>> fnCache = new ConcurrentHashMap<>();
private final ConcurrentMap<String, Function<String, List<String>>> reverseFnCache= new ConcurrentHashMap<>();
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.<Module>of(
new SimpleModule("DruidNamespacedExtractionModule")
{
@Override
public void setupModule(SetupContext context)
{
context.registerSubtypes(NamespacedExtractor.class);
context.registerSubtypes(ExtractionNamespace.class);
}
}
);
}
public static MapBinder<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>> getNamespaceFactoryMapBinder(
final Binder binder
)
{
return MapBinder.newMapBinder(
binder,
new TypeLiteral<Class<? extends ExtractionNamespace>>()
{
},
new TypeLiteral<ExtractionNamespaceFunctionFactory<?>>()
{
}
);
}
@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,
Key.get(NamespaceExtractionCacheManager.class),
Key.get(OnHeapNamespaceExtractionCacheManager.class),
"onheap"
).in(LazySingleton.class);
PolyBind
.optionBinder(binder, Key.get(NamespaceExtractionCacheManager.class))
.addBinding("offheap")
.to(OffHeapNamespaceExtractionCacheManager.class)
.in(LazySingleton.class);
getNamespaceFactoryMapBinder(binder)
.addBinding(JDBCExtractionNamespace.class)
.to(JDBCExtractionNamespaceFunctionFactory.class)
.in(LazySingleton.class);
getNamespaceFactoryMapBinder(binder)
.addBinding(URIExtractionNamespace.class)
.to(URIExtractionNamespaceFunctionFactory.class)
.in(LazySingleton.class);
LifecycleModule.register(binder, NamespaceStaticConfiguration.class);
Jerseys.addResource(binder, NamespacesCacheResource.class);
}
@Provides
@Named("namespaceVersionMap")
@LazySingleton
public ConcurrentMap<String, String> getVersionMap()
{
return new ConcurrentHashMap<>();
}
@Provides
@Named("namespaceExtractionFunctionCache")
public ConcurrentMap<String, Function<String, String>> getFnCache()
{
return fnCache;
}
@Provides
@Named("namespaceReverseExtractionFunctionCache")
public ConcurrentMap<String, Function<String, List<String>>> getReverseFnCache()
{
return reverseFnCache;
}
@Provides
@Named("dimExtractionNamespace")
@LazySingleton
public Function<String, Function<String, String>> getFunctionMaker(
@Named("namespaceExtractionFunctionCache")
final ConcurrentMap<String, Function<String, String>> fnCache
)
{
return new Function<String, Function<String, String>>()
{
@Nullable
@Override
public Function<String, String> apply(final String namespace)
{
Function<String, String> fn = fnCache.get(namespace);
if (fn == null) {
throw new IAE("Namespace [%s] not found", namespace);
}
return fn;
}
};
}
@Provides
@Named("dimReverseExtractionNamespace")
@LazySingleton
public Function<String, Function<String, List<String>>> getReverseFunctionMaker(
@Named("namespaceReverseExtractionFunctionCache")
final ConcurrentMap<String, Function<String, List<String>>> reverseFn
)
{
return new Function<String, Function<String, List<String>>>()
{
@Nullable
@Override
public Function<String, List<String>> apply(final String namespace)
{
Function<String, List<String>> fn = reverseFn.get(namespace);
if (fn == null) {
throw new IAE("Namespace reverse function [%s] not found", namespace);
}
return fn;
}
};
}
}

View File

@ -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();
}
}
}

View File

@ -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<String, Function<String, String>> defaultMap = new ConcurrentHashMap<>();
private static final Function<String, String> NOOP_FN = new Function<String, String>()
{
@Nullable
@Override
public String apply(@Nullable String input)
{
return Strings.isNullOrEmpty(input) ? null : input;
}
};
private static final Function<String, List<String>> NOOP_REVERSE_FN = new Function<String, List<String>>()
{
@Nullable
@Override
public List<String> apply(@Nullable String input)
{
return Strings.isNullOrEmpty(input) ? Collections.<String>emptyList() : Arrays.asList(input);
}
};
private static final Function<String, Function<String, String>> defaultFnFinder = new Function<String, Function<String, String>>()
{
@Nullable
@Override
public Function<String, String> apply(@Nullable String input)
{
Function<String, String> fn = defaultMap.get(input);
return fn == null ? NOOP_FN : fn;
}
};
private static final Function<String,Function<String, List<String>>> defaultReverseFnFinder = new Function<String, Function<String,List<String>>>()
{
@Nullable
@Override
public Function<String, java.util.List<String>> apply(@Nullable final String value)
{
return NOOP_REVERSE_FN;
}
};
@BeforeClass
public static void setupStatic()
{
defaultMap.put(
"noop", new Function<String, String>()
{
@Nullable
@Override
public String apply(String input)
{
return input;
}
}
);
defaultMap.put(
"null", new Function<String, String>()
{
@Nullable
@Override
public String apply(@Nullable String input)
{
return null;
}
}
);
defaultMap.put(
"turtles", new Function<String, String>()
{
@Nullable
@Override
public String apply(@Nullable String input)
{
return "turtle";
}
}
);
defaultMap.put(
"empty", new Function<String, String>()
{
@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));
}
}

View File

@ -91,7 +91,7 @@
<module>extensions-core/kafka-indexing-service</module>
<module>extensions-core/mysql-metadata-storage</module>
<module>extensions-core/postgresql-metadata-storage</module>
<module>extensions-core/namespace-lookup</module>
<module>extensions-core/lookups-cached-global</module>
<module>extensions-core/s3-extensions</module>
<!-- Community extensions -->
<module>extensions-contrib/azure-extensions</module>