Details
-
Bug
-
Status: Resolved
-
Blocker
-
Resolution: Fixed
-
Impala 2.6.0
Description
Over night, the query generator hit the L141 DCHECK on 3 different queries:
138 TupleRow* ALWAYS_INLINE GetRow(int row_idx) { 139 DCHECK(tuple_ptrs_ != NULL); 140 DCHECK_GE(row_idx, 0); 141 DCHECK_LT(row_idx, capacity_); 142 return reinterpret_cast<TupleRow*>(tuple_ptrs_ + row_idx * num_tuples_per_row_); 143 }
The logs show the following for all cases:
row-batch.h:141] Check failed: row_idx < capacity_ (0 vs. 0)
Here is one of the queries:
SELECT a3.int_col_2, a3.tinyint_col_17, (311) / (AVG(COALESCE(-753, -636, NULL)) OVER (PARTITION BY a3.tinyint_col_17)) AS float_col, a3.tinyint_col_17 AS int_col FROM ( SELECT MAX(a2.timestamp_col_4) AS timestamp_col, GREATEST(COALESCE(a1.int_col_36, 686), COALESCE(a1.int_col_35, -31)) AS int_col, GREATEST(-349, COALESCE(FIRST_VALUE(-841) OVER (PARTITION BY GREATEST(COALESCE(COUNT(a1.bigint_col_10), -571), COALESCE(GREATEST(COALESCE(a1.int_col_36, 686), COALESCE(a1.int_col_35, -31)), 300)) ORDER BY GREATEST(COALESCE(COUNT(a1.bigint_col_10), -571), COALESCE(GREATEST(COALESCE(a1.int_col_36, 686), COALESCE(a1.int_col_35, -31)), 300)) DESC, a1.tinyint_col_17 ASC), -563)) AS int_col_1, GREATEST(COALESCE(COUNT(a1.bigint_col_10), -571), COALESCE(GREATEST(COALESCE(a1.int_col_36, 686), COALESCE(a1.int_col_35, -31)), 300)) AS int_col_2, a1.tinyint_col_17 FROM table_2 a1 INNER JOIN table_10 a2 ON (((TRIM(a2.char0052_col_15)) = (a1.string_col_65)) AND ((a2.decimal3213_col_13) = (a1.decimal1204_col_21))) AND ((a2.decimal0303_col_31) = (a1.decimal1606_col_57)) GROUP BY GREATEST(COALESCE(a1.int_col_36, 686), COALESCE(a1.int_col_35, -31)), a1.tinyint_col_17 ) a3 INNER JOIN table_8 a4 ON ((a4.smallint_col_8) = (a3.int_col_1)) AND ((a4.timestamp_col_11) = (a3.timestamp_col))
...and the corresponding backtrace:
google::DumpStackTraceAndExit() () google::LogMessage::Fail() () google::LogMessage::SendToLog() () google::LogMessage::Flush() () google::LogMessageFatal::~LogMessageFatal() () GetRow (row_idx=0, this=0x17ab7200) at /home/dev/Impala/be/src/runtime/row-batch.h:141 impala::PartitionedHashJoinNode::ProcessProbeBatch<0> (this=0xdd90d80, out_batch=0x17ab9840, ht_ctx=0x1773d0e0, status=0x7fc71f6e2640) at /home/dev/Impala/be/src/exec/partitioned-hash-join-node-ir.cc:335 impala::PartitionedHashJoinNode::ProcessProbeBatch (this=0xdd90d80, join_op=impala::TJoinOp::INNER_JOIN, out_batch=0x17ab9840, ht_ctx=0x1773d0e0, status=0x7fc71f6e2640) at /home/dev/Impala/be/src/exec/partitioned-hash-join-node-ir.cc:354 impala::PartitionedHashJoinNode::GetNext (this=0xdd90d80, state=0xf120400, out_batch=0x17ab9840, eos=0xff7f099) at /home/dev/Impala/be/src/exec/partitioned-hash-join-node.cc:956 impala::PlanFragmentExecutor::GetNextInternal (this=0xff7ef70, batch=0x7fc71f6e2e58) at /home/dev/Impala/be/src/runtime/plan-fragment-executor.cc:491 impala::PlanFragmentExecutor::OpenInternal (this=0xff7ef70) at /home/dev/Impala/be/src/runtime/plan-fragment-executor.cc:364 impala::PlanFragmentExecutor::Open (this=0xff7ef70) at /home/dev/Impala/be/src/runtime/plan-fragment-executor.cc:327 impala::FragmentMgr::FragmentExecState::Exec (this=0xff7ed00) at /home/dev/Impala/be/src/service/fragment-exec-state.cc:54 impala::FragmentMgr::FragmentThread (this=0xdeead80, fragment_instance_id=...) at /home/dev/Impala/be/src/service/fragment-mgr.cc:86 boost::_mfi::mf1::operator() (this=0xdcf9260, p=0xdeead80, a1=...) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/bind/mem_fn_template.hpp:165 boost::_bi::list2, boost::_bi::value >::operator(), boost::_bi::list0> (this=0xdcf9270, f=..., a=...) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/bind/bind.hpp:313 boost::_bi::bind_t, boost::_bi::list2, boost::_bi::value > >::operator() (this=0xdcf9260) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/bind/bind_template.hpp:20 boost::detail::function::void_function_obj_invoker0, boost::_bi::list2, boost::_bi::value > >, void>::invoke (function_obj_ptr=...) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/function/function_template.hpp:153 boost::function0::operator() (this=0x7fc71f6e3da0) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/function/function_template.hpp:767 impala::Thread::SuperviseThread(std::string const&, std::string const&, boost::function, impala::Promise*) (name=..., category=..., functor=..., thread_started=0x7fc720ee5aa0) at /home/dev/Impala/be/src/util/thread.cc:315 boost::_bi::list4, boost::_bi::value, boost::_bi::value >, boost::_bi::value*> >::operator(), impala::Promise*), boost::_bi::list0>(boost::_bi::type, void (*&)(std::string const&, std::string const&, boost::function, impala::Promise*), boost::_bi::list0&, int) (this=0xe66a7c0, f=@0xe66a7b8: 0x156b200 , impala::Promise*)>, a=...) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/bind/bind.hpp:457 boost::_bi::bind_t, impala::Promise*), boost::_bi::list4, boost::_bi::value, boost::_bi::value >, boost::_bi::value*> > >::operator()() (this=0xe66a7b8) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/bind/bind_template.hpp:20 boost::detail::thread_data, impala::Promise*), boost::_bi::list4, boost::_bi::value, boost::_bi::value >, boost::_bi::value*> > > >::run() (this=0xe66a600) at /opt/Impala-Toolchain/boost-1.57.0/include/boost/thread/detail/thread.hpp:116 thread_proxy () start_thread (arg=0x7fc71f6e4700) at pthread_create.c:312 clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111
The hash of the build was 8c22e795d77d29229136c61295bc9f7d4c8024f9.
I was able to reproduce these crashes.
$ ssh vd0206.halxg.cloudera.com $ ssh dev@192.168.123.3 # sorry, I didn't set up port forwarding to this docker container
There are core files in /tmp/core_files if you want to inspect them. You can also build Impala and run queries....
$ cd $IMPALA_HOME $ source bin/impala-config.sh $ export KUDU_IS_SUPPORTED=false # KUDU-1419 $ ./buildall.sh -notests $ testdata/bin/run-all.sh $ bin/start-impala-cluster.py $ impala-shell.sh > USE `randomness`;