[ https://issues.apache.org/jira/browse/IMPALA-8322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16811027#comment-16811027 ]
Joe McDonnell commented on IMPALA-8322: --------------------------------------- One way to fix this is to remove the thread available callback earlier. Currently is it done in HdfsScanNode::Close(), but it might be possible to move it to SetDoneInternal(). If HdfsScanNode::SetDoneInternal() calls it before RequestContext::Cancel(), then that reduces the window where a caller in ThreadResourceMgr::DestroyPool() can call it and get stuck trying to get the lock. An alternative is for HdfsScanNode::ThreadTokenAvailableCb() to do a dirty check on done_ without holding the lock and bail if it is true. In addition, it might make sense to look at how RequestContext::Cancel() works when there are multiple scan ranges and multiple IO threads. It immediately sets state that would prevent IO threads from getting more scan ranges from this RequestContext. But several IOs might be in progress. Right now, it does this loop: {code:java} for (ScanRange* range : active_scan_ranges_) { range->CancelInternal(CONTEXT_CANCELLED, false); } {code} Each ScanRange::CancelInternal() needs to get a lock on the scan range and the HdfsFileReader. The HdfsFileReader lock is held during opens and reads, so this will wait on IO. While the cancel thread is waiting on the first HdfsFileReader, another IO thread might be finishing an open on the second HdfsFileReader. Right now, it can proceed to do a read, because it doesn't know that it is cancelled yet (it is looking at ScanRange::cancel_status_). So, after waiting on the first HdfsFileReader, it might also need to wait on the second HdfsFileReader. (There is a limit, because IOs only take so long, and the IO threads can't get more ranges.) If we looped over the ScanRanges setting cancel_status_ (or some other state) without grabbing the HdfsFileReader lock, the extra read could be avoided, because the IO thread would know immediately that it is cancelled (and it can work on scan ranges from other queries). Then, the cancel thread could loop over to wait for the IO threads to finish. > S3 tests encounter "timed out waiting for receiver fragment instance" > --------------------------------------------------------------------- > > Key: IMPALA-8322 > URL: https://issues.apache.org/jira/browse/IMPALA-8322 > Project: IMPALA > Issue Type: Bug > Components: Backend > Affects Versions: Impala 3.3.0 > Reporter: Joe McDonnell > Priority: Blocker > Labels: broken-build > Attachments: fb5b9729-2d7a-4590-ea365b87-d2ead75e.dmp_dumped, > run_tests_swimlane.json.gz > > > This has been seen multiple times when running s3 tests: > {noformat} > query_test/test_join_queries.py:57: in test_basic_joins > self.run_test_case('QueryTest/joins', new_vector) > common/impala_test_suite.py:472: in run_test_case > result = self.__execute_query(target_impalad_client, query, user=user) > common/impala_test_suite.py:699: in __execute_query > return impalad_client.execute(query, user=user) > common/impala_connection.py:174: in execute > return self.__beeswax_client.execute(sql_stmt, user=user) > beeswax/impala_beeswax.py:183: in execute > handle = self.__execute_query(query_string.strip(), user=user) > beeswax/impala_beeswax.py:360: in __execute_query > self.wait_for_finished(handle) > beeswax/impala_beeswax.py:381: in wait_for_finished > raise ImpalaBeeswaxException("Query aborted:" + error_log, None) > E ImpalaBeeswaxException: ImpalaBeeswaxException: > E Query aborted:Sender 127.0.0.1 timed out waiting for receiver fragment > instance: 6c40d992bb87af2f:0ce96e5d00000007, dest node: 4{noformat} > This is related to IMPALA-6818. On a bad run, there are various time outs in > the impalad logs: > {noformat} > I0316 10:47:16.359313 20175 krpc-data-stream-mgr.cc:354] Sender 127.0.0.1 > timed out waiting for receiver fragment instance: > ef4a5dc32a6565bd:a8720b8500000007, dest node: 5 > I0316 10:47:16.359345 20175 rpcz_store.cc:265] Call > impala.DataStreamService.TransmitData from 127.0.0.1:40030 (request call id > 14881) took 120182ms. Request Metrics: {} > I0316 10:47:16.359380 20175 krpc-data-stream-mgr.cc:354] Sender 127.0.0.1 > timed out waiting for receiver fragment instance: > d148d83e11a4603d:54dc35f700000004, dest node: 3 > I0316 10:47:16.359395 20175 rpcz_store.cc:265] Call > impala.DataStreamService.TransmitData from 127.0.0.1:40030 (request call id > 14880) took 123097ms. Request Metrics: {} > ... various messages ... > I0316 10:47:56.364990 20154 kudu-util.h:108] Cancel() RPC failed: Timed out: > CancelQueryFInstances RPC to 127.0.0.1:27000 timed out after 10.000s (SENT) > ... various messages ... > W0316 10:48:15.056421 20150 rpcz_store.cc:251] Call > impala.ControlService.CancelQueryFInstances from 127.0.0.1:40912 (request > call id 202) took 48695ms (client timeout 10000). > W0316 10:48:15.056473 20150 rpcz_store.cc:255] Trace: > 0316 10:47:26.361265 (+ 0us) impala-service-pool.cc:165] Inserting onto call > queue > 0316 10:47:26.361285 (+ 20us) impala-service-pool.cc:245] Handling call > 0316 10:48:15.056398 (+48695113us) inbound_call.cc:162] Queueing success > response > Metrics: {} > I0316 10:48:15.057087 20139 connection.cc:584] Got response to call id 202 > after client already timed out or cancelled{noformat} > So far, this has only happened on s3. The system load at the time is not > higher than normal. If anything it is lower than normal. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-all-unsubscr...@impala.apache.org For additional commands, e-mail: issues-all-h...@impala.apache.org