Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
40 changes: 39 additions & 1 deletion docs/content/querying/dimensionspecs.md
Original file line number Diff line number Diff line change
Expand Up @@ -252,7 +252,7 @@ It is illegal to set `retainMissingValue = true` and also specify a `replaceMiss

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 = false`).
A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is an incompatible change for configs that had relied on defaulting to false. Can you explain more on why this won't impact configs that were relying on that behavior?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@drcrallen lookups is an experimental feature, so changes like that are expected to happen.
I have set this to true, after had been tested.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LookupExtractor is not listed as experimental, and neither is the "optimize" flag (as far as I can tell).

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like it was added in 032d3bf which is in 0.9.0 As such it can change just fine, but the default for an experimental feature should be legacy behavior.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

well... it can be changed in 0.9.0 (which is not yet released). Does the default behavior need to change before we release 0.9.0?

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

Expand Down Expand Up @@ -393,3 +393,41 @@ or without setting "locale" (in this case, the current value of the default loca
"type" : "lower"
}
```

### Lookup DimensionSpecs

Lookup DimensionSpecs can be used to define directly a lookup implementation as dimension spec.
Generally speaking there is two different kind of lookups implementations.
The first kind is passed at the query time like `map` implementation.

```json
{
"type":"lookup",
"dimension":"dimensionName",
"outputName":"dimensionOutputName",
"replaceMissingValuesWith":"missing_value",
"retainMissingValue":false,
"lookup":{"type": "map", "map":{"key":"value"}, "isOneToOne":false}
}
```

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 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.

```json
{
"type":"lookup"
"dimension":"dimensionName"
"outputName":"dimensionOutputName"
"name":"lookupName"
}
```
Original file line number Diff line number Diff line change
Expand Up @@ -87,4 +87,5 @@ public List<String> unapply(@NotNull String value)
{
return reverseExtractionFunction.apply(value);
}

}
6 changes: 6 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -585,6 +585,12 @@
<version>1.3</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<version>1.0.4</version>
<scope>test</scope>
</dependency>
</dependencies>
</dependencyManagement>

Expand Down
5 changes: 5 additions & 0 deletions processing/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,11 @@
<artifactId>caliper</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@
@JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class),
@JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class),
@JsonSubTypes.Type(name = "regexFiltered", value = RegexFilteredDimensionSpec.class),
@JsonSubTypes.Type(name = "listFiltered", value = ListFilteredDimensionSpec.class)
@JsonSubTypes.Type(name = "listFiltered", value = ListFilteredDimensionSpec.class),
@JsonSubTypes.Type(name = "lookup", value = LookupDimensionSpec.class)
})
public interface DimensionSpec
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,235 @@
/*
* 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.dimension;

import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.metamx.common.StringUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.LookupExtractionFn;
import io.druid.query.extraction.LookupExtractor;
import io.druid.query.extraction.LookupReferencesManager;
import io.druid.query.filter.DimFilterCacheHelper;
import io.druid.segment.DimensionSelector;

import javax.annotation.Nullable;
import java.nio.ByteBuffer;

public class LookupDimensionSpec implements DimensionSpec
{
private static final byte CACHE_TYPE_ID = 0x4;

@JsonProperty
private final String dimension;

@JsonProperty
private final String outputName;

@JsonProperty
private final LookupExtractor lookup;

@JsonProperty
private final boolean retainMissingValue;

@JsonProperty
private final String replaceMissingValueWith;

@JsonProperty
private final String name;

@JsonProperty
private final boolean optimize;

private final LookupReferencesManager lookupReferencesManager;

@JsonCreator
public LookupDimensionSpec(
@JsonProperty("dimension") String dimension,
@JsonProperty("outputName") String outputName,
@JsonProperty("lookup") LookupExtractor lookup,
@JsonProperty("retainMissingValue") boolean retainMissingValue,
@JsonProperty("replaceMissingValueWith") String replaceMissingValueWith,
@JsonProperty("name") String name,
@JacksonInject LookupReferencesManager lookupReferencesManager,
@JsonProperty("optimize") Boolean optimize
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is missing from hashcode and equals

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

)
{
this.retainMissingValue = retainMissingValue;
this.optimize = optimize == null ? true : optimize;
this.replaceMissingValueWith = Strings.emptyToNull(replaceMissingValueWith);
this.dimension = Preconditions.checkNotNull(dimension, "dimension can not be Null");
this.outputName = Preconditions.checkNotNull(outputName, "outputName can not be Null");
this.lookupReferencesManager = lookupReferencesManager;
this.name = name;
this.lookup = lookup;
Preconditions.checkArgument(
Strings.isNullOrEmpty(name) ^ (lookup == null),
"name [%s] and lookup [%s] are mutually exclusive please provide either a name or a lookup", name, lookup
);

if (!Strings.isNullOrEmpty(name)) {
Preconditions.checkNotNull(
this.lookupReferencesManager,
"The system is not configured to allow for lookups, please read about configuring a lookup manager in the docs"
);
}
}

@Override
@JsonProperty
public String getDimension()
{
return dimension;
}

@Override
@JsonProperty
public String getOutputName()
{
return outputName;
}

@JsonProperty
@Nullable
public LookupExtractor getLookup()
{
return lookup;
}

@JsonProperty
@Nullable
public String getName()
{
return name;
}

@Override
public ExtractionFn getExtractionFn()
{
final LookupExtractor lookupExtractor = Strings.isNullOrEmpty(name)
? this.lookup
: Preconditions.checkNotNull(
this.lookupReferencesManager.get(name).get(),
"can not find lookup with name [%s]",
name
);
return new LookupExtractionFn(
lookupExtractor,
retainMissingValue,
replaceMissingValueWith,
lookupExtractor.isOneToOne(),
optimize
);

}

@Override
public DimensionSelector decorate(DimensionSelector selector)
{
return selector;
}

@Override
public byte[] getCacheKey()
{
byte[] dimensionBytes = StringUtils.toUtf8(dimension);
byte[] dimExtractionFnBytes = Strings.isNullOrEmpty(name)
? getLookup().getCacheKey()
: StringUtils.toUtf8(name);
byte[] outputNameBytes = StringUtils.toUtf8(outputName);
byte[] replaceWithBytes = StringUtils.toUtf8(Strings.nullToEmpty(replaceMissingValueWith));


return ByteBuffer.allocate(6
+ dimensionBytes.length
+ outputNameBytes.length
+ dimExtractionFnBytes.length
+ replaceWithBytes.length)
.put(CACHE_TYPE_ID)
.put(dimensionBytes)
.put(DimFilterCacheHelper.STRING_SEPARATOR)
.put(outputNameBytes)
.put(DimFilterCacheHelper.STRING_SEPARATOR)
.put(dimExtractionFnBytes)
.put(DimFilterCacheHelper.STRING_SEPARATOR)
.put(replaceWithBytes)
.put(DimFilterCacheHelper.STRING_SEPARATOR)
.put(retainMissingValue == true ? (byte) 1 : (byte) 0)
.array();
}

@Override
public boolean preservesOrdering()
{
return getExtractionFn().preservesOrdering();
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (!(o instanceof LookupDimensionSpec)) {
return false;
}

LookupDimensionSpec that = (LookupDimensionSpec) o;

if (retainMissingValue != that.retainMissingValue) {
return false;
}
if (optimize != that.optimize) {
return false;
}
if (!getDimension().equals(that.getDimension())) {
return false;
}
if (!getOutputName().equals(that.getOutputName())) {
return false;
}
if (getLookup() != null ? !getLookup().equals(that.getLookup()) : that.getLookup() != null) {
return false;
}
if (replaceMissingValueWith != null
? !replaceMissingValueWith.equals(that.replaceMissingValueWith)
: that.replaceMissingValueWith != null) {
return false;
}
return getName() != null ? getName().equals(that.getName()) : that.getName() == null;

}

@Override
public int hashCode()
{
int result = getDimension().hashCode();
result = 31 * result + getOutputName().hashCode();
result = 31 * result + (getLookup() != null ? getLookup().hashCode() : 0);
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 + (optimize ? 1 : 0);
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,10 +65,6 @@ public DimensionSelector decorate(final DimensionSelector selector)
return selector;
}

if (selector == null) {
return selector;
}

int count = 0;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why was this removed?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it is not needed

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can be more clear as in : Code is duplicated immediately prior to here.

The duplicate code is hidden in the fold

final Map<Integer,Integer> forwardMapping = new HashMap<>();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ public String apply(String input)
injective
);
this.lookup = lookup;
this.optimize = optimize == null ? false : optimize;
this.optimize = optimize == null ? true : optimize;
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,11 @@ public Map<String, List<String>> unapplyAll(Iterable<String> values)
* @return A byte array that can be used to uniquely identify if results of a prior lookup can use the cached values
*/

@Nullable
public abstract byte[] getCacheKey();

// make this abstract again once @drcrallen fix the metmax lookup implementation.
public boolean isOneToOne()
{
return false;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why is this hard coded to false?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it is a default implementation, to avoid breaking up your experimental implementation of lookupExtractor.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@drcrallen i dropped a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

((as per conversation)) suggest adding docs to indicate when and if this should be removed or changed in the future.

}
}
Loading