Uploaded image for project: 'Kudu'
  1. Kudu
  2. KUDU-1753

Impala query fails: Unable to advance iterator: Illegal state: Tablet is not running

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • None
    • 1.2.0
    • client, impala
    • None

    Description

      We were running queries (with mem limits set in Impala) like the following one after another (only one query was executing at the same time at any point).

      upsert into table lineitem select * from lineitem_original where l_orderkey % 11 = 0
      

      and

      delete from lineitem where l_orderkey % 11 = 0
      

      One of the queries failed with the following error:

      Unable to advance iterator: Illegal state: Tablet is not running
      

      from kudu-tserver.INFO:

      I1122 17:54:10.659071  7021 tablet.cc:1160] T 5e7e168f1919488ba902f87b32be6065 Flush:      entering phase 2 (starting to duplicate updates in new rowsets)
      I1122 17:54:10.740268 16858 log.cc:513] Max segment size reached. Starting new segment     allocation. 
      I1122 17:54:10.772285  7086 raft_consensus.cc:1192] Rejecting consensus request: Soft      memory limit exceeded (at 82.22% of capacity) [suppressed 5 similar messages]
      I1122 17:54:11.809483  7094 raft_consensus.cc:1192] Rejecting consensus request: Soft      memory limit exceeded (at 84.45% of capacity) [suppressed 5 similar messages]
      I1122 17:54:12.095155 16858 log.cc:398] Rolled over to a new segment: /data1/kudu/tserver/ wals/5e7e168f1919488ba902f87b32be6065/wal-000000082
      I1122 17:54:13.143566  7088 raft_consensus.cc:1192] Rejecting consensus request: Soft      memory limit exceeded (at 82.84% of capacity) [suppressed 6 similar messages]
      I1122 17:54:13.633990  7021 tablet.cc:1215] T 5e7e168f1919488ba902f87b32be6065 Flush Phase 2: carrying over any updates which arrived during Phase 1
      I1122 17:54:13.634016  7021 tablet.cc:1217] T 5e7e168f1919488ba902f87b32be6065 Phase 2     snapshot: MvccSnapshot[committed={T|T < 1298537 or (T in {1298538,1298537})}]
      I1122 17:54:14.154855  7094 raft_consensus.cc:1192] Rejecting consensus request: Soft      memory limit exceeded (at 83.44% of capacity) [suppressed 5 similar messages]
      I1122 17:54:15.068207  7095 tablet_copy_client.cc:273] T 44cf60f4c34841899e3a32852365b112  P 8e6f4cf16c544ef3b7913f6cda1bfe54: Tablet Copy client: Tablet Copy complete. Replacing    tablet superblock.
      I1122 17:54:15.071156  9784 ts_tablet_manager.cc:633] T 44cf60f4c34841899e3a32852365b112 P 8e6f4cf16c544ef3b7913f6cda1bfe54: Bootstrapping tablet
      I1122 17:54:15.071321  9784 tablet_bootstrap.cc:380] T 44cf60f4c34841899e3a32852365b112 P  8e6f4cf16c544ef3b7913f6cda1bfe54: Bootstrap starting.
      I1122 17:54:15.081435  9784 tablet_bootstrap.cc:542] T 44cf60f4c34841899e3a32852365b112 P  8e6f4cf16c544ef3b7913f6cda1bfe54: Time spent opening tablet: real 0.010s user 0.006s sys 0.004s
      .....
      .....
      I1122 17:54:15.086483  9784 tablet_bootstrap.cc:606] T 44cf60f4c34841899e3a32852365b112 P  8e6f4cf16c544ef3b7913f6cda1bfe54: Moving log directory /data1/kudu/tserver/wals/           44cf60f4c34841899e3a32852365b112 to recovery directory /data1/kudu/tserver/wals/           44cf60f4c34841899e3a32852365b112.recovery in preparation for log replay
      I1122 17:54:15.087378  9784 log_util.cc:312] Log segment /data1/kudu/tserver/wals/         44cf60f4c34841899e3a32852365b112.recovery/wal-000000087 has no footer. This segment was    likely being written when the server previously shut down.
      I1122 17:54:15.087385  9784 log_reader.cc:151] Log segment /data1/kudu/tserver/wals/       44cf60f4c34841899e3a32852365b112.recovery/wal-000000087 was likely left in-progress after  a previous crash. Will try to rebuild footer by scanning data.
      W1122 17:54:15.119765  7095 rpcz_store.cc:234] Call kudu.consensus.ConsensusService.       StartTabletCopy from 172.28.195.11:33598 (request call id 247782) took 56098ms (client     timeout 1000).
      W1122 17:54:15.119963  7095 rpcz_store.cc:238] Trace:                                      
      1122 17:53:19.020889 (+     0us) service_pool.cc:143] Inserting onto call queue            
      1122 17:53:19.020911 (+    22us) service_pool.cc:202] Handling call                        
      1122 17:53:19.022175 (+  1264us) ts_tablet_manager.cc:418] T                               44cf60f4c34841899e3a32852365b112 P 8e6f4cf16c544ef3b7913f6cda1bfe54: Initiating tablet     copy from Peer bb2ce221ba374ff4bff7ff5c56946f45 (kudu-stress-8.vpc.cloudera.com:7050)
      1122 17:53:19.064954 (+ 42779us) tablet_metadata.cc:461] Metadata flushed                  
      1122 17:53:19.065703 (+   749us) tablet_metadata.cc:461] Metadata flushed                  
      1122 17:54:15.071167 (+56005464us) ts_tablet_manager.cc:634] Bootstrapping tablet          
      1122 17:54:15.119736 (+ 48569us) inbound_call.cc:130] Queueing success response            
      Metrics: {"fdatasync":967,"fdatasync_us":32775470,"spinlock_wait_cycles":4352,"tablet-     bootstrap.queue_time_us":248,"tcmalloc_contention_cycles":527821056,"thread_start_us":229, "threads_started":1}
      W1122 17:54:15.251955  7021 tablet_peer.cc:656] Time spent Committing in-flights took a    long time.: real 1.530s  user 0.000s sys 0.000s
      

      from impalad.INFO:

      8e6f4cf16c544ef3b7913f6cda1bfe54 (kudu-stress-6.vpc.cloudera.com:7050) of tablet impala::  tpch_10_kudu.lineitem:  failed: Illegal state: Tablet is not running
      I1122 17:53:23.382364  9488 status.cc:114] Unable to advance iterator: Illegal state:      Tablet is not running                                                                      
          @          0x11c14e3  impala::Status::Status()                                         
          @          0x17eace2  impala::KuduScanner::GetNextScannerBatch()                       
          @          0x17e93fa  impala::KuduScanner::GetNext()                                   
          @          0x1785893  impala::KuduScanNode::ProcessScanToken()                         
          @          0x1785bb2  impala::KuduScanNode::RunScannerThread()                         
          @          0x1787ad2  boost::_mfi::mf2<>::operator()()                                 
          @          0x1787998  boost::_bi::list3<>::operator()<>()                              
          @          0x1787519  boost::_bi::bind_t<>::operator()()                               
          @          0x178734f  boost::detail::function::void_function_obj_invoker0<>::invoke()  
          @          0x133ce98  boost::function0<>::operator()()                                 
          @          0x15e8921  impala::Thread::SuperviseThread()                                
          @          0x15ef8fa  boost::_bi::list4<>::operator()<>()                              
          @          0x15ef83d  boost::_bi::bind_t<>::operator()()                               
          @          0x15ef798  boost::detail::thread_data<>::run()                              
          @          0x1a3c34a  thread_proxy                                                     
          @       0x38382079d1  (unknown)                                                        
          @       0x3837ee88fd  (unknown)                                                        
      I1122 17:53:23.392840  9459 runtime-state.cc:208] Error from query 6e47c2d5c47683e8:       5c1a097d00000000: Unable to advance iterator: Illegal state: Tablet is not running
      I1122 17:53:23.451575  9459 fragment-mgr.cc:99] PlanFragment completed.                    instance_id=6e47c2d5c47683e8:5c1a097d00000000
      

      Attachments

        1. kudu-tserver.INFO
          1.89 MB
          Taras Bobrovytsky
        2. impalad.INFO
          125 kB
          Taras Bobrovytsky

        Issue Links

          Activity

            People

              aserbin Alexey Serbin
              tarasbob Taras Bobrovytsky
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: