Details
-
Bug
-
Status: Resolved
-
Blocker
-
Resolution: Cannot Reproduce
-
Impala 3.0, Impala 2.12.0
-
None
-
ghx-label-1
Description
Hit this when running an exhaustive build at this commit. KRPC was disabled.
query_test/test_queries.py:111: in test_subquery self.run_test_case('QueryTest/subquery', vector) common/impala_test_suite.py:397: in run_test_case result = self.__execute_query(target_impalad_client, query, user=user) common/impala_test_suite.py:612: in __execute_query return impalad_client.execute(query, user=user) common/impala_connection.py:160: in execute return self.__beeswax_client.execute(sql_stmt, user=user) beeswax/impala_beeswax.py:173: in execute handle = self.__execute_query(query_string.strip(), user=user) beeswax/impala_beeswax.py:339: in __execute_query handle = self.execute_query_async(query_string, user=user) beeswax/impala_beeswax.py:335: in execute_query_async return self.__do_rpc(lambda: self.imp_service.query(query,)) beeswax/impala_beeswax.py:460: in __do_rpc raise ImpalaBeeswaxException(self.__build_error_message(b), b) E ImpalaBeeswaxException: ImpalaBeeswaxException: E INNER EXCEPTION: <class 'beeswaxd.ttypes.BeeswaxException'> E MESSAGE: IllegalStateException: null Standard Error -- executing against localhost:21000 use functional_rc_def; SET disable_codegen_rows_threshold=0; SET disable_codegen=False; SET abort_on_error=1; SET exec_single_node_rows_threshold=100; SET batch_size=0; SET num_nodes=0; -- executing against localhost:21000 select a.id, a.int_col, a.string_col from functional.alltypessmall a where a.id in (select id from functional.alltypestiny where bool_col = false) and a.id < 5;
I0306 10:59:28.357015 15877 Frontend.java:952] Analyzing query: select a.id, a.int_col, a.string_col from functional.alltypessmall a where a.id in (select id from functional.alltypestiny where bool_col = false) and a.id < 5 I0306 10:59:28.358779 15877 Frontend.java:964] Analysis finished. I0306 10:59:28.603314 27013 data-stream-mgr.cc:236] Reduced stream ID cache from 1891 items, to 1887, eviction took: 0 I0306 10:59:29.028396 15877 jni-util.cc:230] java.lang.IllegalStateException at com.google.common.base.Preconditions.checkState(Preconditions.java:129) at org.apache.impala.service.FeSupport.EvalExprWithoutRow(FeSupport.java:169) at org.apache.impala.service.FeSupport.EvalPredicate(FeSupport.java:218) at org.apache.impala.analysis.Analyzer.isTrueWithNullSlots(Analyzer.java:1917) at org.apache.impala.planner.HdfsScanNode.addDictionaryFilter(HdfsScanNode.java:659) at org.apache.impala.planner.HdfsScanNode.computeDictionaryFilterConjuncts(HdfsScanNode.java:685) at org.apache.impala.planner.HdfsScanNode.init(HdfsScanNode.java:329) at org.apache.impala.planner.SingleNodePlanner.createHdfsScanPlan(SingleNodePlanner.java:1255) at org.apache.impala.planner.SingleNodePlanner.createScanNode(SingleNodePlanner.java:1298) at org.apache.impala.planner.SingleNodePlanner.createTableRefNode(SingleNodePlanner.java:1506) at org.apache.impala.planner.SingleNodePlanner.createTableRefsPlan(SingleNodePlanner.java:776) at org.apache.impala.planner.SingleNodePlanner.createSelectPlan(SingleNodePlanner.java:614) at org.apache.impala.planner.SingleNodePlanner.createQueryPlan(SingleNodePlanner.java:257) at org.apache.impala.planner.SingleNodePlanner.createSingleNodePlan(SingleNodePlanner.java:147) at org.apache.impala.planner.Planner.createPlan(Planner.java:101) at org.apache.impala.service.Frontend.createExecRequest(Frontend.java:896) at org.apache.impala.service.Frontend.createExecRequest(Frontend.java:1017) at org.apache.impala.service.JniFrontend.createExecRequest(JniFrontend.java:156) I0306 10:59:30.051102 15877 status.cc:125] IllegalStateException: null @ 0x1676fad impala::Status::Status() @ 0x1ad6032 impala::JniUtil::GetJniExceptionMsg() @ 0x196ebe3 impala::JniUtil::CallJniMethod<>() @ 0x196b451 impala::Frontend::GetExecRequest() @ 0x198e5a6 impala::ImpalaServer::ExecuteInternal() @ 0x198e0ee impala::ImpalaServer::Execute() @ 0x1a1833c impala::ImpalaServer::query() @ 0x2b2169e beeswax::BeeswaxServiceProcessor::process_query() @ 0x2b213ec beeswax::BeeswaxServiceProcessor::dispatchCall() @ 0x2b093db impala::ImpalaServiceProcessor::dispatchCall() @ 0x161bb70 apache::thrift::TDispatchProcessor::process() @ 0x17f9347 apache::thrift::server::TAcceptQueueServer::Task::run() @ 0x17f0fab impala::ThriftThread::RunRunnable() @ 0x17f26af boost::_mfi::mf2<>::operator()() @ 0x17f2545 boost::_bi::list3<>::operator()<>() @ 0x17f2291 boost::_bi::bind_t<>::operator()() @ 0x17f21a4 boost::detail::function::void_function_obj_invoker0<>::invoke() @ 0x183639c boost::function0<>::operator()() @ 0x1b42041 impala::Thread::SuperviseThread() @ 0x1b4a517 boost::_bi::list5<>::operator()<>() @ 0x1b4a43b boost::_bi::bind_t<>::operator()() @ 0x1b4a3fe boost::detail::thread_data<>::run() @ 0x2dcfd5a thread_proxy @ 0x3ceb607851 (unknown) @ 0x3ceb2e894d (unknown) I0306 10:59:30.523000 15877 impala-server.cc:1006] UnregisterQuery(): query_id=354679e4021b51f5:fe427b2100000000 I0306 10:59:30.523033 15877 impala-server.cc:1093] Cancel(): query_id=354679e4021b51f5:fe427b2100000000
The precondition check fired appears to be the following line:
public static TColumnValue EvalExprWithoutRow(Expr expr, TQueryCtx queryCtx) throws InternalException { ...... Preconditions.checkState(val.getColValsSize() == 1); <<<------ ....... }
Attachments
Issue Links
- blocks
-
IMPALA-6626 Failure to assign dictionary predicates should not result in query failure - testing needed
- Open
-
IMPALA-6625 Skip dictionary and collection conjunct assignment for non-Parquet scans.
- Resolved