diff --git a/docs/content/development/extensions-core/kafka-extraction-namespace.md b/docs/content/development/extensions-core/kafka-extraction-namespace.md index fbf8915f4248..bedf75f173ac 100644 --- a/docs/content/development/extensions-core/kafka-extraction-namespace.md +++ b/docs/content/development/extensions-core/kafka-extraction-namespace.md @@ -38,6 +38,7 @@ See [lookups](../../querying/lookups.html) for how to configure and use lookups. Currently the Kafka lookup extractor feeds the entire kafka stream into a local cache. If you are using OnHeap caching, this can easily clobber your java heap if the kafka stream spews a lot of unique keys. OffHeap caching should alleviate these concerns, but there is still a limit to the quantity of data that can be stored. There is currently no eviction policy. +Even though global cached lookup can support multiple maps in one lookup namespace, Kafka lookup extractor currently provides only one map for each lookup, the default map. ## Testing the Kafka rename functionality diff --git a/docs/content/development/extensions-core/lookups-cached-global.md b/docs/content/development/extensions-core/lookups-cached-global.md index 72d346201c1d..9d3f4597a164 100644 --- a/docs/content/development/extensions-core/lookups-cached-global.md +++ b/docs/content/development/extensions-core/lookups-cached-global.md @@ -34,8 +34,21 @@ Globally cached lookups can be specified as part of the [cluster wide config for "format": "csv", "columns": [ "key", - "value" - ] + "val1", + "val2" + ], + "maps":[ + { + "mapName":"__default", + "keyColumn":"key", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"key", + "valueColumn":"val2" + } + ] }, "pollPeriod": "PT5M" }, @@ -55,8 +68,18 @@ Globally cached lookups can be specified as part of the [cluster wide config for "password": "diurd" }, "table": "lookupTable", - "keyColumn": "mykeyColumn", - "valueColumn": "MyValueColumn", + "maps":[ + { + "mapName":"__default", + "keyColumn":"lookupKey", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"lookupKey2", + "valueColumn":"val2" + } + ], "tsColumn": "timeColumn" }, "firstCacheTimeout": 120000, @@ -95,8 +118,18 @@ In a simple case where only one [tier](../../querying/lookups.html#dynamic-confi "password": "diurd" }, "table": "lookupTable", - "keyColumn": "country_id", - "valueColumn": "country_name", + "maps":[ + { + "mapName":"name", + "keyColumn":"country_id", + "valueColumn":"coutry_name" + }, + { + "mapName":"capital", + "keyColumn":"country_id", + "valueColumn":"capital_city" + } + ], "tsColumn": "timeColumn" }, "firstCacheTimeout": 120000, @@ -120,8 +153,18 @@ Where the coordinator endpoint `/druid/coordinator/v1/lookups/realtime_customer2 "password": "diurd" }, "table": "lookupTable", - "keyColumn": "country_id", - "valueColumn": "country_name", + "maps":[ + { + "mapName":"name", + "keyColumn":"country_id", + "valueColumn":"coutry_name" + }, + { + "mapName":"capital", + "keyColumn":"country_id", + "valueColumn":"capital_city" + } + ], "tsColumn": "timeColumn" }, "firstCacheTimeout": 120000, @@ -159,7 +202,19 @@ The remapping values for each globally cached lookup can be specified by a json "uri": "s3://bucket/some/key/prefix/renames-0003.gz", "namespaceParseSpec":{ "format":"csv", - "columns":["key","value"] + "columns":["key","val1","val2"], + "maps":[ + { + "mapName":"__default", + "keyColumn":"key", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"key", + "valueColumn":"val2" + } + ] }, "pollPeriod":"PT5M" } @@ -172,7 +227,19 @@ The remapping values for each globally cached lookup can be specified by a json "fileRegex":"renames-[0-9]*\\.gz", "namespaceParseSpec":{ "format":"csv", - "columns":["key","value"] + "columns":["key","val1","val2"], + "maps":[ + { + "mapName":"__default", + "keyColumn":"key", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"key", + "valueColumn":"val2" + } + ] }, "pollPeriod":"PT5M" } @@ -199,8 +266,11 @@ Only ONE file which matches the search will be used. For most implementations, t |Parameter|Description|Required|Default| |---------|-----------|--------|-------| |`columns`|The list of columns in the csv file|yes|`null`| -|`keyColumn`|The name of the column containing the key|no|The first column| -|`valueColumn`|The name of the column containing the value|no|The second column| +|`maps`|Map name and key/value columns or fields within data source used in constructing lookup maps|No|`{"mapName":"__default","keyColumn":"key","valueColumn":"value"}`| + +`columns` should contain all the columns specified in `maps`. +`maps` specifies lookup maps under one lookup name. One lookup can have multiple maps that share the source of lookup. It is a list of map specs called `keyValueMap` that has three entries, `mapName`, `keyColumn`, and `valueColumn`. `mapName` is the name of map inside the lookup and `keyColumn` is key column or field name within the lookup source and `valueColumn` is value column or field name within the lookup source. +At lookup, user can specify which map they use or default map `__default` is used if they do not specify explicitly. *example input* @@ -215,9 +285,19 @@ truck,something3,buck ```json "namespaceParseSpec": { "format": "csv", - "columns": ["value","somethingElse","key"], - "keyColumn": "key", - "valueColumn": "value" + "columns": ["val1","val2","key"], + "maps":[ + { + "mapName":"__default", + "keyColumn":"key", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"key", + "valueColumn":"val2" + } + ] } ``` @@ -226,11 +306,11 @@ truck,something3,buck |Parameter|Description|Required|Default| |---------|-----------|--------|-------| |`columns`|The list of columns in the csv file|yes|`null`| -|`keyColumn`|The name of the column containing the key|no|The first column| -|`valueColumn`|The name of the column containing the value|no|The second column| |`delimiter`|The delimiter in the file|no|tab (`\t`)| |`listDelimiter`|The list delimiter in the file|no| (`\u0001`)| +|`maps`|Map name and key/value columns or fields within data source used in constructing lookup maps|No|`{"mapName":"__default","keyColumn":"key","valueColumn":"value"}`| +`columns` should contain all the columns specified in `maps`. *example input* @@ -245,26 +325,36 @@ truck|something,3|buck ```json "namespaceParseSpec": { "format": "tsv", - "columns": ["value","somethingElse","key"], - "keyColumn": "key", - "valueColumn": "value", - "delimiter": "|" + "columns": ["val1","val2","key"], + "delimiter": "|", + "maps":[ + { + "mapName":"__default", + "keyColumn":"key", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"key", + "valueColumn":"val2" + } + ] } ``` ### customJson lookupParseSpec - |Parameter|Description|Required|Default| |---------|-----------|--------|-------| -|`keyFieldName`|The field name of the key|yes|null| -|`valueFieldName`|The field name of the value|yes|null| +|`maps`|Map name and key/value columns or fields within data source used in constructing lookup maps|No|`{"mapName":"__default","keyColumn":"key","valueColumn":"value"}`| + +Inputs should have all the fields used in `maps`. *example input* ```json -{"key": "foo", "value": "bar", "somethingElse" : "something"} -{"key": "baz", "value": "bat", "somethingElse" : "something"} -{"key": "buck", "somethingElse": "something", "value": "truck"} +{"key": "foo", "val1": "bar", "val2" : "something"} +{"key": "baz", "val1": "bat", "val2" : "something"} +{"key": "buck", "val2": "value2", "val1": "truck"} ``` *example namespaceParseSpec* @@ -272,8 +362,18 @@ truck|something,3|buck ```json "namespaceParseSpec": { "format": "customJson", - "keyFieldName": "key", - "valueFieldName": "value" + "maps":[ + { + "mapName":"__default", + "keyColumn":"key", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"key", + "valueColumn":"val2" + } + ] } ``` @@ -281,7 +381,7 @@ With customJson parsing, if the value field for a particular row is missing or n will not be included in the lookup. ### simpleJson lookupParseSpec -The `simpleJson` lookupParseSpec does not take any parameters. It is simply a line delimited json file where the field is the key, and the field's value is the value. +The `simpleJson` lookupParseSpec does not take any parameters. It is simply a line delimited json file where the field is the key, and the field's value is the value. `maps` is ignored and input data is inserted to default map. *example input* @@ -308,8 +408,7 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol |`namespace`|The namespace to define|Yes|| |`connectorConfig`|The connector config to use|Yes|| |`table`|The table which contains the key value pairs|Yes|| -|`keyColumn`|The column in `table` which contains the keys|Yes|| -|`valueColumn`|The column in `table` which contains the values|Yes|| +|`maps`|Map name and key/value columns of `table` used in constructing lookup maps|No|`{"mapName":"__default","keyColumn":"key","valueColumn":"value"}`| |`tsColumn`| The column in `table` which contains when the key was updated|No|Not used| |`pollPeriod`|How often to poll the DB|No|0 (only once)| @@ -324,8 +423,18 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol "password":"diurd" }, "table":"some_lookup_table", - "keyColumn":"the_old_dim_value", - "valueColumn":"the_new_dim_value", + "maps":[ + { + "mapName":"__default", + "keyColumn":"lookupKey", + "valueColumn":"val1" + }, + { + "mapName":"another", + "keyColumn":"lookupKey2", + "valueColumn":"val2" + } + ], "tsColumn":"timestamp_column", "pollPeriod":600000 } @@ -333,4 +442,4 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol # Introspection -Globally cached 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. +Globally cached lookups have introspection points at `/keys` and `/values` which return a complete set of the map names and key/value maps(respectively) in the lookup. Introspection to `/` returns the entire map. Introspection to `/version` returns the version indicator for the lookup. diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md index 6ab8524ced39..fe80d6c7aa76 100644 --- a/docs/content/querying/dimensionspecs.md +++ b/docs/content/querying/dimensionspecs.md @@ -232,26 +232,7 @@ Explicit lookups allow you to specify a set of keys and values to use when perfo } ``` -```json -{ - "type":"lookup", - "lookup":{"type":"namespace","namespace":"some_lookup"}, - "replaceMissingValueWith":"Unknown", - "injective":false -} -``` - -```json -{ - "type":"lookup", - "lookup":{"type":"namespace","namespace":"some_lookup"}, - "retainMissingValue":true, - "injective":false -} -``` - -A lookup can be of type `namespace` or `map`. A `map` lookup is passed as part of the query. -A `namespace` lookup is populated on all the nodes which handle queries as per [lookups](../querying/lookups.html) +A lookup has only one type `map`. A `map` lookup is passed as part of the query. A property of `retainMissingValue` and `replaceMissingValueWith` can be specified at query time to hint how to handle missing values. Setting `replaceMissingValueWith` to `""` has the same effect as setting it to `null` or omitting the property. Setting `retainMissingValue` to true will use the dimension's original value if it is not found in the lookup. The default values are `replaceMissingValueWith = null` and `retainMissingValue = false` which causes missing values to be treated as missing. @@ -479,3 +460,15 @@ The second kind where it is not possible to pass at query time due to their size "name":"lookupName" } ``` + +User can specify the `mapName` inside the given lookup as [Globally Cached Lookups](../development/extensions-core/lookups-cached-global.html) can have multiple maps in each lookup. + +```json +{ + "type":"lookup", + "dimension":"dimensionName", + "outputName":"dimensionOutputName", + "name":"lookupName", + "mapName":"map1" +} +``` \ No newline at end of file diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index cd57d92e5021..b6bbc934fc17 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -108,8 +108,18 @@ So a config might look something like: "password": "diurd" }, "table": "lookupTable", - "keyColumn": "country_id", - "valueColumn": "country_name", + "keyValueMaps":[ + { + "mapName":"name", + "keyName":"country_id", + "valueName":"coutry_name" + }, + { + "mapName":"capital", + "keyName":"country_id", + "valueName":"capital_city" + } + ], "tsColumn": "timeColumn" }, "firstCacheTimeout": 120000, diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java index 2caa482ee738..aef50088245f 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java @@ -38,6 +38,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.query.extraction.MapLookupExtractor; +import io.druid.query.lookup.namespace.KeyValueMap; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaStream; @@ -64,7 +65,7 @@ import java.util.regex.Pattern; @JsonTypeName("kafka") -public class KafkaLookupExtractorFactory implements LookupExtractorFactory +public class KafkaLookupExtractorFactory extends LookupExtractorFactory { private static final Logger LOG = new Logger(KafkaLookupExtractorFactory.class); static final Decoder DEFAULT_STRING_DECODER = new Decoder() @@ -182,7 +183,7 @@ public boolean start() kafkaProperties.setProperty("group.id", factoryId); final String topic = getKafkaTopic(); LOG.debug("About to listen to topic [%s] with group.id [%s]", topic, factoryId); - final Map map = cacheManager.getCacheMap(factoryId); + final Map map = cacheManager.getInnerCacheMap(factoryId, KeyValueMap.DEFAULT_MAPNAME); mapRef.set(map); // Enable publish-subscribe kafkaProperties.setProperty("auto.offset.reset", "smallest"); diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java index 26dbf4186124..c2391114b66c 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java @@ -26,7 +26,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.StringUtils; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; @@ -270,7 +269,7 @@ public void testStartStop() )).andReturn(ImmutableList.of(kafkaStream)).once(); EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); EasyMock.expect(consumerIterator.hasNext()).andAnswer(getBlockingAnswer()).anyTimes(); - EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + EasyMock.expect(cacheManager.getInnerCacheMap(EasyMock.anyString(), EasyMock.anyString())) .andReturn(new ConcurrentHashMap()) .once(); EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); @@ -314,7 +313,7 @@ ConsumerConnector buildConnector(Properties properties) @Test public void testStartFailsFromTimeout() throws Exception { - EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + EasyMock.expect(cacheManager.getInnerCacheMap(EasyMock.anyString(), EasyMock.anyString())) .andReturn(new ConcurrentHashMap()) .once(); EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); @@ -361,7 +360,7 @@ public void testStopDeleteError() )).andReturn(ImmutableList.of(kafkaStream)).once(); EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); EasyMock.expect(consumerIterator.hasNext()).andAnswer(getBlockingAnswer()).anyTimes(); - EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + EasyMock.expect(cacheManager.getInnerCacheMap(EasyMock.anyString(), EasyMock.anyString())) .andReturn(new ConcurrentHashMap()) .once(); EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(false).once(); @@ -402,7 +401,7 @@ public void testStartStopStart() )).andReturn(ImmutableList.of(kafkaStream)).once(); EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); EasyMock.expect(consumerIterator.hasNext()).andAnswer(getBlockingAnswer()).anyTimes(); - EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + EasyMock.expect(cacheManager.getInnerCacheMap(EasyMock.anyString(), EasyMock.anyString())) .andReturn(new ConcurrentHashMap()) .once(); EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); @@ -442,7 +441,7 @@ public void testStartStartStop() )).andReturn(ImmutableList.of(kafkaStream)).once(); EasyMock.expect(kafkaStream.iterator()).andReturn(consumerIterator).anyTimes(); EasyMock.expect(consumerIterator.hasNext()).andAnswer(getBlockingAnswer()).anyTimes(); - EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) + EasyMock.expect(cacheManager.getInnerCacheMap(EasyMock.anyString(), EasyMock.anyString())) .andReturn(new ConcurrentHashMap()) .once(); EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index 0cbae359cc2d..2d8a61962262 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -52,6 +52,11 @@ ${project.parent.version} provided + + com.querydsl + querydsl-sql + 4.1.3 + diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MultiMapsPopulator.java similarity index 64% rename from extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MultiMapsPopulator.java index dcf90849c6bc..96b3e1e17e46 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MultiMapsPopulator.java @@ -20,29 +20,34 @@ package io.druid.data.input; import com.google.common.base.Charsets; +import com.google.common.base.Function; import com.google.common.io.ByteSource; import com.google.common.io.LineProcessor; -import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.parsers.Parser; import java.io.IOException; import java.util.Map; +import java.util.concurrent.ConcurrentMap; /** - * Simple class that takes a `ByteSource` and uses a `Parser` to populate a `Map` + * Simple class that takes a `ByteSource` and uses a `Parser>` to populate multiple `Map`s * The `ByteSource` must be UTF-8 encoded - *

- * If this is handy for other use cases pleaes move this class into a common module + * + * If this is handy for other use cases please move this class into a common module */ -public class MapPopulator +public class MultiMapsPopulator { - private final Parser parser; + private final Parser> parser; + private Function> mapAllocator; - public MapPopulator( - Parser parser + public MultiMapsPopulator( + Parser> parser, + Function> mapAllocator ) { this.parser = parser; + this.mapAllocator = mapAllocator; } public static class PopulateResult @@ -68,16 +73,16 @@ public int getEntries() } /** - * Read through the `source` line by line and populate `map` with the data returned from the `parser` + * Read through the `source` line by line and populate `map`s with the data returned from the `parser` * * @param source The ByteSource to read lines from - * @param map The map to populate + * @param maps The map to populate * - * @return number of lines read and entries parsed + * @return The number of entries parsed * * @throws IOException */ - public PopulateResult populate(final ByteSource source, final Map map) throws IOException + public PopulateResult populate(final ByteSource source, final ConcurrentMap> maps) throws IOException { return source.asCharSource(Charsets.UTF_8).readLines( new LineProcessor() @@ -88,13 +93,19 @@ public PopulateResult populate(final ByteSource source, final Map map) thr @Override public boolean processLine(String line) throws IOException { - if (lines == Integer.MAX_VALUE) { - throw new ISE("Cannot read more than %,d lines", Integer.MAX_VALUE); + Map> parseResult = parser.parse(line); + for (Map.Entry> entry: parseResult.entrySet()) { + Map map = maps.get(entry.getKey()); + if (map == null) + { + map = mapAllocator.apply(entry.getKey()); + maps.put(entry.getKey(), map); + } + Map kvMap = entry.getValue(); + map.putAll(kvMap); + entries += kvMap.size(); } - final Map kvMap = parser.parse(line); - map.putAll(kvMap); lines++; - entries += kvMap.size(); return true; } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index f02ecb85b7cb..e8bb5dfc3cd1 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -26,12 +26,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; - import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.KeyValueMap; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import javax.annotation.Nullable; @@ -43,7 +44,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; @JsonTypeName("cachedNamespace") -public class NamespaceLookupExtractorFactory implements LookupExtractorFactory +public class NamespaceLookupExtractorFactory extends LookupExtractorFactory { private static final Logger LOG = new Logger(NamespaceLookupExtractorFactory.class); @@ -191,9 +192,15 @@ public boolean isInjective() return injective; } - // Grab the latest snapshot from the cache manager @Override public LookupExtractor get() + { + return get(KeyValueMap.DEFAULT_MAPNAME); + } + + // Grab the latest snapshot from the cache manager + @Override + public LookupExtractor get(String innerMapName) { final Lock readLock = startStopSync.readLock(); try { @@ -214,7 +221,10 @@ public LookupExtractor get() if (preVersion == null) { throw new ISE("Namespace vanished for [%s]", extractorID); } - map = manager.getCacheMap(extractorID); + map = manager.getCacheMap(extractorID).get(new Pair(extractorID, innerMapName)); + if (map == null) { + throw new ISE("map [%s] does not exists in [%s]", innerMapName, extractorID); + } postVersion = manager.getVersion(extractorID); if (postVersion == null) { // We lost some horrible race... make sure we clean up @@ -243,4 +253,35 @@ public byte[] getCacheKey() readLock.unlock(); } } + + public Map> getAllMaps() + { + final Lock readLock = startStopSync.readLock(); + readLock.lock(); + try { + if (!started) { + throw new ISE("Factory [%s] not started", extractorID); + } + String preVersion = null, postVersion = null; + Map> mapMap = 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); + } + mapMap = 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)); + return mapMap; + } + finally { + readLock.unlock(); + } + } } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupIntrospectHandler.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupIntrospectHandler.java index 4f1c83d56531..7558c262440e 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupIntrospectHandler.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupIntrospectHandler.java @@ -20,14 +20,15 @@ package io.druid.query.lookup; import com.google.common.collect.ImmutableMap; - +import com.google.common.collect.Maps; import io.druid.common.utils.ServletResourceUtils; -import io.druid.java.util.common.ISE; -import io.druid.query.extraction.MapLookupExtractor; +import io.druid.java.util.common.Pair; +import io.druid.query.lookup.namespace.KeyValueMap; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import javax.ws.rs.GET; import javax.ws.rs.Path; +import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; @@ -47,60 +48,122 @@ public NamespaceLookupIntrospectHandler( this.extractorID = extractorID; this.manager = manager; } + + @GET + @Path("/") + @Produces(MediaType.APPLICATION_JSON) + public Response getDefault() + { + return getMap(KeyValueMap.DEFAULT_MAPNAME); + } + @GET @Path("/keys") @Produces(MediaType.APPLICATION_JSON) - public Response getKeys() + public Response getDefaultKeys() + { + return getKeys(KeyValueMap.DEFAULT_MAPNAME); + } + + @GET + @Path("/values") + @Produces(MediaType.APPLICATION_JSON) + public Response getDefaultValues() + { + return getValues(KeyValueMap.DEFAULT_MAPNAME); + } + + @GET + @Path("/maps") + @Produces(MediaType.APPLICATION_JSON) + public Response getMaps( + ) { try { - return Response.ok(getLatest().keySet()).build(); + return Response.ok(getIdCleanedMaps().keySet()).build(); } - catch (ISE e) { + catch (Exception e) { return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build(); } } @GET - @Path("/values") + @Path("/maps/{mapName}") @Produces(MediaType.APPLICATION_JSON) - public Response getValues() + public Response getMap( + @PathParam("mapName") final String mapName + ) { try { - return Response.ok(getLatest().values()).build(); + return Response.ok(getIdCleanedMaps().get(mapName)).build(); } - catch (ISE e) { + catch (Exception e) { return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build(); } } @GET - @Path("/version") + @Path("/maps/{mapName}/keys") @Produces(MediaType.APPLICATION_JSON) - public Response getVersion() + public Response getKeys( + @PathParam("mapName") final String mapName + ) { - 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(); + try { + Map map = getIdCleanedMaps().get(mapName); + if (map == null) { + return Response.noContent().build(); + } + + return Response.ok(map.keySet()).build(); + } + catch (Exception e) { + return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build(); } } @GET + @Path("/maps/{mapName}/values") @Produces(MediaType.APPLICATION_JSON) - public Response getMap() + public Response getValues( + @PathParam("mapName") final String mapName + ) { try { - return Response.ok(getLatest()).build(); + Map map = getIdCleanedMaps().get(mapName); + if (map == null) { + return Response.noContent().build(); + } + + return Response.ok(map.values()).build(); } - catch (ISE e) { + catch (Exception e) { return Response.status(Response.Status.NOT_FOUND).entity(ServletResourceUtils.sanitizeException(e)).build(); } } - private Map getLatest() + @GET + @Path("/version") + @Produces(MediaType.APPLICATION_JSON) + public Response getVersion() { - return ((MapLookupExtractor) factory.get()).getMap(); + 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(); + } + } + + private Map> getIdCleanedMaps() + { + Map> mapMap = factory.getAllMaps(); + Map> retMap = Maps.newHashMap(); + for (Pair key :mapMap.keySet()) { + String mapName = (String) key.rhs; + retMap.put(mapName, mapMap.get(key)); + } + return retMap; } } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java index 8eb26137295e..f23a6e29df72 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java @@ -33,7 +33,7 @@ * It is expected to have an accompanying ExtractionNamespaceFunctionFactory which handles the actual * extraction implementation. */ -public interface ExtractionNamespace +public abstract class ExtractionNamespace { - long getPollMs(); + public abstract long getPollMs(); } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java index 3fba935824c1..e8f175ff3c64 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java @@ -19,13 +19,18 @@ package io.druid.query.lookup.namespace; +import com.google.common.base.Function; +import io.druid.java.util.common.Pair; + import java.util.Map; +import java.util.concurrent.ConcurrentMap; /** * */ -public interface ExtractionNamespaceCacheFactory +public class ExtractionNamespaceCacheFactory { + /** * This function is called once if `ExtractionNamespace.getUpdateMs() == 0`, or every update if * `ExtractionNamespace.getUpdateMs() > 0` @@ -36,15 +41,57 @@ public interface ExtractionNamespaceCacheFactory * If the returned version is the same as what is passed in as lastVersion, then no swap takes place, and the swap * is discarded. * + * This method is used to support previously implemented {@link #populateCache(String, ExtractionNamespace, String, Map) populateCache()} + * that did not support multiple maps in one lookup. + * This method injects the result of that populateCache() as a default map of lookup. + * If you want to support multiple maps in one lookup, you should override this method as onheap and offheap factories do. + * * @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 * namespace Map in NamespaceExtractionCacheManager. Implementations which cannot offer - * a swappable cache of the data may ignore this but must make sure `buildFn(...)` returns + * a swappable cache of the data may ignore this but must make sure `buildFnMap(...)` returns * a proper Function. * * @return return the (new) version string used in the populating */ - String populateCache(String id, T extractionNamespace, String lastVersion, Map swap) throws Exception; + public String populateCache( + String id, + T extractionNamespace, + String lastVersion, + ConcurrentMap> swap, + Function> mapAllocator + ) throws Exception + { + Pair key = new Pair(id, KeyValueMap.DEFAULT_MAPNAME); + Map cache = swap.get(key); + if (cache == null) + { + cache = mapAllocator.apply(key); + swap.put(key, cache); + } + return populateCache(id, extractionNamespace, lastVersion, cache); + } + + /** + * For minimal changes of other extensions that already defined populateCache() + * based on the previous implementation of globally cached lookup + * + * {@link #populateCache(String, ExtractionNamespace, String, ConcurrentMap, Function) populateCache()} + * will wrap this method to enable mapName support. + * + * @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 + * namespace Map in NamespaceExtractionCacheManager. Implementations which cannot offer + * a swappable cache of the data may ignore this but must make sure `buildFnMap(...)` returns + * a proper Function. + * @return return the (new) version string used in the populating + */ + protected String populateCache(String id, T extractionNamespace, String lastVersion, Map swap) + { + return null; + } } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java index 9fb9c3c5de8c..6a14997b21b2 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java @@ -23,27 +23,28 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import io.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.commons.lang.StringUtils; import org.joda.time.Period; import javax.annotation.Nullable; import javax.validation.constraints.Min; import javax.validation.constraints.NotNull; +import java.util.List; /** * */ @JsonTypeName("jdbc") -public class JDBCExtractionNamespace implements ExtractionNamespace +public class JDBCExtractionNamespace extends ExtractionNamespace { @JsonProperty private final MetadataStorageConnectorConfig connectorConfig; @JsonProperty private final String table; @JsonProperty - private final String keyColumn; - @JsonProperty - private final String valueColumn; + private final List maps; @JsonProperty private final String tsColumn; @JsonProperty @@ -55,10 +56,12 @@ public JDBCExtractionNamespace( final MetadataStorageConnectorConfig connectorConfig, @NotNull @JsonProperty(value = "table", required = true) final String table, - @NotNull @JsonProperty(value = "keyColumn", required = true) + @JsonProperty(value = "keyColumn", required = true) final String keyColumn, - @NotNull @JsonProperty(value = "valueColumn", required = true) + @JsonProperty(value = "valueColumn", required = true) final String valueColumn, + @JsonProperty(value = "maps", required = true) + List maps, @Nullable @JsonProperty(value = "tsColumn", required = false) final String tsColumn, @Min(0) @Nullable @JsonProperty(value = "pollPeriod", required = false) @@ -68,8 +71,10 @@ public JDBCExtractionNamespace( this.connectorConfig = Preconditions.checkNotNull(connectorConfig, "connectorConfig"); Preconditions.checkNotNull(connectorConfig.getConnectURI(), "connectorConfig.connectURI"); this.table = Preconditions.checkNotNull(table, "table"); - this.keyColumn = Preconditions.checkNotNull(keyColumn, "keyColumn"); - this.valueColumn = Preconditions.checkNotNull(valueColumn, "valueColumn"); + Preconditions.checkArgument((keyColumn != null && valueColumn != null) || maps != null, + "Either keyColumn & valueColumn or maps should be specified"); + this.maps = (maps != null) ? maps + : ImmutableList.of(new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, keyColumn, valueColumn)); this.tsColumn = tsColumn; this.pollPeriod = pollPeriod == null ? new Period(0L) : pollPeriod; } @@ -84,14 +89,9 @@ public String getTable() return table; } - public String getKeyColumn() - { - return keyColumn; - } - - public String getValueColumn() + public List getMaps() { - return valueColumn; + return maps; } public String getTsColumn() @@ -109,11 +109,10 @@ public long getPollMs() public String toString() { return String.format( - "JDBCExtractionNamespace = { connectorConfig = { %s }, table = %s, keyColumn = %s, valueColumn = %s, tsColumn = %s, pollPeriod = %s}", + "JDBCExtractionNamespace = { connectorConfig = { %s }, table = %s, key & value Columns = [%s], tsColumn = %s, pollPeriod = %s}", connectorConfig.toString(), table, - keyColumn, - valueColumn, + StringUtils.join(maps, ','), tsColumn, pollPeriod ); @@ -137,17 +136,13 @@ public boolean equals(Object o) if (!table.equals(that.table)) { return false; } - if (!keyColumn.equals(that.keyColumn)) { - return false; - } - if (!valueColumn.equals(that.valueColumn)) { + if (!maps.equals(that.maps)) { return false; } if (tsColumn != null ? !tsColumn.equals(that.tsColumn) : that.tsColumn != null) { return false; } return pollPeriod.equals(that.pollPeriod); - } @Override @@ -155,8 +150,7 @@ public int hashCode() { int result = connectorConfig.hashCode(); result = 31 * result + table.hashCode(); - result = 31 * result + keyColumn.hashCode(); - result = 31 * result + valueColumn.hashCode(); + result = 31 * result + maps.hashCode(); result = 31 * result + (tsColumn != null ? tsColumn.hashCode() : 0); result = 31 * result + pollPeriod.hashCode(); return result; diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/KeyValueMap.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/KeyValueMap.java new file mode 100644 index 000000000000..ab3a9e252dee --- /dev/null +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/KeyValueMap.java @@ -0,0 +1,137 @@ +/* + * 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.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableSet; + +import javax.annotation.Nullable; +import java.util.List; + +public class KeyValueMap +{ + private final String mapName; + private final String keyColumn; + private final String valueColumn; + + public static String DEFAULT_MAPNAME = "__default"; + + @JsonCreator + public KeyValueMap( + @JsonProperty("mapName") String mapName, + @JsonProperty("keyColumn") String keyColumn, + @JsonProperty("valueColumn") String valueColumn + ) + { + this.mapName = mapName; + this.keyColumn = keyColumn; + this.valueColumn = valueColumn; + } + + @JsonProperty + public String getMapName() + { + return mapName; + } + + @JsonProperty + public String getKeyColumn() + { + return keyColumn; + } + + @JsonProperty + public String getValueColumn() + { + return valueColumn; + } + + @Override + public String toString() + { + return "KeyValueMap{" + + "mapName='" + mapName + '\'' + + ", keyColumn='" + keyColumn + '\'' + + ", valueColumn='" + valueColumn + '\'' + + '}'; + } + + @Override + public int hashCode() + { + int result = mapName.hashCode(); + result = 31 * result + keyColumn.hashCode(); + result = 31 * result + valueColumn.hashCode(); + + return result; + } + + @Override + public boolean equals(Object o) + { + if (o == null) { + return false; + } + if (!(o instanceof KeyValueMap)) { + return false; + } + KeyValueMap that = (KeyValueMap)o; + + if (!getMapName().equals(that.getMapName())) { + return false; + } + if (!getKeyColumn().equals(that.getKeyColumn())) { + return false; + } + if (!getValueColumn().equals(that.getValueColumn())) { + return false; + } + + return true; + } + + public static List getRequiredFields(List keyValueMaps) + { + ImmutableSet keySet = FluentIterable.from(keyValueMaps) + .transform( + new Function() { + @Nullable + @Override + public String apply(@Nullable KeyValueMap input) { + return input.keyColumn; + } + }) + .toSet(); + ImmutableSet valueSet = FluentIterable.from(keyValueMaps) + .transform( + new Function() { + @Override + public String apply(KeyValueMap input) { + return input.valueColumn; + } + }) + .toSet(); + + return new ImmutableSet.Builder().addAll(keySet).addAll(valueSet).build().asList(); + } +} diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java index ae05d5c931c3..39daba4b40c6 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/StaticMapExtractionNamespace.java @@ -33,7 +33,7 @@ * Any desire to test the *caching mechanisms in this extension* can use this class. */ @JsonTypeName(StaticMapExtractionNamespace.TYPE_NAME) -public class StaticMapExtractionNamespace implements ExtractionNamespace +public class StaticMapExtractionNamespace extends ExtractionNamespace { static final String TYPE_NAME = "staticMap"; private final Map map; diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java index a7c27c66638e..44c485b8a8c9 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java @@ -27,31 +27,35 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.base.Throwables; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import io.druid.guice.annotations.Json; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.UOE; import io.druid.java.util.common.parsers.CSVParser; import io.druid.java.util.common.parsers.DelimitedParser; import io.druid.java.util.common.parsers.JSONParser; import io.druid.java.util.common.parsers.Parser; - +import org.apache.commons.lang.StringUtils; import org.joda.time.Period; import javax.annotation.Nullable; import javax.validation.constraints.Min; -import javax.validation.constraints.NotNull; import java.io.IOException; import java.net.URI; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; @@ -59,7 +63,7 @@ * */ @JsonTypeName("uri") -public class URIExtractionNamespace implements ExtractionNamespace +public class URIExtractionNamespace extends ExtractionNamespace { @JsonProperty private final URI uri; @@ -68,9 +72,9 @@ public class URIExtractionNamespace implements ExtractionNamespace @JsonProperty private final FlatDataParser namespaceParseSpec; @JsonProperty - private final String fileRegex; - @JsonProperty private final Period pollPeriod; + @JsonProperty + private final String fileRegex; @JsonCreator public URIExtractionNamespace( @@ -177,6 +181,7 @@ public boolean equals(Object o) if (getFileRegex() != null ? !getFileRegex().equals(that.getFileRegex()) : that.getFileRegex() != null) { return false; } + return pollPeriod.equals(that.pollPeriod); } @@ -192,39 +197,74 @@ public int hashCode() return result; } - private static class DelegateParser implements Parser + public FlatDataParser getParser() + { + return namespaceParseSpec; + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format") + @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "csv", value = CSVFlatDataParser.class), + @JsonSubTypes.Type(name = "tsv", value = TSVFlatDataParser.class), + @JsonSubTypes.Type(name = "customJson", value = JSONFlatDataParser.class), + @JsonSubTypes.Type(name = "simpleJson", value = ObjectMapperFlatDataParser.class) + }) + public static abstract class FlatDataParser implements Parser> + { + protected String id = null; + + public FlatDataParser withID(String id) + { + this.id = id; + return this; + } + + @Override + public Map> parse(String input) + { + Preconditions.checkArgument(id != null, "ID should be set before parse()."); + return parseInternal(input); + } + + public abstract Map> parseInternal(String input); + } + + public static class MultiMapFlatDataParser extends FlatDataParser { - private final Parser delegate; - private final String key; - private final String value; + protected final List maps; + protected final Parser delegate; - private DelegateParser( + public MultiMapFlatDataParser( Parser delegate, - @NotNull String key, - @NotNull String value + List maps ) { this.delegate = delegate; - this.key = key; - this.value = value; + this.maps = maps; } @Override - public Map parse(String input) + public Map> parseInternal(String input) { final Map inner = delegate.parse(input); - final String k = Preconditions.checkNotNull( - inner.get(key), - "Key column [%s] missing data in line [%s]", - key, - input - ).toString(); // Just in case is long - final Object val = inner.get(value); - if (val == null) { + ImmutableMap.Builder> builder = new ImmutableMap.Builder<>(); + + for (KeyValueMap map: maps) + { + final String k = Preconditions.checkNotNull( + inner.get(map.getKeyColumn()), + "Key column [%s] missing data in line [%s]", + map.getKeyColumn(), + input + ).toString();// Just in case is long + final Object val = inner.get(map.getValueColumn()); // Skip null or missing values, treat them as if there were no row at all. - return ImmutableMap.of(); + if (val != null) { + builder.put(new Pair(id, map.getMapName()), ImmutableMap.of(k, val.toString())); + } } - return ImmutableMap.of(k, val.toString()); + + return builder.build(); } @Override @@ -238,65 +278,31 @@ public List getFieldNames() { return delegate.getFieldNames(); } - } - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format") - @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "csv", value = CSVFlatDataParser.class), - @JsonSubTypes.Type(name = "tsv", value = TSVFlatDataParser.class), - @JsonSubTypes.Type(name = "customJson", value = JSONFlatDataParser.class), - @JsonSubTypes.Type(name = "simpleJson", value = ObjectMapperFlatDataParser.class) - }) - public static interface FlatDataParser - { - Parser getParser(); + @JsonProperty + public List getMaps() + { + return maps; + } } @JsonTypeName("csv") - public static class CSVFlatDataParser implements FlatDataParser + public static class CSVFlatDataParser extends MultiMapFlatDataParser { - private final Parser parser; private final List columns; - private final String keyColumn; - private final String valueColumn; @JsonCreator public CSVFlatDataParser( @JsonProperty("columns") List columns, - @JsonProperty("keyColumn") final String keyColumn, - @JsonProperty("valueColumn") final String valueColumn + @JsonProperty("maps") List maps ) { - Preconditions.checkArgument( - Preconditions.checkNotNull(columns, "`columns` list required").size() > 1, - "Must specify more than one column to have a key value pair" + super( + new CSVParser(Optional.absent(), columns), + URIExtractionNamespace.getOrCreateKeyVauleMaps(maps, columns) ); - Preconditions.checkArgument( - !(Strings.isNullOrEmpty(keyColumn) ^ Strings.isNullOrEmpty(valueColumn)), - "Must specify both `keyColumn` and `valueColumn` or neither `keyColumn` nor `valueColumn`" - ); this.columns = columns; - this.keyColumn = Strings.isNullOrEmpty(keyColumn) ? columns.get(0) : keyColumn; - this.valueColumn = Strings.isNullOrEmpty(valueColumn) ? columns.get(1) : valueColumn; - Preconditions.checkArgument( - columns.contains(this.keyColumn), - "Column [%s] not found int columns: %s", - this.keyColumn, - Arrays.toString(columns.toArray()) - ); - Preconditions.checkArgument( - columns.contains(this.valueColumn), - "Column [%s] not found int columns: %s", - this.valueColumn, - Arrays.toString(columns.toArray()) - ); - - this.parser = new DelegateParser( - new CSVParser(Optional.absent(), columns), - this.keyColumn, - this.valueColumn - ); } @JsonProperty @@ -305,24 +311,6 @@ public List getColumns() return columns; } - @JsonProperty - public String getKeyColumn() - { - return this.keyColumn; - } - - @JsonProperty - public String getValueColumn() - { - return this.valueColumn; - } - - @Override - public Parser getParser() - { - return parser; - } - @Override public boolean equals(Object o) { @@ -335,79 +323,53 @@ public boolean equals(Object o) CSVFlatDataParser that = (CSVFlatDataParser) o; - if (!getColumns().equals(that.getColumns())) { - return false; - } - if (!getKeyColumn().equals(that.getKeyColumn())) { + if (!getMaps().containsAll(that.getMaps()) || !that.getMaps().containsAll(getMaps())) { return false; } - return getValueColumn().equals(that.getValueColumn()); + return getColumns().equals(that.getColumns()); } @Override public String toString() { return String.format( - "CSVFlatDataParser = { columns = %s, keyColumn = %s, valueColumn = %s }", - Arrays.toString(columns.toArray()), - keyColumn, - valueColumn + "CSVFlatDataParser = { " + + "columns = %s " + + ", maps = [" + StringUtils.join(maps, ',') + "]" + + "}", + Arrays.toString(columns.toArray()) ); } } @JsonTypeName("tsv") - public static class TSVFlatDataParser implements FlatDataParser + public static class TSVFlatDataParser extends MultiMapFlatDataParser { - private final Parser parser; private final List columns; private final String delimiter; private final String listDelimiter; - private final String keyColumn; - private final String valueColumn; @JsonCreator public TSVFlatDataParser( @JsonProperty("columns") List columns, @JsonProperty("delimiter") String delimiter, @JsonProperty("listDelimiter") String listDelimiter, - @JsonProperty("keyColumn") final String keyColumn, - @JsonProperty("valueColumn") final String valueColumn + @JsonProperty("maps") List maps ) { - Preconditions.checkArgument( - Preconditions.checkNotNull(columns, "`columns` list required").size() > 1, - "Must specify more than one column to have a key value pair" - ); - final DelimitedParser delegate = new DelimitedParser( - Optional.fromNullable(Strings.emptyToNull(delimiter)), - Optional.fromNullable(Strings.emptyToNull(listDelimiter)) + super( + new DelimitedParser( + Optional.fromNullable(Strings.emptyToNull(delimiter)), + Optional.fromNullable(Strings.emptyToNull(listDelimiter)), + columns + ), + URIExtractionNamespace.getOrCreateKeyVauleMaps(maps, columns) ); - Preconditions.checkArgument( - !(Strings.isNullOrEmpty(keyColumn) ^ Strings.isNullOrEmpty(valueColumn)), - "Must specify both `keyColumn` and `valueColumn` or neither `keyColumn` nor `valueColumn`" - ); - delegate.setFieldNames(columns); + this.columns = columns; this.delimiter = delimiter; this.listDelimiter = listDelimiter; - this.keyColumn = Strings.isNullOrEmpty(keyColumn) ? columns.get(0) : keyColumn; - this.valueColumn = Strings.isNullOrEmpty(valueColumn) ? columns.get(1) : valueColumn; - Preconditions.checkArgument( - columns.contains(this.keyColumn), - "Column [%s] not found int columns: %s", - this.keyColumn, - Arrays.toString(columns.toArray()) - ); - Preconditions.checkArgument( - columns.contains(this.valueColumn), - "Column [%s] not found int columns: %s", - this.valueColumn, - Arrays.toString(columns.toArray()) - ); - - this.parser = new DelegateParser(delegate, this.keyColumn, this.valueColumn); } @JsonProperty @@ -416,18 +378,6 @@ public List getColumns() return columns; } - @JsonProperty - public String getKeyColumn() - { - return this.keyColumn; - } - - @JsonProperty - public String getValueColumn() - { - return this.valueColumn; - } - @JsonProperty public String getListDelimiter() { @@ -440,12 +390,6 @@ public String getDelimiter() return delimiter; } - @Override - public Parser getParser() - { - return parser; - } - @Override public boolean equals(Object o) { @@ -458,76 +402,51 @@ public boolean equals(Object o) TSVFlatDataParser that = (TSVFlatDataParser) o; - if (!getColumns().equals(that.getColumns())) { + if (!getMaps().containsAll(that.getMaps()) || !that.getMaps().containsAll(getMaps())) { return false; } - if ((getDelimiter() == null) ? that.getDelimiter() == null : getDelimiter().equals(that.getDelimiter())) { + + if (!getColumns().equals(that.getColumns())) { return false; } - if (!getKeyColumn().equals(that.getKeyColumn())) { + if ((getDelimiter() == null) ? that.getDelimiter() == null : getDelimiter().equals(that.getDelimiter())) { return false; } - - return getValueColumn().equals(that.getValueColumn()); + return true; } @Override public String toString() { return String.format( - "TSVFlatDataParser = { columns = %s, delimiter = '%s', listDelimiter = '%s',keyColumn = %s, valueColumn = %s }", + "TSVFlatDataParser = { " + + "columns = %s" + + ", delimiter = '%s'" + + ", listDelimiter = '%s'" + + ", maps = [" + StringUtils.join(maps, ',') + "]" + + "}", Arrays.toString(columns.toArray()), delimiter, - listDelimiter, - keyColumn, - valueColumn + listDelimiter ); } } @JsonTypeName("customJson") - public static class JSONFlatDataParser implements FlatDataParser + public static class JSONFlatDataParser extends MultiMapFlatDataParser { - private final Parser parser; - private final String keyFieldName; - private final String valueFieldName; - @JsonCreator public JSONFlatDataParser( @JacksonInject @Json ObjectMapper jsonMapper, - @JsonProperty("keyFieldName") final String keyFieldName, - @JsonProperty("valueFieldName") final String valueFieldName + @JsonProperty("maps") List maps ) { - Preconditions.checkArgument(!Strings.isNullOrEmpty(keyFieldName), "[keyFieldName] cannot be empty"); - Preconditions.checkArgument(!Strings.isNullOrEmpty(valueFieldName), "[valueFieldName] cannot be empty"); - this.keyFieldName = keyFieldName; - this.valueFieldName = valueFieldName; - this.parser = new DelegateParser( - new JSONParser(jsonMapper, ImmutableList.of(keyFieldName, valueFieldName)), - keyFieldName, - valueFieldName + super( + new JSONParser(jsonMapper, getNeededColumns(maps)), + maps ); } - @JsonProperty - public String getKeyFieldName() - { - return this.keyFieldName; - } - - @JsonProperty - public String getValueFieldName() - { - return this.valueFieldName; - } - - @Override - public Parser getParser() - { - return this.parser; - } - @Override public boolean equals(Object o) { @@ -538,72 +457,77 @@ public boolean equals(Object o) return false; } - JSONFlatDataParser that = (JSONFlatDataParser) o; + JSONFlatDataParser that = (JSONFlatDataParser)o; - if (!getKeyFieldName().equals(that.getKeyFieldName())) { - return false; - } - - return getValueFieldName().equals(that.getValueFieldName()); + return (getMaps().containsAll(that.getMaps()) && that.getMaps().containsAll(getMaps())); } @Override public String toString() { return String.format( - "JSONFlatDataParser = { keyFieldName = %s, valueFieldName = %s }", - keyFieldName, - valueFieldName + "JSONFlatDataParser = { " + + "maps = [" + StringUtils.join(maps, ',') + "]" + + "}" ); } + + private static Set getNeededColumns(List maps) + { + Set neededColumns = Sets.newHashSet(); + Preconditions.checkArgument(maps != null, "key/value map should be specified"); + for (KeyValueMap map: maps) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(map.getKeyColumn()), "key cannot be empty"); + Preconditions.checkArgument(!Strings.isNullOrEmpty(map.getValueColumn()), "value cannot be empty"); + neededColumns.add(map.getKeyColumn()); + neededColumns.add(map.getValueColumn()); + } + + return neededColumns; + } } @JsonTypeName("simpleJson") - public static class ObjectMapperFlatDataParser implements FlatDataParser + public static class ObjectMapperFlatDataParser extends FlatDataParser { - - private final Parser parser; + private final ObjectMapper jsonMapper; @JsonCreator public ObjectMapperFlatDataParser( final @JacksonInject @Json ObjectMapper jsonMapper ) { - parser = new Parser() - { - @Override - public Map parse(String input) - { - try { - return jsonMapper.readValue( - input, new TypeReference>() - { - } - ); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } + this.jsonMapper = jsonMapper; + } - @Override - public void setFieldNames(Iterable fieldNames) - { - throw new UOE("No field names available"); - } + @Override + public Map> parseInternal(String input) + { + Preconditions.checkArgument(id != null, "ID should be set before."); - @Override - public List getFieldNames() - { - throw new UOE("No field names available"); - } - }; + try { + Map kvMap = jsonMapper.readValue( + input, new TypeReference>() {} + ); + return ImmutableMap.of( + new Pair(id, KeyValueMap.DEFAULT_MAPNAME), + kvMap + ); + } catch (IOException e) { + throw Throwables.propagate(e); + } } @Override - public Parser getParser() + public void setFieldNames(Iterable fieldNames) + { + throw new UOE("No field names available"); + } + + @Override + public List getFieldNames() { - return parser; + throw new UOE("No field names available"); } @Override @@ -625,4 +549,51 @@ public String toString() return "ObjectMapperFlatDataParser = { }"; } } + + private static List getOrCreateKeyVauleMaps(List maps, List columns) + { + Preconditions.checkArgument( + Preconditions.checkNotNull(columns, "`columns` list required").size() > 1, + "Must specify more than one column to have a key value pair" + ); + + if (maps == null) { + return ImmutableList.of(new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, columns.get(0), columns.get(1))); + } + + Set neededColumns = FluentIterable + .from(maps) + .transformAndConcat( + new Function>() + { + @Override + public List apply(KeyValueMap input) + { + return ImmutableList.of(input.getKeyColumn(), input.getValueColumn()); + } + } + ) + .toSet(); + Preconditions.checkArgument( + columns.containsAll(neededColumns), + "columns should contains all the key/value columns specified in keyValueMaps: columns[%s], keyValueMaps[%s]", + StringUtils.join(columns, ","), StringUtils.join(maps, ",") + ); + + List defaultFilledMaps = Lists.newArrayListWithCapacity(maps.size()); + for (KeyValueMap map: maps) { + String key = map.getKeyColumn(); + String value = map.getValueColumn(); + if (Strings.isNullOrEmpty(key)) { + key = columns.get(0); + } + if (Strings.isNullOrEmpty(value)) { + value = columns.get(1); + } + + defaultFilledMaps.add(new KeyValueMap(map.getMapName(), key, value)); + } + + return defaultFilledMaps; + } } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java index 870e6adb4607..c16696ca79b4 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java @@ -19,11 +19,25 @@ package io.druid.server.lookup.namespace; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.querydsl.sql.DB2Templates; +import com.querydsl.sql.DerbyTemplates; +import com.querydsl.sql.HSQLDBTemplates; +import com.querydsl.sql.MySQLTemplates; +import com.querydsl.sql.OracleTemplates; +import com.querydsl.sql.PostgreSQLTemplates; +import com.querydsl.sql.SQLServerTemplates; +import com.querydsl.sql.SQLTemplates; import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.KeyValueMap; +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.commons.lang.StringUtils; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.StatementContext; @@ -34,6 +48,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Timestamp; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -43,7 +58,7 @@ * */ public class JDBCExtractionNamespaceCacheFactory - implements ExtractionNamespaceCacheFactory + extends ExtractionNamespaceCacheFactory { private static final Logger LOG = new Logger(JDBCExtractionNamespaceCacheFactory.class); private final ConcurrentMap dbiCache = new ConcurrentHashMap<>(); @@ -53,7 +68,8 @@ public String populateCache( final String id, final JDBCExtractionNamespace namespace, final String lastVersion, - final Map cache + final ConcurrentMap> cache, + final Function> mapAllocator ) throws Exception { final long lastCheck = lastVersion == null ? JodaUtils.MIN_INSTANT : Long.parseLong(lastVersion); @@ -64,46 +80,64 @@ public String populateCache( final long dbQueryStart = System.currentTimeMillis(); 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]", id); - final List> pairs = dbi.withHandle( - new HandleCallback>>() + final List requiredFields = KeyValueMap.getRequiredFields(namespace.getMaps()); + final Map fieldMap = new HashMap<>(); + final int numFields = requiredFields.size(); + for (int idx = 0; idx < numFields; idx++) + { + fieldMap.put(requiredFields.get(idx), idx); + } + + LOG.debug("Updating [%s]", namespace.toString()); + final List columns = dbi.withHandle( + new HandleCallback>() { @Override - public List> withHandle(Handle handle) throws Exception + public List withHandle(Handle handle) throws Exception { - final String query; - query = String.format( - "SELECT %s, %s FROM %s", - keyColumn, - valueColumn, - table - ); + final String query = makeQueryString(namespace.getConnectorConfig().getConnectURI(), requiredFields, table); return handle .createQuery( query ).map( - new ResultSetMapper>() + new ResultSetMapper() { @Override - public Pair map( + public String[] map( final int index, final ResultSet r, final StatementContext ctx ) throws SQLException { - return new Pair<>(r.getString(keyColumn), r.getString(valueColumn)); + String[] values = new String[numFields]; + int idx = 0; + + for (String field: requiredFields) + { + values[idx++] = r.getString(field); + } + return values; } } ).list(); } } ); - for (Pair pair : pairs) { - cache.put(pair.lhs, pair.rhs); + for (String[] values : columns) { + for (KeyValueMap keyValueMap: namespace.getMaps()) + { + String mapName = keyValueMap.getMapName(); + Pair key = new Pair(id, mapName); + Map innerMap = cache.get(key); + if (innerMap == null) + { + innerMap = mapAllocator.apply(key); + cache.put(key, innerMap); + } + innerMap.put(values[fieldMap.get(keyValueMap.getKeyColumn())], values[fieldMap.get(keyValueMap.getValueColumn())]); + } } LOG.info("Finished loading %d values for namespace[%s]", cache.size(), id); if (lastDBUpdate != null) { @@ -113,6 +147,69 @@ public Pair map( } } + private String makeQueryString(String url, List requiredFields, String table) + { + final SQLTemplates dialect = Preconditions.checkNotNull(getSQLTemplates(getDBType(url)), "unknown JDBC type: %s", url); + + List escapedFields = Lists.transform( + requiredFields, + new Function() + { + @Override + public String apply(String field) + { + return StringEscapeUtils.escapeJava(dialect.quoteIdentifier(dialect.escapeLiteral(field))); + } + } + ); + + String query = "SELECT "; + query += StringUtils.join(escapedFields, ", "); + query += " from " + StringEscapeUtils.escapeJava(dialect.quoteIdentifier(dialect.escapeLiteral(table))); + + return query; + } + + private String getDBType(String url) + { + String[] tokens = url.split(":"); + Preconditions.checkArgument((tokens.length > 2) && ("jdbc".equals(tokens[0])), "malformed JDBC url %s", url); + + return tokens[1]; + } + + private SQLTemplates getSQLTemplates(String type) + { + switch(type.toLowerCase()) { + case "db2": + return new DB2Templates(true); + + case "derby": + return new DerbyTemplates(true); + + case "hsqldb": + return new HSQLDBTemplates(true); + + case "jtds": + case "microsoft": + case "sqlserver": + return new SQLServerTemplates(true); + + case "mariadb": + case "mysql": + return new MySQLTemplates(true); + + case "oracle": + return new OracleTemplates(true); + + case "postgresql": + return new PostgreSQLTemplates(true); + } + LOG.warn("Unsupported DB type :%s - try MySQL", type); + // default - mysql template + return new MySQLTemplates(true); + } + private DBI ensureDBI(String id, JDBCExtractionNamespace namespace) { final String key = id; @@ -148,7 +245,7 @@ private Long lastUpdates(String id, JDBCExtractionNamespace namespace) public Timestamp withHandle(Handle handle) throws Exception { final String query = String.format( - "SELECT MAX(%s) FROM %s", + "SELECT MAX(\"%s\") FROM \"%s\"", tsColumn, table ); return handle diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java index dd4d5adc306d..598d50e09769 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java @@ -26,7 +26,7 @@ import java.util.UUID; public class StaticMapExtractionNamespaceCacheFactory - implements ExtractionNamespaceCacheFactory + extends ExtractionNamespaceCacheFactory { private final String version = UUID.randomUUID().toString(); diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java index a97fcff1da1a..99a83fd89a50 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java @@ -19,12 +19,14 @@ package io.druid.server.lookup.namespace; +import com.google.common.base.Function; import com.google.common.io.ByteSource; import com.google.inject.Inject; import io.druid.data.SearchableVersionedDataFinder; -import io.druid.data.input.MapPopulator; +import io.druid.data.input.MultiMapsPopulator; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.RetryUtils; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; @@ -38,12 +40,13 @@ import java.net.URI; import java.util.Map; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentMap; import java.util.regex.Pattern; /** * */ -public class URIExtractionNamespaceCacheFactory implements ExtractionNamespaceCacheFactory +public class URIExtractionNamespaceCacheFactory extends ExtractionNamespaceCacheFactory { private static final int DEFAULT_NUM_RETRIES = 3; private static final Logger log = new Logger(URIExtractionNamespaceCacheFactory.class); @@ -62,7 +65,8 @@ public String populateCache( final String id, final URIExtractionNamespace extractionNamespace, @Nullable final String lastVersion, - final Map cache + final ConcurrentMap> cache, + final Function> mapAllocator ) throws Exception { final boolean doSearch = extractionNamespace.getUriPrefix() != null; @@ -158,9 +162,9 @@ public InputStream openStream() throws IOException } }; } - final MapPopulator.PopulateResult populateResult = new MapPopulator<>( - extractionNamespace.getNamespaceParseSpec() - .getParser() + final MultiMapsPopulator.PopulateResult populateResult = new MultiMapsPopulator<>( + extractionNamespace.getParser().withID(id), + mapAllocator ).populate(source, cache); log.info( "Finished loading %,d values from %,d lines for namespace [%s]", diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java index ba50e846edeb..87ade2eccd3f 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java @@ -19,23 +19,27 @@ package io.druid.server.lookup.namespace.cache; +import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.Striped; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.concurrent.ExecutorServices; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import java.util.Collection; import java.util.Map; @@ -49,6 +53,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; /** * @@ -82,8 +87,22 @@ public NamespaceImplData( protected final ConcurrentMap implData = new ConcurrentHashMap<>(); protected final AtomicLong tasksStarted = new AtomicLong(0); protected final ServiceEmitter serviceEmitter; + private final ConcurrentHashMap lastVersion = new ConcurrentHashMap<>(); + private final ConcurrentMap>> mapMap = new ConcurrentHashMap<>(); + private final Striped nsLocks = Striped.lock(32); private final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap; + public static Function> getMapAllocator(final NamespaceExtractionCacheManager manager) + { + return new Function>() { + @Nullable + @Override + public Map apply(@Nullable Pair key) { + return manager.getOrAllocateInnerCacheMap(key); + } + }; + } + public NamespaceExtractionCacheManager( Lifecycle lifecycle, final ServiceEmitter serviceEmitter, @@ -353,11 +372,12 @@ public void run() // should never happen throw new NullPointerException(String.format("No data for namespace [%s]", id)); } - final Map cache = getCacheMap(cacheId); + final ConcurrentMap> cache = getCacheMap(cacheId); final String preVersion = implData.latestVersion; tasksStarted.incrementAndGet(); - final String newVersion = factory.populateCache(id, namespace, preVersion, cache); + final String newVersion = factory.populateCache(id, namespace, preVersion, cache, + getMapAllocator(NamespaceExtractionCacheManager.this)); if (newVersion.equals(preVersion)) { log.debug("Version `%s` already exists, skipping updating cache", preVersion); } else { @@ -416,26 +436,90 @@ public void run() } /** - * This method is expected to swap the cacheKey into the active namespace, and leave future requests for new cacheKey available. getCacheMap(cacheKey) should return empty data after this call. + * This method swaps the cacheKey into the active namespace, and leaves future requests for new cacheKey available. getInnerCacheMap(cacheKey) should return empty data after this call. * * @param namespaceKey The namespace to swap the cache into * @param cacheKey The cacheKey that contains the data of interest * * @return true if old data was cleared. False if no old data was found */ - protected abstract boolean swapAndClearCache(String namespaceKey, String cacheKey); + protected boolean swapAndClearCache(String namespaceKey, String cacheKey) + { + final Lock lock = nsLocks.get(namespaceKey); + lock.lock(); + try { + ConcurrentMap> cacheMap = mapMap.get(cacheKey); + if (cacheMap == null) { + throw new IAE("Extraction Cache [%s] does not exist", cacheKey); + } + ConcurrentMap> prior = mapMap.put(namespaceKey, cacheMap); + mapMap.remove(cacheKey); + if (prior != null) { + // Old map will get GC'd when it is not used anymore + return true; + } else { + return false; + } + } + finally { + lock.unlock(); + } + } /** * Return a ConcurrentMap with the specified ID (either namespace's name or a cache key ID) + * It contains key-value map for all names of maps + * Create empty map if no matching map exists * * @param namespaceOrCacheKey Either a namespace or cache key should be acceptable here. * - * @return A ConcurrentMap that is backed by the impl which implements this method. + * @return A ConcurrentMap> + */ + public ConcurrentMap> getCacheMap(String namespaceOrCacheKey) + { + ConcurrentMap> map = mapMap.get(namespaceOrCacheKey); + + if (map == null) { + mapMap.putIfAbsent(namespaceOrCacheKey, new ConcurrentHashMap>()); + map = mapMap.get(namespaceOrCacheKey); + } + return map; + } + + /** + * Return a ConcurrentMap with the specified ID and mapName + * Create empty map if no matching mapName exists + * + * @param id A namespace key should be acceptable here. + * + * @param mapName A name of map in a given namespace. + * + * @return A Map */ - public abstract ConcurrentMap getCacheMap(String namespaceOrCacheKey); + public Map getInnerCacheMap(String id, String mapName) + { + ConcurrentMap> maps = getCacheMap(id); + + // Cannot use mapName as a key for inner cache lookup + // due to mapName collision when multiple namespace use the same mapName + Pair key = new Pair(id, mapName); + Map map = maps.get(key); + if (map == null) + { + maps.putIfAbsent(key, getOrAllocateInnerCacheMap(key)); + map = maps.get(key); + } + + return map; + } /** - * Clears out resources used by the namespace such as threads. Implementations may override this and call super.delete(...) if they have resources of their own which need cleared. + * Clear out resources used by the namespace such as threads. + * + * Cached key-value map is two-level, outer map is namespace-level(onheap) and inner map is mapName-level(onheap or offheap). + * First, this method deletes all the inner maps of the given namespace + * with {@link #deleteInnerCacheMaps(ConcurrentMap) deleteInnerCacheMaps()} which is implemented differently for each cache type. + * Then, it deletes namespace-level map entry of the given namespace. * * @param ns The namespace to be deleted * @@ -453,7 +537,20 @@ public boolean delete(final String ns) synchronized (implDatum.changeLock) { if (removeNamespaceLocalMetadata(implDatum)) { log.info("Deleted namespace [%s]", ns); - return true; + final Lock lock = nsLocks.get(ns); + lock.lock(); + try { + ConcurrentMap> map = mapMap.get(ns); + if (map != null) { + deleteInnerCacheMaps(map); + mapMap.remove(ns); + return true; + } + return false; + } + finally { + lock.unlock(); + } } else { log.debug("Did not delete namespace [%s]", ns); return false; @@ -461,6 +558,24 @@ public boolean delete(final String ns) } } + /** + * Delete all the inner maps associated with the given outer map + * + * @param map outer map that contains all the inner maps to delete + * @return True if success, false if fail + */ + protected abstract boolean deleteInnerCacheMaps(final ConcurrentMap> map); + + /** + * Get the map associated with the given key + * + * Implementation of this method should return non-null Map. + * + * @param key key represented by Pair(namespace, mapName) + * @return A ConcurrentMap + */ + public abstract ConcurrentMap getOrAllocateInnerCacheMap(Pair key); + public String getVersion(String namespace) { if (namespace == null) { diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java index a51e4b837462..ff74fbc8c3e0 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java @@ -19,13 +19,12 @@ package io.druid.server.lookup.namespace.cache; -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.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; - +import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.namespace.ExtractionNamespace; @@ -36,8 +35,6 @@ import java.io.File; import java.io.IOException; import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.Lock; @@ -48,7 +45,6 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC { private static final Logger log = new Logger(OffHeapNamespaceExtractionCacheManager.class); private final DB mmapDB; - private ConcurrentMap currentNamespaceCache = new ConcurrentHashMap<>(); private Striped nsLocks = Striped.lazyWeakLock(1024); // Needed to make sure delete() doesn't do weird things private final File tmpFile; @@ -107,48 +103,32 @@ public synchronized void stop() } @Override - protected boolean swapAndClearCache(String namespaceKey, String cacheKey) + protected boolean deleteInnerCacheMaps(final ConcurrentMap> map) { - final Lock lock = nsLocks.get(namespaceKey); - lock.lock(); - try { - Preconditions.checkArgument(mmapDB.exists(cacheKey), "Namespace [%s] does not exist", cacheKey); - - final String swapCacheKey = UUID.randomUUID().toString(); - mmapDB.rename(cacheKey, swapCacheKey); - - final String priorCache = currentNamespaceCache.put(namespaceKey, swapCacheKey); - if (priorCache != null) { - // TODO: resolve what happens here if query is actively going on - mmapDB.delete(priorCache); - return true; - } else { - return false; + if (map != null) { + for (Pair key: map.keySet()) { + log.debug("deleting map[%s] of namespace[%s]", key.rhs, key.lhs); + deleteSingleMap(key.toString()); } - } - finally { - lock.unlock(); + return true; + } else { + return false; } } - @Override - public boolean delete(final String namespaceKey) + private boolean deleteSingleMap(final String mapKey) { - // `super.delete` has a synchronization in it, don't call it in the lock. - if (!super.delete(namespaceKey)) { - return false; - } - final Lock lock = nsLocks.get(namespaceKey); + final Lock lock = nsLocks.get(mapKey); lock.lock(); try { - final String mmapDBkey = currentNamespaceCache.remove(namespaceKey); - if (mmapDBkey == null) { + if (mapKey != null) { + final long pre = tmpFile.length(); + mmapDB.delete(mapKey); + log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length()); + return true; + } else { return false; } - final long pre = tmpFile.length(); - mmapDB.delete(mmapDBkey); - log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length()); - return true; } finally { lock.unlock(); @@ -156,17 +136,12 @@ public boolean delete(final String namespaceKey) } @Override - public ConcurrentMap getCacheMap(String namespaceKey) + public ConcurrentMap getOrAllocateInnerCacheMap(Pair key) { - final Lock lock = nsLocks.get(namespaceKey); + final Lock lock = nsLocks.get(key); lock.lock(); try { - String mapDBKey = currentNamespaceCache.get(namespaceKey); - if (mapDBKey == null) { - // Not something created by swapAndClearCache - mapDBKey = namespaceKey; - } - return mmapDB.createHashMap(mapDBKey).makeOrGet(); + return mmapDB.createHashMap(key.toString()).makeOrGet(); } finally { lock.unlock(); diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java index 070d79087c1d..3f02f5f47601 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java @@ -20,12 +20,10 @@ package io.druid.server.lookup.namespace.cache; import com.google.common.primitives.Chars; -import com.google.common.util.concurrent.Striped; import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; - -import io.druid.java.util.common.IAE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.namespace.ExtractionNamespace; @@ -34,7 +32,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.locks.Lock; /** * @@ -42,8 +39,7 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCacheManager { private static final Logger LOG = new Logger(OnHeapNamespaceExtractionCacheManager.class); - private final ConcurrentMap> mapMap = new ConcurrentHashMap<>(); - private final Striped nsLocks = Striped.lock(32); + private final ConcurrentMap> mapMap = new ConcurrentHashMap<>(); @Inject public OnHeapNamespaceExtractionCacheManager( @@ -56,55 +52,22 @@ public OnHeapNamespaceExtractionCacheManager( } @Override - protected boolean swapAndClearCache(String namespaceKey, String cacheKey) - { - final Lock lock = nsLocks.get(namespaceKey); - lock.lock(); - try { - ConcurrentMap cacheMap = mapMap.get(cacheKey); - if (cacheMap == null) { - throw new IAE("Extraction Cache [%s] does not exist", cacheKey); - } - ConcurrentMap prior = mapMap.put(namespaceKey, cacheMap); - mapMap.remove(cacheKey); - if (prior != null) { - // Old map will get GC'd when it is not used anymore - return true; - } else { - return false; - } - } - finally { - lock.unlock(); - } - } - - @Override - public ConcurrentMap getCacheMap(String namespaceOrCacheKey) + public ConcurrentMap getOrAllocateInnerCacheMap(Pair key) { - ConcurrentMap map = mapMap.get(namespaceOrCacheKey); + ConcurrentMap map = mapMap.get(key); if (map == null) { - mapMap.putIfAbsent(namespaceOrCacheKey, new ConcurrentHashMap()); - map = mapMap.get(namespaceOrCacheKey); + mapMap.putIfAbsent(key, new ConcurrentHashMap()); + map = mapMap.get(key); } return map; } @Override - public boolean delete(final String namespaceKey) + protected boolean deleteInnerCacheMaps(final ConcurrentMap> map) { - // `super.delete` has a synchronization in it, don't call it in the lock. - if (!super.delete(namespaceKey)) { - return false; - } - final Lock lock = nsLocks.get(namespaceKey); - lock.lock(); - try { - return mapMap.remove(namespaceKey) != null; - } - finally { - lock.unlock(); - } + // do nothing because references to maps will be removed by NamesapceExtractionCacheManager after the return of this method + // and GC will clean maps eventually + return true; } @Override @@ -112,10 +75,11 @@ protected void monitor(ServiceEmitter serviceEmitter) { long numEntries = 0; long size = 0; - for (Map.Entry> entry : mapMap.entrySet()) { + for (Map.Entry> entry : mapMap.entrySet()) { final ConcurrentMap map = entry.getValue(); + final Pair cacheKey = entry.getKey(); if (map == null) { - LOG.debug("missing cache key for reporting [%s]", entry.getKey()); + LOG.debug("missing cache key for reporting [%s of %s]", cacheKey.rhs, cacheKey.lhs); continue; } numEntries += map.size(); @@ -123,7 +87,7 @@ protected void monitor(ServiceEmitter serviceEmitter) 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()); + LOG.debug("Missing entries for cache key [%s of %s]", cacheKey.rhs, cacheKey.lhs); continue; } size += key.length() + value.length(); diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java index 76005722c487..1e8580383817 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java @@ -27,11 +27,11 @@ 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.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.Json; @@ -53,9 +53,11 @@ import org.junit.rules.TemporaryFolder; import javax.ws.rs.core.Response; +import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; public class NamespaceLookupExtractorFactoryTest { @@ -94,7 +96,6 @@ public void testSimpleSerde() throws Exception temporaryFolder.newFolder().toURI(), null, null, new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper), - Period.millis(0), null ); @@ -282,8 +283,15 @@ public long getPollMs() EasyMock.eq(60000L) )).andReturn(true).once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); + final ConcurrentMap map = new ConcurrentHashMap>() { + @Override + public ConcurrentMap get(Object key) + { + return new ConcurrentHashMap(); + } + }; EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) - .andReturn(new ConcurrentHashMap()) + .andReturn(map) .once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); EasyMock.expect( @@ -320,9 +328,16 @@ public long getPollMs() EasyMock.eq(60000L) )).andReturn(true).once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); + final ConcurrentMap map = new ConcurrentHashMap>() { + @Override + public ConcurrentMap get(Object key) + { + return new ConcurrentHashMap(); + } + }; EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) - .andReturn(new ConcurrentHashMap()) - .once(); + .andReturn(map) + .once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn(null).once(); EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once(); EasyMock.replay(cacheManager); @@ -361,15 +376,29 @@ public long getPollMs() EasyMock.eq(60000L) )).andReturn(true).once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("0").once(); + final ConcurrentMap map = new ConcurrentHashMap>() { + @Override + public ConcurrentMap get(Object key) + { + return new ConcurrentHashMap(ImmutableMap.of("foo", "bar")); + } + }; EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) - .andReturn(new ConcurrentHashMap(ImmutableMap.of("foo", "bar"))) - .once(); + .andReturn(map) + .once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("1").once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("2").once(); + final ConcurrentMap map2 = new ConcurrentHashMap>() { + @Override + public ConcurrentMap get(Object key) + { + return new ConcurrentHashMap(); + } + }; EasyMock.expect(cacheManager.getCacheMap(EasyMock.anyString())) - .andReturn(new ConcurrentHashMap()) - .once(); + .andReturn(map2) + .once(); EasyMock.expect(cacheManager.getVersion(EasyMock.anyString())).andReturn("2").once(); EasyMock.expect(cacheManager.checkedDelete(EasyMock.anyString())).andReturn(true).once(); EasyMock.replay(cacheManager); @@ -514,7 +543,7 @@ public void configure(Binder binder) ); 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 String str = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"maps\":[{\"mapName\":\"__default\",\"keyColumn\":\"someKey\", \"valueColumn\":\"someVal\"}]}, \"pollPeriod\": \"PT5M\" } } }"; final LookupExtractorFactory factory = mapper.readValue(str, LookupExtractorFactory.class); Assert.assertTrue(factory instanceof NamespaceLookupExtractorFactory); final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = (NamespaceLookupExtractorFactory) factory; @@ -566,14 +595,14 @@ public void configure(Binder binder) Assert.assertNotNull(handler); final Class clazz = handler.getClass(); Assert.assertNotNull(clazz.getMethod("getVersion").invoke(handler)); - Assert.assertEquals(ImmutableSet.of("foo"), ((Response) clazz.getMethod("getKeys").invoke(handler)).getEntity()); + Assert.assertEquals(ImmutableSet.of("foo"), ((Response) clazz.getMethod("getDefaultKeys").invoke(handler)).getEntity()); Assert.assertEquals( - ImmutableSet.of("bar"), - ((Response) clazz.getMethod("getValues").invoke(handler)).getEntity() + ImmutableList.of("bar"), + Lists.newArrayList((Collection) ((Response) clazz.getMethod("getDefaultValues").invoke(handler)).getEntity()) ); Assert.assertEquals( - ImmutableMap.builder().put("foo", "bar").build(), - ((Response) clazz.getMethod("getMap").invoke(handler)).getEntity() + ImmutableList.of(ImmutableMap.of("foo", "bar")), + Lists.newArrayList((Map>) ((Response) clazz.getMethod("getDefault").invoke(handler)).getEntity()) ); } finally { @@ -613,7 +642,7 @@ public void testExceptionalIntrospectionHandler() throws Exception validateCode( new ISE("some exception"), 404, - "getKeys", + "getDefaultKeys", handler, manager, clazz @@ -622,7 +651,7 @@ public void testExceptionalIntrospectionHandler() throws Exception validateCode( new ISE("some exception"), 404, - "getValues", + "getDefaultValues", handler, manager, clazz @@ -631,7 +660,7 @@ public void testExceptionalIntrospectionHandler() throws Exception validateCode( new ISE("some exception"), 404, - "getMap", + "getDefault", handler, manager, clazz diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupReferenceManagerTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupReferenceManagerTest.java new file mode 100644 index 000000000000..b6d46cb7c4c0 --- /dev/null +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupReferenceManagerTest.java @@ -0,0 +1,279 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.lookup; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.io.Files; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.guice.GuiceInjectors; +import io.druid.initialization.Initialization; +import io.druid.metadata.MetadataStorageConnectorConfig; +import io.druid.metadata.TestDerbyConnector; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.dimension.LookupDimensionSpec; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.lookup.namespace.ExtractionNamespace; +import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.KeyValueMap; +import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.server.lookup.namespace.NamespaceExtractionModule; +import org.apache.commons.lang.StringUtils; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.skife.jdbi.v2.Handle; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.OutputStreamWriter; + +public class NamespaceLookupReferenceManagerTest +{ + LookupReferencesManager lookupReferencesManager; + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + Injector injector; + ObjectMapper mapper; + + @Before + public void setUp() throws IOException + { + System.setProperty("druid.extensions.searchCurrentClassloader", "false"); + + injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of() + ), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + } + }, + new NamespaceExtractionModule() + ) + ); + mapper = injector.getInstance(ObjectMapper.class); + + lookupReferencesManager = new LookupReferencesManager(new LookupConfig(Files.createTempDir().getAbsolutePath()), mapper); + Assert.assertTrue("must be closed before start call", lookupReferencesManager.isClosed()); + lookupReferencesManager.start(); + Assert.assertFalse("must start after start call", lookupReferencesManager.isClosed()); + + } + + @After + public void tearDown() + { + lookupReferencesManager.stop(); + Assert.assertTrue("stop call should close it", lookupReferencesManager.isClosed()); + System.setProperty("druid.extensions.searchCurrentClassloader", "true"); + } + + @Test + public void registerUriNamespaceAndLookupTest() throws IOException + { + final File tmpFile = temporaryFolder.newFile(); + try (OutputStreamWriter out = new FileWriter(tmpFile)) { + out.write(StringUtils.join(new String[] {"foo", "bar", "baz"}, ',')); + out.write("\n"); + out.write(StringUtils.join(new String[] {"work", "bad", "good"}, ',')); + } + ExtractionNamespace uriExtractionNamespace = new URIExtractionNamespace( + tmpFile.toURI(), + null, null, + new URIExtractionNamespace.CSVFlatDataParser( + ImmutableList.of("key", "val1", "val2"), + ImmutableList.of ( + new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, "key", "val1"), + new KeyValueMap("another", "key", "val2") + ) + ), + new Period(0), + null + ); + String uriNamespaceJson = mapper.writeValueAsString(uriExtractionNamespace); + String lookupExtractorFactoryJson = "{\"type\":\"cachedNamespace\",\"extractionNamespace\":"+ uriNamespaceJson +", \"firstCacheTimeout\":1000, \"injective\":false}"; + NamespaceLookupExtractorFactory lookupExtractorFactory = + mapper.readValue(lookupExtractorFactoryJson, NamespaceLookupExtractorFactory.class); + Assert.assertEquals(uriExtractionNamespace, lookupExtractorFactory.getExtractionNamespace()); + + lookupReferencesManager.put("test", lookupExtractorFactory); + LookupExtractor lookupExtractor1 = lookupReferencesManager.get("test").get(KeyValueMap.DEFAULT_MAPNAME); + Assert.assertEquals("bar", lookupExtractor1.apply("foo")); + Assert.assertEquals("bad", lookupExtractor1.apply("work")); + LookupExtractor lookupExtractor2 = lookupReferencesManager.get("test").get("another"); + Assert.assertEquals("baz", lookupExtractor2.apply("foo")); + Assert.assertEquals("good", lookupExtractor2.apply("work")); + // when no map name is specified, KeyValueMap.DEFAULT_MAPNAME is used as default map name + // however, this usage is not recommended + LookupExtractor lookupExtractor3 = lookupReferencesManager.get("test").get(); + Assert.assertEquals(lookupExtractor1, lookupExtractor3); + + DimensionSpec dimensionSpec = new LookupDimensionSpec( + "dim", + "out", + null, + false, + null, + "test", + "another", + lookupReferencesManager, + false + ); + + ExtractionFn fn = dimensionSpec.getExtractionFn(); + Assert.assertEquals("baz", fn.apply("foo")); + Assert.assertEquals("good", fn.apply("work")); + + DimensionSpec defaultDimSpec = new LookupDimensionSpec( + "dim", + "out", + null, + false, + null, + "test", + null, + lookupReferencesManager, + false + ); + + ExtractionFn fn2 = defaultDimSpec.getExtractionFn(); + Assert.assertEquals("bar", fn2.apply("foo")); + Assert.assertEquals("bad", fn2.apply("work")); + } + + @Test + public void registerJDBCNamespaceAndLookupTest() throws IOException + { + String name = "jdbcTest"; + + createTable(name); + + MetadataStorageConnectorConfig connectorConfig = mapper.readValue( + String.format( + "{\"connectURI\":\"%s\"}", + derbyConnectorRule.getMetadataConnectorConfig().getConnectURI() + ), + MetadataStorageConnectorConfig.class + ); + ExtractionNamespace jdbcExtractionNamespace = new JDBCExtractionNamespace( + connectorConfig, + name, + null, + null, + ImmutableList.of( + new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, "lookupKey", "val1"), + new KeyValueMap("another", "lookupKey", "val2") + ), + null, + new Period(0) + ); + String jdbcNamespaceJson = mapper.writeValueAsString(jdbcExtractionNamespace); + String lookupExtractorFactoryJson = "{\"type\":\"cachedNamespace\",\"extractionNamespace\":"+ jdbcNamespaceJson +", \"firstCacheTimeout\":100000, \"injective\":false}"; + NamespaceLookupExtractorFactory lookupExtractorFactory = + mapper.readValue(lookupExtractorFactoryJson, NamespaceLookupExtractorFactory.class); + + lookupReferencesManager.put(name, lookupExtractorFactory); + LookupExtractor lookupExtractor1 = lookupReferencesManager.get(name).get(KeyValueMap.DEFAULT_MAPNAME); + Assert.assertEquals("bar", lookupExtractor1.apply("foo")); + Assert.assertEquals("bad", lookupExtractor1.apply("work")); + LookupExtractor lookupExtractor2 = lookupReferencesManager.get(name).get("another"); + Assert.assertEquals("baz", lookupExtractor2.apply("foo")); + Assert.assertEquals("good", lookupExtractor2.apply("work")); + // when no map name is specified, KeyValueMap.DEFAULT_MAPNAME is used as default map name + // however, this usage is not recommended + LookupExtractor lookupExtractor3 = lookupReferencesManager.get(name).get(); + Assert.assertEquals(lookupExtractor1, lookupExtractor3); + + DimensionSpec dimensionSpec = new LookupDimensionSpec( + "dim", + "out", + null, + false, + null, + name, + "another", + lookupReferencesManager, + false + ); + + ExtractionFn fn = dimensionSpec.getExtractionFn(); + Assert.assertEquals("baz", fn.apply("foo")); + Assert.assertEquals("good", fn.apply("work")); + + DimensionSpec defaultDimSpec = new LookupDimensionSpec( + "dim", + "out", + null, + false, + null, + name, + null, + lookupReferencesManager, + false + ); + + ExtractionFn fn2 = defaultDimSpec.getExtractionFn(); + Assert.assertEquals("bar", fn2.apply("foo")); + Assert.assertEquals("bad", fn2.apply("work")); + + dropTable(name); + } + + private void createTable(String name) + { + Handle handle = derbyConnectorRule.getConnector().getDBI().open(); + Assert.assertEquals( + 0, + handle.createStatement( + String.format("CREATE TABLE \"%s\" (\"lookupKey\" VARCHAR(10), \"val1\" VARCHAR(10), \"val2\" VARCHAR(10))", name) + ).setQueryTimeout(1).execute() + ); + handle.commit(); + handle.createStatement(String.format("TRUNCATE TABLE \"%s\"", name)).setQueryTimeout(1).execute(); + handle.createStatement(String.format("INSERT INTO \"%s\" (\"lookupKey\", \"val1\", \"val2\") VALUES ('foo', 'bar', 'baz')", name)).setQueryTimeout(1).execute(); + handle.createStatement(String.format("INSERT INTO \"%s\" (\"lookupKey\", \"val1\", \"val2\") VALUES ('work', 'bad', 'good')", name)).setQueryTimeout(1).execute(); + handle.close(); + } + + private void dropTable(String name) + { + Handle handle = derbyConnectorRule.getConnector().getDBI().open(); + handle.createStatement(String.format("DROP TABLE \"%s\"", name)).setQueryTimeout(1).execute(); + handle.close(); + } +} diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java index d26ec74cbea9..cb4c68b8c24f 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java @@ -27,10 +27,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import com.google.common.io.CharSink; import com.google.common.io.Files; -import io.druid.data.input.MapPopulator; +import io.druid.data.input.MultiMapsPopulator; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Pair; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -39,9 +41,10 @@ import org.junit.rules.TemporaryFolder; import java.io.File; -import java.util.HashMap; +import java.net.URI; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentMap; public class JSONFlatDataParserTest { @@ -92,58 +95,108 @@ public CharSequence apply(Map input) @Test public void testSimpleParse() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( - MAPPER, - "key", - "val" + final URIExtractionNamespace extractionNamespace = new URIExtractionNamespace( + new URI("dummy"), + null, + null, + new URIExtractionNamespace.JSONFlatDataParser( + MAPPER, + ImmutableList.of(new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, "key", "val")) + ), + null, + null ); - final Map map = new HashMap<>(); - new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map); - Assert.assertEquals(VAL1, map.get(KEY1)); - Assert.assertEquals(VAL2, map.get(KEY2)); + final ConcurrentMap> map = populateMap(extractionNamespace, "test"); + final Map resultMap = map.get(new Pair("test", KeyValueMap.DEFAULT_MAPNAME)); + Assert.assertEquals(VAL1, resultMap.get(KEY1)); + Assert.assertEquals(VAL2, resultMap.get(KEY2)); } @Test public void testParseWithNullValues() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( - MAPPER, - "key", - "otherVal" + final URIExtractionNamespace extractionNamespace = new URIExtractionNamespace( + new URI("dummy"), + null, + null, + new URIExtractionNamespace.JSONFlatDataParser( + MAPPER, + ImmutableList.of(new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, "key", "otherVal")) + ), + null, + null ); - final Map map = new HashMap<>(); - new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map); - Assert.assertEquals(OTHERVAL1, map.get(KEY1)); - Assert.assertEquals(OTHERVAL2, map.get(KEY2)); + final ConcurrentMap> map = populateMap(extractionNamespace, "test2"); + final Map resultMap = map.get(new Pair("test2", KeyValueMap.DEFAULT_MAPNAME)); + Assert.assertEquals(OTHERVAL1, resultMap.get(KEY1)); + Assert.assertEquals(OTHERVAL2, resultMap.get(KEY2)); } @Test public void testParseWithEmptyValues() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( - MAPPER, - "key", - "canBeEmpty" + final URIExtractionNamespace extractionNamespace = new URIExtractionNamespace( + new URI("dummy"), + null, + null, + new URIExtractionNamespace.JSONFlatDataParser( + MAPPER, + ImmutableList.of(new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, "key", "canBeEmpty")) + ), + null, + null ); - final Map map = new HashMap<>(); - new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map); - Assert.assertEquals(CANBEEMPTY1, map.get(KEY1)); - Assert.assertEquals(CANBEEMPTY2, map.get(KEY2)); + final ConcurrentMap> map = populateMap(extractionNamespace, "test3"); + final Map resultMap = map.get(new Pair("test3", KeyValueMap.DEFAULT_MAPNAME)); + Assert.assertEquals(CANBEEMPTY1, resultMap.get(KEY1)); + Assert.assertEquals(CANBEEMPTY2, resultMap.get(KEY2)); } @Test public void testFailParseOnKeyMissing() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( - MAPPER, - "keyWHOOPS", - "val" + final URIExtractionNamespace extractionNamespace = new URIExtractionNamespace( + new URI("dummy"), + null, + null, + new URIExtractionNamespace.JSONFlatDataParser( + MAPPER, + ImmutableList.of(new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, "keyWHOOPS", "val")) + ), + null, + null ); - final Map map = new HashMap<>(); - expectedException.expect(NullPointerException.class); expectedException.expectMessage("Key column [keyWHOOPS] missing data in line"); - new MapPopulator<>(parser.getParser()).populate(Files.asByteSource(tmpFile), map); + final ConcurrentMap> map = populateMap(extractionNamespace, "test4"); + final Map resultMap = map.get(new Pair("test4", KeyValueMap.DEFAULT_MAPNAME)); + Assert.assertEquals(null, resultMap.get(KEY1)); + } + + private ConcurrentMap> populateMap( + URIExtractionNamespace extractionNamespace, + String id + ) throws Exception + { + final ConcurrentMap> map = Maps.newConcurrentMap(); + + new MultiMapsPopulator<>(extractionNamespace.getParser().withID(id), + new Function>() + { + @Override + public Map apply(Pair key) + { + Map inner = map.get(key); + if (inner == null) { + map.putIfAbsent(key, Maps.newHashMap()); + inner = map.get(key); + } + return inner; + } + }) + .populate(Files.asByteSource(tmpFile), map); + + return map; } } diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java index 6d620b73e0b1..ece1e974c58c 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java @@ -31,10 +31,12 @@ import io.druid.guice.GuiceInjectableValues; import io.druid.guice.annotations.Json; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Pair; import org.junit.Assert; import org.junit.Test; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -63,6 +65,7 @@ public void configure(Binder binder) ) ) ).registerSubtypes(URIExtractionNamespace.class, URIExtractionNamespace.FlatDataParser.class); + mapper.registerSubtypes(URIExtractionNamespace.class, URIExtractionNamespace.CSVFlatDataParser.class); final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); mapper.setAnnotationIntrospectors( @@ -76,94 +79,100 @@ public void configure(Binder binder) return mapper; } + final String id = "test"; + final String mapName = "testMap"; + @Test public void testCSV() { - URIExtractionNamespace.CSVFlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", "col3" - ), "col2", "col3" - ); - Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("A,B,C")); + ), + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id); + Assert.assertEquals(ImmutableMap.of("B", "C"), parser.parse("A,B,C").get(new Pair(id, mapName))); } @Test(expected = IllegalArgumentException.class) public void testBadCSV() { - URIExtractionNamespace.CSVFlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", "col3" - ), "col2", "col3ADFSDF" - ); - Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("A,B,C")); + ), + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3ADFSDF")) + ).withID(id); + Assert.assertEquals(ImmutableMap.of("B", "C"), parser.parse("A,B,C").get(new Pair(id, mapName))); } @Test(expected = NullPointerException.class) public void testBadCSV2() { - URIExtractionNamespace.CSVFlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", "col3" - ), "col2", "col3" - ); - Map map = parser.getParser().parse("A"); + ), + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id); + Assert.assertEquals(ImmutableMap.of("B", "C"), parser.parse("A").get(new Pair(id, mapName))); } @Test public void testTSV() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3"), "|", - null, "col2", - "col3" - ); - Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("A|B|C")); + null, + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id); + Assert.assertEquals(ImmutableMap.of("B", "C"), parser.parse("A|B|C").get(new Pair(id, mapName))); } @Test public void testWithListDelimiterTSV() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3"), "\\u0001", - "\\u0002", "col2", - "col3" - ); - Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("A\\u0001B\\u0001C")); + "\\u0002", + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id); + Assert.assertEquals(ImmutableMap.of("B", "C"), parser.parse("A\\u0001B\\u0001C").get(new Pair(id, mapName))); } + @Test(expected = IllegalArgumentException.class) public void testBadTSV() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3fdsfds"), ",", - null, "col2", - "col3" - ); - Map map = parser.getParser().parse("A,B,C"); - Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("A,B,C")); + null, + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id); + Map map = parser.parse("A,B,C").get(new Pair(id, mapName)); + Assert.assertEquals(ImmutableMap.of("B", "C"), parser.parse("A,B,C").get(new Pair(id, mapName))); } - @Test(expected = NullPointerException.class) public void testBadTSV2() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3"), ",", - null, "col2", - "col3" - ); - Map map = parser.getParser().parse("A"); - Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("A,B,C")); + null, + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id); + Map map = parser.parse("A").get(new Pair(id, mapName)); + Assert.assertEquals(ImmutableMap.of("B", "C"), parser.parse("A,B,C").get(new Pair(id, mapName))); } @Test @@ -171,45 +180,40 @@ public void testJSONFlatDataParser() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( new ObjectMapper(), - keyField, - valueField - ); + ImmutableList.of(new KeyValueMap(mapName, keyField, valueField)) + ).withID(id); Assert.assertEquals( ImmutableMap.of("B", "C"), - parser.getParser() - .parse( - String.format( - "{\"%s\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", - keyField, - valueField - ) - ) + parser.parse( + String.format( + "{\"%s\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", + keyField, + valueField + ) + ).get(new Pair(id, mapName)) ); } - @Test(expected = NullPointerException.class) public void testJSONFlatDataParserBad() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( new ObjectMapper(), - keyField, - valueField - ); + ImmutableList.of(new KeyValueMap(mapName, keyField, valueField)) + ).withID(id); Assert.assertEquals( ImmutableMap.of("B", "C"), - parser.getParser() - .parse( - String.format( - "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", - keyField, - valueField - ) - ) + parser.parse( + String.format( + "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", + keyField, + valueField + ) + ).get(new Pair(id, mapName)) ); } @@ -218,21 +222,19 @@ public void testJSONFlatDataParserBad2() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( - registerTypes(new ObjectMapper()), - null, - valueField - ); + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + new ObjectMapper(), + ImmutableList.of(new KeyValueMap(mapName, null, valueField)) + ).withID(id); Assert.assertEquals( ImmutableMap.of("B", "C"), - parser.getParser() - .parse( - String.format( - "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", - keyField, - valueField - ) - ) + parser.parse( + String.format( + "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", + keyField, + valueField + ) + ).get(new Pair(id, mapName)) ); } @@ -241,21 +243,19 @@ public void testJSONFlatDataParserBad3() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( - registerTypes(new ObjectMapper()), - keyField, - null - ); + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + new ObjectMapper(), + ImmutableList.of(new KeyValueMap(mapName, keyField, null)) + ).withID(id); Assert.assertEquals( ImmutableMap.of("B", "C"), - parser.getParser() - .parse( - String.format( - "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", - keyField, - valueField - ) - ) + parser.parse( + String.format( + "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", + keyField, + valueField + ) + ).get(new Pair(id, mapName)) ); } @@ -264,31 +264,32 @@ public void testJSONFlatDataParserBad4() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( - registerTypes(new ObjectMapper()), - "", - "" - ); + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + new ObjectMapper(), + ImmutableList.of(new KeyValueMap(mapName, "", "")) + ).withID(id); Assert.assertEquals( ImmutableMap.of("B", "C"), - parser.getParser() - .parse( - String.format( - "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", - keyField, - valueField - ) - ) + parser.parse( + String.format( + "{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}", + keyField, + valueField + ) + ).get(new Pair(id, mapName)) ); } @Test public void testObjectMapperFlatDataParser() { - URIExtractionNamespace.ObjectMapperFlatDataParser parser = new URIExtractionNamespace.ObjectMapperFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.ObjectMapperFlatDataParser( registerTypes(new ObjectMapper()) + ).withID(id); + Assert.assertEquals( + ImmutableMap.of("B", "C"), + parser.parse("{\"B\":\"C\"}").get(new Pair(id, KeyValueMap.DEFAULT_MAPNAME)) ); - Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("{\"B\":\"C\"}")); } @Test @@ -301,17 +302,30 @@ public void testSimpleJSONSerDe() throws IOException "col1", "col2", "col3" - ), "col2", "col3" - ), - new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper), - new URIExtractionNamespace.JSONFlatDataParser(mapper, "keyField", "valueField"), - new URIExtractionNamespace.TSVFlatDataParser(ImmutableList.of("A", "B"), ",", null, "A", "B") + ), + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id), + new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper).withID(id), + new URIExtractionNamespace.JSONFlatDataParser( + mapper, + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id), + new URIExtractionNamespace.TSVFlatDataParser( + ImmutableList.of( + "col1", + "col2", + "col3" + ), + ",", + null, + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id) )) { final String str = mapper.writeValueAsString(parser); final URIExtractionNamespace.FlatDataParser parser2 = mapper.readValue( str, URIExtractionNamespace.FlatDataParser.class - ); + ).withID(id); Assert.assertEquals(str, mapper.writeValueAsString(parser2)); } } @@ -326,11 +340,24 @@ public void testSimpleToString() throws IOException "col1", "col2", "col3" - ), "col2", "col3" - ), - new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper), - new URIExtractionNamespace.JSONFlatDataParser(mapper, "keyField", "valueField"), - new URIExtractionNamespace.TSVFlatDataParser(ImmutableList.of("A", "B"), ",", null, "A", "B") + ), + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id), + new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper).withID(id), + new URIExtractionNamespace.JSONFlatDataParser( + mapper, + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id), + new URIExtractionNamespace.TSVFlatDataParser( + ImmutableList.of( + "col1", + "col2", + "col3" + ), + ",", + null, + ImmutableList.of(new KeyValueMap(mapName, "col2", "col3")) + ).withID(id) )) { Assert.assertFalse(parser.toString().contains("@")); } @@ -341,7 +368,7 @@ public void testMatchedJson() throws IOException { final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); URIExtractionNamespace namespace = mapper.readValue( - "{\"type\":\"uri\", \"uriPrefix\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\", \"namespace\":\"testNamespace\"}", + "{\"type\":\"uri\", \"uriPrefix\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\"}", URIExtractionNamespace.class ); @@ -376,11 +403,37 @@ public void testExplicitJsonException() throws IOException { final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); mapper.readValue( - "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\", \"namespace\":\"testNamespace\"}", + "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\"}", URIExtractionNamespace.class ); } + @Test + public void testExplicitJsonWithMultipleMaps() throws IOException + { + final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); + URIExtractionNamespace namespace = mapper.readValue( + "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"csv\", \"columns\":[\"key\", \"value1\", \"value2\"], \"maps\":[{\"mapName\":\"map1\",\"keyColumn\":\"key\", \"valueColumn\":\"value1\"}, {\"mapName\":\"map2\",\"keyColumn\":\"key\", \"valueColumn\":\"value2\"}]}, \"pollPeriod\":\"PT5M\"}", + URIExtractionNamespace.class + ); + + Assert.assertEquals( + URIExtractionNamespace.CSVFlatDataParser.class.getCanonicalName(), + namespace.getNamespaceParseSpec().getClass().getCanonicalName() + ); + Assert.assertEquals("file:/foo", namespace.getUri().toString()); + URIExtractionNamespace.CSVFlatDataParser parser = (URIExtractionNamespace.CSVFlatDataParser)namespace.getNamespaceParseSpec(); + List keyValueMaps = parser.getMaps(); + Assert.assertEquals(2, keyValueMaps.size()); + Assert.assertEquals("map1", keyValueMaps.get(0).getMapName()); + Assert.assertEquals("key", keyValueMaps.get(0).getKeyColumn()); + Assert.assertEquals("value1", keyValueMaps.get(0).getValueColumn()); + Assert.assertEquals("map2", keyValueMaps.get(1).getMapName()); + Assert.assertEquals("key", keyValueMaps.get(1).getKeyColumn()); + Assert.assertEquals("value2", keyValueMaps.get(1).getValueColumn()); + Assert.assertEquals(5L * 60_000L, namespace.getPollMs()); + } + @Test public void testFlatDataNumeric() { @@ -388,76 +441,71 @@ public void testFlatDataNumeric() final String valueField = "valueField"; final int n = 341879; final String nString = String.format("%d", n); - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( new ObjectMapper(), - keyField, - valueField - ); + ImmutableList.of(new KeyValueMap(mapName, keyField, valueField)) + ).withID(id); Assert.assertEquals( "num string value", ImmutableMap.of("B", nString), - parser.getParser() - .parse( - String.format( - "{\"%s\":\"B\", \"%s\":\"%d\", \"FOO\":\"BAR\"}", - keyField, - valueField, - n - ) - ) + parser.parse( + String.format( + "{\"%s\":\"B\", \"%s\":\"%d\", \"FOO\":\"BAR\"}", + keyField, + valueField, + n + ) + ).get(new Pair(id, mapName)) ); Assert.assertEquals( "num string key", ImmutableMap.of(nString, "C"), - parser.getParser() - .parse( - String.format( - "{\"%s\":\"%d\", \"%s\":\"C\", \"FOO\":\"BAR\"}", - keyField, - n, - valueField - ) - ) + parser.parse( + String.format( + "{\"%s\":\"%d\", \"%s\":\"C\", \"FOO\":\"BAR\"}", + keyField, + n, + valueField + ) + ).get(new Pair(id, mapName)) ); Assert.assertEquals( "num value", ImmutableMap.of("B", nString), - parser.getParser() - .parse( - String.format( - "{\"%s\":\"B\", \"%s\":%d, \"FOO\":\"BAR\"}", - keyField, - valueField, - n - ) - ) + parser.parse( + String.format( + "{\"%s\":\"B\", \"%s\":%d, \"FOO\":\"BAR\"}", + keyField, + valueField, + n + ) + ).get(new Pair(id, mapName)) ); Assert.assertEquals( "num key", ImmutableMap.of(nString, "C"), - parser.getParser() - .parse( - String.format( - "{\"%s\":%d, \"%s\":\"C\", \"FOO\":\"BAR\"}", - keyField, - n, - valueField - ) - ) + parser.parse( + String.format( + "{\"%s\":%d, \"%s\":\"C\", \"FOO\":\"BAR\"}", + keyField, + n, + valueField + ) + ).get(new Pair(id, mapName)) ); } @Test public void testSimpleJsonNumeric() { - final URIExtractionNamespace.ObjectMapperFlatDataParser parser = new URIExtractionNamespace.ObjectMapperFlatDataParser( + final URIExtractionNamespace.FlatDataParser parser = new URIExtractionNamespace.ObjectMapperFlatDataParser( registerTypes(new DefaultObjectMapper()) - ); + ).withID(id); final int n = 341879; final String nString = String.format("%d", n); Assert.assertEquals( ImmutableMap.of("key", nString), - parser.getParser().parse(String.format("{\"key\":%d}", n)) + parser.parse(String.format("{\"key\":%d}", n)).get(new Pair(id, KeyValueMap.DEFAULT_MAPNAME)) ); } } diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java index f231c86e77c3..d4dc1bb0e08f 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java @@ -21,13 +21,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; - import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.Lifecycle; 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.KeyValueMap; import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; @@ -45,8 +46,9 @@ import java.io.File; import java.io.FileWriter; import java.io.OutputStreamWriter; -import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; /** * @@ -104,8 +106,9 @@ public void testNewTask() throws Exception new Period(0), null ); - Map map = new HashMap<>(); - factory.populateCache(namespaceID, namespace, null, map); + ConcurrentMap> mapMap = new ConcurrentHashMap<>(); + factory.populateCache(namespaceID, namespace, null, mapMap, NamespaceExtractionCacheManager.getMapAllocator(cacheManager)); + Map map = mapMap.get(new Pair(namespaceID, KeyValueMap.DEFAULT_MAPNAME)); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); cacheManager.delete(namespaceID); diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java index 1d7542f09f3b..c3851c0c88a9 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java @@ -28,10 +28,12 @@ import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.UOE; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.KeyValueMap; import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; @@ -221,7 +223,7 @@ public Object[] next() catch (Exception e) { throw Throwables.propagate(e); } - ConcurrentHashMap> fnCache = new ConcurrentHashMap>(); + ConcurrentHashMap>> fnCacheMap = new ConcurrentHashMap<>(); try { return new Object[]{ String.format( @@ -267,6 +269,7 @@ public URIExtractionNamespaceCacheFactoryTest( new NoopServiceEmitter(), namespaceFunctionFactoryMap ); + this.mapAllocator = NamespaceExtractionCacheManager.getMapAllocator(manager); namespaceFunctionFactoryMap.put( URIExtractionNamespace.class, @@ -280,11 +283,12 @@ public URIExtractionNamespaceCacheFactoryTest( private final String suffix; private final Function outStreamSupplier; private Lifecycle lifecycle; - private NamespaceExtractionCacheManager manager; + private final NamespaceExtractionCacheManager manager; private File tmpFile; private File tmpFileParent; private URIExtractionNamespaceCacheFactory factory; private URIExtractionNamespace namespace; + private final Function> mapAllocator; private String id; @Before @@ -334,7 +338,8 @@ public void simpleTest() throws IOException, ExecutionException, InterruptedExce { Assert.assertTrue(manager.getKnownIDs().isEmpty()); NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(id, namespace)); - Map map = manager.getCacheMap(id); + Pair key = new Pair(id, KeyValueMap.DEFAULT_MAPNAME); + Map map = manager.getCacheMap(id).get(key); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); } @@ -353,7 +358,8 @@ public void simpleTestRegex() throws IOException, ExecutionException, Interrupte ); Assert.assertTrue(!manager.getKnownIDs().contains(regexID)); NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(regexID, namespace)); - Map map = manager.getCacheMap(regexID); + Pair key = new Pair(regexID, KeyValueMap.DEFAULT_MAPNAME); + Map map = manager.getCacheMap(regexID).get(key); Assert.assertNotNull(map); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); @@ -382,7 +388,8 @@ public void simplePileONamespacesTest() throws InterruptedException } for (String id : ids) { - final Map map = manager.getCacheMap(id); + Pair key = new Pair(id, KeyValueMap.DEFAULT_MAPNAME); + final Map map = manager.getCacheMap(id).get(key); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); manager.delete(id); @@ -395,14 +402,15 @@ public void testLoadOnlyOnce() throws Exception { Assert.assertTrue(manager.getKnownIDs().isEmpty()); - ConcurrentMap map = new ConcurrentHashMap<>(); + ConcurrentMap> mapMap = new ConcurrentHashMap<>(); + String v = factory.populateCache(id, namespace, null, mapMap, mapAllocator); - String v = factory.populateCache(id, namespace, null, map); + Map map = mapMap.get(new Pair(id, KeyValueMap.DEFAULT_MAPNAME)); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); Assert.assertNotNull(v); - String v2 = factory.populateCache(id, namespace, v, map); + String v2 = factory.populateCache(id, namespace, v, mapMap, mapAllocator); Assert.assertEquals(v, v2); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); @@ -419,8 +427,8 @@ public void testMissing() throws Exception null ); Assert.assertTrue(new File(namespace.getUri()).delete()); - ConcurrentMap map = new ConcurrentHashMap<>(); - factory.populateCache(id, badNamespace, null, map); + ConcurrentMap> map = new ConcurrentHashMap<>(); + factory.populateCache(id, badNamespace, null, map, mapAllocator); } @Test(expected = FileNotFoundException.class) @@ -436,8 +444,8 @@ public void testMissingRegex() throws Exception null ); Assert.assertTrue(new File(namespace.getUri()).delete()); - ConcurrentMap map = new ConcurrentHashMap<>(); - factory.populateCache(badId, badNamespace, null, map); + ConcurrentMap> map = new ConcurrentHashMap<>(); + factory.populateCache(badId, badNamespace, null, map, mapAllocator); } @Test(expected = IAE.class) @@ -525,7 +533,7 @@ public void testWeirdSchemaOnExactURI() throws Exception Period.millis((int) namespace.getPollMs()), null ); - final Map map = new HashMap<>(); - Assert.assertNotNull(factory.populateCache(id, extractionNamespace, null, map)); + final ConcurrentMap> map = new ConcurrentHashMap<>(); + Assert.assertNotNull(factory.populateCache(id, extractionNamespace, null, map, mapAllocator)); } } diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java index 5215228dc8f2..9cec2b7f2dd0 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java @@ -19,6 +19,7 @@ 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; @@ -27,14 +28,15 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; - import io.druid.concurrent.Execs; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.TestDerbyConnector; 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.KeyValueMap; import io.druid.server.lookup.namespace.JDBCExtractionNamespaceCacheFactory; import io.druid.server.metrics.NoopServiceEmitter; import org.joda.time.Period; @@ -50,8 +52,10 @@ import java.io.Closeable; import java.io.IOException; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -71,7 +75,9 @@ public class JDBCExtractionNamespaceTest private static final String namespace = "testNamespace"; private static final String tableName = "abstractDbRenameTest"; private static final String keyName = "keyName"; - private static final String valName = "valName"; + private static final String valName1 = "valName1"; + private static final String valName2 = "valName2"; + private static final String valName3 = "valName3"; private static final String tsColumn_ = "tsColumn"; private static final Map renames = ImmutableMap.of( "foo", "bar", @@ -79,6 +85,36 @@ public class JDBCExtractionNamespaceTest "how about that", "foo", "empty string", "" ); + private static final Map appends = ImmutableMap.of( + "foo", "foo?", + "bad", "bad?", + "how about that", "how about that?", + "empty string", "empty string?" + ); + private static final Map get1stWord = ImmutableMap.of( + "foo", "foo", + "bad", "bad", + "how about that", "how", + "empty string", "empty" + ); + private static final Map> mapOfMap = ImmutableMap.of( + new Pair(namespace, KeyValueMap.DEFAULT_MAPNAME), renames, + new Pair(namespace, "appends"), appends, + new Pair(namespace, "get1st"), get1stWord + ); + + private static final List DEFAULT_ONLY_MAP = ImmutableList.of ( + new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, keyName, valName1) + ); + private static final List ALL_MAPS = ImmutableList.of ( + new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, keyName, valName1), + new KeyValueMap("appends", keyName, valName2), + new KeyValueMap("get1st", keyName, valName3) + ); + private static final List NO_DEFAULT_MAPS = ImmutableList.of ( + new KeyValueMap("appends", keyName, valName2), + new KeyValueMap("get1st", keyName, valName3) + ); @Parameterized.Parameters(name = "{0}") public static Collection getParameters() @@ -120,22 +156,24 @@ public Handle call() 0, handle.createStatement( String.format( - "CREATE TABLE %s (%s TIMESTAMP, %s VARCHAR(64), %s VARCHAR(64))", + "CREATE TABLE \"%s\" (\"%s\" TIMESTAMP, \"%s\" VARCHAR(64), \"%s\" VARCHAR(64), \"%s\" VARCHAR(64), \"%s\" VARCHAR(64))", tableName, tsColumn_, keyName, - valName + valName1, + valName2, + valName3 ) ).setQueryTimeout(1).execute() ); - handle.createStatement(String.format("TRUNCATE TABLE %s", tableName)).setQueryTimeout(1).execute(); + handle.createStatement(String.format("TRUNCATE TABLE \"%s\"", tableName)).setQueryTimeout(1).execute(); handle.commit(); closer.register(new Closeable() { @Override public void close() throws IOException { - handle.createStatement("DROP TABLE " + tableName).setQueryTimeout(1).execute(); + handle.createStatement("DROP TABLE \"" + tableName + "\"").setQueryTimeout(1).execute(); final ListenableFuture future = setupTeardownService.submit(new Runnable() { @Override @@ -175,9 +213,9 @@ public void close() throws IOException } } }); - for (Map.Entry entry : renames.entrySet()) { + for (String key : renames.keySet()) { try { - insertValues(handle, entry.getKey(), entry.getValue(), "2015-01-01 00:00:00"); + insertValues(handle, key, renames.get(key), appends.get(key), get1stWord.get(key), "2015-01-01 00:00:00"); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -197,7 +235,8 @@ public String populateCache( final String id, final JDBCExtractionNamespace namespace, final String lastVersion, - final Map cache + final ConcurrentMap> cache, + final Function> mapAllocator ) throws Exception { updateLock.lockInterruptibly(); @@ -208,7 +247,8 @@ public String populateCache( id, namespace, lastVersion, - cache + cache, + mapAllocator ); } finally { @@ -325,26 +365,26 @@ public void close() throws IOException } } - private void insertValues(final Handle handle, final String key, final String val, final String updateTs) + private void insertValues(final Handle handle, final String key, final String val1, final String val2, final String val3, final String updateTs) throws InterruptedException { final String query; if (tsColumn == null) { handle.createStatement( - String.format("DELETE FROM %s WHERE %s='%s'", tableName, keyName, key) + String.format("DELETE FROM \"%s\" WHERE \"%s\"='%s'", tableName, keyName, key) ).setQueryTimeout(1).execute(); query = String.format( - "INSERT INTO %s (%s, %s) VALUES ('%s', '%s')", + "INSERT INTO \"%s\" (\"%s\", \"%s\", \"%s\", \"%s\") VALUES ('%s', '%s', '%s', '%s')", tableName, - keyName, valName, - key, val + keyName, valName1, valName2, valName3, + key, val1, val2, val3 ); } else { query = String.format( - "INSERT INTO %s (%s, %s, %s) VALUES ('%s', '%s', '%s')", + "INSERT INTO \"%s\" (\"%s\", \"%s\", \"%s\", \"%s\", \"%s\") VALUES ('%s', '%s', '%s', '%s', '%s')", tableName, - tsColumn, keyName, valName, - updateTs, key, val + tsColumn, keyName, valName1, valName2, valName3, + updateTs, key, val1, val2, val3 ); } Assert.assertEquals(1, handle.createStatement(query).setQueryTimeout(1).execute()); @@ -355,65 +395,132 @@ private void insertValues(final Handle handle, final String key, final String va } @Test(timeout = 10_000L) - public void testMapping() + public void testMapping1() throws ClassNotFoundException, NoSuchFieldException, IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( derbyConnectorRule.getMetadataConnectorConfig(), tableName, - keyName, - valName, + null, + null, + DEFAULT_ONLY_MAP, tsColumn, new Period(0) ); NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); - final Map map = extractionCacheManager.getCacheMap(namespace); + checkExtractFunction(namespace, extractionNamespace); + } - for (Map.Entry entry : renames.entrySet()) { - String key = entry.getKey(); - String val = entry.getValue(); - Assert.assertEquals("non-null check", Strings.emptyToNull(val), Strings.emptyToNull(map.get(key))); - } - Assert.assertEquals("null check", null, map.get("baz")); + @Test(timeout = 10_000L) + public void testMapping2() + throws ClassNotFoundException, NoSuchFieldException, IllegalAccessException, ExecutionException, + InterruptedException, TimeoutException + { + final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( + derbyConnectorRule.getMetadataConnectorConfig(), + tableName, + null, + null, + ALL_MAPS, + tsColumn, + new Period(0) + ); + NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); + checkExtractFunction(namespace, extractionNamespace); + } + + @Test(timeout = 10_000L) + public void testMapping3() + throws ClassNotFoundException, NoSuchFieldException, IllegalAccessException, ExecutionException, + InterruptedException, TimeoutException + { + final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( + derbyConnectorRule.getMetadataConnectorConfig(), + tableName, + null, + null, + NO_DEFAULT_MAPS, + tsColumn, + new Period(0) + ); + NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); + checkExtractFunction(namespace, extractionNamespace); + } + + + @Test(timeout = 10_000L) + public void testOldSpec() + throws ClassNotFoundException, NoSuchFieldException, IllegalAccessException, ExecutionException, + InterruptedException, TimeoutException + { + final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( + derbyConnectorRule.getMetadataConnectorConfig(), + tableName, + keyName, + valName1, + null, + tsColumn, + new Period(0) + ); + NamespaceExtractionCacheManagersTest.waitFor(extractionCacheManager.schedule(namespace, extractionNamespace)); + checkExtractFunction(namespace, extractionNamespace); } + @Test(timeout = 10_000L) public void testSkipOld() throws NoSuchFieldException, IllegalAccessException, ExecutionException, InterruptedException { - final JDBCExtractionNamespace extractionNamespace = ensureNamespace(); + ensureNamespace(ALL_MAPS); - assertUpdated(namespace, "foo", "bar"); + assertUpdated(namespace, "get1st", "foo", "foo"); if (tsColumn != null) { - insertValues(handleRef, "foo", "baz", "1900-01-01 00:00:00"); + insertValues(handleRef, "foo", "baz", "foo?", "foox", "1900-01-01 00:00:00"); } - assertUpdated(namespace, "foo", "bar"); + assertUpdated(namespace, "get1st", "foo", "foo"); } @Test(timeout = 60_000L) public void testFindNew() throws NoSuchFieldException, IllegalAccessException, ExecutionException, InterruptedException { - final JDBCExtractionNamespace extractionNamespace = ensureNamespace(); + ensureNamespace(ALL_MAPS); + + assertUpdated(namespace, "appends", "foo", "foo?"); - assertUpdated(namespace, "foo", "bar"); + insertValues(handleRef, "foo", "baz", "foo!", "foo", "2900-01-01 00:00:00"); - insertValues(handleRef, "foo", "baz", "2900-01-01 00:00:00"); + assertUpdated(namespace, "appends", "foo", "foo!"); + } + + private void checkExtractFunction(String id, JDBCExtractionNamespace namespace) + { + for (KeyValueMap keyValueMap: namespace.getMaps()) + { + Pair mapKey = new Pair(id, keyValueMap.getMapName()); + Map map = extractionCacheManager.getCacheMap(id).get(mapKey); - assertUpdated(namespace, "foo", "baz"); + for (Map.Entry entry : mapOfMap.get(mapKey).entrySet()) { + String key = entry.getKey(); + String val = entry.getValue(); + Assert.assertEquals("non-null check", Strings.emptyToNull(val), Strings.emptyToNull(map.get(key))); + } + Assert.assertEquals("null check", null, map.get("baz")); + } } - private JDBCExtractionNamespace ensureNamespace() + private JDBCExtractionNamespace ensureNamespace(List keyValueMaps) throws NoSuchFieldException, IllegalAccessException, InterruptedException { final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( derbyConnectorRule.getMetadataConnectorConfig(), tableName, - keyName, - valName, + null, + null, + keyValueMaps, tsColumn, new Period(10) ); @@ -421,10 +528,11 @@ private JDBCExtractionNamespace ensureNamespace() waitForUpdates(1_000L, 2L); + Pair mapKey = new Pair(namespace, KeyValueMap.DEFAULT_MAPNAME); Assert.assertEquals( "sanity check not correct", "bar", - extractionCacheManager.getCacheMap(namespace).get("foo") + extractionCacheManager.getCacheMap(namespace).get(mapKey).get("foo") ); return extractionNamespace; } @@ -456,16 +564,16 @@ private void waitForUpdates(long timeout, long numUpdates) throws InterruptedExc } while (post < pre + numUpdates); } - private void assertUpdated(String namespace, String key, String expected) throws InterruptedException + private void assertUpdated(String namespace, String mapName, String key, String expected) throws InterruptedException { waitForUpdates(1_000L, 2L); - Map map = extractionCacheManager.getCacheMap(namespace); + Map map = extractionCacheManager.getCacheMap(namespace).get(new Pair(namespace, mapName)); // rely on test timeout to break out of this loop while (!expected.equals(map.get(key))) { Thread.sleep(100); - map = extractionCacheManager.getCacheMap(namespace); + map = extractionCacheManager.getCacheMap(namespace).get(mapName); } Assert.assertEquals( diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java index f3fbe88eae8d..83c8236080fd 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java @@ -20,6 +20,7 @@ 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; @@ -29,9 +30,11 @@ import io.druid.concurrent.Execs; import io.druid.data.SearchableVersionedDataFinder; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.KeyValueMap; import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; @@ -96,11 +99,14 @@ public String populateCache( final String id, final URIExtractionNamespace extractionNamespace, final String lastVersion, - final Map cache + final ConcurrentMap> cache, + final Function> mapAllocator ) throws Exception { + Map keyValue = new ConcurrentHashMap<>(); + keyValue.put(KEY, VALUE); // Don't actually read off disk because TravisCI doesn't like that - cache.put(KEY,VALUE); + cache.put(new Pair(id, KeyValueMap.DEFAULT_MAPNAME), keyValue); Thread.sleep(2);// To make absolutely sure there is a unique currentTimeMillis return Long.toString(System.currentTimeMillis()); } @@ -373,7 +379,7 @@ public void testDelete(final String ns) } } while (!manager.implData.containsKey(ns) || !manager.implData.get(ns).enabled.get()); - Assert.assertEquals(VALUE, manager.getCacheMap(ns).get(KEY)); + Assert.assertEquals(VALUE, manager.getInnerCacheMap(ns, KeyValueMap.DEFAULT_MAPNAME).get(KEY)); Assert.assertTrue(manager.implData.containsKey(ns)); diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java index 051a3f7096f9..f374ae65ddc5 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -24,13 +24,14 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; - import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.KeyValueMap; import io.druid.query.lookup.namespace.URIExtractionNamespace; import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.lookup.namespace.URIExtractionNamespaceCacheFactory; @@ -49,6 +50,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -105,13 +107,17 @@ public NamespaceExtractionCacheManagersTest( private static final List nsList = ImmutableList.of("testNs", "test.ns", "//tes-tn!s"); + private final String TEST_MAP_NAME = "nsTest"; + @Before public void setup() { // prepopulate caches for (String ns : nsList) { - final ConcurrentMap map = extractionCacheManager.getCacheMap(ns); - map.put("oldNameSeed1", "oldNameSeed2"); + final ConcurrentMap> map = extractionCacheManager.getCacheMap(ns); + ConcurrentMap mapEntry = new ConcurrentHashMap<>(); + mapEntry.put("oldNameSeed1", "oldNameSeed2"); + map.put(new Pair(ns, TEST_MAP_NAME), mapEntry); } } @@ -119,28 +125,37 @@ public void setup() public void testSimpleCacheCreate() { for (String ns : nsList) { - ConcurrentMap map = extractionCacheManager.getCacheMap(ns); + Map map = extractionCacheManager.getInnerCacheMap(ns, TEST_MAP_NAME); map.put("key", "val"); Assert.assertEquals("val", map.get("key")); - Assert.assertEquals("val", extractionCacheManager.getCacheMap(ns).get("key")); + Pair mapKey = new Pair(ns, TEST_MAP_NAME); + Assert.assertEquals("val", extractionCacheManager.getCacheMap(ns).get(mapKey).get("key")); } } + // following looks weird because of following new restriction: + // swapAndClearCache() is called only once and called before the actual usage + // New implementation of cacheMap needs namespace embedded in itself + // and upon the above assumption, cacheMap is constructed with new namespace (not cacheKey) + // + // Original test violates the restriction so that is changed to work @Test public void testSimpleCacheSwap() { for (String ns : nsList) { - ConcurrentMap map = extractionCacheManager.getCacheMap(ns + "old_cache"); + Map map = extractionCacheManager.getInnerCacheMap(ns + "old_cache", TEST_MAP_NAME); map.put("key", "val"); extractionCacheManager.swapAndClearCache(ns, ns + "old_cache"); Assert.assertEquals("val", map.get("key")); - Assert.assertEquals("val", extractionCacheManager.getCacheMap(ns).get("key")); + Pair mapKey = new Pair(ns + "old_cache", TEST_MAP_NAME); + Assert.assertEquals("val", extractionCacheManager.getCacheMap(ns).get(mapKey).get("key")); - ConcurrentMap map2 = extractionCacheManager.getCacheMap(ns + "cache"); + Map map2 = extractionCacheManager.getInnerCacheMap(ns + "cache", TEST_MAP_NAME); map2.put("key", "val2"); Assert.assertTrue(extractionCacheManager.swapAndClearCache(ns, ns + "cache")); Assert.assertEquals("val2", map2.get("key")); - Assert.assertEquals("val2", extractionCacheManager.getCacheMap(ns).get("key")); + mapKey = new Pair(ns + "cache", TEST_MAP_NAME); + Assert.assertEquals("val2", extractionCacheManager.getCacheMap(ns).get(mapKey).get("key")); } } @@ -148,10 +163,11 @@ public void testSimpleCacheSwap() public void testMissingCacheThrowsIAE() { for (String ns : nsList) { - ConcurrentMap map = extractionCacheManager.getCacheMap(ns); + Map map = extractionCacheManager.getInnerCacheMap(ns, TEST_MAP_NAME); map.put("key", "val"); Assert.assertEquals("val", map.get("key")); - Assert.assertEquals("val", extractionCacheManager.getCacheMap(ns).get("key")); + Pair mapKey = new Pair(ns, TEST_MAP_NAME); + Assert.assertEquals("val", extractionCacheManager.getCacheMap(ns).get(mapKey).get("key")); Assert.assertFalse(extractionCacheManager.swapAndClearCache(ns, "I don't exist")); } } @@ -188,8 +204,7 @@ public void testDeleteOnScheduleFail() throws Exception null, new URIExtractionNamespace.JSONFlatDataParser( new DefaultObjectMapper(), - "key", - "val" + ImmutableList.of(new KeyValueMap(KeyValueMap.DEFAULT_MAPNAME, "key", "val")) ), Period.millis(10000), null diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java index 7cdfe8fece48..04bfd25730a4 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManagerTest.java @@ -39,6 +39,7 @@ import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.KeyValueMap; import io.druid.server.DruidNode; import io.druid.server.lookup.namespace.NamespaceExtractionModule; import io.druid.server.metrics.NoopServiceEmitter; @@ -101,7 +102,7 @@ public void testRacyCreation() throws Exception final String namespace = "namespace-" + UUID.randomUUID().toString(); final String cacheKey = "initial-cache-" + namespace; namespaceIds.add(namespace); - manager.getCacheMap(cacheKey).put("foo", "bar"); + manager.getInnerCacheMap(cacheKey, KeyValueMap.DEFAULT_MAPNAME).put("foo", "bar"); Assert.assertFalse(manager.swapAndClearCache(namespace, cacheKey)); } final Random random = new Random(3748218904L); @@ -123,7 +124,7 @@ public void run() } for (int i = 0; i < 1000; ++i) { final String cacheKey = String.format("%s-%d-key-%d", namespace, j, i); - manager.getCacheMap(cacheKey).put("foo", "bar" + Integer.toString(i)); + manager.getInnerCacheMap(cacheKey, KeyValueMap.DEFAULT_MAPNAME).put("foo", "bar" + Integer.toString(i)); Assert.assertTrue(manager.swapAndClearCache(namespace, cacheKey)); } } @@ -138,7 +139,7 @@ public void run() } for (final String namespace : namespaceIds) { - Assert.assertEquals(ImmutableMap.of("foo", "bar999"), manager.getCacheMap(namespace)); + Assert.assertEquals(ImmutableMap.of("foo", "bar999"), manager.getCacheMap(namespace).values().iterator().next()); } } } diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java index 4c6b57bffa42..2d303d6b25cc 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LoadingLookupFactory.java @@ -33,7 +33,7 @@ import java.util.concurrent.atomic.AtomicBoolean; @JsonTypeName("loadingLookup") -public class LoadingLookupFactory implements LookupExtractorFactory +public class LoadingLookupFactory extends LookupExtractorFactory { private final static Logger LOGGER = new Logger(LoadingLookupFactory.class); diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java index 3766120ff79f..b841b45a7a61 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/PollingLookupFactory.java @@ -34,7 +34,7 @@ import java.util.concurrent.atomic.AtomicBoolean; @JsonTypeName("pollingLookup") -public class PollingLookupFactory implements LookupExtractorFactory +public class PollingLookupFactory extends LookupExtractorFactory { private static final Logger LOGGER = new Logger(PollingLookupFactory.class); diff --git a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java index fd41aeeba03c..48175e0bd86c 100644 --- a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java @@ -24,11 +24,13 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.primitives.Bytes; import io.druid.java.util.common.StringUtils; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.DimFilterUtils; import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.lookup.LookupExtractor; +import io.druid.query.lookup.LookupExtractorFactory; import io.druid.query.lookup.LookupReferencesManager; import io.druid.segment.DimensionSelector; @@ -57,6 +59,9 @@ public class LookupDimensionSpec implements DimensionSpec @JsonProperty private final String name; + @JsonProperty + private final String innerMapName; + @JsonProperty private final boolean optimize; @@ -70,6 +75,7 @@ public LookupDimensionSpec( @JsonProperty("retainMissingValue") boolean retainMissingValue, @JsonProperty("replaceMissingValueWith") String replaceMissingValueWith, @JsonProperty("name") String name, + @JsonProperty("innerMapName") String innerMapName, @JacksonInject LookupReferencesManager lookupReferencesManager, @JsonProperty("optimize") Boolean optimize ) @@ -81,6 +87,7 @@ public LookupDimensionSpec( this.outputName = Preconditions.checkNotNull(outputName, "outputName can not be Null"); this.lookupReferencesManager = lookupReferencesManager; this.name = name; + this.innerMapName = innerMapName; this.lookup = lookup; Preconditions.checkArgument( Strings.isNullOrEmpty(name) ^ (lookup == null), @@ -123,16 +130,27 @@ public String getName() return name; } + @JsonProperty + @Nullable + public String getInnerMapName() + { + return innerMapName; + } + @Override public ExtractionFn getExtractionFn() { - final LookupExtractor lookupExtractor = Strings.isNullOrEmpty(name) - ? this.lookup - : Preconditions.checkNotNull( - lookupReferencesManager.get(name), - "Lookup [%s] not found", - name - ).get(); + LookupExtractor lookupExtractor = null; + if (Strings.isNullOrEmpty(name)) { + lookupExtractor = this.lookup; + } else { + LookupExtractorFactory factory = Preconditions.checkNotNull( + lookupReferencesManager.get(name), + "Lookup [%s] not found", + name + ); + lookupExtractor = Strings.isNullOrEmpty(innerMapName) ? factory.get() : factory.get(innerMapName); + } return new LookupExtractionFn( lookupExtractor, @@ -155,7 +173,9 @@ public byte[] getCacheKey() byte[] dimensionBytes = StringUtils.toUtf8(dimension); byte[] dimExtractionFnBytes = Strings.isNullOrEmpty(name) ? getLookup().getCacheKey() - : StringUtils.toUtf8(name); + : Strings.isNullOrEmpty(innerMapName) + ? StringUtils.toUtf8(name) + : Bytes.concat(StringUtils.toUtf8(name), StringUtils.toUtf8(innerMapName)); byte[] outputNameBytes = StringUtils.toUtf8(outputName); byte[] replaceWithBytes = StringUtils.toUtf8(Strings.nullToEmpty(replaceMissingValueWith)); @@ -216,7 +236,22 @@ public boolean equals(Object o) : that.replaceMissingValueWith != null) { return false; } - return getName() != null ? getName().equals(that.getName()) : that.getName() == null; + + if (getName() != null) { + if (!getName().equals(that.getName())) { + return false; + } + if (getInnerMapName() != null) { + if (!getInnerMapName().equals(that.getInnerMapName())) { + return false; + } + } else if(that.getInnerMapName() != null) { + return false; + } + } else if (that.getName() != null) { + return false; + } + return true; } @@ -229,6 +264,7 @@ public int hashCode() result = 31 * result + (retainMissingValue ? 1 : 0); result = 31 * result + (replaceMissingValueWith != null ? replaceMissingValueWith.hashCode() : 0); result = 31 * result + (getName() != null ? getName().hashCode() : 0); + result = 31 * result + (getInnerMapName() != null ? getInnerMapName().hashCode() : 0); result = 31 * result + (optimize ? 1 : 0); return result; } diff --git a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java index bec1e07feb07..d315a58074e1 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java @@ -20,7 +20,7 @@ package io.druid.query.lookup; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.base.Supplier; +import com.metamx.common.ISE; import javax.annotation.Nullable; @@ -30,7 +30,7 @@ * If a LookupExtractorFactory wishes to support idempotent updates, it needs to implement the `replaces` method */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -public interface LookupExtractorFactory extends Supplier +public abstract class LookupExtractorFactory { /** *

@@ -40,7 +40,7 @@ public interface LookupExtractorFactory extends Supplier * * @return Returns false if is not successfully started the {@link LookupExtractor} otherwise returns true. */ - public boolean start(); + public abstract boolean start(); /** *

@@ -49,19 +49,35 @@ public interface LookupExtractorFactory extends Supplier *

* @return Returns false if not successfully closed the {@link LookupExtractor} otherwise returns true */ - public boolean close(); + public abstract boolean close(); /** * Determine if this LookupExtractorFactory should replace some other LookupExtractorFactory. * This is used to implement no-down-time * @param other Some other LookupExtractorFactory which might need replaced * @return `true` if the other should be replaced by this one. `false` if this one should not replace the other factory */ - boolean replaces(@Nullable LookupExtractorFactory other); + public abstract boolean replaces(@Nullable LookupExtractorFactory other); + + /** + * @return Returns LookupExtractor associated with this factory + */ + public abstract LookupExtractor get(); + + /** + * Some LookupExtractorFactories have multiple inner maps in one namespace. + * For those factories, innerMapName should be additionally given to get LookupExtractor + * @param innerMapName name of map in the namespace + * @return LookupExtractor for the given mapName + */ + public LookupExtractor get(String innerMapName) + { + throw new ISE("innerMapName not supported"); + } /** * @return Returns the actual introspection request handler, can return {@code null} if it is not supported. * This will be called once per HTTP request to introspect the actual lookup. */ @Nullable - public LookupIntrospectHandler getIntrospectHandler(); + public abstract LookupIntrospectHandler getIntrospectHandler(); } diff --git a/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java b/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java index 09977983c9b9..bf67987c2780 100644 --- a/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java +++ b/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java @@ -36,6 +36,7 @@ public class RegisteredLookupExtractionFn implements ExtractionFn private final Object delegateLock = new Object(); private final LookupReferencesManager manager; private final String lookup; + private final String mapName; private final boolean retainMissingValue; private final String replaceMissingValueWith; private final boolean injective; @@ -45,6 +46,7 @@ public class RegisteredLookupExtractionFn implements ExtractionFn public RegisteredLookupExtractionFn( @JacksonInject LookupReferencesManager manager, @JsonProperty("lookup") String lookup, + @JsonProperty("mapName") String mapName, @JsonProperty("retainMissingValue") final boolean retainMissingValue, @Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith, @JsonProperty("injective") final boolean injective, @@ -58,6 +60,7 @@ public RegisteredLookupExtractionFn( this.injective = injective; this.optimize = optimize == null ? true : optimize; this.lookup = lookup; + this.mapName = mapName; } @JsonProperty("lookup") @@ -66,6 +69,12 @@ public String getLookup() return lookup; } + @JsonProperty("mapName") + public String getMapName() + { + return mapName; + } + @JsonProperty("retainMissingValue") public boolean isRetainMissingValue() { @@ -95,11 +104,13 @@ public byte[] getCacheKey() { final byte[] keyPrefix = StringUtils.toUtf8(getClass().getCanonicalName()); final byte[] lookupName = StringUtils.toUtf8(getLookup()); + final byte[] mapName = getMapName() == null ? new byte[] {} : StringUtils.toUtf8(getMapName()); final byte[] delegateKey = ensureDelegate().getCacheKey(); return ByteBuffer - .allocate(keyPrefix.length + 1 + lookupName.length + 1 + delegateKey.length) + .allocate(keyPrefix.length + 1 + lookupName.length + 1 + mapName.length + 1 + delegateKey.length) .put(keyPrefix).put((byte) 0xFF) .put(lookupName).put((byte) 0xFF) + .put(mapName).put((byte) 0xFF) .put(delegateKey) .array(); } @@ -140,8 +151,10 @@ private LookupExtractionFn ensureDelegate() // http://www.javamex.com/tutorials/double_checked_locking.shtml synchronized (delegateLock) { if (null == delegate) { + LookupExtractorFactory factory = + Preconditions.checkNotNull(manager.get(getLookup()), "Lookup [%s] not found", getLookup()); delegate = new LookupExtractionFn( - Preconditions.checkNotNull(manager.get(getLookup()), "Lookup [%s] not found", getLookup()).get(), + getMapName() == null ? factory.get() : factory.get(getMapName()), isRetainMissingValue(), getReplaceMissingValueWith(), isInjective(), @@ -177,6 +190,14 @@ public boolean equals(Object o) if (!getLookup().equals(that.getLookup())) { return false; } + if (getMapName() != null) { + if (!getMapName().equals(that.getMapName())) { + return false; + } + } else if(that.getMapName() != null) { + return false; + } + return getReplaceMissingValueWith() != null ? getReplaceMissingValueWith().equals(that.getReplaceMissingValueWith()) : that.getReplaceMissingValueWith() == null; @@ -186,6 +207,7 @@ public boolean equals(Object o) public int hashCode() { int result = getLookup().hashCode(); + result = 31 * result + (getMapName() != null ? getMapName().hashCode() : 0); result = 31 * result + (isRetainMissingValue() ? 1 : 0); result = 31 * result + (getReplaceMissingValueWith() != null ? getReplaceMissingValueWith().hashCode() : 0); result = 31 * result + (isInjective() ? 1 : 0); @@ -199,6 +221,7 @@ public String toString() return "RegisteredLookupExtractionFn{" + "delegate=" + delegate + ", lookup='" + lookup + '\'' + + ", mapName='" + mapName + '\'' + ", retainMissingValue=" + retainMissingValue + ", replaceMissingValueWith='" + replaceMissingValueWith + '\'' + ", injective=" + injective + diff --git a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java index 609139df81f3..db2c391273d2 100644 --- a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java @@ -40,7 +40,12 @@ public class RegisteredLookupExtractionFnTest "foo", "bar", "bat", "baz" ); + private static Map MAP2 = ImmutableMap.of( + "foo", "bal", + "bat", "bau" + ); private static final LookupExtractor LOOKUP_EXTRACTOR = new MapLookupExtractor(MAP, true); + private static final LookupExtractor LOOKUP_EXTRACTOR2 = new MapLookupExtractor(MAP2, true); private static final String LOOKUP_NAME = "some lookup"; @Rule @@ -55,6 +60,7 @@ public void testSimpleDelegation() final RegisteredLookupExtractionFn fn = new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, true, null, true, @@ -67,6 +73,40 @@ public void testSimpleDelegation() Assert.assertEquals("not in the map", fn.apply("not in the map")); } + @Test + public void testMultipleMapsInOneLookup() + { + final LookupReferencesManager manager = EasyMock.createStrictMock(LookupReferencesManager.class); + managerReturnsMultiMaps(manager); + EasyMock.replay(manager); + final RegisteredLookupExtractionFn fn1 = new RegisteredLookupExtractionFn( + manager, + LOOKUP_NAME, + "map1", + true, + null, + true, + false + ); + final RegisteredLookupExtractionFn fn2 = new RegisteredLookupExtractionFn( + manager, + LOOKUP_NAME, + "map2", + true, + null, + true, + false + ); + EasyMock.verify(manager); + for (String orig : Arrays.asList("", "foo", "bat")) { + Assert.assertEquals(LOOKUP_EXTRACTOR.apply(orig), fn1.apply(orig)); + } + Assert.assertEquals("not in the map", fn1.apply("not in the map")); + for (String orig : Arrays.asList("", "foo", "bat")) { + Assert.assertEquals(LOOKUP_EXTRACTOR2.apply(orig), fn2.apply(orig)); + } + Assert.assertEquals("not in the map", fn2.apply("not in the map")); + } @Test public void testMissingDelegation() @@ -80,6 +120,7 @@ public void testMissingDelegation() new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, true, null, true, @@ -96,6 +137,7 @@ public void testNullLookup() { expectedException.expectMessage("`lookup` required"); new RegisteredLookupExtractionFn( + null, null, null, true, @@ -116,6 +158,7 @@ public void testSerDe() throws Exception final RegisteredLookupExtractionFn fn = new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, true, null, true, @@ -144,6 +187,7 @@ public void testEquals() final RegisteredLookupExtractionFn fn = new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, false, "something", true, @@ -154,6 +198,7 @@ public void testEquals() new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, false, "something", true, @@ -165,6 +210,7 @@ public void testEquals() new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, true, null, true, @@ -177,6 +223,7 @@ public void testEquals() new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, false, "something else", true, @@ -190,6 +237,7 @@ public void testEquals() new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, false, "something", false, @@ -202,6 +250,7 @@ public void testEquals() new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, false, "something", true, @@ -215,6 +264,7 @@ public void testEquals() new RegisteredLookupExtractionFn( manager, LOOKUP_NAME, + null, false, null, true, @@ -260,4 +310,53 @@ public LookupExtractor get() } }).anyTimes(); } + + private void managerReturnsMultiMaps(LookupReferencesManager manager) + { + EasyMock.expect(manager.get(EasyMock.eq(LOOKUP_NAME))).andReturn(new LookupExtractorFactory() + { + @Override + public boolean start() + { + return false; + } + + @Override + public boolean close() + { + return false; + } + + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return false; + } + + @Nullable + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + return null; + } + + @Override + public LookupExtractor get() + { + return null; + } + + @Override + public LookupExtractor get(String innerMapName) + { + if (innerMapName.equals("map1")) { + return LOOKUP_EXTRACTOR; + } + if (innerMapName.equals("map2")) { + return LOOKUP_EXTRACTOR2; + } + return null; + } + }).anyTimes(); + } } diff --git a/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java b/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java index 5ac15478ca38..925f43dedfaa 100644 --- a/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java +++ b/server/src/main/java/io/druid/query/lookup/MapLookupExtractorFactory.java @@ -34,7 +34,7 @@ import java.util.Map; @JsonTypeName("map") -public class MapLookupExtractorFactory implements LookupExtractorFactory +public class MapLookupExtractorFactory extends LookupExtractorFactory { @JsonProperty private final Map map; diff --git a/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java index 061bbefbd2e3..31f369196418 100644 --- a/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java +++ b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java @@ -55,7 +55,7 @@ public class LookupDimensionSpecTest EasyMock.replay(LOOKUP_REF_MANAGER); } - private final DimensionSpec lookupDimSpec = new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, + private final DimensionSpec lookupDimSpec = new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, null, true ); @@ -76,23 +76,23 @@ public void testSerDesr(DimensionSpec lookupDimSpec) throws IOException private Object[] parametersForTestSerDesr() { return new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true), - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, true), - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), - new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", LOOKUP_REF_MANAGER, true) + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", null, LOOKUP_REF_MANAGER, true) }; } @Test(expected = Exception.class) public void testExceptionWhenNameAndLookupNotNull() { - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "replace", "name", null, true); + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "replace", "name", null, null, true); } @Test(expected = Exception.class) public void testExceptionWhenNameAndLookupNull() { - new LookupDimensionSpec("dimName", "outputName", null, false, "replace", "", null, true); + new LookupDimensionSpec("dimName", "outputName", null, false, "replace", "", null, null, true); } @Test @@ -111,39 +111,39 @@ public Object[] parametersForTestApply() { return new Object[]{ new Object[]{ - new LookupDimensionSpec("dimName", "outputName", null, true, null, "lookupName", LOOKUP_REF_MANAGER, true), + new LookupDimensionSpec("dimName", "outputName", null, true, null, "lookupName", null, LOOKUP_REF_MANAGER, true), STRING_MAP }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, null, true), STRING_MAP }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, null, true), ImmutableMap.of("not there", "") }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", null, false, null, "lookupName", LOOKUP_REF_MANAGER, true), + new LookupDimensionSpec("dimName", "outputName", null, false, null, "lookupName", null, LOOKUP_REF_MANAGER, true), ImmutableMap.of("not there", "") }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, null, true ), ImmutableMap.of("not there", "Missing_value") }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", null, false, "Missing_value", "lookupName", LOOKUP_REF_MANAGER, + new LookupDimensionSpec("dimName", "outputName", null, false, "Missing_value", "lookupName", null, LOOKUP_REF_MANAGER, true ), ImmutableMap.of("not there", "Missing_value") }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", null, true, null, "lookupName", LOOKUP_REF_MANAGER, true), + new LookupDimensionSpec("dimName", "outputName", null, true, null, "lookupName", null, LOOKUP_REF_MANAGER, true), ImmutableMap.of("not there", "not there") }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, "", null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, "", null, null, true), ImmutableMap.of("not there", "not there") } @@ -164,29 +164,29 @@ public Object[] parametersForTestGetCacheKey() { return new Object[]{ new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, null, true), false }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, null, true ), false }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName2", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName2", MAP_LOOKUP_EXTRACTOR, false, null, null, null, null, true), false }, new Object[]{ - new LookupDimensionSpec("dimName2", "outputName2", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + new LookupDimensionSpec("dimName2", "outputName2", MAP_LOOKUP_EXTRACTOR, false, null, null, null, null, true), false }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, null, true), true }, new Object[]{ - new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", LOOKUP_REF_MANAGER, true), + new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", null, LOOKUP_REF_MANAGER, true), false } };