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 "ASF subversion and git services (Jira)" <ji...@apache.org> on 2020/06/03 18:13:00 UTC

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

    [ https://issues.apache.org/jira/browse/IMPALA-9809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17125201#comment-17125201 ] 

ASF subversion and git services commented on IMPALA-9809:
---------------------------------------------------------

Commit 37b5599a7a25536a6e5265788cf4e139db3b0cfa in impala's branch refs/heads/master from Yongzhi Chen
[ https://gitbox.apache.org/repos/asf?p=impala.git;h=37b5599 ]

IMPALA-9809: Multi-aggregation query on particular dataset crashes impalad

In streaming-aggregation-node.cc , when replicate_input_ is true
and num_aggs > 1, it will call AddBatchStreaming several
times(more than 1), each time, the out_batch will be used.
If a row is not cached, the value will be saved in the out_batch,
and out_batch's row count will be increased.
The row_count did not set back to 0 when next while loop. Therefore
in out_batch, it is possible that not all the tuples are non-null.
(For example the rows added when agg_idx = 1, only tuple with 1 not
null; the rows added when when agg_idx = 2, only tuple with 2 not
null). But in grouping-aggregation-ir.cc, the serialize out code is
start from very beginning of out_batch for a agg_idx, it has good
chance to hit null tuple.

Fix the issue by only serialize the tuples being added by
current function call.

Tests:
Manual tests
Unit tests

Change-Id: I06d73171cdc40bdbb15960573030ac7fc94a7e16
Reviewed-on: http://gerrit.cloudera.org:8080/16019
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


> 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
>            Priority: Critical
>
> 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