Tyson Hamilton created BEAM-9816:
------------------------------------
Summary: JOIN ON inequality results in RexInputRef cannot be cast
to RexCall
Key: BEAM-9816
URL: https://issues.apache.org/jira/browse/BEAM-9816
Project: Beam
Issue Type: Bug
Components: dsl-sql
Reporter: Tyson Hamilton
Attempting to use an inequality in a JOIN ON clause results in an unexpected
exception. Adding the following test in
org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java:
{code:java}
@Test
public void testInnerJoinWithInequality() throws Exception {
String sql =
"SELECT * "
+ "FROM ORDER_DETAILS1 o1 "
+ "JOIN ORDER_DETAILS2 o2 "
+ "ON "
+ "(o1.order_id=o2.site_id AND o2.price=o1.site_id) "
+ "AND o1.order_id>1";
PAssert.that(queryFromOrderTables(sql))
.containsInAnyOrder(
TestUtils.RowsBuilder.of(RESULT_ROW_TYPE).addRows(2, 3, 3, 1, 2,
3).getRows());
pipeline.run();
}
{code}
results in the following error:
{code:java}
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef
cannot be cast to
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCallorg.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef
cannot be cast to
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCalljava.lang.ClassCastException:
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef
cannot be cast to
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel.extractJoinRexNodes(BeamJoinRel.java:171)
at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamCoGBKJoinRel$StandardJoin.expand(BeamCoGBKJoinRel.java:111)
at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamCoGBKJoinRel$StandardJoin.expand(BeamCoGBKJoinRel.java:90)
at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:542) at
org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:493) at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils.toPCollection(BeamSqlRelUtils.java:69)
at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils.lambda$buildPCollectionList$0(BeamSqlRelUtils.java:50)
at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
at java.util.Iterator.forEachRemaining(Iterator.java:116) at
java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) at
java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) at
java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) at
java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) at
java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566) at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils.buildPCollectionList(BeamSqlRelUtils.java:51)
at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils.toPCollection(BeamSqlRelUtils.java:67)
at
org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils.toPCollection(BeamSqlRelUtils.java:39)
at
org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:128)
at org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:84)
at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:542) at
org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:493) at
org.apache.beam.sdk.values.PCollectionTuple.apply(PCollectionTuple.java:261) at
org.apache.beam.sdk.extensions.sql.BeamSqlDslJoinTest.queryFromOrderTables(BeamSqlDslJoinTest.java:408)
at
org.apache.beam.sdk.extensions.sql.BeamSqlDslJoinTest.testInnerJoinWithInequality(BeamSqlDslJoinTest.java:92)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498) at
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:319)
at
org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:266)
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:305) at
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:365) at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
at org.junit.runners.ParentRunner$4.run(ParentRunner.java:330) at
org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:78) at
org.junit.runners.ParentRunner.runChildren(ParentRunner.java:328) at
org.junit.runners.ParentRunner.access$100(ParentRunner.java:65) at
org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:292) at
org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:305) at
org.junit.runners.ParentRunner.run(ParentRunner.java:412) at
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
at
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
at
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
at
org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
at
org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498) at
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
at
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
at
org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
at
org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
at com.sun.proxy.$Proxy2.processTestClass(Unknown Source) at
org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498) at
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
at
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
at
org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
at
org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
at
org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
at
org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
at
org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at
org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
at java.lang.Thread.run(Thread.java:748){code}
however moving the inequality to a WHERE predicate after the join results in a
passing test. Also using a JOIN WHERE syntax, instead of ON, results in a
passing test.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)