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

Cancelling query with group_concat causes crash

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • Impala 2.3.0
    • Impala 2.3.0
    • None

    Description

      The query below reliably causes a crash if it is cancelled. This was run through the impala-shell, then ctrl-c when the results start to stream back.

      use tpch;
      
      select c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, 
      group_concat(
        concat(
          cast(o_orderkey as string), "\003",
          cast(o_custkey as string), "\003",
          cast(o_orderstatus as string), "\003",
          cast(o_totalprice as string), "\003",
          cast(o_orderdate as string), "\003",
          cast(o_orderpriority as string), "\003",
          cast(o_clerk as string), "\003",
          cast(o_shippriority as string), "\003",
          cast(o_comment as string), "\003",
          cast(lineitem_string as string)
        ), "\002"
      )
      from customer inner join nested_orders on c_custkey = o_custkey
      group by c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment;
      

      Backtrace

      #6  0x0000000002025c2e in google::LogMessageFatal::~LogMessageFatal (
          this=0x7f6a29d9add0, __in_chrg=<optimized out>) at src/logging.cc:1836
      #7  0x00000000013330c0 in impala::FreePool::CheckValidAllocation (this=0x9c68240, 
          computed_list_ptr=0x9c68708, allocation=0x2e7acc30 "\001")
          at /home/dev/Impala/be/src/runtime/free-pool.h:147
      #8  0x0000000001332e01 in impala::FreePool::Free (this=0x9c68240, ptr=0x2e7acc30 "\001")
          at /home/dev/Impala/be/src/runtime/free-pool.h:92
      #9  0x0000000001331991 in impala_udf::FunctionContext::Free (this=0xad68a70, 
          buffer=0x2e7acc30 "\001") at /home/dev/Impala/be/src/udf/udf.cc:308
      #10 0x0000000000fbc446 in impala::AggregateFunctions::StringConcatFinalize (
          ctx=0xad68a70, src=...) at /home/dev/Impala/be/src/exprs/aggregate-functions.cc:543
      #11 0x0000000001624192 in impala::AggFnEvaluator::SerializeOrFinalize (this=0xac1a000, 
          agg_fn_ctx=0xad68a70, src=0x3ea4a28d, dst_slot_desc=0xa845260, dst=0x2c01f000, 
          fn=0xfbc278 <impala::AggregateFunctions::StringConcatFinalize(impala_udf::FunctionContext*, impala_udf::StringVal const&)>)
          at /home/dev/Impala/be/src/exprs/agg-fn-evaluator.cc:487
      #12 0x00000000015baf84 in impala::AggFnEvaluator::Finalize (this=0xac1a000, 
          agg_fn_ctx=0xad68a70, src=0x3ea4a28d, dst=0x2c01f000)
          at /home/dev/Impala/be/src/exprs/agg-fn-evaluator.h:254
      #13 0x00000000015bb1ae in impala::AggFnEvaluator::Finalize (evaluators=..., fn_ctxs=..., 
          src=0x3ea4a28d, dst=0x2c01f000)
          at /home/dev/Impala/be/src/exprs/agg-fn-evaluator.h:300
      #14 0x00000000015b17a0 in impala::PartitionedAggregationNode::CleanupHashTbl (
          this=0xb206f00, ctxs=..., it=...)
          at /home/dev/Impala/be/src/exec/partitioned-aggregation-node.cc:367
      #15 0x00000000015b19f6 in impala::PartitionedAggregationNode::Close (this=0xb206f00, 
          state=0x578c000) at /home/dev/Impala/be/src/exec/partitioned-aggregation-node.cc:391
      #16 0x00000000014cd5a6 in impala::PlanFragmentExecutor::Close (this=0x9eeaf10)
          at /home/dev/Impala/be/src/runtime/plan-fragment-executor.cc:573
      

      Failed DCHECK

        void CheckValidAllocation(FreeListNode* computed_list_ptr, uint8_t* allocation) const { 
          // On debug, check that list is valid.                                                
          bool found = false;                                                                   
          for (int i = 0; i < NUM_LISTS && !found; ++i) {                                       
            if (computed_list_ptr == &lists_[i]) found = true;                                  
          }                                                                                     
          DCHECK(found) << "Could not find list for ptr: "                                      
                        << reinterpret_cast<void*>(allocation)                                  
                        << ". Allocation could have already been freed." << std::endl           
                        << DebugString();
      

      Attachments

        Activity

          People

            caseyc casey
            caseyc casey
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: