Skip to content

Commit

Permalink
Remove unused BETWEEN operator
Browse files Browse the repository at this point in the history
It was only being used during analysis to check for support
for the operator. At runtime, it gets desugard into a comparison
using the "less than or equal" operator.

Remove usages of the "between" operator and check for existance
of "less than or equal" operator instead.
  • Loading branch information
martint committed May 31, 2020
1 parent 18a2638 commit 44d4eb6
Show file tree
Hide file tree
Showing 32 changed files with 53 additions and 238 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,6 @@
import static io.prestosql.type.DecimalCasts.SMALLINT_TO_DECIMAL_CAST;
import static io.prestosql.type.DecimalCasts.TINYINT_TO_DECIMAL_CAST;
import static io.prestosql.type.DecimalCasts.VARCHAR_TO_DECIMAL_CAST;
import static io.prestosql.type.DecimalInequalityOperators.DECIMAL_BETWEEN_OPERATOR;
import static io.prestosql.type.DecimalInequalityOperators.DECIMAL_DISTINCT_FROM_OPERATOR;
import static io.prestosql.type.DecimalInequalityOperators.DECIMAL_EQUAL_OPERATOR;
import static io.prestosql.type.DecimalInequalityOperators.DECIMAL_GREATER_THAN_OPERATOR;
Expand Down Expand Up @@ -568,7 +567,6 @@ public FunctionRegistry(Metadata metadata, FeaturesConfig featuresConfig)
.functions(DECIMAL_TO_INTEGER_SATURATED_FLOOR_CAST, INTEGER_TO_DECIMAL_SATURATED_FLOOR_CAST)
.functions(DECIMAL_TO_SMALLINT_SATURATED_FLOOR_CAST, SMALLINT_TO_DECIMAL_SATURATED_FLOOR_CAST)
.functions(DECIMAL_TO_TINYINT_SATURATED_FLOOR_CAST, TINYINT_TO_DECIMAL_SATURATED_FLOOR_CAST)
.function(DECIMAL_BETWEEN_OPERATOR)
.function(DECIMAL_DISTINCT_FROM_OPERATOR)
.function(new Histogram(featuresConfig.getHistogramGroupImplementation()))
.function(CHECKSUM_AGGREGATION)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,6 @@
import static io.prestosql.spi.function.InvocationConvention.InvocationArgumentConvention.NEVER_NULL;
import static io.prestosql.spi.function.InvocationConvention.InvocationReturnConvention.FAIL_ON_NULL;
import static io.prestosql.spi.function.InvocationConvention.InvocationReturnConvention.NULLABLE_RETURN;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.EQUAL;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN_OR_EQUAL;
Expand Down Expand Up @@ -1384,9 +1383,6 @@ public void verifyComparableOrderableContract()
missingOperators.put(type, operator);
}
}
if (!canResolveOperator(BETWEEN, BOOLEAN, ImmutableList.of(type, type, type))) {
missingOperators.put(type, BETWEEN);
}
}
}
// TODO: verify the parametric types too
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,14 +53,11 @@ public OperatorNotFoundException(OperatorType operatorType, List<? extends Type>
private static String formatErrorMessage(OperatorType operatorType, List<? extends Type> argumentTypes, Optional<TypeSignature> returnType)
{
String operatorString;
switch (operatorType) {
case BETWEEN:
return format("Cannot check if %s is BETWEEN %s and %s", argumentTypes.get(0), argumentTypes.get(1), argumentTypes.get(2));
case CAST:
operatorString = format("%s%s", operatorType.getOperator(), returnType.map(value -> " to " + value).orElse(""));
break;
default:
operatorString = format("'%s'%s", operatorType.getOperator(), returnType.map(value -> ":" + value).orElse(""));
if (operatorType == OperatorType.CAST) {
operatorString = format("%s%s", operatorType.getOperator(), returnType.map(value -> " to " + value).orElse(""));
}
else {
operatorString = format("'%s'%s", operatorType.getOperator(), returnType.map(value -> ":" + value).orElse(""));
}
return format("%s cannot be applied to %s", operatorString, Joiner.on(", ").join(argumentTypes));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,6 @@
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static io.prestosql.operator.TypeSignatureParser.parseTypeSignature;
import static io.prestosql.operator.annotations.ImplementationDependency.isImplementationDependencyAnnotation;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.EQUAL;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN_OR_EQUAL;
Expand All @@ -69,7 +68,7 @@
public final class FunctionsParserHelper
{
private static final Set<OperatorType> COMPARABLE_TYPE_OPERATORS = ImmutableSet.of(EQUAL, NOT_EQUAL, HASH_CODE, XX_HASH_64, IS_DISTINCT_FROM, INDETERMINATE);
private static final Set<OperatorType> ORDERABLE_TYPE_OPERATORS = ImmutableSet.of(LESS_THAN, LESS_THAN_OR_EQUAL, GREATER_THAN, GREATER_THAN_OR_EQUAL, BETWEEN);
private static final Set<OperatorType> ORDERABLE_TYPE_OPERATORS = ImmutableSet.of(LESS_THAN, LESS_THAN_OR_EQUAL, GREATER_THAN, GREATER_THAN_OR_EQUAL);

private FunctionsParserHelper()
{}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,9 +48,6 @@ public static void validateOperator(OperatorType operatorType, TypeSignature ret
case GREATER_THAN_OR_EQUAL:
validateComparisonOperatorSignature(operatorType, returnType, argumentTypes, 2);
break;
case BETWEEN:
validateComparisonOperatorSignature(operatorType, returnType, argumentTypes, 3);
break;
case CAST:
validateOperatorSignature(operatorType, returnType, argumentTypes, 1);
break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1122,7 +1122,35 @@ private boolean isDateTimeType(Type type)
@Override
protected Type visitBetweenPredicate(BetweenPredicate node, StackableAstVisitorContext<Context> context)
{
return getOperator(context, node, OperatorType.BETWEEN, node.getValue(), node.getMin(), node.getMax());
Type valueType = process(node.getValue(), context);
Type minType = process(node.getMin(), context);
Type maxType = process(node.getMax(), context);

Optional<Type> commonType = typeCoercion.getCommonSuperType(valueType, minType)
.flatMap(type -> typeCoercion.getCommonSuperType(type, maxType));

if (commonType.isEmpty()) {
semanticException(TYPE_MISMATCH, node, "Cannot check if %s is BETWEEN %s and %s", valueType, minType, maxType);
}

try {
metadata.resolveOperator(OperatorType.LESS_THAN_OR_EQUAL, List.of(commonType.get(), commonType.get()));
}
catch (OperatorNotFoundException e) {
semanticException(TYPE_MISMATCH, node, "Cannot check if %s is BETWEEN %s and %s", valueType, minType, maxType);
}

if (!valueType.equals(commonType.get())) {
addOrReplaceExpressionCoercion(node.getValue(), valueType, commonType.get());
}
if (!minType.equals(commonType.get())) {
addOrReplaceExpressionCoercion(node.getMin(), minType, commonType.get());
}
if (!maxType.equals(commonType.get())) {
addOrReplaceExpressionCoercion(node.getMax(), maxType, commonType.get());
}

return setExpressionType(node, BOOLEAN);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,10 +51,10 @@ public BytecodeNode generateExpression(ResolvedFunction resolvedFunction, Byteco
AND,
BOOLEAN,
call(
standardFunctionResolution.comparisonFunction(Operator.GREATER_THAN_OR_EQUAL, value.getType(), min.getType()),
standardFunctionResolution.comparisonFunction(Operator.LESS_THAN_OR_EQUAL, min.getType(), value.getType()),
BOOLEAN,
valueReference,
min),
min,
valueReference),
call(
standardFunctionResolution.comparisonFunction(Operator.LESS_THAN_OR_EQUAL, value.getType(), max.getType()),
BOOLEAN,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -775,7 +775,7 @@ protected Object visitBetweenPredicate(BetweenPredicate node, Object context)

Boolean greaterOrEqualToMin = null;
if (min != null) {
greaterOrEqualToMin = (Boolean) invokeOperator(OperatorType.GREATER_THAN_OR_EQUAL, types(node.getValue(), node.getMin()), ImmutableList.of(value, min));
greaterOrEqualToMin = (Boolean) invokeOperator(OperatorType.LESS_THAN_OR_EQUAL, types(node.getMin(), node.getValue()), ImmutableList.of(min, value));
}
Boolean lessThanOrEqualToMax = null;
if (max != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import static io.prestosql.spi.StandardErrorCode.DIVISION_BY_ZERO;
import static io.prestosql.spi.StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE;
import static io.prestosql.spi.function.OperatorType.ADD;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.DIVIDE;
import static io.prestosql.spi.function.OperatorType.EQUAL;
Expand Down Expand Up @@ -180,13 +179,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.BIGINT) long lef
return left >= right;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType(StandardTypes.BIGINT) long value, @SqlType(StandardTypes.BIGINT) long min, @SqlType(StandardTypes.BIGINT) long max)
{
return min <= value && value <= max;
}

@ScalarOperator(CAST)
@SqlType(StandardTypes.BOOLEAN)
public static boolean castToBoolean(@SqlType(StandardTypes.BIGINT) long value)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import io.prestosql.spi.function.SqlType;
import io.prestosql.spi.type.StandardTypes;

import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.EQUAL;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN;
Expand Down Expand Up @@ -99,13 +98,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.BOOLEAN) boolean
return left || !right;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType(StandardTypes.BOOLEAN) boolean value, @SqlType(StandardTypes.BOOLEAN) boolean min, @SqlType(StandardTypes.BOOLEAN) boolean max)
{
return (value && max) || (!value && !min);
}

@ScalarOperator(CAST)
@SqlType(StandardTypes.DOUBLE)
public static double castToDouble(@SqlType(StandardTypes.BOOLEAN) boolean value)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import io.prestosql.spi.function.SqlType;
import io.prestosql.spi.type.StandardTypes;

import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.EQUAL;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN_OR_EQUAL;
Expand Down Expand Up @@ -92,14 +91,6 @@ public static boolean greaterThanOrEqual(@SqlType("char(x)") Slice left, @SqlTyp
return compareChars(left, right) >= 0;
}

@LiteralParameters("x")
@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType("char(x)") Slice value, @SqlType("char(x)") Slice min, @SqlType("char(x)") Slice max)
{
return compareChars(min, value) <= 0 && compareChars(value, max) <= 0;
}

@LiteralParameters("x")
@ScalarOperator(HASH_CODE)
@SqlType(StandardTypes.BIGINT)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
import static io.airlift.slice.SliceUtf8.trim;
import static io.airlift.slice.Slices.utf8Slice;
import static io.prestosql.spi.StandardErrorCode.INVALID_CAST_ARGUMENT;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.EQUAL;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN;
Expand Down Expand Up @@ -101,13 +100,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.DATE) long left,
return left >= right;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType(StandardTypes.DATE) long value, @SqlType(StandardTypes.DATE) long min, @SqlType(StandardTypes.DATE) long max)
{
return min <= value && value <= max;
}

@ScalarOperator(CAST)
@SqlType(StandardTypes.TIMESTAMP)
public static long castToTimestamp(ConnectorSession session, @SqlType(StandardTypes.DATE) long value)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import static io.prestosql.operator.scalar.ScalarFunctionImplementation.NullConvention.BLOCK_AND_POSITION;
import static io.prestosql.operator.scalar.ScalarFunctionImplementation.NullConvention.USE_NULL_FLAG;
import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.EQUAL;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN_OR_EQUAL;
Expand Down Expand Up @@ -67,7 +66,6 @@ public final class DecimalInequalityOperators
public static final SqlScalarFunction DECIMAL_LESS_THAN_OR_EQUAL_OPERATOR = comparisonOperator(LESS_THAN_OR_EQUAL, IS_RESULT_LESS_THAN_OR_EQUAL);
public static final SqlScalarFunction DECIMAL_GREATER_THAN_OPERATOR = comparisonOperator(GREATER_THAN, IS_RESULT_GREATER_THAN);
public static final SqlScalarFunction DECIMAL_GREATER_THAN_OR_EQUAL_OPERATOR = comparisonOperator(GREATER_THAN_OR_EQUAL, IS_RESULT_GREATER_THAN_OR_EQUAL);
public static final SqlScalarFunction DECIMAL_BETWEEN_OPERATOR = betweenOperator();
public static final SqlScalarFunction DECIMAL_DISTINCT_FROM_OPERATOR = distinctOperator();

private DecimalInequalityOperators() {}
Expand Down Expand Up @@ -261,23 +259,6 @@ private static boolean invokeGetResult(MethodHandle getResultMethodHandle, int c
}
}

private static SqlScalarFunction betweenOperator()
{
Signature signature = Signature.builder()
.kind(SCALAR)
.operatorType(BETWEEN)
.argumentTypes(DECIMAL_SIGNATURE, DECIMAL_SIGNATURE, DECIMAL_SIGNATURE)
.returnType(BOOLEAN.getTypeSignature())
.build();
return SqlScalarFunction.builder(DecimalInequalityOperators.class)
.signature(signature)
.deterministic(true)
.choice(choice -> choice
.implementation(methodsGroup -> methodsGroup
.methods("betweenShortShortShort", "betweenLongLongLong")))
.build();
}

@UsedByGeneratedCode
public static boolean betweenShortShortShort(long value, long low, long high)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import static io.prestosql.spi.StandardErrorCode.INVALID_CAST_ARGUMENT;
import static io.prestosql.spi.StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE;
import static io.prestosql.spi.function.OperatorType.ADD;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.DIVIDE;
import static io.prestosql.spi.function.OperatorType.EQUAL;
Expand Down Expand Up @@ -166,13 +165,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.DOUBLE) double l
return left >= right;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType(StandardTypes.DOUBLE) double value, @SqlType(StandardTypes.DOUBLE) double min, @SqlType(StandardTypes.DOUBLE) double max)
{
return min <= value && value <= max;
}

@ScalarOperator(CAST)
@SqlType(StandardTypes.BOOLEAN)
public static boolean castToBoolean(@SqlType(StandardTypes.DOUBLE) double value)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import static io.prestosql.spi.StandardErrorCode.DIVISION_BY_ZERO;
import static io.prestosql.spi.StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE;
import static io.prestosql.spi.function.OperatorType.ADD;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.DIVIDE;
import static io.prestosql.spi.function.OperatorType.EQUAL;
Expand Down Expand Up @@ -175,13 +174,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.INTEGER) long le
return left >= right;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType(StandardTypes.INTEGER) long value, @SqlType(StandardTypes.INTEGER) long min, @SqlType(StandardTypes.INTEGER) long max)
{
return min <= value && value <= max;
}

@ScalarOperator(CAST)
@SqlType(StandardTypes.BIGINT)
public static long castToBigint(@SqlType(StandardTypes.INTEGER) long value)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import static io.prestosql.client.IntervalDayTime.formatMillis;
import static io.prestosql.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static io.prestosql.spi.function.OperatorType.ADD;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.DIVIDE;
import static io.prestosql.spi.function.OperatorType.EQUAL;
Expand Down Expand Up @@ -163,16 +162,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.INTERVAL_DAY_TO_
return left >= right;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(
@SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long value,
@SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long min,
@SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long max)
{
return min <= value && value <= max;
}

@ScalarOperator(CAST)
@LiteralParameters("x")
@SqlType("varchar(x)")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import static io.airlift.slice.Slices.utf8Slice;
import static io.prestosql.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static io.prestosql.spi.function.OperatorType.ADD;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.DIVIDE;
import static io.prestosql.spi.function.OperatorType.EQUAL;
Expand Down Expand Up @@ -164,16 +163,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.INTERVAL_YEAR_TO
return left >= right;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(
@SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long value,
@SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long min,
@SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long max)
{
return min <= value && value <= max;
}

@ScalarOperator(CAST)
@LiteralParameters("x")
@SqlType("varchar(x)")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import static io.airlift.slice.Slices.wrappedBuffer;
import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static io.prestosql.spi.StandardErrorCode.INVALID_CAST_ARGUMENT;
import static io.prestosql.spi.function.OperatorType.BETWEEN;
import static io.prestosql.spi.function.OperatorType.CAST;
import static io.prestosql.spi.function.OperatorType.EQUAL;
import static io.prestosql.spi.function.OperatorType.GREATER_THAN;
Expand Down Expand Up @@ -97,13 +96,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.IPADDRESS) Slice
return left.compareTo(right) >= 0;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType(StandardTypes.IPADDRESS) Slice value, @SqlType(StandardTypes.IPADDRESS) Slice min, @SqlType(StandardTypes.IPADDRESS) Slice max)
{
return min.compareTo(value) <= 0 && value.compareTo(max) <= 0;
}

@ScalarOperator(HASH_CODE)
@SqlType(StandardTypes.BIGINT)
public static long hashCode(@SqlType(StandardTypes.IPADDRESS) Slice value)
Expand Down
Loading

0 comments on commit 44d4eb6

Please sign in to comment.