Skip to content

Commit

Permalink
Add config to Re2JRegexpType
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Feb 19, 2020
1 parent ae407a1 commit b839a3d
Show file tree
Hide file tree
Showing 8 changed files with 26 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ public final class MetadataManager
private final ColumnPropertyManager columnPropertyManager;
private final AnalyzePropertyManager analyzePropertyManager;
private final TransactionManager transactionManager;
private final TypeRegistry typeRegistry = new TypeRegistry();
private final TypeRegistry typeRegistry;

private final ConcurrentMap<String, BlockEncoding> blockEncodings = new ConcurrentHashMap<>();
private final ConcurrentMap<QueryId, QueryCatalogs> catalogsByQueryId = new ConcurrentHashMap<>();
Expand All @@ -174,6 +174,7 @@ public MetadataManager(
AnalyzePropertyManager analyzePropertyManager,
TransactionManager transactionManager)
{
typeRegistry = new TypeRegistry(featuresConfig);
functions = new FunctionRegistry(this, featuresConfig);
functionResolver = new FunctionResolver(this);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,11 @@
import io.prestosql.spi.type.TypeParameter;
import io.prestosql.spi.type.TypeSignature;
import io.prestosql.spi.type.TypeSignatureParameter;
import io.prestosql.sql.analyzer.FeaturesConfig;
import io.prestosql.sql.parser.SqlParser;
import io.prestosql.type.CharParametricType;
import io.prestosql.type.DecimalParametricType;
import io.prestosql.type.Re2JRegexpType;
import io.prestosql.type.VarcharParametricType;

import javax.annotation.concurrent.ThreadSafe;
Expand Down Expand Up @@ -72,7 +74,6 @@
import static io.prestosql.type.JsonType.JSON;
import static io.prestosql.type.LikePatternType.LIKE_PATTERN;
import static io.prestosql.type.MapParametricType.MAP;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP;
import static io.prestosql.type.RowParametricType.ROW;
import static io.prestosql.type.UnknownType.UNKNOWN;
import static io.prestosql.type.UuidType.UUID;
Expand All @@ -89,7 +90,7 @@ final class TypeRegistry

private final Cache<TypeSignature, Type> parametricTypeCache;

public TypeRegistry()
public TypeRegistry(FeaturesConfig featuresConfig)
{
// Manually register UNKNOWN type without a verifyTypeClass call since it is a special type that cannot be used by functions
this.types.put(UNKNOWN.getTypeSignature(), UNKNOWN);
Expand All @@ -115,7 +116,7 @@ public TypeRegistry()
addType(SET_DIGEST);
addType(P4_HYPER_LOG_LOG);
addType(JONI_REGEXP);
addType(RE2J_REGEXP);
addType(new Re2JRegexpType(featuresConfig.getRe2JDfaStatesLimit(), featuresConfig.getRe2JDfaRetries()));
addType(LIKE_PATTERN);
addType(JSON_PATH);
addType(COLOR);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
import static io.prestosql.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.type.Chars.padSpaces;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP_SIGNATURE;
import static io.prestosql.util.Reflection.methodHandle;
import static java.lang.invoke.MethodHandles.insertArguments;
import static java.util.Collections.emptyList;
Expand All @@ -57,7 +57,7 @@ public static SqlOperator castCharToRe2JRegexp(int dfaStatesLimit, int dfaRetrie

private Re2JCastToRegexpFunction(String sourceType, int dfaStatesLimit, int dfaRetries, boolean padSpaces)
{
super(CAST, emptyList(), emptyList(), RE2J_REGEXP.getTypeSignature(), ImmutableList.of(parseTypeSignature(sourceType, ImmutableSet.of("x"))), false);
super(CAST, emptyList(), emptyList(), RE2J_REGEXP_SIGNATURE, ImmutableList.of(parseTypeSignature(sourceType, ImmutableSet.of("x"))), false);
this.dfaStatesLimit = dfaStatesLimit;
this.dfaRetries = dfaRetries;
this.padSpaces = padSpaces;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,6 @@
import static io.prestosql.type.JoniRegexpType.JONI_REGEXP;
import static io.prestosql.type.JsonPathType.JSON_PATH;
import static io.prestosql.type.LikePatternType.LIKE_PATTERN;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP;
import static io.prestosql.type.UnknownType.UNKNOWN;
import static java.lang.Float.intBitsToFloat;
import static java.lang.Math.toIntExact;
Expand Down Expand Up @@ -226,7 +225,7 @@ public Expression toExpression(Object object, Type type)
return new GenericLiteral(type.getBaseName(), ((JoniRegexp) object).pattern().toStringUtf8());
}

if (type.equals(RE2J_REGEXP)) {
if (type instanceof Re2JRegexpType) {
return new GenericLiteral(Re2JRegexpType.NAME, ((Re2JRegexp) object).pattern());
}

Expand Down
11 changes: 8 additions & 3 deletions presto-main/src/main/java/io/prestosql/type/Re2JRegexpType.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,12 +26,17 @@
public class Re2JRegexpType
extends AbstractType
{
public static final Re2JRegexpType RE2J_REGEXP = new Re2JRegexpType();
public static final String NAME = "Re2JRegExp";
public static final TypeSignature RE2J_REGEXP_SIGNATURE = new TypeSignature(NAME);

public Re2JRegexpType()
private final int dfaStatesLimit;
private final int dfaRetries;

public Re2JRegexpType(int dfaStatesLimit, int dfaRetries)
{
super(new TypeSignature(NAME), Re2JRegexp.class);
super(RE2J_REGEXP_SIGNATURE, Re2JRegexp.class);
this.dfaStatesLimit = dfaStatesLimit;
this.dfaRetries = dfaRetries;
}

@Override
Expand Down
6 changes: 3 additions & 3 deletions presto-main/src/main/java/io/prestosql/type/TypeCoercion.java
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
import static io.prestosql.type.CodePointsType.CODE_POINTS;
import static io.prestosql.type.JoniRegexpType.JONI_REGEXP;
import static io.prestosql.type.JsonPathType.JSON_PATH;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP_SIGNATURE;
import static java.util.Objects.requireNonNull;

public final class TypeCoercion
Expand Down Expand Up @@ -420,7 +420,7 @@ public Optional<Type> coerceTypeBase(Type sourceType, String resultTypeBase)
case JoniRegexpType.NAME:
return Optional.of(JONI_REGEXP);
case Re2JRegexpType.NAME:
return Optional.of(RE2J_REGEXP);
return Optional.of(lookupType.apply(RE2J_REGEXP_SIGNATURE));
case JsonPathType.NAME:
return Optional.of(JSON_PATH);
case CodePointsType.NAME:
Expand All @@ -436,7 +436,7 @@ public Optional<Type> coerceTypeBase(Type sourceType, String resultTypeBase)
case JoniRegexpType.NAME:
return Optional.of(JONI_REGEXP);
case Re2JRegexpType.NAME:
return Optional.of(RE2J_REGEXP);
return Optional.of(lookupType.apply(RE2J_REGEXP_SIGNATURE));
case JsonPathType.NAME:
return Optional.of(JSON_PATH);
case CodePointsType.NAME:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
import static io.prestosql.type.JoniRegexpType.JONI_REGEXP;
import static io.prestosql.type.JsonPathType.JSON_PATH;
import static io.prestosql.type.LikePatternType.LIKE_PATTERN;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP_SIGNATURE;
import static io.prestosql.type.UnknownType.UNKNOWN;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.testng.Assert.assertEquals;
Expand All @@ -57,7 +57,7 @@ public void testEncode()
assertEncode(utf8Slice("hello"), VARBINARY, literalVarbinary("hello".getBytes(UTF_8)));
assertEncode(castVarcharToJoniRegexp(utf8Slice("[a-z]")), LIKE_PATTERN, "LikePattern '[a-z]'");
assertEncode(castVarcharToJoniRegexp(utf8Slice("[a-z]")), JONI_REGEXP, "JoniRegExp '[a-z]'");
assertEncode(castVarcharToRe2JRegexp(utf8Slice("[a-z]")), RE2J_REGEXP, "Re2JRegExp '[a-z]'");
assertEncode(castVarcharToRe2JRegexp(utf8Slice("[a-z]")), metadata.getType(RE2J_REGEXP_SIGNATURE), "Re2JRegExp '[a-z]'");
assertEncode(castVarcharToJsonPath(utf8Slice("$.foo")), JSON_PATH, "JsonPath '$.foo'");
assertEncode(castVarcharToCodePoints(utf8Slice("hello")), CODE_POINTS, "CodePoints 'hello'");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@
import static io.prestosql.spi.type.VarcharType.createVarcharType;
import static io.prestosql.type.JoniRegexpType.JONI_REGEXP;
import static io.prestosql.type.JsonPathType.JSON_PATH;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP;
import static io.prestosql.type.Re2JRegexpType.RE2J_REGEXP_SIGNATURE;
import static io.prestosql.type.UnknownType.UNKNOWN;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
Expand All @@ -58,6 +58,7 @@
public class TestTypeCoercion
{
private final Metadata metadata = createTestMetadataManager();
private final Type re2jType = metadata.getType(RE2J_REGEXP_SIGNATURE);
private final TypeCoercion typeCoercion = new TypeCoercion(metadata::getType);

@Test
Expand Down Expand Up @@ -122,7 +123,7 @@ public void testTypeCompatibility()
assertThat(TIME, TIME_WITH_TIME_ZONE).hasCommonSuperType(TIME_WITH_TIME_ZONE).canCoerceFirstToSecondOnly();
assertThat(TIMESTAMP, TIMESTAMP_WITH_TIME_ZONE).hasCommonSuperType(TIMESTAMP_WITH_TIME_ZONE).canCoerceFirstToSecondOnly();
assertThat(VARCHAR, JONI_REGEXP).hasCommonSuperType(JONI_REGEXP).canCoerceFirstToSecondOnly();
assertThat(VARCHAR, RE2J_REGEXP).hasCommonSuperType(RE2J_REGEXP).canCoerceFirstToSecondOnly();
assertThat(VARCHAR, re2jType).hasCommonSuperType(re2jType).canCoerceFirstToSecondOnly();
assertThat(VARCHAR, JSON_PATH).hasCommonSuperType(JSON_PATH).canCoerceFirstToSecondOnly();

assertThat(REAL, DOUBLE).hasCommonSuperType(DOUBLE).canCoerceFirstToSecondOnly();
Expand Down Expand Up @@ -193,7 +194,7 @@ public void testTypeCompatibility()

assertThat(createCharType(42), JONI_REGEXP).hasCommonSuperType(JONI_REGEXP).canCoerceFirstToSecondOnly();
assertThat(createCharType(42), JSON_PATH).hasCommonSuperType(JSON_PATH).canCoerceFirstToSecondOnly();
assertThat(createCharType(42), RE2J_REGEXP).hasCommonSuperType(RE2J_REGEXP).canCoerceFirstToSecondOnly();
assertThat(createCharType(42), re2jType).hasCommonSuperType(re2jType).canCoerceFirstToSecondOnly();

assertThat(anonymousRow(createVarcharType(2)), anonymousRow(createVarcharType(5)))
.hasCommonSuperType(anonymousRow(createVarcharType(5)))
Expand Down

0 comments on commit b839a3d

Please sign in to comment.