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

Crash: impala::HdfsScanNode::Init

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Duplicate
    • Impala 2.3.0
    • None
    • None
    • Randomly generated data, I can provide access to the machine that has it loaded.

    Description

      Query:

      SELECT
      t1162.field_59.field_63 AS float_col
      FROM table_3 t1162
      LEFT JOIN (
      WITH with_1 AS (SELECT
      MIN(COALESCE(LEAST(392, -808), -814, 695)) OVER (ORDER BY t1166.pos DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 91 FOLLOWING) AS int_col,
      FIRST_VALUE(-963) OVER (ORDER BY t1166.pos DESC) AS int_col_t1167,
      t1166.pos AS int_col_t1168
      FROM t1162.field_58 t1163
      INNER JOIN t1163.value t1164
      INNER JOIN t1163.value t1165 ON (((t1165.pos) = (t1164.pos)) AND ((t1165.pos) = (t1164.pos))) AND ((t1165.pos) = (t1164.pos))
      INNER JOIN t1162.field_74 t1166 ON ((t1166.pos) = (t1165.pos)) AND ((t1166.pos) = (t1165.pos)))
      SELECT
      COALESCE(LAG(SUM(688), 80) OVER (PARTITION BY MIN(t1172.pos) ORDER BY MIN(t1172.pos) DESC, (t1170.pos) + (t1170.pos) DESC), -457, 49) AS int_col,
      CAST(FIRST_VALUE(SUM(-265)) OVER (PARTITION BY MIN(t1172.pos) ORDER BY MIN(t1172.pos) ASC, COALESCE((t1170.pos) + (t1170.pos), MAX(COALESCE(t1170.pos, t1171.pos, t1171.pos)), (t1170.pos) + (t1170.pos)) ASC ROWS BETWEEN 64 FOLLOWING AND UNBOUNDED FOLLOWING) AS STRING) AS char_col,
      MIN(t1172.pos) AS int_col_t1173,
      (t1170.pos) + (t1170.pos) AS int_col_t1174,
      COALESCE((t1170.pos) + (t1170.pos), MAX(COALESCE(t1170.pos, t1171.pos, t1171.pos)), (t1170.pos) + (t1170.pos)) AS int_col_t1175
      FROM t1162.field_58 t1169
      INNER JOIN t1169.value t1170
      INNER JOIN t1162.field_74 t1171 ON ((t1171.pos) = (t1170.pos)) AND ((t1171.pos) = (t1170.pos))
      LEFT JOIN t1169.value t1172 ON (t1172.pos) = (t1170.pos)
      GROUP BY
      (t1170.pos) + (t1170.pos)
      HAVING
      (CAST('2029-04-30 00:00:00' AS TIMESTAMP)) < (MAX(CAST('2004-12-18 00:00:00' AS TIMESTAMP)))
      ) t1176
      INNER JOIN t1162.field_74 t1177 ON (t1177.pos) = (t1162.field_71)
      INNER JOIN table_4 t1178 ON (((t1178.field_75) = (t1162.field_71)) AND ((t1178.field_75) = (t1162.field_73))) AND ((t1178.field_75) = (t1162.field_59.field_66))
      

      Stack Trace:

      #0  0x00007f51fa0c7cc9 in __GI_raise (sig=sig@entry=6) at ../nptl/sysdeps/unix/sysv/linux/raise.c:56
      #1  0x00007f51fa0cb0d8 in __GI_abort () at abort.c:89
      #2  0x0000000002118579 in google::DumpStackTraceAndExit () at src/utilities.cc:147
      #3  0x000000000211162d in google::LogMessage::Fail () at src/logging.cc:1315
      #4  0x00000000021134b5 in google::LogMessage::SendToLog (this=0x7f5146a90750) at src/logging.cc:1269
      #5  0x0000000002111183 in google::LogMessage::Flush (this=this@entry=0x7f5146a90750) at src/logging.cc:1138
      #6  0x0000000002113e0e in google::LogMessageFatal::~LogMessageFatal (this=0x7f5146a90750, __in_chrg=<optimized out>) at src/logging.cc:1836
      #7  0x00000000015eed17 in impala::HdfsScanNode::Init (this=0x8351000, tnode=...) at /home/dev/Impala/be/src/exec/hdfs-scan-node.cc:132
      #8  0x00000000015d2d2f in impala::ExecNode::CreateTreeHelper (pool=0x93a15e0, tnodes=..., descs=..., parent=0xc49bb80, node_idx=0x7f5146a9097c, root=0x0) at /home/dev/Impala/be/src/exec/exec-node.cc:258
      #9  0x00000000015d2c3d in impala::ExecNode::CreateTreeHelper (pool=0x93a15e0, tnodes=..., descs=..., parent=0xcc0ac00, node_idx=0x7f5146a9097c, root=0x0) at /home/dev/Impala/be/src/exec/exec-node.cc:249
      #10 0x00000000015d2c3d in impala::ExecNode::CreateTreeHelper (pool=0x93a15e0, tnodes=..., descs=..., parent=0x0, node_idx=0x7f5146a9097c, root=0xd218f30) at /home/dev/Impala/be/src/exec/exec-node.cc:249
      #11 0x00000000015d28c2 in impala::ExecNode::CreateTree (pool=0x93a15e0, plan=..., descs=..., root=0xd218f30) at /home/dev/Impala/be/src/exec/exec-node.cc:214
      #12 0x00000000015962d4 in impala::PlanFragmentExecutor::Prepare (this=0xd218f28, request=...) at /home/dev/Impala/be/src/runtime/plan-fragment-executor.cc:199
      #13 0x000000000137d9e5 in impala::FragmentMgr::FragmentExecState::Prepare (this=0xd218d00, exec_params=...) at /home/dev/Impala/be/src/service/fragment-exec-state.cc:44
      #14 0x0000000001376165 in impala::FragmentMgr::ExecPlanFragment (this=0xcb28de0, exec_params=...) at /home/dev/Impala/be/src/service/fragment-mgr.cc:51
      #15 0x00000000012ef7a9 in impala::ImpalaInternalService::ExecPlanFragment (this=0xcb2b350, return_val=..., params=...) at /home/dev/Impala/be/src/service/impala-internal-service.h:37
      #16 0x00000000014c6367 in impala::ImpalaInternalServiceProcessor::process_ExecPlanFragment (this=0xcb28d80, seqid=0, iprot=0xcc57c80, oprot=0xcc57c40, callContext=0xd2aa2c0) at /home/dev/Impala/be/generated-sources/gen-cpp/ImpalaInternalService.cpp:949
      #17 0x00000000014c6078 in impala::ImpalaInternalServiceProcessor::dispatchCall (this=0xcb28d80, iprot=0xcc57c80, oprot=0xcc57c40, fname=..., seqid=0, callContext=0xd2aa2c0) at /home/dev/Impala/be/generated-sources/gen-cpp/ImpalaInternalService.cpp:922
      #18 0x00000000012e631d in apache::thrift::TDispatchProcessor::process (this=0xcb28d80, in=..., out=..., connectionContext=0xd2aa2c0) at /home/dev/Impala/thirdparty/thrift-0.9.0/build/include/thrift/TDispatchProcessor.h:121
      #19 0x00000000020ba553 in apache::thrift::server::TThreadedServer::Task::run (this=0xccf58c0) at src/thrift/server/TThreadedServer.cpp:70
      #20 0x00000000012114c1 in impala::ThriftThread::RunRunnable (this=0xd2aa200, runnable=..., promise=0x7f519d33e790) at /home/dev/Impala/be/src/rpc/thrift-thread.cc:61
      #21 0x0000000001212c89 in boost::_mfi::mf2<void, impala::ThriftThread, boost::shared_ptr<apache::thrift::concurrency::Runnable>, impala::Promise<unsigned long>*>::operator() (this=0xdcaa840, p=0xd2aa200, a1=..., a2=0x7f519d33e790) at /usr/include/boost/bind/mem_fn_template.hpp:280
      #22 0x0000000001212ae4 in boost::_bi::list3<boost::_bi::value<impala::ThriftThread*>, boost::_bi::value<boost::shared_ptr<apache::thrift::concurrency::Runnable> >, boost::_bi::value<impala::Promise<unsigned long>*> >::operator()<boost::_mfi::mf2<void, impala::ThriftThread, boost::shared_ptr<apache::thrift::concurrency::Runnable>, impala::Promise<unsigned long>*>, boost::_bi::list0> (this=0xdcaa850, f=..., a=...) at /usr/include/boost/bind/bind.hpp:392
      #23 0x0000000001212875 in boost::_bi::bind_t<void, boost::_mfi::mf2<void, impala::ThriftThread, boost::shared_ptr<apache::thrift::concurrency::Runnable>, impala::Promise<unsigned long>*>, boost::_bi::list3<boost::_bi::value<impala::ThriftThread*>, boost::_bi::value<boost::shared_ptr<apache::thrift::concurrency::Runnable> >, boost::_bi::value<impala::Promise<unsigned long>*> > >::operator() (this=0xdcaa840) at /usr/include/boost/bind/bind_template.hpp:20
      #24 0x0000000001212796 in boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf2<void, impala::ThriftThread, boost::shared_ptr<apache::thrift::concurrency::Runnable>, impala::Promise<unsigned long>*>, boost::_bi::list3<boost::_bi::value<impala::ThriftThread*>, boost::_bi::value<boost::shared_ptr<apache::thrift::concurrency::Runnable> >, boost::_bi::value<impala::Promise<unsigned long>*> > >, void>::invoke (function_obj_ptr=...) at /usr/include/boost/function/function_template.hpp:153
      #25 0x0000000001247b28 in boost::function0<void>::operator() (this=0x7f5146a91e00) at /usr/include/boost/function/function_template.hpp:767
      #26 0x0000000001457d5b in impala::Thread::SuperviseThread(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*) (name=..., category=..., functor=..., thread_started=0x7f519d33e5d0) at /home/dev/Impala/be/src/util/thread.cc:314
      #27 0x0000000001460f91 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=0xd2ac9c0, f=@0xd2ac9b8: 0x1457a52 <impala::Thread::SuperviseThread(std::string const&, std::string const&, boost::function<void ()>, impala::Promise<long>*)>, a=...) at /usr/include/boost/bind/bind.hpp:457
      #28 0x0000000001460edb 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=0xd2ac9b8) at /usr/include/boost/bind/bind_template.hpp:20
      #29 0x0000000001460ea0 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=0xd2ac800) at /usr/include/boost/thread/detail/thread.hpp:117
      #30 0x00007f51fd091a4a in ?? () from /usr/lib/x86_64-linux-gnu/libboost_thread.so.1.54.0
      #31 0x00007f51fc53e182 in start_thread (arg=0x7f5146a92700) at pthread_create.c:312
      #32 0x00007f51fa18b47d in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111
      

      impalad.FATAL:

      F0924 13:16:48.641844 15662 hdfs-scan-node.cc:132] Check failed: conjuncts_map_[tuple_id_].empty()
      

      Attachments

        Activity

          People

            alex.behm Alexander Behm
            tarasbob Taras Bobrovytsky
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: