You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues-all@impala.apache.org by "Yongzhi Chen (Jira)" <ji...@apache.org> on 2020/06/01 20:57:00 UTC

[jira] [Created] (IMPALA-9809) A query with multi-aggregation functions on particular dataset crashes impala daemon

Yongzhi Chen created IMPALA-9809:
------------------------------------

             Summary: A query with multi-aggregation functions on  particular dataset crashes impala daemon
                 Key: IMPALA-9809
                 URL: https://issues.apache.org/jira/browse/IMPALA-9809
             Project: IMPALA
          Issue Type: Bug
          Components: Backend
    Affects Versions: Impala 3.4.0
            Reporter: Yongzhi Chen
            Assignee: Yongzhi Chen


Table :
create external table tbl_agg (
  c0 string, c1 string,
  c2 int, c3 string) stored as parquet location
  'file://$IMPALA_HOME/testdata/data/local_parquet_tbl';

Query:
select
c0,
min(case when c1>'2000-01-01' then c1 else null end) as flt,
count(distinct case when c2 >70 then c3 end) as cl,
count(distinct c3) as cnl
from tbl_agg
group by c0;

With particular dataset crashes impalad:
{noformat}
(gdb) bt
#0  0x00000000022dff50 in impala::Tuple::IsNull (this=0x0, offset=...) at /home/yongzhi/Impala/be/src/runtime/tuple.h:250
#1  0x000000000296fe28 in impala::AggFnEvaluator::SerializeOrFinalize (this=0xe888760, src=0x0, dst_slot_desc=..., dst=0x0, fn=
    0x2974198 <impala::AggregateFunctions::UpdateVal<impala_udf::StringVal>(impala_udf::FunctionContext*, impala_udf::StringVal const&, impala_udf::StringVal*)+434>)
    at /home/yongzhi/Impala/be/src/exprs/agg-fn-evaluator.cc:399
#2  0x0000000002931679 in impala::AggFnEvaluator::Serialize (this=0xe888760, tuple=0x0) at /home/yongzhi/Impala/be/src/exprs/agg-fn-evaluator.h:269
#3  0x00000000029316ed in impala::AggFnEvaluator::Serialize (evals=..., dst=0x0) at /home/yongzhi/Impala/be/src/exprs/agg-fn-evaluator.h:300
#4  0x00000000029379a6 in impala::GroupingAggregator::AddBatchStreamingImpl (this=0x7ff6e91458d0, agg_idx=0, needs_serialize=166, prefetch_mode=(unknown: 32758), in_batch=0x0, out_batch=0xc144480, 
    ht_ctx=0xd939200, remaining_capacity=0x7ff6e9145c30) at /home/yongzhi/Impala/be/src/exec/grouping-aggregator-ir.cc:207
#5  0x0000000002928cd2 in impala::GroupingAggregator::AddBatchStreaming (this=0xfc81b00, state=0xd386e00, out_batch=0xc144480, child_batch=0xc144540, eos=0x7ff6e9145eff)
    at /home/yongzhi/Impala/be/src/exec/grouping-aggregator.cc:486
#6  0x0000000002961eeb in impala::StreamingAggregationNode::GetRowsStreaming (this=0xdd8a5a0, state=0xd386e00, out_batch=0xc144480)
    at /home/yongzhi/Impala/be/src/exec/streaming-aggregation-node.cc:125
#7  0x00000000029614dc in impala::Status::Status (status=..., this=0xd436de0) at /home/yongzhi/Impala/be/src/common/status.h:105
#8  impala::StreamingAggregationNode::GetNext (this=0xdd8a5a0, state=0xd386e00, row_batch=0xc144480, eos=0x7ff6e91464b7) at /home/yongzhi/Impala/be/src/exec/streaming-aggregation-node.cc:63
#9  0x0000000002320cea in impala::FragmentInstanceState::ExecInternal (this=0xbf8b5c0) at /home/yongzhi/Impala/be/src/runtime/fragment-instance-state.cc:364
#10 0x000000000231da4d in impala::FragmentInstanceState::Exec (this=0xbf8b5c0) at /home/yongzhi/Impala/be/src/runtime/fragment-instance-state.cc:98
#11 0x0000000002331c35 in impala::QueryState::ExecFInstance (this=0xe84d000, fis=0xbf8b5c0) at /home/yongzhi/Impala/be/src/runtime/query-state.cc:754
#12 0x000000000232ffa8 in impala::QueryState::<lambda()>::operator()(void) const (__closure=0x7ff6e9146c98) at /home/yongzhi/Impala/be/src/runtime/query-state.cc:662
#13 0x000000000233381d in boost::detail::function::void_function_obj_invoker0<impala::QueryState::StartFInstances()::<lambda()>, void>::invoke(boost::detail::function::function_buffer &) (
    function_obj_ptr=...) at /home/yongzhi/Impala/toolchain/boost-1.61.0-p2/include/boost/function/function_template.hpp:159
#14 0x000000000210fdac in boost::function0<void>::operator() (this=0x7ff6e9146c90) at /home/yongzhi/Impala/toolchain/boost-1.61.0-p2/include/boost/function/function_template.hpp:771
#15 0x00000000026f35c1 in impala::Thread::SuperviseThread(std::string const&, std::string const&, boost::function<void ()>, impala::ThreadDebugInfo const*, impala::Promise<long, (impala::PromiseMode)0>*) (name=..., category=..., functor=..., parent_thread_info=0x7ff6e9947940, thread_started=0x7ff6e99468e0) at /home/yongzhi/Impala/be/src/util/thread.cc:360
#16 0x00000000026fb845 in boost::_bi::list5<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void ()> >, boost::_bi::value<impala::ThreadDebugInfo*>, boost::_bi::value<impala::Promise<long, (impala::PromiseMode)0>*> >::operator()<void (*)(std::string const&, std::string const&, boost::function<void ()>, impala::ThreadDebugInfo const*, impala::Promise<long, (impala::PromiseMode)0>*), boost::_bi::list0>(boost::_bi::type<void>, void (*&)(std::string const&, std::string const&, boost::function<void ()>, impala::ThreadDebugInfo const*, impala::Promise<long, (impala::PromiseMode)0>*), boost::_bi::list0&, int) (this=0xd3861c0, f=@0xd3861b8: <error reading variable>, a=...)
    at /home/yongzhi/Impala/toolchain/boost-1.61.0-p2/include/boost/bind/bind.hpp:531
#17 0x00000000026fb769 in boost::_bi::bind_t<void, void (*)(std::string const&, std::string const&, boost::function<void ()>, impala::ThreadDebugInfo const*, impala::Promise<long, (impala::PromiseMode)0>*), boost::_bi::list5<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void ()> >, boost::_bi::value<impala::ThreadDebugInfo*>, boost::_bi::value<impala::Promise<long, (impala::PromiseMode)0>*> > >::operator()() (this=0xd3861b8) at /home/yongzhi/Impala/toolchain/boost-1.61.0-p2/include/boost/bind/bind.hpp:1222
#18 0x00000000026fb72c in boost::detail::thread_data<boost::_bi::bind_t<void, void (*)(std::string const&, std::string const&, boost::function<void ()>, impala::ThreadDebugInfo const*, impala::Promise<long, (impala::PromiseMode)0>*), boost::_bi::list5<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void ()> >, boost::_bi::value<impala::ThreadDebugInfo*>, boost::_bi::value<impala::Promise<long, (impala::PromiseMode)0>*> > > >::run() (this=0xd386000) at /home/yongzhi/Impala/toolchain/boost-1.61.0-p2/include/boost/thread/detail/thread.hpp:116
#19 0x0000000003f5957a in thread_proxy ()
#20 0x00007ff7aa8546ba in start_thread () from /lib/x86_64-linux-gnu/libpthread.so.0
#21 0x00007ff7a74ae41d in clone () from /lib/x86_64-linux-gnu/libc.so.6
{noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-all-unsubscribe@impala.apache.org
For additional commands, e-mail: issues-all-help@impala.apache.org