You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by Yash Sharma <ya...@gmail.com> on 2015/01/07 19:06:45 UTC

Drill Cassandra Storage - Unexpected exception during fragment initialization

Hi All,
I am writing Cassandra storage for Drill and am facing this error while
firing a simple select query against Cassandra table on Sqlline.

I am able to get the expected results by using a physical plan directly.

Any thoughts on what I might be missing.

Thanks


Some more info for troubleshooting -

Query:
0: jdbc:drill:zk=local> select id from cassandra.recsys.`trending_now`;

Query failed: Query failed: Unexpected exception during fragment
initialization: Internal error: Error while applying rule
DrillPushProjIntoScan, args
[rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
recsys, trending_now])]

Error: exception while executing query: Failure while executing query.
(state=,code=0)

----------
Physical Plan:
{
  head : {
    type : "APACHE_DRILL_PHYSICAL",
    version : 1,
    generator : {
      type : "manual"
    }
  },
  graph : [ {
    pop : "cassandra-scan",
    @id : 1,
    cassandraScanSpec : {
      keyspace : "[KEYSPACE_NAME]",
      table : "[TABLE_NAME]"
    },
    storage:
    {
      "type":"cassandra",
      host: "localhost",
      port : 9042
    },
    columns: [
    "id", "pog_id"
    ]
  },
  {
    pop : "screen",
    @id : 2,
    child : 1
  } ]
}
----------
Info:
KEYSPACE_NAME - recsys
TABLE_NAME - trending_now
----------
I am able to view cassandra and all the Keyspaces in Cassandra via Sqlline.
Describing the table trending_now gives me the below result:
0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
+-------------+------------+-------------+
| COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
+-------------+------------+-------------+
| *           | ANY        | YES         |
+-------------+------------+-------------+

Code:
https://github.com/yssharma/drill/tree/cassandra-storage
https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42

Re: Drill Cassandra Storage - Unexpected exception during fragment initialization

Posted by Yash Sharma <ya...@gmail.com>.
Well that worked. Thanks Guys.

Thanks Jason :)
There is still lot of work to be done. Have put down a skeleton that lets
me fire simple projections on Cassandra for now.Will have a demo once it
comes to a good shape.
Will keep you all posted.

On Thu, Jan 8, 2015 at 12:09 AM, Jason Altekruse <al...@gmail.com>
wrote:

> Jinfeng beat me to it. This looks really cool so far, can't wait to see a
> demo!
>
> -Jason
>
> On Wed, Jan 7, 2015 at 10:32 AM, Jinfeng Ni <ji...@gmail.com> wrote:
>
> > Caused by: java.lang.NullPointerException
> > at
> > org.apache.drill.exec.planner.logical.DrillScanRel.
> > computeSelfCost(DrillScanRel.java:149)
> > ~[classes/:na]
> >
> > This line of code points to :
> >
> >  double rowCount = stats.getRecordCount();
> >
> > Did you override  groupScan.getScanStats() ?  If not, you can try to
> return
> > a default ScanStat instance.
> >
> >
> >
> > On Wed, Jan 7, 2015 at 10:24 AM, Yash Sharma <ya...@gmail.com> wrote:
> >
> > > I see a NPE somewhere in middle. Will try to crack it tomorrow.
> > > Let me know if you guys find any hints.
> > >
> > > Thanks
> > >
> > > Caused by: java.lang.NullPointerException
> > > at
> > >
> > >
> >
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> > > ~[classes/:na]
> > > at
> > >
> > >
> >
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > > ~[na:1.7.0_45]
> > > at
> > >
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > > ~[na:1.7.0_45]
> > > at
> > >
> > >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > > ~[na:1.7.0_45]
> > > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > >
> > > On Wed, Jan 7, 2015 at 11:46 PM, Yash Sharma <ya...@gmail.com>
> wrote:
> > >
> > > > -------------------------
> > > > Lilith Log:
> > > > -------------------------
> > > >
> > > > org.apache.drill.exec.work.foreman.ForemanException: Unexpected
> > exception
> > > > during fragment initialization: Internal error: Error while applying
> > rule
> > > > DrillPushProjIntoScan, args
> > > >
> > >
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > > >
> rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > > > recsys, trending_now])]
> > > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:194)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > > [na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > > [na:1.7.0_45]
> > > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > > Caused by: java.lang.AssertionError: Internal error: Error while
> > applying
> > > > rule DrillPushProjIntoScan, args
> > > >
> > >
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > > >
> rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > > > recsys, trending_now])]
> > > > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:246)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > > ~[classes/:na]
> > > > at
> org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > > [classes/:na]
> > > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > > [na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > > [na:1.7.0_45]
> > > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > > ... 4 more
> > > > Caused by: java.lang.AssertionError: Internal error: Error occurred
> > while
> > > > applying rule DrillPushProjIntoScan
> > > > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:143)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > > ~[classes/:na]
> > > > at
> org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > > [classes/:na]
> > > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > > [na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > > [na:1.7.0_45]
> > > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > > ... 12 more
> > > > Caused by: java.lang.reflect.UndeclaredThrowableException
> > > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source)
> ~[na:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > > ~[classes/:na]
> > > > at
> org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > > [classes/:na]
> > > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > > [na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > > [na:1.7.0_45]
> > > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > > ... 15 more
> > > > Caused by: java.lang.reflect.InvocationTargetException
> > > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > > > ~[na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > > > ~[na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > > > ~[na:1.7.0_45]
> > > > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source)
> ~[na:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > > ~[classes/:na]
> > > > at
> org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > > [classes/:na]
> > > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > > [na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > > [na:1.7.0_45]
> > > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > > ... 31 more
> > > > Caused by: java.lang.NullPointerException
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > > > ~[na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > > > ~[na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > > > ~[na:1.7.0_45]
> > > > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source)
> ~[na:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > > ~[classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > > ~[classes/:na]
> > > > at
> org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > > [classes/:na]
> > > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > > [classes/:na]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > > [na:1.7.0_45]
> > > > at
> > > >
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > > [na:1.7.0_45]
> > > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > > ... 36 more
> > > >
> > > >
> > > >
> > > > On Wed, Jan 7, 2015 at 11:39 PM, Timothy Chen <tn...@gmail.com>
> > wrote:
> > > >
> > > >> Is there a stack trace or inner exceptions that you can see from
> > Lilith?
> > > >>
> > > >> Tim
> > > >>
> > > >> On Wed, Jan 7, 2015 at 10:06 AM, Yash Sharma <ya...@gmail.com>
> > wrote:
> > > >> > Hi All,
> > > >> > I am writing Cassandra storage for Drill and am facing this error
> > > while
> > > >> > firing a simple select query against Cassandra table on Sqlline.
> > > >> >
> > > >> > I am able to get the expected results by using a physical plan
> > > directly.
> > > >> >
> > > >> > Any thoughts on what I might be missing.
> > > >> >
> > > >> > Thanks
> > > >> >
> > > >> >
> > > >> > Some more info for troubleshooting -
> > > >> >
> > > >> > Query:
> > > >> > 0: jdbc:drill:zk=local> select id from
> > > cassandra.recsys.`trending_now`;
> > > >> >
> > > >> > Query failed: Query failed: Unexpected exception during fragment
> > > >> > initialization: Internal error: Error while applying rule
> > > >> > DrillPushProjIntoScan, args
> > > >> >
> > > >>
> > >
> >
> [rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
> > > >> >
> > > >>
> > >
> rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > > >> > recsys, trending_now])]
> > > >> >
> > > >> > Error: exception while executing query: Failure while executing
> > query.
> > > >> > (state=,code=0)
> > > >> >
> > > >> > ----------
> > > >> > Physical Plan:
> > > >> > {
> > > >> >   head : {
> > > >> >     type : "APACHE_DRILL_PHYSICAL",
> > > >> >     version : 1,
> > > >> >     generator : {
> > > >> >       type : "manual"
> > > >> >     }
> > > >> >   },
> > > >> >   graph : [ {
> > > >> >     pop : "cassandra-scan",
> > > >> >     @id : 1,
> > > >> >     cassandraScanSpec : {
> > > >> >       keyspace : "[KEYSPACE_NAME]",
> > > >> >       table : "[TABLE_NAME]"
> > > >> >     },
> > > >> >     storage:
> > > >> >     {
> > > >> >       "type":"cassandra",
> > > >> >       host: "localhost",
> > > >> >       port : 9042
> > > >> >     },
> > > >> >     columns: [
> > > >> >     "id", "pog_id"
> > > >> >     ]
> > > >> >   },
> > > >> >   {
> > > >> >     pop : "screen",
> > > >> >     @id : 2,
> > > >> >     child : 1
> > > >> >   } ]
> > > >> > }
> > > >> > ----------
> > > >> > Info:
> > > >> > KEYSPACE_NAME - recsys
> > > >> > TABLE_NAME - trending_now
> > > >> > ----------
> > > >> > I am able to view cassandra and all the Keyspaces in Cassandra via
> > > >> Sqlline.
> > > >> > Describing the table trending_now gives me the below result:
> > > >> > 0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
> > > >> > +-------------+------------+-------------+
> > > >> > | COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
> > > >> > +-------------+------------+-------------+
> > > >> > | *           | ANY        | YES         |
> > > >> > +-------------+------------+-------------+
> > > >> >
> > > >> > Code:
> > > >> > https://github.com/yssharma/drill/tree/cassandra-storage
> > > >> >
> > > >>
> > >
> >
> https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42
> > > >>
> > > >
> > > >
> > >
> >
>

Re: Drill Cassandra Storage - Unexpected exception during fragment initialization

Posted by Jason Altekruse <al...@gmail.com>.
Jinfeng beat me to it. This looks really cool so far, can't wait to see a
demo!

-Jason

On Wed, Jan 7, 2015 at 10:32 AM, Jinfeng Ni <ji...@gmail.com> wrote:

> Caused by: java.lang.NullPointerException
> at
> org.apache.drill.exec.planner.logical.DrillScanRel.
> computeSelfCost(DrillScanRel.java:149)
> ~[classes/:na]
>
> This line of code points to :
>
>  double rowCount = stats.getRecordCount();
>
> Did you override  groupScan.getScanStats() ?  If not, you can try to return
> a default ScanStat instance.
>
>
>
> On Wed, Jan 7, 2015 at 10:24 AM, Yash Sharma <ya...@gmail.com> wrote:
>
> > I see a NPE somewhere in middle. Will try to crack it tomorrow.
> > Let me know if you guys find any hints.
> >
> > Thanks
> >
> > Caused by: java.lang.NullPointerException
> > at
> >
> >
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> > ~[classes/:na]
> > at
> >
> >
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > ~[na:1.7.0_45]
> > at
> >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > ~[na:1.7.0_45]
> > at
> >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > ~[na:1.7.0_45]
> > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> >
> > On Wed, Jan 7, 2015 at 11:46 PM, Yash Sharma <ya...@gmail.com> wrote:
> >
> > > -------------------------
> > > Lilith Log:
> > > -------------------------
> > >
> > > org.apache.drill.exec.work.foreman.ForemanException: Unexpected
> exception
> > > during fragment initialization: Internal error: Error while applying
> rule
> > > DrillPushProjIntoScan, args
> > >
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > > rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > > recsys, trending_now])]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:194)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > Caused by: java.lang.AssertionError: Internal error: Error while
> applying
> > > rule DrillPushProjIntoScan, args
> > >
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > > rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > > recsys, trending_now])]
> > > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:246)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 4 more
> > > Caused by: java.lang.AssertionError: Internal error: Error occurred
> while
> > > applying rule DrillPushProjIntoScan
> > > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:143)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 12 more
> > > Caused by: java.lang.reflect.UndeclaredThrowableException
> > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 15 more
> > > Caused by: java.lang.reflect.InvocationTargetException
> > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > > ~[na:1.7.0_45]
> > > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 31 more
> > > Caused by: java.lang.NullPointerException
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > > ~[na:1.7.0_45]
> > > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 36 more
> > >
> > >
> > >
> > > On Wed, Jan 7, 2015 at 11:39 PM, Timothy Chen <tn...@gmail.com>
> wrote:
> > >
> > >> Is there a stack trace or inner exceptions that you can see from
> Lilith?
> > >>
> > >> Tim
> > >>
> > >> On Wed, Jan 7, 2015 at 10:06 AM, Yash Sharma <ya...@gmail.com>
> wrote:
> > >> > Hi All,
> > >> > I am writing Cassandra storage for Drill and am facing this error
> > while
> > >> > firing a simple select query against Cassandra table on Sqlline.
> > >> >
> > >> > I am able to get the expected results by using a physical plan
> > directly.
> > >> >
> > >> > Any thoughts on what I might be missing.
> > >> >
> > >> > Thanks
> > >> >
> > >> >
> > >> > Some more info for troubleshooting -
> > >> >
> > >> > Query:
> > >> > 0: jdbc:drill:zk=local> select id from
> > cassandra.recsys.`trending_now`;
> > >> >
> > >> > Query failed: Query failed: Unexpected exception during fragment
> > >> > initialization: Internal error: Error while applying rule
> > >> > DrillPushProjIntoScan, args
> > >> >
> > >>
> >
> [rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
> > >> >
> > >>
> > rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > >> > recsys, trending_now])]
> > >> >
> > >> > Error: exception while executing query: Failure while executing
> query.
> > >> > (state=,code=0)
> > >> >
> > >> > ----------
> > >> > Physical Plan:
> > >> > {
> > >> >   head : {
> > >> >     type : "APACHE_DRILL_PHYSICAL",
> > >> >     version : 1,
> > >> >     generator : {
> > >> >       type : "manual"
> > >> >     }
> > >> >   },
> > >> >   graph : [ {
> > >> >     pop : "cassandra-scan",
> > >> >     @id : 1,
> > >> >     cassandraScanSpec : {
> > >> >       keyspace : "[KEYSPACE_NAME]",
> > >> >       table : "[TABLE_NAME]"
> > >> >     },
> > >> >     storage:
> > >> >     {
> > >> >       "type":"cassandra",
> > >> >       host: "localhost",
> > >> >       port : 9042
> > >> >     },
> > >> >     columns: [
> > >> >     "id", "pog_id"
> > >> >     ]
> > >> >   },
> > >> >   {
> > >> >     pop : "screen",
> > >> >     @id : 2,
> > >> >     child : 1
> > >> >   } ]
> > >> > }
> > >> > ----------
> > >> > Info:
> > >> > KEYSPACE_NAME - recsys
> > >> > TABLE_NAME - trending_now
> > >> > ----------
> > >> > I am able to view cassandra and all the Keyspaces in Cassandra via
> > >> Sqlline.
> > >> > Describing the table trending_now gives me the below result:
> > >> > 0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
> > >> > +-------------+------------+-------------+
> > >> > | COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
> > >> > +-------------+------------+-------------+
> > >> > | *           | ANY        | YES         |
> > >> > +-------------+------------+-------------+
> > >> >
> > >> > Code:
> > >> > https://github.com/yssharma/drill/tree/cassandra-storage
> > >> >
> > >>
> >
> https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42
> > >>
> > >
> > >
> >
>

Re: Drill Cassandra Storage - Unexpected exception during fragment initialization

Posted by Yash Sharma <ya...@gmail.com>.
Cool. Will check this tomm.

Thanks Tim and Jinfeng.
On 08/01/2015 12:02 am, "Jinfeng Ni" <ji...@gmail.com> wrote:

> Caused by: java.lang.NullPointerException
> at
> org.apache.drill.exec.planner.logical.DrillScanRel.
> computeSelfCost(DrillScanRel.java:149)
> ~[classes/:na]
>
> This line of code points to :
>
>  double rowCount = stats.getRecordCount();
>
> Did you override  groupScan.getScanStats() ?  If not, you can try to return
> a default ScanStat instance.
>
>
>
> On Wed, Jan 7, 2015 at 10:24 AM, Yash Sharma <ya...@gmail.com> wrote:
>
> > I see a NPE somewhere in middle. Will try to crack it tomorrow.
> > Let me know if you guys find any hints.
> >
> > Thanks
> >
> > Caused by: java.lang.NullPointerException
> > at
> >
> >
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> > ~[classes/:na]
> > at
> >
> >
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > ~[na:1.7.0_45]
> > at
> >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > ~[na:1.7.0_45]
> > at
> >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > ~[na:1.7.0_45]
> > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> >
> > On Wed, Jan 7, 2015 at 11:46 PM, Yash Sharma <ya...@gmail.com> wrote:
> >
> > > -------------------------
> > > Lilith Log:
> > > -------------------------
> > >
> > > org.apache.drill.exec.work.foreman.ForemanException: Unexpected
> exception
> > > during fragment initialization: Internal error: Error while applying
> rule
> > > DrillPushProjIntoScan, args
> > >
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > > rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > > recsys, trending_now])]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:194)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > Caused by: java.lang.AssertionError: Internal error: Error while
> applying
> > > rule DrillPushProjIntoScan, args
> > >
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > > rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > > recsys, trending_now])]
> > > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:246)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 4 more
> > > Caused by: java.lang.AssertionError: Internal error: Error occurred
> while
> > > applying rule DrillPushProjIntoScan
> > > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:143)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 12 more
> > > Caused by: java.lang.reflect.UndeclaredThrowableException
> > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 15 more
> > > Caused by: java.lang.reflect.InvocationTargetException
> > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > > ~[na:1.7.0_45]
> > > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 31 more
> > > Caused by: java.lang.NullPointerException
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > > ~[na:1.7.0_45]
> > > at
> > >
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > > ~[na:1.7.0_45]
> > > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > > at
> > >
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > > ~[optiq-core-0.9-drill-r16.jar:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > > ~[classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > > ~[classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > > [classes/:na]
> > > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > > [classes/:na]
> > > at
> > >
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > > [classes/:na]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > > [na:1.7.0_45]
> > > at
> > >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > > [na:1.7.0_45]
> > > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > > ... 36 more
> > >
> > >
> > >
> > > On Wed, Jan 7, 2015 at 11:39 PM, Timothy Chen <tn...@gmail.com>
> wrote:
> > >
> > >> Is there a stack trace or inner exceptions that you can see from
> Lilith?
> > >>
> > >> Tim
> > >>
> > >> On Wed, Jan 7, 2015 at 10:06 AM, Yash Sharma <ya...@gmail.com>
> wrote:
> > >> > Hi All,
> > >> > I am writing Cassandra storage for Drill and am facing this error
> > while
> > >> > firing a simple select query against Cassandra table on Sqlline.
> > >> >
> > >> > I am able to get the expected results by using a physical plan
> > directly.
> > >> >
> > >> > Any thoughts on what I might be missing.
> > >> >
> > >> > Thanks
> > >> >
> > >> >
> > >> > Some more info for troubleshooting -
> > >> >
> > >> > Query:
> > >> > 0: jdbc:drill:zk=local> select id from
> > cassandra.recsys.`trending_now`;
> > >> >
> > >> > Query failed: Query failed: Unexpected exception during fragment
> > >> > initialization: Internal error: Error while applying rule
> > >> > DrillPushProjIntoScan, args
> > >> >
> > >>
> >
> [rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
> > >> >
> > >>
> > rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > >> > recsys, trending_now])]
> > >> >
> > >> > Error: exception while executing query: Failure while executing
> query.
> > >> > (state=,code=0)
> > >> >
> > >> > ----------
> > >> > Physical Plan:
> > >> > {
> > >> >   head : {
> > >> >     type : "APACHE_DRILL_PHYSICAL",
> > >> >     version : 1,
> > >> >     generator : {
> > >> >       type : "manual"
> > >> >     }
> > >> >   },
> > >> >   graph : [ {
> > >> >     pop : "cassandra-scan",
> > >> >     @id : 1,
> > >> >     cassandraScanSpec : {
> > >> >       keyspace : "[KEYSPACE_NAME]",
> > >> >       table : "[TABLE_NAME]"
> > >> >     },
> > >> >     storage:
> > >> >     {
> > >> >       "type":"cassandra",
> > >> >       host: "localhost",
> > >> >       port : 9042
> > >> >     },
> > >> >     columns: [
> > >> >     "id", "pog_id"
> > >> >     ]
> > >> >   },
> > >> >   {
> > >> >     pop : "screen",
> > >> >     @id : 2,
> > >> >     child : 1
> > >> >   } ]
> > >> > }
> > >> > ----------
> > >> > Info:
> > >> > KEYSPACE_NAME - recsys
> > >> > TABLE_NAME - trending_now
> > >> > ----------
> > >> > I am able to view cassandra and all the Keyspaces in Cassandra via
> > >> Sqlline.
> > >> > Describing the table trending_now gives me the below result:
> > >> > 0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
> > >> > +-------------+------------+-------------+
> > >> > | COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
> > >> > +-------------+------------+-------------+
> > >> > | *           | ANY        | YES         |
> > >> > +-------------+------------+-------------+
> > >> >
> > >> > Code:
> > >> > https://github.com/yssharma/drill/tree/cassandra-storage
> > >> >
> > >>
> >
> https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42
> > >>
> > >
> > >
> >
>

Re: Drill Cassandra Storage - Unexpected exception during fragment initialization

Posted by Jinfeng Ni <ji...@gmail.com>.
Caused by: java.lang.NullPointerException
at
org.apache.drill.exec.planner.logical.DrillScanRel.
computeSelfCost(DrillScanRel.java:149)
~[classes/:na]

This line of code points to :

 double rowCount = stats.getRecordCount();

Did you override  groupScan.getScanStats() ?  If not, you can try to return
a default ScanStat instance.



On Wed, Jan 7, 2015 at 10:24 AM, Yash Sharma <ya...@gmail.com> wrote:

> I see a NPE somewhere in middle. Will try to crack it tomorrow.
> Let me know if you guys find any hints.
>
> Thanks
>
> Caused by: java.lang.NullPointerException
> at
>
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> ~[classes/:na]
> at
>
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> ~[na:1.7.0_45]
> at
>
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> ~[na:1.7.0_45]
> at
>
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> ~[na:1.7.0_45]
> at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
>
> On Wed, Jan 7, 2015 at 11:46 PM, Yash Sharma <ya...@gmail.com> wrote:
>
> > -------------------------
> > Lilith Log:
> > -------------------------
> >
> > org.apache.drill.exec.work.foreman.ForemanException: Unexpected exception
> > during fragment initialization: Internal error: Error while applying rule
> > DrillPushProjIntoScan, args
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > recsys, trending_now])]
> > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:194)
> > [classes/:na]
> > at
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > [classes/:na]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > [na:1.7.0_45]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > [na:1.7.0_45]
> > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > Caused by: java.lang.AssertionError: Internal error: Error while applying
> > rule DrillPushProjIntoScan, args
> >
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> > rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > recsys, trending_now])]
> > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:246)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > ~[classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > [classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > [classes/:na]
> > at
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > [classes/:na]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > [na:1.7.0_45]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > [na:1.7.0_45]
> > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > ... 4 more
> > Caused by: java.lang.AssertionError: Internal error: Error occurred while
> > applying rule DrillPushProjIntoScan
> > at org.eigenbase.util.Util.newInternal(Util.java:750)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:143)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > ~[classes/:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > ~[classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > [classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > [classes/:na]
> > at
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > [classes/:na]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > [na:1.7.0_45]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > [na:1.7.0_45]
> > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > ... 12 more
> > Caused by: java.lang.reflect.UndeclaredThrowableException
> > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > at
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > ~[classes/:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > ~[classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > [classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > [classes/:na]
> > at
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > [classes/:na]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > [na:1.7.0_45]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > [na:1.7.0_45]
> > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > ... 15 more
> > Caused by: java.lang.reflect.InvocationTargetException
> > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > ~[na:1.7.0_45]
> > at
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > ~[na:1.7.0_45]
> > at
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > ~[na:1.7.0_45]
> > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > at
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > at
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > ~[classes/:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > ~[classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > [classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > [classes/:na]
> > at
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > [classes/:na]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > [na:1.7.0_45]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > [na:1.7.0_45]
> > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > ... 31 more
> > Caused by: java.lang.NullPointerException
> > at
> >
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> > ~[classes/:na]
> > at
> >
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> > ~[na:1.7.0_45]
> > at
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> > ~[na:1.7.0_45]
> > at
> >
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> > ~[na:1.7.0_45]
> > at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> > at
> >
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> > at
> >
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> > ~[classes/:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> > net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> > ~[optiq-core-0.9-drill-r16.jar:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> > ~[classes/:na]
> > at
> >
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> > ~[classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> > [classes/:na]
> > at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> > [classes/:na]
> > at
> >
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> > [classes/:na]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> > [na:1.7.0_45]
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> > [na:1.7.0_45]
> > at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> > ... 36 more
> >
> >
> >
> > On Wed, Jan 7, 2015 at 11:39 PM, Timothy Chen <tn...@gmail.com> wrote:
> >
> >> Is there a stack trace or inner exceptions that you can see from Lilith?
> >>
> >> Tim
> >>
> >> On Wed, Jan 7, 2015 at 10:06 AM, Yash Sharma <ya...@gmail.com> wrote:
> >> > Hi All,
> >> > I am writing Cassandra storage for Drill and am facing this error
> while
> >> > firing a simple select query against Cassandra table on Sqlline.
> >> >
> >> > I am able to get the expected results by using a physical plan
> directly.
> >> >
> >> > Any thoughts on what I might be missing.
> >> >
> >> > Thanks
> >> >
> >> >
> >> > Some more info for troubleshooting -
> >> >
> >> > Query:
> >> > 0: jdbc:drill:zk=local> select id from
> cassandra.recsys.`trending_now`;
> >> >
> >> > Query failed: Query failed: Unexpected exception during fragment
> >> > initialization: Internal error: Error while applying rule
> >> > DrillPushProjIntoScan, args
> >> >
> >>
> [rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
> >> >
> >>
> rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> >> > recsys, trending_now])]
> >> >
> >> > Error: exception while executing query: Failure while executing query.
> >> > (state=,code=0)
> >> >
> >> > ----------
> >> > Physical Plan:
> >> > {
> >> >   head : {
> >> >     type : "APACHE_DRILL_PHYSICAL",
> >> >     version : 1,
> >> >     generator : {
> >> >       type : "manual"
> >> >     }
> >> >   },
> >> >   graph : [ {
> >> >     pop : "cassandra-scan",
> >> >     @id : 1,
> >> >     cassandraScanSpec : {
> >> >       keyspace : "[KEYSPACE_NAME]",
> >> >       table : "[TABLE_NAME]"
> >> >     },
> >> >     storage:
> >> >     {
> >> >       "type":"cassandra",
> >> >       host: "localhost",
> >> >       port : 9042
> >> >     },
> >> >     columns: [
> >> >     "id", "pog_id"
> >> >     ]
> >> >   },
> >> >   {
> >> >     pop : "screen",
> >> >     @id : 2,
> >> >     child : 1
> >> >   } ]
> >> > }
> >> > ----------
> >> > Info:
> >> > KEYSPACE_NAME - recsys
> >> > TABLE_NAME - trending_now
> >> > ----------
> >> > I am able to view cassandra and all the Keyspaces in Cassandra via
> >> Sqlline.
> >> > Describing the table trending_now gives me the below result:
> >> > 0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
> >> > +-------------+------------+-------------+
> >> > | COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
> >> > +-------------+------------+-------------+
> >> > | *           | ANY        | YES         |
> >> > +-------------+------------+-------------+
> >> >
> >> > Code:
> >> > https://github.com/yssharma/drill/tree/cassandra-storage
> >> >
> >>
> https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42
> >>
> >
> >
>

Re: Drill Cassandra Storage - Unexpected exception during fragment initialization

Posted by Yash Sharma <ya...@gmail.com>.
I see a NPE somewhere in middle. Will try to crack it tomorrow.
Let me know if you guys find any hints.

Thanks

Caused by: java.lang.NullPointerException
at
org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
~[classes/:na]
at
org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
~[optiq-core-0.9-drill-r16.jar:na]
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
~[na:1.7.0_45]
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
~[na:1.7.0_45]
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
~[na:1.7.0_45]
at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]

On Wed, Jan 7, 2015 at 11:46 PM, Yash Sharma <ya...@gmail.com> wrote:

> -------------------------
> Lilith Log:
> -------------------------
>
> org.apache.drill.exec.work.foreman.ForemanException: Unexpected exception
> during fragment initialization: Internal error: Error while applying rule
> DrillPushProjIntoScan, args
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> recsys, trending_now])]
> at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:194)
> [classes/:na]
> at
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> [classes/:na]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> [na:1.7.0_45]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> [na:1.7.0_45]
> at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> Caused by: java.lang.AssertionError: Internal error: Error while applying
> rule DrillPushProjIntoScan, args
> [rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
> rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> recsys, trending_now])]
> at org.eigenbase.util.Util.newInternal(Util.java:750)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:246)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> ~[classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> [classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> [classes/:na]
> at
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> [classes/:na]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> [na:1.7.0_45]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> [na:1.7.0_45]
> at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> ... 4 more
> Caused by: java.lang.AssertionError: Internal error: Error occurred while
> applying rule DrillPushProjIntoScan
> at org.eigenbase.util.Util.newInternal(Util.java:750)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:143)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> ~[classes/:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> ~[classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> [classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> [classes/:na]
> at
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> [classes/:na]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> [na:1.7.0_45]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> [na:1.7.0_45]
> at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> ... 12 more
> Caused by: java.lang.reflect.UndeclaredThrowableException
> at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> at
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> ~[classes/:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> ~[classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> [classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> [classes/:na]
> at
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> [classes/:na]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> [na:1.7.0_45]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> [na:1.7.0_45]
> at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> ... 15 more
> Caused by: java.lang.reflect.InvocationTargetException
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> ~[na:1.7.0_45]
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> ~[na:1.7.0_45]
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> ~[na:1.7.0_45]
> at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> at
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> at
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> ~[classes/:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> ~[classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> [classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> [classes/:na]
> at
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> [classes/:na]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> [na:1.7.0_45]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> [na:1.7.0_45]
> at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> ... 31 more
> Caused by: java.lang.NullPointerException
> at
> org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
> ~[classes/:na]
> at
> org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> ~[na:1.7.0_45]
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> ~[na:1.7.0_45]
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> ~[na:1.7.0_45]
> at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
> at
> org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
> at
> org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
> ~[classes/:na]
> at
> org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
> ~[optiq-core-0.9-drill-r16.jar:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
> ~[classes/:na]
> at
> org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
> ~[classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
> [classes/:na]
> at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
> [classes/:na]
> at
> org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
> [classes/:na]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> [na:1.7.0_45]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> [na:1.7.0_45]
> at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> ... 36 more
>
>
>
> On Wed, Jan 7, 2015 at 11:39 PM, Timothy Chen <tn...@gmail.com> wrote:
>
>> Is there a stack trace or inner exceptions that you can see from Lilith?
>>
>> Tim
>>
>> On Wed, Jan 7, 2015 at 10:06 AM, Yash Sharma <ya...@gmail.com> wrote:
>> > Hi All,
>> > I am writing Cassandra storage for Drill and am facing this error while
>> > firing a simple select query against Cassandra table on Sqlline.
>> >
>> > I am able to get the expected results by using a physical plan directly.
>> >
>> > Any thoughts on what I might be missing.
>> >
>> > Thanks
>> >
>> >
>> > Some more info for troubleshooting -
>> >
>> > Query:
>> > 0: jdbc:drill:zk=local> select id from cassandra.recsys.`trending_now`;
>> >
>> > Query failed: Query failed: Unexpected exception during fragment
>> > initialization: Internal error: Error while applying rule
>> > DrillPushProjIntoScan, args
>> >
>> [rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
>> >
>> rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
>> > recsys, trending_now])]
>> >
>> > Error: exception while executing query: Failure while executing query.
>> > (state=,code=0)
>> >
>> > ----------
>> > Physical Plan:
>> > {
>> >   head : {
>> >     type : "APACHE_DRILL_PHYSICAL",
>> >     version : 1,
>> >     generator : {
>> >       type : "manual"
>> >     }
>> >   },
>> >   graph : [ {
>> >     pop : "cassandra-scan",
>> >     @id : 1,
>> >     cassandraScanSpec : {
>> >       keyspace : "[KEYSPACE_NAME]",
>> >       table : "[TABLE_NAME]"
>> >     },
>> >     storage:
>> >     {
>> >       "type":"cassandra",
>> >       host: "localhost",
>> >       port : 9042
>> >     },
>> >     columns: [
>> >     "id", "pog_id"
>> >     ]
>> >   },
>> >   {
>> >     pop : "screen",
>> >     @id : 2,
>> >     child : 1
>> >   } ]
>> > }
>> > ----------
>> > Info:
>> > KEYSPACE_NAME - recsys
>> > TABLE_NAME - trending_now
>> > ----------
>> > I am able to view cassandra and all the Keyspaces in Cassandra via
>> Sqlline.
>> > Describing the table trending_now gives me the below result:
>> > 0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
>> > +-------------+------------+-------------+
>> > | COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
>> > +-------------+------------+-------------+
>> > | *           | ANY        | YES         |
>> > +-------------+------------+-------------+
>> >
>> > Code:
>> > https://github.com/yssharma/drill/tree/cassandra-storage
>> >
>> https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42
>>
>
>

Re: Drill Cassandra Storage - Unexpected exception during fragment initialization

Posted by Yash Sharma <ya...@gmail.com>.
-------------------------
Lilith Log:
-------------------------

org.apache.drill.exec.work.foreman.ForemanException: Unexpected exception
during fragment initialization: Internal error: Error while applying rule
DrillPushProjIntoScan, args
[rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
recsys, trending_now])]
at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:194)
[classes/:na]
at
org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
[classes/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[na:1.7.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
Caused by: java.lang.AssertionError: Internal error: Error while applying
rule DrillPushProjIntoScan, args
[rel#8:ProjectRel.NONE.ANY([]).[](child=rel#7:Subset#0.ENUMERABLE.ANY([]).[],id=$1,pog_id=$2),
rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
recsys, trending_now])]
at org.eigenbase.util.Util.newInternal(Util.java:750)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:246)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
~[optiq-core-0.9-drill-r16.jar:na]
at
net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
~[optiq-core-0.9-drill-r16.jar:na]
at net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
~[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
[classes/:na]
at
org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
[classes/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[na:1.7.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
... 4 more
Caused by: java.lang.AssertionError: Internal error: Error occurred while
applying rule DrillPushProjIntoScan
at org.eigenbase.util.Util.newInternal(Util.java:750)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:143)
~[optiq-core-0.9-drill-r16.jar:na]
at org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
~[classes/:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
~[optiq-core-0.9-drill-r16.jar:na]
at
net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
~[optiq-core-0.9-drill-r16.jar:na]
at net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
~[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
[classes/:na]
at
org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
[classes/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[na:1.7.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
... 12 more
Caused by: java.lang.reflect.UndeclaredThrowableException
at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
at
org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
~[optiq-core-0.9-drill-r16.jar:na]
at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
~[optiq-core-0.9-drill-r16.jar:na]
at org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
~[classes/:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
~[optiq-core-0.9-drill-r16.jar:na]
at
net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
~[optiq-core-0.9-drill-r16.jar:na]
at net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
~[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
[classes/:na]
at
org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
[classes/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[na:1.7.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
... 15 more
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
~[na:1.7.0_45]
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
~[na:1.7.0_45]
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
~[na:1.7.0_45]
at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
at
org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
~[optiq-core-0.9-drill-r16.jar:na]
at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
at
org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
~[optiq-core-0.9-drill-r16.jar:na]
at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
~[optiq-core-0.9-drill-r16.jar:na]
at org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
~[classes/:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
~[optiq-core-0.9-drill-r16.jar:na]
at
net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
~[optiq-core-0.9-drill-r16.jar:na]
at net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
~[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
[classes/:na]
at
org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
[classes/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[na:1.7.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
... 31 more
Caused by: java.lang.NullPointerException
at
org.apache.drill.exec.planner.logical.DrillScanRel.computeSelfCost(DrillScanRel.java:149)
~[classes/:na]
at
org.eigenbase.rel.metadata.RelMdPercentageOriginalRows.getNonCumulativeCost(RelMdPercentageOriginalRows.java:159)
~[optiq-core-0.9-drill-r16.jar:na]
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
~[na:1.7.0_45]
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
~[na:1.7.0_45]
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
~[na:1.7.0_45]
at java.lang.reflect.Method.invoke(Method.java:606) ~[na:1.7.0_45]
at
org.eigenbase.rel.metadata.ReflectiveRelMetadataProvider$2$1.invoke(ReflectiveRelMetadataProvider.java:135)
~[optiq-core-0.9-drill-r16.jar:na]
at com.sun.proxy.$Proxy39.getNonCumulativeCost(Unknown Source) ~[na:na]
at
org.eigenbase.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:125)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:926)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:333)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.RelSubset.propagateCostImprovements(RelSubset.java:314)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.asd(VolcanoPlanner.java:1635)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1568)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1)
~[optiq-core-0.9-drill-r16.jar:na]
at org.eigenbase.rel.AbstractRelNode.onRegister(AbstractRelNode.java:301)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1464)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.register(VolcanoPlanner.java:837)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:860)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1750)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:131)
~[optiq-core-0.9-drill-r16.jar:na]
at org.eigenbase.relopt.RelOptRuleCall.transformTo(RelOptRuleCall.java:210)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:85)
~[classes/:na]
at
org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:223)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:661)
~[optiq-core-0.9-drill-r16.jar:na]
at
net.hydromatic.optiq.tools.Programs$RuleSetProgram.run(Programs.java:165)
~[optiq-core-0.9-drill-r16.jar:na]
at net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:273)
~[optiq-core-0.9-drill-r16.jar:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:155)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:134)
~[classes/:na]
at
org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:145)
~[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:507)
[classes/:na]
at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:185)
[classes/:na]
at
org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:254)
[classes/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[na:1.7.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
... 36 more



On Wed, Jan 7, 2015 at 11:39 PM, Timothy Chen <tn...@gmail.com> wrote:

> Is there a stack trace or inner exceptions that you can see from Lilith?
>
> Tim
>
> On Wed, Jan 7, 2015 at 10:06 AM, Yash Sharma <ya...@gmail.com> wrote:
> > Hi All,
> > I am writing Cassandra storage for Drill and am facing this error while
> > firing a simple select query against Cassandra table on Sqlline.
> >
> > I am able to get the expected results by using a physical plan directly.
> >
> > Any thoughts on what I might be missing.
> >
> > Thanks
> >
> >
> > Some more info for troubleshooting -
> >
> > Query:
> > 0: jdbc:drill:zk=local> select id from cassandra.recsys.`trending_now`;
> >
> > Query failed: Query failed: Unexpected exception during fragment
> > initialization: Internal error: Error while applying rule
> > DrillPushProjIntoScan, args
> >
> [rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
> > rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> > recsys, trending_now])]
> >
> > Error: exception while executing query: Failure while executing query.
> > (state=,code=0)
> >
> > ----------
> > Physical Plan:
> > {
> >   head : {
> >     type : "APACHE_DRILL_PHYSICAL",
> >     version : 1,
> >     generator : {
> >       type : "manual"
> >     }
> >   },
> >   graph : [ {
> >     pop : "cassandra-scan",
> >     @id : 1,
> >     cassandraScanSpec : {
> >       keyspace : "[KEYSPACE_NAME]",
> >       table : "[TABLE_NAME]"
> >     },
> >     storage:
> >     {
> >       "type":"cassandra",
> >       host: "localhost",
> >       port : 9042
> >     },
> >     columns: [
> >     "id", "pog_id"
> >     ]
> >   },
> >   {
> >     pop : "screen",
> >     @id : 2,
> >     child : 1
> >   } ]
> > }
> > ----------
> > Info:
> > KEYSPACE_NAME - recsys
> > TABLE_NAME - trending_now
> > ----------
> > I am able to view cassandra and all the Keyspaces in Cassandra via
> Sqlline.
> > Describing the table trending_now gives me the below result:
> > 0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
> > +-------------+------------+-------------+
> > | COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
> > +-------------+------------+-------------+
> > | *           | ANY        | YES         |
> > +-------------+------------+-------------+
> >
> > Code:
> > https://github.com/yssharma/drill/tree/cassandra-storage
> >
> https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42
>

Re: Drill Cassandra Storage - Unexpected exception during fragment initialization

Posted by Timothy Chen <tn...@gmail.com>.
Is there a stack trace or inner exceptions that you can see from Lilith?

Tim

On Wed, Jan 7, 2015 at 10:06 AM, Yash Sharma <ya...@gmail.com> wrote:
> Hi All,
> I am writing Cassandra storage for Drill and am facing this error while
> firing a simple select query against Cassandra table on Sqlline.
>
> I am able to get the expected results by using a physical plan directly.
>
> Any thoughts on what I might be missing.
>
> Thanks
>
>
> Some more info for troubleshooting -
>
> Query:
> 0: jdbc:drill:zk=local> select id from cassandra.recsys.`trending_now`;
>
> Query failed: Query failed: Unexpected exception during fragment
> initialization: Internal error: Error while applying rule
> DrillPushProjIntoScan, args
> [rel#2753:ProjectRel.NONE.ANY([]).[](child=rel#2752:Subset#0.ENUMERABLE.ANY([]).[],id=$1),
> rel#2747:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[cassandra,
> recsys, trending_now])]
>
> Error: exception while executing query: Failure while executing query.
> (state=,code=0)
>
> ----------
> Physical Plan:
> {
>   head : {
>     type : "APACHE_DRILL_PHYSICAL",
>     version : 1,
>     generator : {
>       type : "manual"
>     }
>   },
>   graph : [ {
>     pop : "cassandra-scan",
>     @id : 1,
>     cassandraScanSpec : {
>       keyspace : "[KEYSPACE_NAME]",
>       table : "[TABLE_NAME]"
>     },
>     storage:
>     {
>       "type":"cassandra",
>       host: "localhost",
>       port : 9042
>     },
>     columns: [
>     "id", "pog_id"
>     ]
>   },
>   {
>     pop : "screen",
>     @id : 2,
>     child : 1
>   } ]
> }
> ----------
> Info:
> KEYSPACE_NAME - recsys
> TABLE_NAME - trending_now
> ----------
> I am able to view cassandra and all the Keyspaces in Cassandra via Sqlline.
> Describing the table trending_now gives me the below result:
> 0: jdbc:drill:zk=local> describe cassandra.recsys.`trending_now`;
> +-------------+------------+-------------+
> | COLUMN_NAME | DATA_TYPE  | IS_NULLABLE |
> +-------------+------------+-------------+
> | *           | ANY        | YES         |
> +-------------+------------+-------------+
>
> Code:
> https://github.com/yssharma/drill/tree/cassandra-storage
> https://github.com/yssharma/drill/commit/494b205d2ad42d740ffad68f8af31421cdf40a42