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

DCHECK failure: FE produces tuple with decimal slot but agg fn returns double

    XMLWordPrintableJSON

Details

    Description

      DCHECK failure in AggFnEvaluator. Intermediate tuple has a DECIMAL slot for the result of an agg fn which returns a double.

      QUERY:

      SELECT
      FIRST_VALUE(-32.9) OVER (ORDER BY month ROWS BETWEEN 92 PRECEDING AND UNBOUNDED FOLLOWING)
      FROM alltypestiny
      

      STACK:

      raise () from /lib64/libc.so.6
      raise () from /lib64/libc.so.6
      abort () from /lib64/libc.so.6
      google::DumpStackTraceAndExit () at src/utilities.cc:147
      google::LogMessage::Fail () at src/logging.cc:1296
      google::LogMessage::SendToLog (this=0x7f6dd18ffde0) at src/logging.cc:1250
      google::LogMessage::Flush (this=0x7f6dd18ffde0) at src/logging.cc:1119
      google::LogMessageFatal::~LogMessageFatal (this=0x7f6dd18ffde0, __in_chrg=<value optimized out>) at src/logging.cc:1817
      impala::AggFnEvaluator::Prepare (this=0x7b814a0, state=0x82e8000, desc=..., intermediate_slot_desc=0x4f7a190, output_slot_desc=0x4f7a190, agg_fn_pool=0x5dbd860, agg_fn_ctx=0x7f6dd1900078) at /data/9/query-gen/Impala/be/src/exprs/agg-fn-evaluator.cc:128
      impala::AnalyticEvalNode::Prepare (this=0x6977180, state=0x82e8000) at /data/9/query-gen/Impala/be/src/exec/analytic-eval-node.cc:137
      impala::PlanFragmentExecutor::Prepare (this=0x4e4cf00, request=...) at /data/9/query-gen/Impala/be/src/runtime/plan-fragment-executor.cc:234
      impala::Coordinator::Exec (this=0x6495000, schedule=..., output_expr_ctxs=0xabbab60) at /data/9/query-gen/Impala/be/src/runtime/coordinator.cc:342
      impala::ImpalaServer::QueryExecState::ExecQueryOrDmlRequest (this=0xabba000, query_exec_request=...) at /data/9/query-gen/Impala/be/src/service/query-exec-state.cc:403
      impala::ImpalaServer::QueryExecState::Exec (this=0xabba000, exec_request=0x7f6dd19028f0) at /data/9/query-gen/Impala/be/src/service/query-exec-state.cc:138
      impala::ImpalaServer::ExecuteInternal (this=0x52d2580, query_ctx=..., session_state=..., registered_exec_state=0x7f6dd1903b67, exec_state=0x7f6dd1903df0) at /data/9/query-gen/Impala/be/src/service/impala-server.cc:607
      impala::ImpalaServer::Execute (this=0x52d2580, query_ctx=0x7f6dd1903c20, session_state=..., exec_state=0x7f6dd1903df0) at /data/9/query-gen/Impala/be/src/service/impala-server.cc:550
      impala::ImpalaServer::ExecuteStatement (this=0x52d2580, return_val=..., request=...) at /data/9/query-gen/Impala/be/src/service/impala-hs2-server.cc:709
      apache::hive::service::cli::thrift::TCLIServiceProcessor::process_ExecuteStatement (this=0x37e50e0, seqid=0, iprot=0x598c800, oprot=0x5934300, callContext=0x598c900) at /data/9/query-gen/Impala/be/generated-sources/gen-cpp/TCLIService.cpp:4695
      apache::hive::service::cli::thrift::TCLIServiceProcessor::dispatchCall (this=0x37e50e0, iprot=0x598c800, oprot=0x5934300, fname="ExecuteStatement", seqid=0, callContext=0x598c900) at /data/9/query-gen/Impala/be/generated-sources/gen-cpp/TCLIService.cpp:4506
      impala::ImpalaHiveServer2ServiceProcessor::dispatchCall (this=0x37e50e0, iprot=0x598c800, oprot=0x5934300, fname="ExecuteStatement", seqid=0, callContext=0x598c900) at /data/9/query-gen/Impala/be/generated-sources/gen-cpp/ImpalaHiveServer2Service.cpp:463
      apache::thrift::TDispatchProcessor::process (this=0x37e50e0, in=..., out=..., connectionContext=0x598c900) at /data/9/query-gen/Impala/thirdparty/thrift-0.9.0/build/include/thrift/TDispatchProcessor.h:121
      apache::thrift::server::TThreadPoolServer::Task::run (this=0x7e6a180) at src/thrift/server/TThreadPoolServer.cpp:70
      apache::thrift::concurrency::ThreadManager::Task::run (this=0x598c880) at src/thrift/concurrency/ThreadManager.cpp:187
      apache::thrift::concurrency::ThreadManager::Worker::run (this=0x5fedef0) at src/thrift/concurrency/ThreadManager.cpp:316
      impala::ThriftThread::RunRunnable (this=0x5b0e7c0, runnable=..., promise=0x7fffdf865f10) at /data/9/query-gen/Impala/be/src/rpc/thrift-thread.cc:61
      boost::_mfi::mf2<void, impala::ThriftThread, boost::shared_ptr<apache::thrift::concurrency::Runnable>, impala::Promise<unsigned long>*>::operator() (this=0x65f3230, p=0x5b0e7c0, a1=..., a2=0x7fffdf865f10) at /usr/include/boost/bind/mem_fn_template.hpp:280
      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=0x65f3240, f=..., a=...) at /usr/include/boost/bind/bind.hpp:392
      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=0x65f3230) at /usr/include/boost/bind/bind_template.hpp:20
      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
      boost::function0<void>::operator() (this=0x7f6dd1904cd0) at /usr/include/boost/function/function_template.hpp:1013
      impala::Thread::SuperviseThread (name="hiveserver2-frontend-5", category="thrift-server", functor=..., thread_started=0x7fffdf865d10) at /data/9/query-gen/Impala/be/src/util/thread.cc:311
      boost::_bi::list4<boost::_bi::value<std::basic_string<char, std::char_traits<char>, std::allocator<char> > >, boost::_bi::value<std::basic_string<char, std::char_traits<char>, std::allocator<char> > >, boost::_bi::value<boost::function<void()> >, boost::_bi::value<impala::Promise<long int>*> >::operator()<void (*)(const std::string&, const std::string&, impala::Thread::ThreadFunctor, impala::Promise<long int>*), boost::_bi::list0>(boost::_bi::type<void>, void (*&)(const std::string &, const std::string &, impala::Thread::ThreadFunctor, impala::Promise<long> *), boost::_bi::list0 &, int) (this=0x6940910, f=@0x6940908, a=...) at /usr/include/boost/bind/bind.hpp:457
      boost::_bi::bind_t<void, void (*)(const std::string&, const std::string&, impala::Thread::ThreadFunctor, impala::Promise<long int>*), boost::_bi::list4<boost::_bi::value<std::basic_string<char, std::char_traits<char>, std::allocator<char> > >, boost::_bi::value<std::basic_string<char, std::char_traits<char>, std::allocator<char> > >, boost::_bi::value<boost::function<void()> >, boost::_bi::value<impala::Promise<long int>*> > >::operator()(void) (this=0x6940908) at /usr/include/boost/bind/bind_template.hpp:20
      boost::detail::thread_data<boost::_bi::bind_t<void, void (*)(const std::string&, const std::string&, impala::Thread::ThreadFunctor, impala::Promise<long int>*), boost::_bi::list4<boost::_bi::value<std::basic_string<char, std::char_traits<char>, std::allocator<char> > >, boost::_bi::value<std::basic_string<char, std::char_traits<char>, std::allocator<char> > >, boost::_bi::value<boost::function<void()> >, boost::_bi::value<impala::Promise<long int>*> > > >::run(void) (this=0x6940780) at /usr/include/boost/thread/detail/thread.hpp:61
      thread_proxy ()
      start_thread () from /lib64/libpthread.so.0
      clone () from /lib64/libc.so.6
      

      DB: Functional
      File Format: Text/None
      git Hash: a95e7ec

      Attachments

        Activity

          People

            mjacobs Matthew Jacobs
            tarasbob Taras Bobrovytsky
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: