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

Operators running on top of selective Parquet scans spend a lot of time calling impala::MemPool::FreeAll on empty batches

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • Impala 2.6.0
    • Impala 2.9.0
    • Backend
    • None

    Description

      Operators that are executed after a highly selective scan node spend a lot of time calling impala::MemPool::FreeAll on row batches with all rows filtered out.

      So even if an operator ends up processing 0 rows it still has to clear the memory allocated for the empty batches created by the HdfsScanNode.

      https://github.com/apache/incubator-impala/blob/2.7.0/be/src/runtime/row-batch.cc#L317

      Should try using Clear() and investigate the repercussions.

      Repro query

      select l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_comment from lineitem where l_orderkey=0 group by l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_comment order by l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_comment limit 10
      
      +---------------------+--------+----------+----------+-------+------------+-----------+---------------+---------------
      | Operator            | #Hosts | Avg Time | Max Time | #Rows | Est. #Rows | Peak Mem  | Est. Peak Mem | Detail       |
      +---------------------+--------+----------+----------+-------+------------+-----------+---------------+--------------
      | 05:MERGING-EXCHANGE | 1      | 34.56us  | 34.56us  | 0     | 4          | 0 B       | -1 B          | UNPARTITIONED|
      | 02:TOP-N            | 7      | 388.39us | 1.71ms   | 0     | 4          | 12.00 KB  | 986 B         |              |
      | 04:AGGREGATE        | 7      | 3.72ms   | 9.59ms   | 0     | 4          | 2.45 MB   | 10.00 MB      | FINALIZE     |
      | 03:EXCHANGE         | 7      | 6.88us   | 8.15us   | 0     | 4          | 0 B       | 0 
      | 01:AGGREGATE        | 7      | 8.42s    | 9.10s    | 0     | 4          | 10.14 MB  | 10.00 MB      | STREAMING                 |
      | 00:SCAN HDFS        | 7      | 34.07s   | 37.75s   | 0     | 4          | 466.98 MB | 176.00 MB     | tpch_300_parquet.lineitem 
      +---------------------+--------+----------+----------+-------+------------+-----------+---------------+---------------------------
      
      CPU Time
      1 of 27: 74.4% (5.990s of 8.050s)
      
      libc.so.6 ! madvise - [unknown source file]
      impalad ! TCMalloc_SystemRelease + 0x79 - [unknown source file]
      impalad ! tcmalloc::PageHeap::DecommitSpan + 0x20 - [unknown source file]
      impalad ! tcmalloc::PageHeap::MergeIntoFreeList + 0x212 - [unknown source file]
      impalad ! tcmalloc::PageHeap::Delete + 0x23 - [unknown source file]
      impalad ! tcmalloc::CentralFreeList::ReleaseToSpans + 0x10f - [unknown source file]
      impalad ! tcmalloc::CentralFreeList::ReleaseListToSpans + 0x1a - [unknown source file]
      impalad ! tcmalloc::CentralFreeList::InsertRange + 0x3b - [unknown source file]
      impalad ! tcmalloc::ThreadCache::ReleaseToCentralCache + 0x103 - [unknown source file]
      impalad ! tcmalloc::ThreadCache::Scavenge + 0x3e - [unknown source file]
      impalad ! operator delete + 0x329 - [unknown source file]
      impalad ! impala::MemPool::FreeAll + 0x59 - mem-pool.cc:90
      impalad ! impala::RowBatch::Reset + 0x2c - row-batch.cc:312
      impalad ! impala::PartitionedAggregationNode::GetRowsStreaming + 0x1af - partitioned-aggregation-node.cc:588
      impalad ! impala::PartitionedAggregationNode::GetNextInternal + 0x260 - partitioned-aggregation-node.cc:451
      impalad ! impala::PartitionedAggregationNode::GetNext + 0x21 - partitioned-aggregation-node.cc:376
      impalad ! impala::PlanFragmentExecutor::ExecInternal + 0x192 - plan-fragment-executor.cc:361
      impalad ! impala::PlanFragmentExecutor::Exec + 0x17e - plan-fragment-executor.cc:339
      impalad ! impala::FragmentMgr::FragmentExecState::Exec + 0xdf - fragment-exec-state.cc:54
      impalad ! impala::FragmentMgr::FragmentThread + 0x39 - fragment-mgr.cc:86
      impalad ! boost::_mfi::mf1<void, impala::FragmentMgr, impala::TUniqueId>::operator() + 0x42 - mem_fn_template.hpp:165
      impalad ! operator()<boost::_mfi::mf1<void, impala::FragmentMgr, impala::TUniqueId>, boost::_bi::list0> - bind.hpp:313
      impalad ! boost::_bi::bind_t<void, boost::_mfi::mf1<void, impala::FragmentMgr, impala::TUniqueId>, boost::_bi::list2<boost::_bi::value<impala::FragmentMgr*>, boost::_bi::value<impala::TUniqueId>>>::operator() - bind_template.hpp:20
      impalad ! boost::detail::function::void_function_obj_invoker0<boost::_bi::bind_t<void, boost::_mfi::mf1<void, impala::FragmentMgr, impala::TUniqueId>, boost::_bi::list2<boost::_bi::value<impala::FragmentMgr*>, boost::_bi::value<impala::TUniqueId>>>, void>::invoke + 0x7 - function_template.hpp:153
      impalad ! boost::function0<void>::operator() + 0x1a - function_template.hpp:767
      impalad ! impala::Thread::SuperviseThread + 0x20e - thread.cc:317
      impalad ! operator()<void (*)(const std::basic_string<char>&, const std::basic_string<char>&, boost::function<void()>, impala::Promise<long int>*), boost::_bi::list0> + 0x5a - bind.hpp:457
      impalad ! boost::_bi::bind_t<void, void (*)(std::string const&, std::string const&, boost::function<void (void)>, impala::Promise<long>*), boost::_bi::list4<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void (void)>>, boost::_bi::value<impala::Promise<long>*>>>::operator() - bind_template.hpp:20
      impalad ! boost::detail::thread_data<boost::_bi::bind_t<void, void (*)(std::string const&, std::string const&, boost::function<void (void)>, impala::Promise<long>*), boost::_bi::list4<boost::_bi::value<std::string>, boost::_bi::value<std::string>, boost::_bi::value<boost::function<void (void)>>, boost::_bi::value<impala::Promise<long>*>>>>::run + 0x19 - thread.hpp:116
      impalad ! thread_proxy + 0xd9 - [unknown source file]
      libpthread.so.0 ! start_thread + 0xd0 - [unknown source file]
      libc.so.6 ! clone + 0x6c - [unknown source file]
      

      Attachments

        Issue Links

          Activity

            People

              tarmstrong Tim Armstrong
              mmokhtar Mostafa Mokhtar
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: