Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
Rename ArithmeticBinaryExpression.Type to Operator
  • Loading branch information
martint committed Jul 9, 2018
1 parent 3b403fd commit 88b93f3
Show file tree
Hide file tree
Showing 18 changed files with 59 additions and 59 deletions.
Expand Up @@ -196,11 +196,11 @@ protected SymbolStatsEstimate visitArithmeticBinary(ArithmeticBinaryExpression n
double leftHigh = left.getHighValue(); double leftHigh = left.getHighValue();
double rightLow = right.getLowValue(); double rightLow = right.getLowValue();
double rightHigh = right.getHighValue(); double rightHigh = right.getHighValue();
if (node.getType() == ArithmeticBinaryExpression.Type.DIVIDE && rightLow < 0 && rightHigh > 0) { if (node.getOperator() == ArithmeticBinaryExpression.Operator.DIVIDE && rightLow < 0 && rightHigh > 0) {
result.setLowValue(Double.NEGATIVE_INFINITY) result.setLowValue(Double.NEGATIVE_INFINITY)
.setHighValue(Double.POSITIVE_INFINITY); .setHighValue(Double.POSITIVE_INFINITY);
} }
else if (node.getType() == ArithmeticBinaryExpression.Type.MODULUS) { else if (node.getOperator() == ArithmeticBinaryExpression.Operator.MODULUS) {
double maxDivisor = max(abs(rightLow), abs(rightHigh)); double maxDivisor = max(abs(rightLow), abs(rightHigh));
if (leftHigh <= 0) { if (leftHigh <= 0) {
result.setLowValue(max(-maxDivisor, leftLow)) result.setLowValue(max(-maxDivisor, leftLow))
Expand All @@ -216,10 +216,10 @@ else if (leftLow >= 0) {
} }
} }
else { else {
double v1 = operate(node.getType(), leftLow, rightLow); double v1 = operate(node.getOperator(), leftLow, rightLow);
double v2 = operate(node.getType(), leftLow, rightHigh); double v2 = operate(node.getOperator(), leftLow, rightHigh);
double v3 = operate(node.getType(), leftHigh, rightLow); double v3 = operate(node.getOperator(), leftHigh, rightLow);
double v4 = operate(node.getType(), leftHigh, rightHigh); double v4 = operate(node.getOperator(), leftHigh, rightHigh);
double lowValue = min(v1, v2, v3, v4); double lowValue = min(v1, v2, v3, v4);
double highValue = max(v1, v2, v3, v4); double highValue = max(v1, v2, v3, v4);


Expand All @@ -230,9 +230,9 @@ else if (leftLow >= 0) {
return result.build(); return result.build();
} }


private double operate(ArithmeticBinaryExpression.Type type, double left, double right) private double operate(ArithmeticBinaryExpression.Operator operator, double left, double right)
{ {
switch (type) { switch (operator) {
case ADD: case ADD:
return left + right; return left + right;
case SUBTRACT: case SUBTRACT:
Expand All @@ -244,7 +244,7 @@ private double operate(ArithmeticBinaryExpression.Type type, double left, double
case MODULUS: case MODULUS:
return left % right; return left % right;
default: default:
throw new IllegalStateException("Unsupported ArithmeticBinaryExpression.Type: " + type); throw new IllegalStateException("Unsupported ArithmeticBinaryExpression.Operator: " + operator);
} }
} }


Expand Down
Expand Up @@ -609,7 +609,7 @@ protected Type visitArithmeticUnary(ArithmeticUnaryExpression node, StackableAst
@Override @Override
protected Type visitArithmeticBinary(ArithmeticBinaryExpression node, StackableAstVisitorContext<Context> context) protected Type visitArithmeticBinary(ArithmeticBinaryExpression node, StackableAstVisitorContext<Context> context)
{ {
return getOperator(context, node, OperatorType.valueOf(node.getType().name()), node.getLeft(), node.getRight()); return getOperator(context, node, OperatorType.valueOf(node.getOperator().name()), node.getLeft(), node.getRight());
} }


@Override @Override
Expand Down
Expand Up @@ -711,10 +711,10 @@ protected Object visitArithmeticBinary(ArithmeticBinaryExpression node, Object c
} }


if (hasUnresolvedValue(left, right)) { if (hasUnresolvedValue(left, right)) {
return new ArithmeticBinaryExpression(node.getType(), toExpression(left, type(node.getLeft())), toExpression(right, type(node.getRight()))); return new ArithmeticBinaryExpression(node.getOperator(), toExpression(left, type(node.getLeft())), toExpression(right, type(node.getRight())));
} }


return invokeOperator(OperatorType.valueOf(node.getType().name()), types(node.getLeft(), node.getRight()), ImmutableList.of(left, right)); return invokeOperator(OperatorType.valueOf(node.getOperator().name()), types(node.getLeft(), node.getRight()), ImmutableList.of(left, right));
} }


@Override @Override
Expand Down
Expand Up @@ -28,7 +28,7 @@
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;


import static com.facebook.presto.sql.tree.ArithmeticBinaryExpression.Type.ADD; import static com.facebook.presto.sql.tree.ArithmeticBinaryExpression.Operator.ADD;
import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableList.toImmutableList;
import static java.lang.Math.toIntExact; import static java.lang.Math.toIntExact;
Expand Down
Expand Up @@ -107,9 +107,9 @@ public static Signature arithmeticNegationSignature(Type returnType, Type valueT
return internalOperator("NEGATION", returnType.getTypeSignature(), valueType.getTypeSignature()); return internalOperator("NEGATION", returnType.getTypeSignature(), valueType.getTypeSignature());
} }


public static Signature arithmeticExpressionSignature(ArithmeticBinaryExpression.Type expressionType, Type returnType, Type leftType, Type rightType) public static Signature arithmeticExpressionSignature(ArithmeticBinaryExpression.Operator operator, Type returnType, Type leftType, Type rightType)
{ {
return internalOperator(expressionType.name(), returnType.getTypeSignature(), leftType.getTypeSignature(), rightType.getTypeSignature()); return internalOperator(operator.name(), returnType.getTypeSignature(), leftType.getTypeSignature(), rightType.getTypeSignature());
} }


public static Signature subscriptSignature(Type returnType, Type leftType, Type rightType) public static Signature subscriptSignature(Type returnType, Type leftType, Type rightType)
Expand Down
Expand Up @@ -408,7 +408,7 @@ protected RowExpression visitArithmeticBinary(ArithmeticBinaryExpression node, V
RowExpression right = process(node.getRight(), context); RowExpression right = process(node.getRight(), context);


return call( return call(
arithmeticExpressionSignature(node.getType(), getType(node), left.getType(), right.getType()), arithmeticExpressionSignature(node.getOperator(), getType(node), left.getType(), right.getType()),
getType(node), getType(node),
left, left,
right); right);
Expand Down
Expand Up @@ -24,7 +24,7 @@
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import org.testng.annotations.Test; import org.testng.annotations.Test;


import static com.facebook.presto.sql.tree.ArithmeticBinaryExpression.Type.ADD; import static com.facebook.presto.sql.tree.ArithmeticBinaryExpression.Operator.ADD;
import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertEquals;


public class TestDesugarTryExpressionRewriter public class TestDesugarTryExpressionRewriter
Expand Down
Expand Up @@ -385,7 +385,7 @@ private static Expression add(String symbol1, String symbol2)


private static Expression add(Expression expression1, Expression expression2) private static Expression add(Expression expression1, Expression expression2)
{ {
return new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.ADD, expression1, expression2); return new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.ADD, expression1, expression2);
} }


private static Expression multiply(String symbol1, String symbol2) private static Expression multiply(String symbol1, String symbol2)
Expand All @@ -395,7 +395,7 @@ private static Expression multiply(String symbol1, String symbol2)


private static Expression multiply(Expression expression1, Expression expression2) private static Expression multiply(Expression expression1, Expression expression2)
{ {
return new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.MULTIPLY, expression1, expression2); return new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.MULTIPLY, expression1, expression2);
} }


private static Expression equals(String symbol1, String symbol2) private static Expression equals(String symbol1, String symbol2)
Expand Down
Expand Up @@ -99,14 +99,14 @@ public void testArithmeticExpression()
@Test @Test
public void testArithmeticExpressionWithNulls() public void testArithmeticExpressionWithNulls()
{ {
for (ArithmeticBinaryExpression.Type type : ArithmeticBinaryExpression.Type.values()) { for (ArithmeticBinaryExpression.Operator operator : ArithmeticBinaryExpression.Operator.values()) {
assertProjection("CAST(NULL AS INTEGER) " + type.getValue() + " CAST(NULL AS INTEGER)", null); assertProjection("CAST(NULL AS INTEGER) " + operator.getValue() + " CAST(NULL AS INTEGER)", null);


assertProjection("42 " + type.getValue() + " NULL", null); assertProjection("42 " + operator.getValue() + " NULL", null);
assertProjection("NULL " + type.getValue() + " 42", null); assertProjection("NULL " + operator.getValue() + " 42", null);


assertProjection("11.1 " + type.getValue() + " CAST(NULL AS INTEGER)", null); assertProjection("11.1 " + operator.getValue() + " CAST(NULL AS INTEGER)", null);
assertProjection("CAST(NULL AS INTEGER) " + type.getValue() + " 11.1", null); assertProjection("CAST(NULL AS INTEGER) " + operator.getValue() + " 11.1", null);
} }
} }


Expand Down
Expand Up @@ -188,7 +188,7 @@ protected Boolean visitArithmeticBinary(ArithmeticBinaryExpression actual, Node
{ {
if (expectedExpression instanceof ArithmeticBinaryExpression) { if (expectedExpression instanceof ArithmeticBinaryExpression) {
ArithmeticBinaryExpression expected = (ArithmeticBinaryExpression) expectedExpression; ArithmeticBinaryExpression expected = (ArithmeticBinaryExpression) expectedExpression;
if (actual.getType() == expected.getType()) { if (actual.getOperator() == expected.getOperator()) {
return process(actual.getLeft(), expected.getLeft()) && process(actual.getRight(), expected.getRight()); return process(actual.getLeft(), expected.getLeft()) && process(actual.getRight(), expected.getRight());
} }
} }
Expand Down
Expand Up @@ -120,9 +120,9 @@ public void testPartitioningColumnAndHashWithoutIdentityMappingInProjection()
Symbol hTimes5 = p.symbol("h_times_5"); Symbol hTimes5 = p.symbol("h_times_5");
return p.project( return p.project(
Assignments.builder() Assignments.builder()
.put(aTimes5, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.MULTIPLY, new SymbolReference("a"), new LongLiteral("5"))) .put(aTimes5, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.MULTIPLY, new SymbolReference("a"), new LongLiteral("5")))
.put(bTimes5, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.MULTIPLY, new SymbolReference("b"), new LongLiteral("5"))) .put(bTimes5, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.MULTIPLY, new SymbolReference("b"), new LongLiteral("5")))
.put(hTimes5, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.MULTIPLY, new SymbolReference("h"), new LongLiteral("5"))) .put(hTimes5, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.MULTIPLY, new SymbolReference("h"), new LongLiteral("5")))
.build(), .build(),
p.exchange(e -> e p.exchange(e -> e
.addSource( .addSource(
Expand Down
Expand Up @@ -52,7 +52,7 @@ public void test()
Symbol c = p.symbol("c"); Symbol c = p.symbol("c");
Symbol cTimes3 = p.symbol("c_times_3"); Symbol cTimes3 = p.symbol("c_times_3");
return p.project( return p.project(
Assignments.of(cTimes3, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.MULTIPLY, c.toSymbolReference(), new LongLiteral("3"))), Assignments.of(cTimes3, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.MULTIPLY, c.toSymbolReference(), new LongLiteral("3"))),
p.union( p.union(
ImmutableListMultimap.<Symbol, Symbol>builder() ImmutableListMultimap.<Symbol, Symbol>builder()
.put(c, a) .put(c, a)
Expand Down
Expand Up @@ -499,7 +499,7 @@ protected String visitArithmeticUnary(ArithmeticUnaryExpression node, Void conte
@Override @Override
protected String visitArithmeticBinary(ArithmeticBinaryExpression node, Void context) protected String visitArithmeticBinary(ArithmeticBinaryExpression node, Void context)
{ {
return formatBinaryExpression(node.getType().getValue(), node.getLeft(), node.getRight()); return formatBinaryExpression(node.getOperator().getValue(), node.getLeft(), node.getRight());
} }


@Override @Override
Expand Down
Expand Up @@ -235,7 +235,7 @@ protected Void visitComparisonExpression(ComparisonExpression node, Integer inde
@Override @Override
protected Void visitArithmeticBinary(ArithmeticBinaryExpression node, Integer indentLevel) protected Void visitArithmeticBinary(ArithmeticBinaryExpression node, Integer indentLevel)
{ {
print(indentLevel, node.getType().toString()); print(indentLevel, node.getOperator().toString());


super.visitArithmeticBinary(node, indentLevel + 1); super.visitArithmeticBinary(node, indentLevel + 1);


Expand Down
Expand Up @@ -1868,19 +1868,19 @@ private static Optional<String> getTextIfPresent(Token token)
.map(Token::getText); .map(Token::getText);
} }


private static ArithmeticBinaryExpression.Type getArithmeticBinaryOperator(Token operator) private static ArithmeticBinaryExpression.Operator getArithmeticBinaryOperator(Token operator)
{ {
switch (operator.getType()) { switch (operator.getType()) {
case SqlBaseLexer.PLUS: case SqlBaseLexer.PLUS:
return ArithmeticBinaryExpression.Type.ADD; return ArithmeticBinaryExpression.Operator.ADD;
case SqlBaseLexer.MINUS: case SqlBaseLexer.MINUS:
return ArithmeticBinaryExpression.Type.SUBTRACT; return ArithmeticBinaryExpression.Operator.SUBTRACT;
case SqlBaseLexer.ASTERISK: case SqlBaseLexer.ASTERISK:
return ArithmeticBinaryExpression.Type.MULTIPLY; return ArithmeticBinaryExpression.Operator.MULTIPLY;
case SqlBaseLexer.SLASH: case SqlBaseLexer.SLASH:
return ArithmeticBinaryExpression.Type.DIVIDE; return ArithmeticBinaryExpression.Operator.DIVIDE;
case SqlBaseLexer.PERCENT: case SqlBaseLexer.PERCENT:
return ArithmeticBinaryExpression.Type.MODULUS; return ArithmeticBinaryExpression.Operator.MODULUS;
} }


throw new UnsupportedOperationException("Unsupported operator: " + operator.getText()); throw new UnsupportedOperationException("Unsupported operator: " + operator.getText());
Expand Down
Expand Up @@ -22,7 +22,7 @@
public class ArithmeticBinaryExpression public class ArithmeticBinaryExpression
extends Expression extends Expression
{ {
public enum Type public enum Operator
{ {
ADD("+"), ADD("+"),
SUBTRACT("-"), SUBTRACT("-"),
Expand All @@ -31,7 +31,7 @@ public enum Type
MODULUS("%"); MODULUS("%");
private final String value; private final String value;


Type(String value) Operator(String value)
{ {
this.value = value; this.value = value;
} }
Expand All @@ -42,31 +42,31 @@ public String getValue()
} }
} }


private final Type type; private final Operator operator;
private final Expression left; private final Expression left;
private final Expression right; private final Expression right;


public ArithmeticBinaryExpression(Type type, Expression left, Expression right) public ArithmeticBinaryExpression(Operator operator, Expression left, Expression right)
{ {
this(Optional.empty(), type, left, right); this(Optional.empty(), operator, left, right);
} }


public ArithmeticBinaryExpression(NodeLocation location, Type type, Expression left, Expression right) public ArithmeticBinaryExpression(NodeLocation location, Operator operator, Expression left, Expression right)
{ {
this(Optional.of(location), type, left, right); this(Optional.of(location), operator, left, right);
} }


private ArithmeticBinaryExpression(Optional<NodeLocation> location, Type type, Expression left, Expression right) private ArithmeticBinaryExpression(Optional<NodeLocation> location, Operator operator, Expression left, Expression right)
{ {
super(location); super(location);
this.type = type; this.operator = operator;
this.left = left; this.left = left;
this.right = right; this.right = right;
} }


public Type getType() public Operator getOperator()
{ {
return type; return operator;
} }


public Expression getLeft() public Expression getLeft()
Expand Down Expand Up @@ -102,14 +102,14 @@ public boolean equals(Object o)
} }


ArithmeticBinaryExpression that = (ArithmeticBinaryExpression) o; ArithmeticBinaryExpression that = (ArithmeticBinaryExpression) o;
return (type == that.type) && return (operator == that.operator) &&
Objects.equals(left, that.left) && Objects.equals(left, that.left) &&
Objects.equals(right, that.right); Objects.equals(right, that.right);
} }


@Override @Override
public int hashCode() public int hashCode()
{ {
return Objects.hash(type, left, right); return Objects.hash(operator, left, right);
} }
} }
Expand Up @@ -134,7 +134,7 @@ public Expression visitArithmeticBinary(ArithmeticBinaryExpression node, Context
Expression right = rewrite(node.getRight(), context.get()); Expression right = rewrite(node.getRight(), context.get());


if (left != node.getLeft() || right != node.getRight()) { if (left != node.getLeft() || right != node.getRight()) {
return new ArithmeticBinaryExpression(node.getType(), left, right); return new ArithmeticBinaryExpression(node.getOperator(), left, right);
} }


return node; return node;
Expand Down
Expand Up @@ -551,25 +551,25 @@ public void testPrecedenceAndAssociativity()
new NotExpression(new LongLiteral("1")), new NotExpression(new LongLiteral("1")),
new LongLiteral("2"))); new LongLiteral("2")));


assertExpression("-1 + 2", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.ADD, assertExpression("-1 + 2", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.ADD,
negative(new LongLiteral("1")), negative(new LongLiteral("1")),
new LongLiteral("2"))); new LongLiteral("2")));


assertExpression("1 - 2 - 3", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.SUBTRACT, assertExpression("1 - 2 - 3", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.SUBTRACT,
new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.SUBTRACT, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.SUBTRACT,
new LongLiteral("1"), new LongLiteral("1"),
new LongLiteral("2")), new LongLiteral("2")),
new LongLiteral("3"))); new LongLiteral("3")));


assertExpression("1 / 2 / 3", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.DIVIDE, assertExpression("1 / 2 / 3", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.DIVIDE,
new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.DIVIDE, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.DIVIDE,
new LongLiteral("1"), new LongLiteral("1"),
new LongLiteral("2")), new LongLiteral("2")),
new LongLiteral("3"))); new LongLiteral("3")));


assertExpression("1 + 2 * 3", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.ADD, assertExpression("1 + 2 * 3", new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.ADD,
new LongLiteral("1"), new LongLiteral("1"),
new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Type.MULTIPLY, new ArithmeticBinaryExpression(ArithmeticBinaryExpression.Operator.MULTIPLY,
new LongLiteral("2"), new LongLiteral("2"),
new LongLiteral("3")))); new LongLiteral("3"))));
} }
Expand Down

0 comments on commit 88b93f3

Please sign in to comment.