Skip to content

Commit

Permalink
Add enum operators
Browse files Browse the repository at this point in the history
Support common operators like `=` on enum types
  • Loading branch information
daniel-ohayon authored and Rongrong Zhong committed Sep 29, 2020
1 parent 9fe32b5 commit cef58ff
Show file tree
Hide file tree
Showing 8 changed files with 721 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@

import java.util.concurrent.TimeUnit;

import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;

@SuppressWarnings("MethodMayBeStatic")
@State(Scope.Thread)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
Expand All @@ -42,13 +44,13 @@ public class BenchmarkTypeSignatureParsing
@Benchmark
public void parseRowTypeSignature(BenchmarkData data)
{
TypeSignature.parseTypeSignature(data.signature);
parseTypeSignature(data.signature);
}

@Benchmark
public void parseRowTypeSignatureWithEnums(BenchmarkData data)
{
TypeSignature.parseTypeSignature(data.enumSignature);
parseTypeSignature(data.enumSignature);
}

@State(Scope.Thread)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,13 +192,15 @@
import com.facebook.presto.type.DateTimeOperators;
import com.facebook.presto.type.DecimalOperators;
import com.facebook.presto.type.DoubleOperators;
import com.facebook.presto.type.EnumCasts;
import com.facebook.presto.type.HyperLogLogOperators;
import com.facebook.presto.type.IntegerOperators;
import com.facebook.presto.type.IntervalDayTimeOperators;
import com.facebook.presto.type.IntervalYearMonthOperators;
import com.facebook.presto.type.IpAddressOperators;
import com.facebook.presto.type.IpPrefixOperators;
import com.facebook.presto.type.LikeFunctions;
import com.facebook.presto.type.LongEnumOperators;
import com.facebook.presto.type.QuantileDigestOperators;
import com.facebook.presto.type.RealOperators;
import com.facebook.presto.type.SmallintOperators;
Expand All @@ -210,6 +212,7 @@
import com.facebook.presto.type.TinyintOperators;
import com.facebook.presto.type.UnknownOperators;
import com.facebook.presto.type.VarbinaryOperators;
import com.facebook.presto.type.VarcharEnumOperators;
import com.facebook.presto.type.VarcharOperators;
import com.facebook.presto.type.khyperloglog.KHyperLogLogAggregationFunction;
import com.facebook.presto.type.khyperloglog.KHyperLogLogFunctions;
Expand Down Expand Up @@ -707,7 +710,10 @@ public BuiltInFunctionNamespaceManager(
.function(MergeTDigestFunction.MERGE)
.sqlInvokedScalar(MapNormalizeFunction.class)
.sqlInvokedScalars(ArrayArithmeticFunctions.class)
.scalar(DynamicFilterPlaceholderFunction.class);
.scalar(DynamicFilterPlaceholderFunction.class)
.scalars(EnumCasts.class)
.scalars(LongEnumOperators.class)
.scalars(VarcharEnumOperators.class);

switch (featuresConfig.getRegexLibrary()) {
case JONI:
Expand Down
111 changes: 111 additions & 0 deletions presto-main/src/main/java/com/facebook/presto/type/EnumCasts.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
/*
* Licensed 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 com.facebook.presto.type;

import com.facebook.presto.common.type.LongEnumType;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.VarcharEnumType;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.function.ScalarOperator;
import com.facebook.presto.spi.function.SqlType;
import com.facebook.presto.spi.function.TypeParameter;
import io.airlift.slice.Slice;

import static com.facebook.presto.common.function.OperatorType.CAST;
import static com.facebook.presto.common.type.StandardTypes.BIGINT;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_CAST_ARGUMENT;

public final class EnumCasts
{
private EnumCasts()
{
}

@ScalarOperator(CAST)
@TypeParameter(value = "T", boundedBy = VarcharEnumType.class)
@SqlType("T")
public static Slice castVarcharToEnum(@TypeParameter("T") Type enumType, @SqlType(StandardTypes.VARCHAR) Slice value)
{
if (!(((VarcharEnumType) enumType).getEnumMap().values().contains(value.toStringUtf8()))) {
throw new PrestoException(INVALID_CAST_ARGUMENT,
String.format(
"No value '%s' in enum '%s'",
value.toStringUtf8(),
enumType.getTypeSignature().getBase()));
}
return value;
}

@ScalarOperator(CAST)
@TypeParameter(value = "T", boundedBy = VarcharEnumType.class)
@SqlType(StandardTypes.VARCHAR)
public static Slice castEnumToVarchar(@SqlType("T") Slice value)
{
return value;
}

@ScalarOperator(CAST)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType("T")
public static long castBigintToEnum(@TypeParameter("T") Type enumType, @SqlType(BIGINT) long value)
{
return castLongToEnum(enumType, value);
}

@ScalarOperator(CAST)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType("T")
public static long castIntegerToEnum(@TypeParameter("T") Type enumType, @SqlType(StandardTypes.INTEGER) long value)
{
return castLongToEnum(enumType, value);
}

@ScalarOperator(CAST)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType("T")
public static long castSmallintToEnum(@TypeParameter("T") Type enumType, @SqlType(StandardTypes.SMALLINT) long value)
{
return castLongToEnum(enumType, value);
}

@ScalarOperator(CAST)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType("T")
public static long castTinyintToEnum(@TypeParameter("T") Type enumType, @SqlType(StandardTypes.TINYINT) long value)
{
return castLongToEnum(enumType, value);
}

private static long castLongToEnum(Type enumType, long value)
{
if (!((LongEnumType) enumType).getEnumMap().values().contains(value)) {
throw new PrestoException(INVALID_CAST_ARGUMENT,
String.format(
"No value '%d' in enum '%s'",
value,
enumType.getTypeSignature().getBase()));
}
return value;
}

@ScalarOperator(CAST)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(BIGINT)
public static long castEnumToBigint(@SqlType("T") long value)
{
return value;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
/*
* Licensed 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 com.facebook.presto.type;

import com.facebook.presto.common.type.AbstractLongType;
import com.facebook.presto.common.type.LongEnumType;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.spi.function.IsNull;
import com.facebook.presto.spi.function.ScalarOperator;
import com.facebook.presto.spi.function.SqlNullable;
import com.facebook.presto.spi.function.SqlType;
import com.facebook.presto.spi.function.TypeParameter;
import io.airlift.slice.XxHash64;

import static com.facebook.presto.common.function.OperatorType.BETWEEN;
import static com.facebook.presto.common.function.OperatorType.EQUAL;
import static com.facebook.presto.common.function.OperatorType.GREATER_THAN;
import static com.facebook.presto.common.function.OperatorType.GREATER_THAN_OR_EQUAL;
import static com.facebook.presto.common.function.OperatorType.HASH_CODE;
import static com.facebook.presto.common.function.OperatorType.INDETERMINATE;
import static com.facebook.presto.common.function.OperatorType.IS_DISTINCT_FROM;
import static com.facebook.presto.common.function.OperatorType.LESS_THAN;
import static com.facebook.presto.common.function.OperatorType.LESS_THAN_OR_EQUAL;
import static com.facebook.presto.common.function.OperatorType.NOT_EQUAL;
import static com.facebook.presto.common.function.OperatorType.XX_HASH_64;
import static com.facebook.presto.common.type.StandardTypes.BIGINT;
import static com.facebook.presto.common.type.StandardTypes.BOOLEAN;

public final class LongEnumOperators
{
private LongEnumOperators() {}

@ScalarOperator(EQUAL)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(BOOLEAN)
@SqlNullable
public static Boolean equal(@SqlType("T") long left, @SqlType("T") long right)
{
return left == right;
}

@ScalarOperator(NOT_EQUAL)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(BOOLEAN)
@SqlNullable
public static Boolean notEqual(@SqlType("T") long left, @SqlType("T") long right)
{
return left != right;
}

@ScalarOperator(IS_DISTINCT_FROM)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(BOOLEAN)
public static boolean isDistinctFrom(
@SqlType("T") long left,
@IsNull boolean leftNull,
@SqlType("T") long right,
@IsNull boolean rightNull)
{
if (leftNull != rightNull) {
return true;
}
if (leftNull) {
return false;
}
return notEqual(left, right);
}

@ScalarOperator(HASH_CODE)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(BIGINT)
public static long hashCode(@SqlType("T") long value)
{
return AbstractLongType.hash(value);
}

@ScalarOperator(XX_HASH_64)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(BIGINT)
public static long xxHash64(@SqlType("T") long value)
{
return XxHash64.hash(value);
}

@ScalarOperator(INDETERMINATE)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(BOOLEAN)
public static boolean indeterminate(@SqlType("T") long value, @IsNull boolean isNull)
{
return isNull;
}

@ScalarOperator(LESS_THAN)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(StandardTypes.BOOLEAN)
public static boolean lessThan(@SqlType("T") long left, @SqlType("T") long right)
{
return left < right;
}

@ScalarOperator(LESS_THAN_OR_EQUAL)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(StandardTypes.BOOLEAN)
public static boolean lessThanOrEqual(@SqlType("T") long left, @SqlType("T") long right)
{
return left <= right;
}

@ScalarOperator(GREATER_THAN)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(StandardTypes.BOOLEAN)
public static boolean greaterThan(@SqlType("T") long left, @SqlType("T") long right)
{
return left > right;
}

@ScalarOperator(GREATER_THAN_OR_EQUAL)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(StandardTypes.BOOLEAN)
public static boolean greaterThanOrEqual(@SqlType("T") long left, @SqlType("T") long right)
{
return left >= right;
}

@ScalarOperator(BETWEEN)
@TypeParameter(value = "T", boundedBy = LongEnumType.class)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType("T") long value, @SqlType("T") long min, @SqlType("T") long max)
{
return min <= value && value <= max;
}
}
Loading

0 comments on commit cef58ff

Please sign in to comment.