mirror of https://github.com/apache/druid.git
[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:
parent
0ac1b27d53
commit
8024b915e2
|
@ -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>
|
||||
|
|
|
@ -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.
|
||||
|
||||
|
|
|
@ -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.
|
|
@ -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)|
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
)
|
||||
);
|
||||
|
|
|
@ -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>
|
|
@ -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>
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
{
|
|
@ -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) {
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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()));
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -17,4 +17,4 @@
|
|||
# under the License.
|
||||
#
|
||||
|
||||
io.druid.server.namespace.NamespacedExtractionModule
|
||||
io.druid.server.lookup.namespace.NamespaceExtractionModule
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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"))
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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();
|
|
@ -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()) {
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
2
pom.xml
2
pom.xml
|
@ -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>
|
||||
|
|
Loading…
Reference in New Issue