Add SQL REGEXP_EXTRACT function; add "index" to "regex" extractionFn. (#4055)

* Add SQL REGEXP_EXTRACT function; add "index" to "regex" extractionFn.

* Fix tests.
This commit is contained in:
Gian Merlino 2017-03-24 17:38:36 -07:00 committed by Jonathan Wei
parent de081c711b
commit dd6c0ab509
8 changed files with 233 additions and 19 deletions

View File

@ -144,7 +144,9 @@ If there is no match, it returns the dimension value as is.
```json
{
"type" : "regex", "expr" : <regular_expression>,
"type" : "regex",
"expr" : <regular_expression>,
"index" : <group to extract, default 1>
"replaceMissingValue" : true,
"replaceMissingValueWith" : "foobar"
}
@ -153,6 +155,9 @@ If there is no match, it returns the dimension value as is.
For example, using `"expr" : "(\\w\\w\\w).*"` will transform
`'Monday'`, `'Tuesday'`, `'Wednesday'` into `'Mon'`, `'Tue'`, `'Wed'`.
If "index" is set, it will control which group from the match to extract. Index zero extracts the string matching the
entire pattern.
If the `replaceMissingValue` property is true, the extraction function will transform dimension values that do not match the regex pattern to a user-specified String. Default value is `false`.
The `replaceMissingValueWith` property sets the String that unmatched dimension values will be replaced with, if `replaceMissingValue` is true. If `replaceMissingValueWith` is not specified, unmatched dimension values will be replaced with nulls.

View File

@ -23,9 +23,11 @@ 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.google.common.primitives.Ints;
import io.druid.java.util.common.StringUtils;
import java.nio.ByteBuffer;
import java.util.Objects;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -36,6 +38,7 @@ public class RegexDimExtractionFn extends DimExtractionFn
private static final byte CACHE_KEY_SEPARATOR = (byte) 0xFF;
private final String expr;
private final int index;
private final Pattern pattern;
private final boolean replaceMissingValue;
private final String replaceMissingValueWith;
@ -43,6 +46,7 @@ public class RegexDimExtractionFn extends DimExtractionFn
@JsonCreator
public RegexDimExtractionFn(
@JsonProperty("expr") String expr,
@JsonProperty("index") Integer index,
@JsonProperty("replaceMissingValue") Boolean replaceMissingValue,
@JsonProperty("replaceMissingValueWith") String replaceMissingValueWith
)
@ -50,15 +54,26 @@ public class RegexDimExtractionFn extends DimExtractionFn
Preconditions.checkNotNull(expr, "expr must not be null");
this.expr = expr;
this.index = index == null ? 1 : index;
this.pattern = Pattern.compile(expr);
this.replaceMissingValue = replaceMissingValue == null ? false : replaceMissingValue;
this.replaceMissingValueWith = replaceMissingValueWith;
}
public RegexDimExtractionFn(
String expr,
Boolean replaceMissingValue,
String replaceMissingValueWith
)
{
this(expr, null, replaceMissingValue, replaceMissingValueWith);
}
@Override
public byte[] getCacheKey()
{
byte[] exprBytes = StringUtils.toUtf8(expr);
byte[] indexBytes = Ints.toByteArray(index);
byte[] replaceBytes = replaceMissingValue ? new byte[]{1} : new byte[]{0};
byte[] replaceStrBytes;
if (replaceMissingValueWith == null) {
@ -69,6 +84,7 @@ public class RegexDimExtractionFn extends DimExtractionFn
int totalLen = 1
+ exprBytes.length
+ indexBytes.length
+ replaceBytes.length
+ replaceStrBytes.length; // fields
totalLen += 2; // separators
@ -77,6 +93,7 @@ public class RegexDimExtractionFn extends DimExtractionFn
.put(ExtractionCacheHelper.CACHE_TYPE_ID_REGEX)
.put(exprBytes)
.put(CACHE_KEY_SEPARATOR)
.put(indexBytes)
.put(replaceStrBytes)
.put(CACHE_KEY_SEPARATOR)
.put(replaceBytes)
@ -89,7 +106,7 @@ public class RegexDimExtractionFn extends DimExtractionFn
final String retVal;
final Matcher matcher = pattern.matcher(Strings.nullToEmpty(dimValue));
if (matcher.find()) {
retVal = matcher.group(1);
retVal = matcher.group(index);
} else {
retVal = replaceMissingValue ? replaceMissingValueWith : dimValue;
}
@ -102,6 +119,12 @@ public class RegexDimExtractionFn extends DimExtractionFn
return expr;
}
@JsonProperty
public int getIndex()
{
return index;
}
@JsonProperty("replaceMissingValue")
public boolean isReplaceMissingValue()
{
@ -129,11 +152,11 @@ public class RegexDimExtractionFn extends DimExtractionFn
@Override
public String toString()
{
return String.format("regex(%s)", expr);
return String.format("regex(/%s/, %d)", expr, index);
}
@Override
public boolean equals(Object o)
public boolean equals(final Object o)
{
if (this == o) {
return true;
@ -141,19 +164,16 @@ public class RegexDimExtractionFn extends DimExtractionFn
if (o == null || getClass() != o.getClass()) {
return false;
}
RegexDimExtractionFn that = (RegexDimExtractionFn) o;
if (!expr.equals(that.expr)) {
return false;
}
return true;
final RegexDimExtractionFn that = (RegexDimExtractionFn) o;
return index == that.index &&
replaceMissingValue == that.replaceMissingValue &&
Objects.equals(expr, that.expr) &&
Objects.equals(replaceMissingValueWith, that.replaceMissingValueWith);
}
@Override
public int hashCode()
{
return expr.hashCode();
return Objects.hash(expr, index, replaceMissingValue, replaceMissingValueWith);
}
}

View File

@ -87,6 +87,43 @@ public class RegexDimExtractionFnTest
Assert.assertEquals(expected, extracted);
}
@Test
public void testIndexZero()
{
String regex = "/([^/]{4})/";
ExtractionFn extractionFn = new RegexDimExtractionFn(regex, 0, true, null);
Set<String> extracted = Sets.newLinkedHashSet();
for (String path : paths) {
extracted.add(extractionFn.apply(path));
}
Set<String> expected = Sets.newLinkedHashSet(
ImmutableList.of("/prod/", "/demo/", "/dash/")
);
Assert.assertEquals(expected, extracted);
}
@Test
public void testIndexTwo()
{
String regex = "^/([^/]+)/([^/]+)";
ExtractionFn extractionFn = new RegexDimExtractionFn(regex, 2, true, null);
Set<String> extracted = Sets.newLinkedHashSet();
for (String path : paths) {
extracted.add(extractionFn.apply(path));
}
Set<String> expected = Sets.newLinkedHashSet(
ImmutableList.of(
"prod", "demo",
"aloe", "baloo"
)
);
Assert.assertEquals(expected, extracted);
}
@Test
public void testStringExtraction()
{
@ -164,8 +201,10 @@ public class RegexDimExtractionFnTest
byte[] cacheKey = extractionFn.getCacheKey();
byte[] expectedCacheKey = new byte[]{
0x01, 0x28, 0x61, 0x5C, 0x77, 0x2A, 0x29, (byte) 0xFF,
0x66, 0x6F, 0x6F, 0x62, 0x61, 0x72, (byte) 0xFF, 0x01
0x01, 0x28, 0x61, 0x5C, 0x77, 0x2A, 0x29, (byte) 0xFF, // expr
0x00, 0x00, 0x00, 0x01, // index
0x66, 0x6F, 0x6F, 0x62, 0x61, 0x72, (byte) 0xFF, // replaceMissingValueWith
0x01 // replaceMissingValue
};
Assert.assertArrayEquals(expectedCacheKey, cacheKey);
@ -181,7 +220,12 @@ public class RegexDimExtractionFnTest
Assert.assertEquals(expected2, extracted2);
cacheKey = nullExtractionFn.getCacheKey();
expectedCacheKey = new byte[]{0x01, 0x28, 0x61, 0x5C, 0x77, 0x2A, 0x29, (byte) 0xFF, (byte) 0xFF, 0x01};
expectedCacheKey = new byte[]{
0x01, 0x28, 0x61, 0x5C, 0x77, 0x2A, 0x29, (byte) 0xFF, // expr
0x00, 0x00, 0x00, 0x01, // index
(byte) 0xFF, // replaceMissingValueWith
0x01 // replaceMissingValue
};
Assert.assertArrayEquals(expectedCacheKey, cacheKey);
}

View File

@ -47,7 +47,7 @@ public class SelectorDimFilterTest
SelectorDimFilter selectorDimFilter2 = new SelectorDimFilter("abc", "d", regexFn);
Assert.assertEquals("abc = d", selectorDimFilter.toString());
Assert.assertEquals("regex(.*)(abc) = d", selectorDimFilter2.toString());
Assert.assertEquals("regex(/.*/, 1)(abc) = d", selectorDimFilter2.toString());
}
@Test

View File

@ -107,7 +107,7 @@ public class DimensionSelectorHavingSpecTest
String expected = "DimensionSelectorHavingSpec{" +
"dimension='gender'," +
" value='m'," +
" extractionFunction='regex(^([^,]*),)'}";
" extractionFunction='regex(/^([^,]*),/, 1)'}";
Assert.assertEquals(new DimensionSelectorHavingSpec("gender", "m", extractionFn).toString(), expected);
expected = "DimensionSelectorHavingSpec{" +

View File

@ -0,0 +1,105 @@
/*
* 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.sql.calcite.expression;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.RegexDimExtractionFn;
import io.druid.sql.calcite.planner.DruidOperatorTable;
import io.druid.sql.calcite.planner.PlannerContext;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import java.util.List;
public class RegexpExtractExtractionOperator implements SqlExtractionOperator
{
private static final String NAME = "REGEXP_EXTRACT";
private static final SqlFunction SQL_FUNCTION = new RegexpExtractSqlFunction();
@Override
public SqlFunction calciteFunction()
{
return SQL_FUNCTION;
}
@Override
public RowExtraction convert(
final DruidOperatorTable operatorTable,
final PlannerContext plannerContext,
final List<String> rowOrder,
final RexNode expression
)
{
final RexCall call = (RexCall) expression;
final RowExtraction rex = Expressions.toRowExtraction(
operatorTable,
plannerContext,
rowOrder,
call.getOperands().get(0)
);
if (rex == null) {
return null;
}
final String pattern = RexLiteral.stringValue(call.getOperands().get(1));
final int index = call.getOperands().size() >= 3 ? RexLiteral.intValue(call.getOperands().get(2)) : 0;
final ExtractionFn extractionFn = new RegexDimExtractionFn(pattern, index, true, null);
return RowExtraction.of(
rex.getColumn(),
ExtractionFns.compose(extractionFn, rex.getExtractionFn())
);
}
private static class RegexpExtractSqlFunction extends SqlFunction
{
private static final String SIGNATURE1 = "'" + NAME + "(subject, pattern)'\n";
private static final String SIGNATURE2 = "'" + NAME + "(subject, pattern, index)'\n";
RegexpExtractSqlFunction()
{
super(
NAME,
SqlKind.OTHER_FUNCTION,
ReturnTypes.explicit(SqlTypeName.VARCHAR),
null,
OperandTypes.or(
OperandTypes.and(
OperandTypes.sequence(SIGNATURE1, OperandTypes.CHARACTER, OperandTypes.LITERAL),
OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER)
),
OperandTypes.and(
OperandTypes.sequence(SIGNATURE2, OperandTypes.CHARACTER, OperandTypes.LITERAL, OperandTypes.LITERAL),
OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER)
)
),
SqlFunctionCategory.STRING
);
}
}
}

View File

@ -40,6 +40,7 @@ import io.druid.sql.calcite.expression.CharacterLengthExtractionOperator;
import io.druid.sql.calcite.expression.ExtractExtractionOperator;
import io.druid.sql.calcite.expression.FloorExtractionOperator;
import io.druid.sql.calcite.expression.LookupExtractionOperator;
import io.druid.sql.calcite.expression.RegexpExtractExtractionOperator;
import io.druid.sql.calcite.expression.SqlExtractionOperator;
import io.druid.sql.calcite.expression.SubstringExtractionOperator;
import io.druid.sql.calcite.planner.Calcites;
@ -64,7 +65,8 @@ public class SqlModule implements Module
ExtractExtractionOperator.class,
FloorExtractionOperator.class,
LookupExtractionOperator.class,
SubstringExtractionOperator.class
SubstringExtractionOperator.class,
RegexpExtractExtractionOperator.class
);
private static final String PROPERTY_SQL_ENABLE = "druid.sql.enable";

View File

@ -53,6 +53,7 @@ import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.extraction.BucketExtractionFn;
import io.druid.query.extraction.CascadeExtractionFn;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.RegexDimExtractionFn;
import io.druid.query.extraction.StrlenExtractionFn;
import io.druid.query.extraction.SubstringDimExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
@ -2792,6 +2793,43 @@ public class CalciteQueryTest
);
}
@Test
public void testRegexpExtract() throws Exception
{
testQuery(
"SELECT DISTINCT REGEXP_EXTRACT(dim1, '^.'), REGEXP_EXTRACT(dim1, '^(.)', 1) FROM foo",
ImmutableList.<Query>of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
DIMS(
new ExtractionDimensionSpec(
"dim1",
"d0",
new RegexDimExtractionFn("^.", 0, true, null)
),
new ExtractionDimensionSpec(
"dim1",
"d1",
new RegexDimExtractionFn("^(.)", 1, true, null)
)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{"", ""},
new Object[]{"1", "1"},
new Object[]{"2", "2"},
new Object[]{"a", "a"},
new Object[]{"d", "d"}
)
);
}
@Test
public void testGroupBySortPushDown() throws Exception
{