Details
-
Bug
-
Status: Resolved
-
Blocker
-
Resolution: Fixed
-
Impala 2.1
-
None
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