You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@impala.apache.org by "Thomas Tauber-Marshall (JIRA)" <ji...@apache.org> on 2017/04/26 18:16:04 UTC

[jira] [Resolved] (IMPALA-5125) Check failed: tuple_desc_map_.back() != __null

     [ https://issues.apache.org/jira/browse/IMPALA-5125?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Thomas Tauber-Marshall resolved IMPALA-5125.
--------------------------------------------
       Resolution: Fixed
    Fix Version/s: Impala 2.9.0

commit 915a16345c9325f29cad2a4c113d960e434b4ba7
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Date:   Fri Apr 14 12:36:46 2017 -0700

    IMPALA-5125: SimplifyConditionalsRule incorrectly handles aggregates
    
    This patch addresses 3 issues:
    - SelectList.reset() didn't properly reset some of its members, though
      they're documented as needing to be reset. This was causing a crash
      when the Planner attempted to make an aggregation node for an agg
      function that had been eliminated by expr rewriting. While I'm here,
      I added resetting of all of SelectList's members that need to be
      reset, and fixed the documentation of one member that shouldn't be
      reset.
    - SimplifyConditionalsRule was changing the meaning of queries that
      contain agg functions, e.g. because "select if(true, 0, sum(id))"
      is not equivalent to "select 0". The fix is to not return the
      simplfied expr if it removes all aggregates.
    - ExprRewriteRulesTest was performing rewrites on the result exprs of
      the SelectStmt, which causes problems if the result exprs have been
      substituted. In normal query execution, we don't rewrite the result
      exprs anyway, so the fix is to match normal query execution and
      rewrite the select list exprs.
    
    Testing:
    - Added e2e test to exprs.test.
    - Added unit test to ExprRewriteRulesTest.
    
    Change-Id: Ic20b1621753980b47a612e0885804363b733f6da
    Reviewed-on: http://gerrit.cloudera.org:8080/6653
    Reviewed-by: Thomas Tauber-Marshall <tm...@cloudera.com>
    Tested-by: Impala Public Jenkins

> Check failed: tuple_desc_map_.back() != __null
> ----------------------------------------------
>
>                 Key: IMPALA-5125
>                 URL: https://issues.apache.org/jira/browse/IMPALA-5125
>             Project: IMPALA
>          Issue Type: Bug
>          Components: Frontend
>    Affects Versions: Impala 2.9.0
>            Reporter: Michael Brown
>            Assignee: Thomas Tauber-Marshall
>            Priority: Blocker
>              Labels: crash, query_generator, regression
>             Fix For: Impala 2.9.0
>
>
> The following query seems to produce this dcheck reliably:
> {noformat}
> USE tpch;
> SELECT
> IF(True, 514, COUNT(a1.ps_partkey)) AS int_col
> FROM partsupp a1
> WHERE
> (a1.ps_availqty) IN (SELECT
> (a2.ps_availqty) * (((a2.ps_partkey) + (538.4658339348)) - (-445.1448857941)) AS decimal_col
> FROM partsupp a2
> WHERE
> (False) AND ((a2.ps_partkey) < (a2.ps_suppkey)))
> {noformat}
> {noformat}
> #6  0x00000000028c316e in google::LogMessageFatal::~LogMessageFatal() ()
> #7  0x00000000013f1d1a in impala::RowDescriptor::RowDescriptor (this=0x93bfe48, desc_tbl=..., row_tuples=..., nullable_tuples=...) at /home/mikeb/Impala/be/src/runtime/descriptors.cc:364
> #8  0x00000000016a94a1 in impala::ExecNode::ExecNode (this=0x93bfe00, pool=0x9752b40, tnode=..., descs=...) at /home/mikeb/Impala/be/src/exec/exec-node.cc:134
> #9  0x000000000179c27a in impala::PartitionedAggregationNode::PartitionedAggregationNode (this=0x93bfe00, pool=0x9752b40, tnode=..., descs=...) at /home/mikeb/Impala/be/src/exec/partitioned-aggregation-node.cc:137
> #10 0x00000000016ab599 in impala::ExecNode::CreateNode (pool=0x9752b40, tnode=..., descs=..., node=0x7fcd18fa4878, state=0x9817500) at /home/mikeb/Impala/be/src/exec/exec-node.cc:303
> #11 0x00000000016aab70 in impala::ExecNode::CreateTreeHelper (state=0x9817500, tnodes=..., descs=..., parent=0x0, node_idx=0x7fcd18fa494c, root=0xad690d0) at /home/mikeb/Impala/be/src/exec/exec-node.cc:239
> #12 0x00000000016aa82e in impala::ExecNode::CreateTree (state=0x9817500, plan=..., descs=..., root=0xad690d0) at /home/mikeb/Impala/be/src/exec/exec-node.cc:217
> #13 0x0000000001a6fe26 in impala::PlanFragmentExecutor::PrepareInternal (this=0xad690d0, qs=0x9183c00, tdesc_tbl=..., fragment_ctx=..., instance_ctx=...) at /home/mikeb/Impala/be/src/runtime/plan-fragment-executor.cc:177
> #14 0x0000000001a6eaff in impala::PlanFragmentExecutor::Prepare (this=0xad690d0, query_state=0x9183c00, desc_tbl=..., fragment_ctx=..., instance_ctx=...) at /home/mikeb/Impala/be/src/runtime/plan-fragment-executor.cc:99
> #15 0x0000000001a6ba7b in impala::FragmentInstanceState::Exec (this=0xad68e00) at /home/mikeb/Impala/be/src/runtime/fragment-instance-state.cc:64
> #16 0x0000000001a77167 in impala::QueryExecMgr::ExecFInstance (this=0x9a46ba0, fis=0xad68e00) at /home/mikeb/Impala/be/src/runtime/query-exec-mgr.cc:110
> #17 0x0000000001a79f90 in boost::_mfi::mf1<void, impala::QueryExecMgr, impala::FragmentInstanceState*>::operator() (this=0x9752ac0, p=0x9a46ba0, a1=0xad68e00)
>     at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/bind/mem_fn_template.hpp:165
> #18 0x0000000001a79e19 in boost::_bi::list2<boost::_bi::value<impala::QueryExecMgr*>, boost::_bi::value<impala::FragmentInstanceState*> >::operator()<boost::_mfi::mf1<void, impala::QueryExecMgr, impala::FragmentInstanceState*>, boost::_bi::list0> (this=0x9752ad0, f=..., a=...) at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/bind/bind.hpp:313
> #19 0x0000000001a7994d in boost::_bi::bind_t<void, boost::_mfi::mf1<void, impala::QueryExecMgr, impala::FragmentInstanceState*>, boost::_bi::list2<boost::_bi::value<impala::QueryExecMgr*>, boost::_bi::value<impala::FragmentInstanceState*> > >::operator() (this=0x9752ac0) at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/bind/bind_template.hpp:20
> #20 0x0000000001a79558 in boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, impala::QueryExecMgr, impala::FragmentInstanceState*>, boost::_bi::list2<boost::_bi::value<impala::QueryExecMgr*>, boost::_bi::value<impala::FragmentInstanceState*> > >, void>::invoke (function_obj_ptr=...) at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/function/function_template.hpp:153
> #21 0x000000000137cb70 in boost::function0<void>::operator() (this=0x7fcd18fa5d20) at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/function/function_template.hpp:767
> #22 0x000000000162f1c7 in impala::Thread::SuperviseThread(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*) (name=..., category=..., functor=..., thread_started=0x7fcd1a7aea80)
>     at /home/mikeb/Impala/be/src/util/thread.cc:325
> #23 0x0000000001637ba2 in boost::_bi::list4<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void ()> >, boost::_bi::value<impala::Promise<long>*> >::operator()<void (*)(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*), boost::_bi::list0>(boost::_bi::type<void>, void (*&)(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*), boost::_bi::list0&, int) (this=0x8a2c5c0, f=@0x8a2c5b8: 0x162ef02 <impala::Thread::SuperviseThread(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*)>, a=...)
>     at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/bind/bind.hpp:457
> #24 0x0000000001637ae5 in boost::_bi::bind_t<void, void (*)(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*), boost::_bi::list4<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void ()> >, boost::_bi::value<impala::Promise<long>*> > >::operator()() (this=0x8a2c5b8) at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/bind/bind_template.hpp:20
> #25 0x0000000001637aa8 in boost::detail::thread_data<boost::_bi::bind_t<void, void (*)(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*), boost::_bi::list4<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void ()> >, boost::_bi::value<impala::Promise<long>*> > > >::run() (this=0x8a2c400) at /home/mikeb/Impala/toolchain/boost-1.57.0-p1/include/boost/thread/detail/thread.hpp:116
> #26 0x0000000001af148a in thread_proxy ()
> #27 0x00007fcda3de6184 in start_thread (arg=0x7fcd18fa6700) at pthread_create.c:312
> #28 0x00007fcda3b1337d in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111
> {noformat}
> Commit: {{IMPALA-5113: fix dirty unpinned invariant}}
> *Workaround*
> {code}
> SET ENABLE_EXPR_REWRITES=FALSE;
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)