Lookups: Inherit "injective" from registered lookups, improve docs. (#5316)

Code changes:
- In the lookup-based extractionFns, inherit injective property from
  the lookup itself if not specified.

Doc changes:
- Add a "Query execution" section to the lookups doc explaining how
  injective lookups and their optimizations work.
- Remove scary warnings against using registeredLookup extractionFns.
  They are necessary and important since they work with filters and
  function cascades -- two things that the dimension specs do not do.
  They deserve to be first class citizens.
- Move the "registeredLookup" fn above the "lookup" fn. It's probably
  more commonly used, so the docs read better this way.
This commit is contained in:
Gian Merlino 2018-02-01 18:30:19 -08:00 committed by GitHub
parent 1fffc681d2
commit ed47a1e1a9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 198 additions and 178 deletions

View File

@ -71,7 +71,7 @@ The parameters are as follows
|--------|-----------|--------|-------| |--------|-----------|--------|-------|
|`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-| |`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-|
|`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`0` (do not wait)| |`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`0` (do not wait)|
|`injective`|If the underlying map is injective (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`| |`injective`|If the underlying map is [injective](../../querying/lookups.html#query-execution) (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`|
If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the lookup nodes should be monitored for repeated failures. If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the lookup nodes should be monitored for repeated failures.

View File

@ -95,8 +95,6 @@ The default values are `replaceMissingValueWith = null` and `retainMissingValue
It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`. It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`.
A property of `injective` specifies if optimizations can be used which assume there is no combining of multiple names into one. For example: If ABC123 is the only key that maps to SomeCompany, that can be optimized since it is a unique lookup. But if both ABC123 and DEF456 BOTH map to SomeCompany, then that is NOT a unique lookup. Setting this value to true and setting `retainMissingValue` to FALSE (the default) may cause undesired behavior.
A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`). A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
The second kind where it is not possible to pass at query time due to their size, will be based on an external lookup table or resource that is already registered via configuration file or/and coordinator. The second kind where it is not possible to pass at query time due to their size, will be based on an external lookup table or resource that is already registered via configuration file or/and coordinator.
@ -316,11 +314,94 @@ Example for the `__time` dimension:
JavaScript-based functionality is disabled by default. Please refer to the Druid <a href="../development/javascript.html">JavaScript programming guide</a> for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it. JavaScript-based functionality is disabled by default. Please refer to the Druid <a href="../development/javascript.html">JavaScript programming guide</a> for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
</div> </div>
### Lookup extraction function ### Registered lookup extraction function
Lookups are a concept in Druid where dimension values are (optionally) replaced with new values. Lookups are a concept in Druid where dimension values are (optionally) replaced with new values.
For more documentation on using lookups, please see [here](../querying/lookups.html). For more documentation on using lookups, please see [Lookups](../querying/lookups.html).
Explicit lookups allow you to specify a set of keys and values to use when performing the extraction. The "registeredLookup" extraction function lets you refer to a lookup that has been registered in the cluster-wide
configuration.
An example:
```json
{
"type":"registeredLookup",
"lookup":"some_lookup_name",
"retainMissingValue":true
}
```
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.
It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`.
A property of `injective` can override the lookup's own sense of whether or not it is
[injective](lookups.html#query-execution). If left unspecified, Druid will use the registered cluster-wide lookup
configuration.
A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
The optimization layer will run on the broker and it will rewrite the extraction filter as clause of selector filters.
For instance the following filter
```json
{
"filter": {
"type": "selector",
"dimension": "product",
"value": "bar_1",
"extractionFn": {
"type": "registeredLookup",
"optimize": true,
"lookup": "some_lookup_name"
}
}
}
```
will be rewritten as the following simpler query, assuming a lookup that maps "product_1" and "product_3" to the value
"bar_1":
```json
{
"filter":{
"type":"or",
"fields":[
{
"filter":{
"type":"selector",
"dimension":"product",
"value":"product_1"
}
},
{
"filter":{
"type":"selector",
"dimension":"product",
"value":"product_3"
}
}
]
}
}
```
A null dimension value can be mapped to a specific value by specifying the empty string as the key in your lookup file.
This allows distinguishing between a null dimension and a lookup resulting in a null.
For example, specifying `{"":"bar","bat":"baz"}` with dimension values `[null, "foo", "bat"]` and replacing missing values with `"oof"` will yield results of `["bar", "oof", "baz"]`.
Omitting the empty string key will cause the missing value to take over. For example, specifying `{"bat":"baz"}` with dimension values `[null, "foo", "bat"]` and replacing missing values with `"oof"` will yield results of `["oof", "oof", "baz"]`.
### Inline lookup extraction function
Lookups are a concept in Druid where dimension values are (optionally) replaced with new values.
For more documentation on using lookups, please see [Lookups](../querying/lookups.html).
The "lookup" extraction function lets you specify an inline lookup map without registering one in the cluster-wide
configuration.
Examples:
```json ```json
{ {
@ -347,108 +428,10 @@ Explicit lookups allow you to specify a set of keys and values to use when perfo
} }
``` ```
```json The inline lookup should be of type `map`.
{
"type":"lookup",
"lookup":{"type":"namespace","namespace":"some_lookup"},
"replaceMissingValueWith":"Unknown",
"injective":false
}
```
```json The properties `retainMissingValue`, `replaceMissingValueWith`, `injective`, and `optimize` behave similarly to the
{ [registered lookup extraction function](#registered-lookup-extraction-function).
"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 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.
It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`.
A property of `injective` specifies if optimizations can be used which assume there is no combining of multiple names into one. For example: If ABC123 is the only key that maps to SomeCompany, that can be optimized since it is a unique lookup. But if both ABC123 and DEF456 BOTH map to SomeCompany, then that is NOT a unique lookup. Setting this value to true and setting `retainMissingValue` to FALSE (the default) may cause undesired behavior.
A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
The optimization layer will run on the broker and it will rewrite the extraction filter as clause of selector filters.
For instance the following filter
```json
{
"filter": {
"type": "selector",
"dimension": "product",
"value": "bar_1",
"extractionFn": {
"type": "lookup",
"optimize": true,
"lookup": {
"type": "map",
"map": {
"product_1": "bar_1",
"product_3": "bar_1"
}
}
}
}
}
```
will be rewritten as
```json
{
"filter":{
"type":"or",
"fields":[
{
"filter":{
"type":"selector",
"dimension":"product",
"value":"product_1"
}
},
{
"filter":{
"type":"selector",
"dimension":"product",
"value":"product_3"
}
}
]
}
}
```
A null dimension value can be mapped to a specific value by specifying the empty string as the key.
This allows distinguishing between a null dimension and a lookup resulting in a null.
For example, specifying `{"":"bar","bat":"baz"}` with dimension values `[null, "foo", "bat"]` and replacing missing values with `"oof"` will yield results of `["bar", "oof", "baz"]`.
Omitting the empty string key will cause the missing value to take over. For example, specifying `{"bat":"baz"}` with dimension values `[null, "foo", "bat"]` and replacing missing values with `"oof"` will yield results of `["oof", "oof", "baz"]`.
### Registered Lookup Extraction Function
While it is recommended that the [lookup dimension spec](#lookup-dimensionspecs) be used whenever possible, any lookup that is registered for use as a lookup dimension spec can be used as a dimension extraction.
The specification for dimension extraction using dimension specification named lookups is formatted as per the following example:
```json
{
"type":"registeredLookup",
"lookup":"some_lookup_name",
"retainMissingValue":true,
"injective":false
}
```
All the flags for [lookup extraction function](#lookup-extraction-function) apply here as well.
In general, the dimension specification should be used. This dimension **extraction** implementation is made available for testing, validation, and transitioning from dimension extraction to the dimension specification style lookups.
There is also a chance that a feature uses dimension extraction in such a way that it is not applied to dimension specification lookups. Such a scenario should be brought to the attention of the development mailing list.
### Cascade Extraction Function ### Cascade Extraction Function

View File

@ -8,15 +8,17 @@ layout: doc_page
Lookups are an <a href="../development/experimental.html">experimental</a> feature. Lookups are an <a href="../development/experimental.html">experimental</a> feature.
</div> </div>
Lookups are a concept in Druid where dimension values are (optionally) replaced with new values. Lookups are a concept in Druid where dimension values are (optionally) replaced with new values, allowing join-like
See [dimension specs](../querying/dimensionspecs.html) for more information. For the purpose of these documents, functionality. Applying lookups in Druid is similar to joining a dimension table in a data warehouse. See
a "key" refers to a dimension value to match, and a "value" refers to its replacement. [dimension specs](../querying/dimensionspecs.html) for more information. For the purpose of these documents, a "key"
So if you wanted to rename `appid-12345` to `Super Mega Awesome App` then the key would be `appid-12345` and the value refers to a dimension value to match, and a "value" refers to its replacement. So if you wanted to map
would be `Super Mega Awesome App`. `appid-12345` to `Super Mega Awesome App` then the key would be `appid-12345` and the value would be
`Super Mega Awesome App`.
It is worth noting that lookups support use cases where keys map to unique values (injective) such as a country code and It is worth noting that lookups support not just use cases where keys map one-to-one to unique values, such as country
a country name, and also supports use cases where multiple IDs map to the same value, e.g. multiple app-ids belonging to code and country name, but also support use cases where multiple IDs map to the same value, e.g. multiple app-ids
a single account manager. mapping to a single account manager. When lookups are one-to-one, Druid is able to apply additional optimizations at
query time; see [Query execution](#query-execution) below for more details.
Lookups do not have history. They always use the current data. This means that if the chief account manager for a Lookups do not have history. They always use the current data. This means that if the chief account manager for a
particular app-id changes, and you issue a query with a lookup to store the app-id to account manager relationship, particular app-id changes, and you issue a query with a lookup to store the app-id to account manager relationship,
@ -33,6 +35,38 @@ Other lookup types are available as extensions, including:
- Globally cached lookups from local files, remote URIs, or JDBC through [lookups-cached-global](../development/extensions-core/lookups-cached-global.html). - Globally cached lookups from local files, remote URIs, or JDBC through [lookups-cached-global](../development/extensions-core/lookups-cached-global.html).
- Globally cached lookups from a Kafka topic through [kafka-extraction-namespace](../development/extensions-core/kafka-extraction-namespace.html). - Globally cached lookups from a Kafka topic through [kafka-extraction-namespace](../development/extensions-core/kafka-extraction-namespace.html).
Query Execution
---------------
When executing an aggregation query involving lookups, Druid can decide to apply lookups either while scanning and
aggregating rows, or to apply them after aggregation is complete. It is more efficient to apply lookups after
aggregation is complete, so Druid will do this if it can. Druid decides this by checking if the lookup is marked
as "injective" or not. In general, you should set this property for any lookup that is naturally one-to-one, to allow
Druid to run your queries as fast as possible.
Injective lookups should include _all_ possible keys that may show up in your dataset, and should also map all keys to
_unique values_. This matters because non-injective lookups may map different keys to the same value, which must be
accounted for during aggregation, lest query results contain two result values that should have been aggregated into
one.
This lookup is injective (assuming it contains all possible keys from your data):
```
1 -> Foo
2 -> Bar
3 -> Billy
```
But this one is not, since both "2" and "3" map to the same key:
```
1 -> Foo
2 -> Bar
3 -> Bar
```
To tell Druid that your lookup is injective, you must specify `"injective" : true` in the lookup configuration. Druid
will not detect this automatically.
Dynamic Configuration Dynamic Configuration
--------------------- ---------------------
<div class="note caution"> <div class="note caution">

View File

@ -37,18 +37,14 @@ public class LookupExtractionFn extends FunctionalExtraction
{ {
private final LookupExtractor lookup; private final LookupExtractor lookup;
private final boolean optimize; private final boolean optimize;
// Thes are retained for auto generated hashCode and Equals
private final boolean retainMissingValue;
private final String replaceMissingValueWith;
private final boolean injective;
@JsonCreator @JsonCreator
public LookupExtractionFn( public LookupExtractionFn(
@JsonProperty("lookup") final LookupExtractor lookup, @JsonProperty("lookup") final LookupExtractor lookup,
@JsonProperty("retainMissingValue") final boolean retainMissingValue, @JsonProperty("retainMissingValue") final boolean retainMissingValue,
@Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith, @Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith,
@JsonProperty("injective") final boolean injective, @JsonProperty("injective") final Boolean injective,
@JsonProperty("optimize") Boolean optimize @JsonProperty("optimize") final Boolean optimize
) )
{ {
super( super(
@ -63,13 +59,10 @@ public class LookupExtractionFn extends FunctionalExtraction
}, },
retainMissingValue, retainMissingValue,
replaceMissingValueWith, replaceMissingValueWith,
injective injective != null ? injective : lookup.isOneToOne()
); );
this.lookup = lookup; this.lookup = lookup;
this.optimize = optimize == null ? true : optimize; this.optimize = optimize == null ? true : optimize;
this.retainMissingValue = retainMissingValue;
this.injective = injective;
this.replaceMissingValueWith = replaceMissingValueWith;
} }
@ -175,9 +168,9 @@ public class LookupExtractionFn extends FunctionalExtraction
return "LookupExtractionFn{" + return "LookupExtractionFn{" +
"lookup=" + lookup + "lookup=" + lookup +
", optimize=" + optimize + ", optimize=" + optimize +
", retainMissingValue=" + retainMissingValue + ", retainMissingValue=" + isRetainMissingValue() +
", replaceMissingValueWith='" + replaceMissingValueWith + '\'' + ", replaceMissingValueWith='" + getReplaceMissingValueWith() + '\'' +
", injective=" + injective + ", injective=" + isInjective() +
'}'; '}';
} }
} }

View File

@ -20,7 +20,6 @@
package io.druid.query.lookup; package io.druid.query.lookup;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Strings; import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
@ -36,11 +35,11 @@ import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class LookupExtractionFnTest public class LookupExtractionFnTest
@ -53,30 +52,23 @@ public class LookupExtractionFnTest
ImmutableList.of( ImmutableList.of(
ImmutableSet.of(true, false), ImmutableSet.of(true, false),
ImmutableSet.of("", "MISSING VALUE"), ImmutableSet.of("", "MISSING VALUE"),
ImmutableSet.of(true, false) ImmutableSet.of(Optional.of(true), Optional.of(false), Optional.empty())
) )
), new Function<List<?>, Object[]>() ),
{ List::toArray
@Nullable
@Override
public Object[] apply(List<?> input)
{
return input.toArray();
}
}
); );
} }
private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper();
private final boolean retainMissing; private final boolean retainMissing;
private final String replaceMissing; private final String replaceMissing;
private final boolean injective; private final Boolean injective;
public LookupExtractionFnTest(boolean retainMissing, String replaceMissing, boolean injective) public LookupExtractionFnTest(boolean retainMissing, String replaceMissing, Optional<Boolean> injective)
{ {
this.replaceMissing = Strings.emptyToNull(replaceMissing); this.replaceMissing = Strings.emptyToNull(replaceMissing);
this.retainMissing = retainMissing; this.retainMissing = retainMissing;
this.injective = injective; this.injective = injective.orElse(null);
} }
@Test @Test
@ -136,7 +128,12 @@ public class LookupExtractionFnTest
Assert.assertEquals(retainMissing, lookupExtractionFn2.isRetainMissingValue()); Assert.assertEquals(retainMissing, lookupExtractionFn2.isRetainMissingValue());
Assert.assertEquals(replaceMissing, lookupExtractionFn2.getReplaceMissingValueWith()); Assert.assertEquals(replaceMissing, lookupExtractionFn2.getReplaceMissingValueWith());
Assert.assertEquals(injective, lookupExtractionFn2.isInjective());
if (injective == null) {
Assert.assertEquals(lookupExtractionFn2.getLookup().isOneToOne(), lookupExtractionFn2.isInjective());
} else {
Assert.assertEquals(injective, lookupExtractionFn2.isInjective());
}
Assert.assertArrayEquals(lookupExtractionFn.getCacheKey(), lookupExtractionFn2.getCacheKey()); Assert.assertArrayEquals(lookupExtractionFn.getCacheKey(), lookupExtractionFn2.getCacheKey());

View File

@ -28,6 +28,7 @@ import io.druid.query.extraction.ExtractionFn;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Objects;
public class RegisteredLookupExtractionFn implements ExtractionFn public class RegisteredLookupExtractionFn implements ExtractionFn
{ {
@ -38,7 +39,7 @@ public class RegisteredLookupExtractionFn implements ExtractionFn
private final String lookup; private final String lookup;
private final boolean retainMissingValue; private final boolean retainMissingValue;
private final String replaceMissingValueWith; private final String replaceMissingValueWith;
private final boolean injective; private final Boolean injective;
private final boolean optimize; private final boolean optimize;
@JsonCreator @JsonCreator
@ -47,7 +48,7 @@ public class RegisteredLookupExtractionFn implements ExtractionFn
@JsonProperty("lookup") String lookup, @JsonProperty("lookup") String lookup,
@JsonProperty("retainMissingValue") final boolean retainMissingValue, @JsonProperty("retainMissingValue") final boolean retainMissingValue,
@Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith, @Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith,
@JsonProperty("injective") final boolean injective, @JsonProperty("injective") final Boolean injective,
@JsonProperty("optimize") Boolean optimize @JsonProperty("optimize") Boolean optimize
) )
{ {
@ -79,7 +80,7 @@ public class RegisteredLookupExtractionFn implements ExtractionFn
} }
@JsonProperty("injective") @JsonProperty("injective")
public boolean isInjective() public Boolean isInjective()
{ {
return injective; return injective;
} }
@ -142,13 +143,17 @@ public class RegisteredLookupExtractionFn implements ExtractionFn
// http://www.javamex.com/tutorials/double_checked_locking.shtml // http://www.javamex.com/tutorials/double_checked_locking.shtml
synchronized (delegateLock) { synchronized (delegateLock) {
if (null == delegate) { if (null == delegate) {
final LookupExtractor factory = Preconditions.checkNotNull(
manager.get(getLookup()),
"Lookup [%s] not found",
getLookup()
).getLookupExtractorFactory().get();
delegate = new LookupExtractionFn( delegate = new LookupExtractionFn(
Preconditions.checkNotNull(manager.get(getLookup()), "Lookup [%s] not found", getLookup()) factory,
.getLookupExtractorFactory()
.get(),
isRetainMissingValue(), isRetainMissingValue(),
getReplaceMissingValueWith(), getReplaceMissingValueWith(),
isInjective(), injective == null ? factory.isOneToOne() : injective,
isOptimize() isOptimize()
); );
} }
@ -158,7 +163,7 @@ public class RegisteredLookupExtractionFn implements ExtractionFn
} }
@Override @Override
public boolean equals(Object o) public boolean equals(final Object o)
{ {
if (this == o) { if (this == o) {
return true; return true;
@ -166,35 +171,19 @@ public class RegisteredLookupExtractionFn implements ExtractionFn
if (o == null || getClass() != o.getClass()) { if (o == null || getClass() != o.getClass()) {
return false; return false;
} }
final RegisteredLookupExtractionFn that = (RegisteredLookupExtractionFn) o;
RegisteredLookupExtractionFn that = (RegisteredLookupExtractionFn) o; return retainMissingValue == that.retainMissingValue &&
optimize == that.optimize &&
if (isRetainMissingValue() != that.isRetainMissingValue()) { Objects.equals(lookup, that.lookup) &&
return false; Objects.equals(replaceMissingValueWith, that.replaceMissingValueWith) &&
} Objects.equals(injective, that.injective);
if (isInjective() != that.isInjective()) {
return false;
}
if (isOptimize() != that.isOptimize()) {
return false;
}
if (!getLookup().equals(that.getLookup())) {
return false;
}
return getReplaceMissingValueWith() != null
? getReplaceMissingValueWith().equals(that.getReplaceMissingValueWith())
: that.getReplaceMissingValueWith() == null;
} }
@Override @Override
public int hashCode() public int hashCode()
{ {
int result = getLookup().hashCode();
result = 31 * result + (isRetainMissingValue() ? 1 : 0); return Objects.hash(lookup, retainMissingValue, replaceMissingValueWith, injective, optimize);
result = 31 * result + (getReplaceMissingValueWith() != null ? getReplaceMissingValueWith().hashCode() : 0);
result = 31 * result + (isInjective() ? 1 : 0);
result = 31 * result + (isOptimize() ? 1 : 0);
return result;
} }
@Override @Override

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.extraction.MapLookupExtractor;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
@ -57,16 +58,39 @@ public class RegisteredLookupExtractionFnTest
LOOKUP_NAME, LOOKUP_NAME,
true, true,
null, null,
true, false,
false false
); );
EasyMock.verify(manager); EasyMock.verify(manager);
Assert.assertEquals(false, fn.isInjective());
Assert.assertEquals(ExtractionFn.ExtractionType.MANY_TO_ONE, fn.getExtractionType());
for (String orig : Arrays.asList("", "foo", "bat")) { for (String orig : Arrays.asList("", "foo", "bat")) {
Assert.assertEquals(LOOKUP_EXTRACTOR.apply(orig), fn.apply(orig)); Assert.assertEquals(LOOKUP_EXTRACTOR.apply(orig), fn.apply(orig));
} }
Assert.assertEquals("not in the map", fn.apply("not in the map")); Assert.assertEquals("not in the map", fn.apply("not in the map"));
} }
@Test
public void testInheritInjective()
{
final LookupReferencesManager manager = EasyMock.createStrictMock(LookupReferencesManager.class);
managerReturnsMap(manager);
EasyMock.replay(manager);
final RegisteredLookupExtractionFn fn = new RegisteredLookupExtractionFn(
manager,
LOOKUP_NAME,
true,
null,
null,
false
);
EasyMock.verify(manager);
Assert.assertNull(fn.isInjective());
Assert.assertEquals(ExtractionFn.ExtractionType.ONE_TO_ONE, fn.getExtractionType());
}
@Test @Test
public void testMissingDelegation() public void testMissingDelegation()