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

Crash in BufferedBlockMgr::WriteComplete due to null buffer_desc_ during large spilling query

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • Impala 2.5.0, Impala 2.6.0, Impala 2.7.0
    • Impala 2.7.0
    • Backend

    Description

      I ran into this running a stress test on a release build. The query was TPC-H Q18 running on scale factor 20. I wasn't able to reproduce on a debug build, but it reproduced reliably on the release build when running the stress test's mem limit binary search.

      The crash was in WriteComplete when dereferencing buffer_desc_:

      void BufferedBlockMgr::WriteComplete(Block* block, const Status& write_status) {
        Status status = Status::OK();
        lock_guard<mutex> lock(lock_);
        outstanding_writes_counter_->Add(-1);
        DCHECK(Validate()) << endl << DebugInternal();
        DCHECK(is_cancelled_ || block->in_write_) << "WriteComplete() for block not in write."
                                                  << endl << block->DebugString();
        if (!block->client_local_) {
          DCHECK_GT(non_local_outstanding_writes_, 0) << block->DebugString();
          --non_local_outstanding_writes_;
        }
        block->in_write_ = false;
      
        // Explicitly release our temporarily allocated buffer here so that it doesn't
        // hang around needlessly.
        if (encryption_) EncryptDone(block);
      
        // ReturnUnusedBlock() will clear the block, so save required state in local vars.
        // state is not valid if the block was deleted because the state may be torn down
        // after the state's fragment has deleted all of its blocks.
        TmpFileMgr::File* tmp_file = block->tmp_file_;
        RuntimeState* state = block->is_deleted_ ? NULL : block->client_->state_;
      
        // If the block was re-pinned when it was in the IOMgr queue, don't free it.
        if (block->is_pinned_) {
          // The number of outstanding writes has decreased but the number of free buffers
          // hasn't.
          DCHECK(!block->is_deleted_);
          DCHECK(!block->client_local_)
              << "Client should be waiting. No one should have pinned this block.";
          if (write_status.ok() && !is_cancelled_ && !state->is_cancelled()) {
            status = WriteUnpinnedBlocks();
          }
        } else if (block->client_local_) {
          DCHECK(!block->is_deleted_)
              << "Client should be waiting. No one should have deleted this block.";
          block->write_complete_cv_.notify_one();
        } else {
          DCHECK_EQ(block->buffer_desc_->len, max_block_size_)
              << "Only io sized buffers should spill";
          free_io_buffers_.Enqueue(block->buffer_desc_); <=== buffer_desc_ is NULL here
          // Finish the DeleteBlock() work.
          if (block->is_deleted_) {
            block->buffer_desc_->block = NULL;
            block->buffer_desc_ = NULL;
            ReturnUnusedBlock(block);
            block = NULL;
          }
          // Multiple threads may be waiting for the same block in FindBuffer().  Wake them
          // all up.  One thread will get this block, and the others will re-evaluate whether
          // they should continue waiting and if another write needs to be initiated.
          buffer_available_cv_.notify_all();
        }
        DCHECK(Validate()) << endl << DebugInternal();
      

      I was able to reproduce the problem with a unit test after some effort. The bug is in TransferBuffer() when deleting the 'src' block while a write is in flight for it. If it gets into TransferBuffer() in that state, there is no code that waits for the write to complete. It seems to be very rare for it to actually go down that path - I think typically it goes down a path in PinBlock() that waits for the write to complete. To reproduce I had to create two clients with overcommitted reservations (this is probably sufficient but not necessary).

      In trying different reservation values I also reproduced a hang, which may be a different problem.

      Attachments

        Activity

          People

            tarmstrong Tim Armstrong
            tarmstrong Tim Armstrong
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: