Skip to content

Commit

Permalink
Fix compilation for try_cast
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed Feb 9, 2015
1 parent 5f52308 commit 90348e3
Show file tree
Hide file tree
Showing 4 changed files with 56 additions and 23 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,14 +59,6 @@ public static boolean greaterThan(int left, int right)
return left > right;
}

public static void propagateInterruptedException(Throwable throwable)
{
if (throwable instanceof InterruptedException) {
Thread.currentThread().interrupt();
throw new RuntimeException(throwable);
}
}

public static boolean in(Object value, Set<?> set)
{
return set.contains(value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,34 +16,67 @@
import com.facebook.presto.byteCode.Block;
import com.facebook.presto.byteCode.ByteCodeNode;
import com.facebook.presto.byteCode.CompilerContext;
import com.facebook.presto.byteCode.control.TryCatch;
import com.facebook.presto.byteCode.instruction.LabelNode;
import com.facebook.presto.metadata.Signature;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.sql.relational.RowExpression;
import com.google.common.primitives.Primitives;

import java.lang.invoke.MethodHandle;
import java.util.List;

import static com.facebook.presto.byteCode.ParameterizedType.type;
import static com.facebook.presto.sql.gen.ByteCodeUtils.ifWasNullPopAndGoto;
import static com.facebook.presto.sql.gen.ByteCodeUtils.invoke;
import static com.facebook.presto.sql.gen.ByteCodeUtils.unboxPrimitiveIfNecessary;
import static com.facebook.presto.type.UnknownType.UNKNOWN;
import static java.lang.invoke.MethodHandles.catchException;
import static java.lang.invoke.MethodHandles.constant;
import static java.lang.invoke.MethodHandles.dropArguments;
import static java.lang.invoke.MethodType.methodType;

public class TryCastCodeGenerator
implements ByteCodeGenerator
{
@Override
public ByteCodeNode generateExpression(Signature signature, ByteCodeGeneratorContext generatorContext, Type returnType, List<RowExpression> arguments)
{
ByteCodeNode castByteCode = new CastCodeGenerator().generateExpression(signature, generatorContext, returnType, arguments);

CompilerContext context = generatorContext.getContext();
Block catchBlock = new Block(context)
.comment("propagate InterruptedException")
.invokeStatic(CompilerOperations.class, "propagateInterruptedException", void.class, Throwable.class)
.comment("wasNull = true;")
.putVariable("wasNull", true)
.comment("restore stack after exception")
.getVariable("output") // TODO: this is quite a strong assumption to make. This code should not be sensitive to whether "output" was on the stack before the call
.comment("return dummy value for null")
.pushJavaDefault(returnType.getJavaType());

return new TryCatch(context, "TRY_CAST", castByteCode, catchBlock, type(Exception.class));
RowExpression argument = arguments.get(0);

Class<?> argumentType = argument.getType().getJavaType();
Class<?> unboxedReturnType = returnType.getJavaType();
Class<?> boxedReturnType = Primitives.wrap(unboxedReturnType);

if (argument.getType().equals(UNKNOWN)) {
return new Block(context)
.putVariable("wasNull", true)
.pushJavaDefault(unboxedReturnType);
}

MethodHandle function = generatorContext
.getRegistry()
.getCoercion(argument.getType(), returnType)
.getMethodHandle()
.asType(methodType(boxedReturnType, argumentType));

MethodHandle tryCast = exceptionToNull(function, boxedReturnType, RuntimeException.class);

Binding tryCastBinding = generatorContext.getCallSiteBinder().bind(tryCast);

LabelNode end = new LabelNode("end");

return new Block(context)
.comment("call tryCast method")
.append(generatorContext.generate(argument))
.append(ifWasNullPopAndGoto(context, end, unboxedReturnType, argumentType))
.append(invoke(generatorContext.getContext(), tryCastBinding, "tryCast"))
.append(unboxPrimitiveIfNecessary(context, boxedReturnType))
.visitLabel(end);
}

private static MethodHandle exceptionToNull(MethodHandle target, Class<?> type, Class<? extends Throwable> throwable)
{
MethodHandle toNull = dropArguments(constant(type, null), 0, throwable);
return catchException(target, throwable, toNull);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -543,8 +543,13 @@ public void testTryCast()
assertExecute("try_cast('foo' as varchar)", "foo");
assertExecute("try_cast('foo' as bigint)", null);
assertExecute("try_cast(bound_string as bigint)", null);
assertExecute("try_cast(cast(null as varchar) as bigint)", null);
assertExecute("try_cast(bound_long / 13 as bigint)", 94);
assertExecute("coalesce(try_cast('123' as bigint), 456)", 123L);
assertExecute("coalesce(try_cast('foo' as bigint), 456)", 456L);
assertExecute("concat('foo', cast('bar' as varchar))", "foobar");
assertExecute("try_cast(try_cast(123 as varchar) as bigint)", 123L);
assertExecute("try_cast('foo' as varchar) || try_cast('bar' as varchar)", "foobar");

Futures.allAsList(futures).get();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2248,6 +2248,9 @@ public void testCast()

assertQuery("SELECT try_cast('foo' AS BIGINT) FROM orders", "SELECT CAST(null AS BIGINT) FROM orders");
assertQuery("SELECT try_cast(clerk AS BIGINT) FROM orders", "SELECT CAST(null AS BIGINT) FROM orders");
assertQuery("SELECT try_cast(orderkey * orderkey AS VARCHAR) FROM orders", "SELECT CAST(orderkey * orderkey AS VARCHAR) FROM orders");
assertQuery("SELECT try_cast(try_cast(orderkey AS VARCHAR) AS BIGINT) FROM orders", "SELECT orderkey FROM orders");
assertQuery("SELECT try_cast(clerk AS VARCHAR) || try_cast(clerk AS VARCHAR) FROM orders", "SELECT clerk || clerk FROM orders");

assertQuery("SELECT coalesce(try_cast('foo' AS BIGINT), 456) FROM orders", "SELECT 456 FROM orders");
assertQuery("SELECT coalesce(try_cast(clerk AS BIGINT), 456) FROM orders", "SELECT 456 FROM orders");
Expand Down

0 comments on commit 90348e3

Please sign in to comment.