Uploaded image for project: 'IMPALA'
  1. IMPALA
  2. IMPALA-3510

qgen: Check failed: row_idx < capacity_ (0 vs. 0)

    XMLWordPrintableJSON

Details

    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`;
      

      Attachments

        Activity

          People

            kwho Michael Ho
            mikeb Michael Brown
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: