Impala
Impalaistheopensource,nativeanalyticdatabaseforApacheHadoop.
 All Classes Namespaces Files Functions Variables Typedefs Enumerations Enumerator Friends Macros
impala::ExecNode Class Referenceabstract

#include <exec-node.h>

Inheritance diagram for impala::ExecNode:
Collaboration diagram for impala::ExecNode:

Classes

class  RowBatchQueue
 

Public Member Functions

 ExecNode (ObjectPool *pool, const TPlanNode &tnode, const DescriptorTbl &descs)
 Init conjuncts. More...
 
virtual ~ExecNode ()
 
virtual Status Init (const TPlanNode &tnode)
 
virtual Status Prepare (RuntimeState *state)
 
virtual Status Open (RuntimeState *state)
 
virtual Status GetNext (RuntimeState *state, RowBatch *row_batch, bool *eos)=0
 
virtual Status Reset (RuntimeState *state)
 
virtual void Close (RuntimeState *state)
 
void CollectNodes (TPlanNodeType::type node_type, std::vector< ExecNode * > *nodes)
 
void CollectScanNodes (std::vector< ExecNode * > *nodes)
 Collect all scan node types. More...
 
std::string DebugString () const
 Returns a string representation in DFS order of the plan rooted at this. More...
 
virtual void DebugString (int indentation_level, std::stringstream *out) const
 
const std::vector< ExprContext * > & conjunct_ctxs () const
 
int id () const
 
TPlanNodeType::type type () const
 
const RowDescriptorrow_desc () const
 
int64_t rows_returned () const
 
int64_t limit () const
 
bool ReachedLimit ()
 
RuntimeProfileruntime_profile ()
 
MemTrackermem_tracker ()
 
MemTrackerexpr_mem_tracker ()
 

Static Public Member Functions

static Status CreateTree (ObjectPool *pool, const TPlan &plan, const DescriptorTbl &descs, ExecNode **root)
 
static void SetDebugOptions (int node_id, TExecNodePhase::type phase, TDebugAction::type action, ExecNode *tree)
 Set debug action for node with given id in 'tree'. More...
 
static bool EvalConjuncts (ExprContext *const *ctxs, int num_ctxs, TupleRow *row)
 
static llvm::Function * CodegenEvalConjuncts (RuntimeState *state, const std::vector< ExprContext * > &conjunct_ctxs, const char *name="EvalConjuncts")
 
static int GetNodeIdFromProfile (RuntimeProfile *p)
 Extract node id from p->name(). More...
 

Static Public Attributes

static const std::string ROW_THROUGHPUT_COUNTER = "RowsReturnedRate"
 Names of counters shared by all exec nodes. More...
 

Protected Member Functions

ExecNodechild (int i)
 
bool is_closed ()
 
virtual bool IsScanNode () const
 
void InitRuntimeProfile (const std::string &name)
 
Status ExecDebugAction (TExecNodePhase::type phase, RuntimeState *state)
 
void AddRuntimeExecOption (const std::string &option)
 Appends option to 'runtime_exec_options_'. More...
 
virtual Status QueryMaintenance (RuntimeState *state)
 
void AddExprCtxToFree (ExprContext *ctx)
 
void AddExprCtxsToFree (const std::vector< ExprContext * > &ctxs)
 
void AddExprCtxsToFree (const SortExecExprs &sort_exec_exprs)
 

Static Protected Member Functions

static Status CreateNode (ObjectPool *pool, const TPlanNode &tnode, const DescriptorTbl &descs, ExecNode **node)
 Create a single exec node derived from thrift node; place exec node in 'pool'. More...
 
static Status CreateTreeHelper (ObjectPool *pool, const std::vector< TPlanNode > &tnodes, const DescriptorTbl &descs, ExecNode *parent, int *node_idx, ExecNode **root)
 

Protected Attributes

int id_
 
TPlanNodeType::type type_
 
ObjectPoolpool_
 
std::vector< ExprContext * > conjunct_ctxs_
 
std::vector< ExecNode * > children_
 
RowDescriptor row_descriptor_
 
TExecNodePhase::type debug_phase_
 
TDebugAction::type debug_action_
 
int64_t limit_
 
int64_t num_rows_returned_
 
boost::scoped_ptr< RuntimeProfileruntime_profile_
 
RuntimeProfile::Counterrows_returned_counter_
 
RuntimeProfile::Counterrows_returned_rate_
 
boost::scoped_ptr< MemTrackermem_tracker_
 Account for peak memory used by this node. More...
 
boost::scoped_ptr< MemTrackerexpr_mem_tracker_
 MemTracker that should be used for ExprContexts. More...
 
boost::mutex exec_options_lock_
 
std::string runtime_exec_options_
 

Private Attributes

bool is_closed_
 
std::vector< ExprContext * > expr_ctxs_to_free_
 Expr contexts whose local allocations are safe to free in the main execution thread. More...
 

Friends

class DataSink
 

Detailed Description

Superclass of all executor nodes. All subclasses need to make sure to check RuntimeState::is_cancelled() periodically in order to ensure timely termination after the cancellation flag gets set.

Definition at line 46 of file exec-node.h.

Constructor & Destructor Documentation

impala::ExecNode::ExecNode ( ObjectPool pool,
const TPlanNode &  tnode,
const DescriptorTbl descs 
)

Init conjuncts.

Definition at line 106 of file exec-node.cc.

References InitRuntimeProfile(), and impala::PrintPlanNodeType().

impala::ExecNode::~ExecNode ( )
virtual

Definition at line 121 of file exec-node.cc.

Member Function Documentation

void impala::ExecNode::AddExprCtxsToFree ( const SortExecExprs sort_exec_exprs)
protected
void impala::ExecNode::AddExprCtxToFree ( ExprContext ctx)
inlineprotected

Add an ExprContext to have its local allocations freed by QueryMaintenance(). Exprs that are evaluated in the main execution thread should be added. Exprs evaluated in a separate thread are generally not safe to add, since a local allocation may be freed while it's being used. Rather than using this mechanism, threads should call FreeLocalAllocations() on local ExprContexts periodically.

Definition at line 276 of file exec-node.h.

References expr_ctxs_to_free_.

Referenced by impala::AnalyticEvalNode::Prepare().

ExecNode* impala::ExecNode::child ( int  i)
inlineprotected

Definition at line 241 of file exec-node.h.

References children_.

Referenced by impala::CrossJoinNode::BuildListDebugString(), impala::BlockingJoinNode::BuildSideThread(), impala::HashJoinNode::CodegenCreateOutputRow(), impala::PartitionedHashJoinNode::CodegenCreateOutputRow(), impala::CrossJoinNode::ConstructBuildSide(), impala::HashJoinNode::ConstructBuildSide(), impala::PartitionedHashJoinNode::ConstructBuildSide(), impala::BlockingJoinNode::GetLeftChildRowString(), impala::SelectNode::GetNext(), impala::UnionNode::GetNext(), impala::CrossJoinNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AnalyticEvalNode::GetNextOutputBatch(), impala::PartitionedAggregationNode::Partition::InitStreams(), impala::HashJoinNode::LeftJoinGetNext(), impala::PartitionedHashJoinNode::NextProbeRowBatch(), impala::SelectNode::Open(), impala::SortNode::Open(), impala::TopNNode::Open(), impala::BlockingJoinNode::Open(), impala::AggregationNode::Open(), impala::AnalyticEvalNode::Open(), impala::PartitionedAggregationNode::Open(), impala::UnionNode::OpenCurrentChild(), impala::SelectNode::Prepare(), impala::SortNode::Prepare(), impala::UnionNode::Prepare(), impala::TopNNode::Prepare(), impala::BlockingJoinNode::Prepare(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::AnalyticEvalNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::PartitionedAggregationNode::Prepare(), impala::PartitionedHashJoinNode::ProcessBuildInput(), impala::AnalyticEvalNode::ProcessChildBatches(), and impala::SortNode::SortInput().

void impala::ExecNode::Close ( RuntimeState state)
virtual

Close() will get called for every exec node, regardless of what else is called and the status of these calls (i.e. Prepare() may never have been called, or Prepare()/Open()/GetNext() returned with an error). Close() releases all resources that were allocated in Open()/GetNext(), even if the latter ended with an error. Close() can be called if the node has been prepared or the node is closed. The default implementation updates runtime profile counters and calls Close() on the children. Subclasses should check if the node has already been closed (is_closed()), then close themselves, then call the base Close(). Nodes that are using tuples returned by a child may call Close() on their children before their own Close() if the child node has returned eos. It is only safe to call Close() on the child node while the parent node is still returning rows if the parent node fully materializes the child's input.

Reimplemented in impala::HdfsScanNode, impala::PartitionedAggregationNode, impala::AnalyticEvalNode, impala::PartitionedHashJoinNode, impala::BlockingJoinNode, impala::AggregationNode, impala::DataSourceScanNode, impala::HashJoinNode, impala::CrossJoinNode, impala::HBaseScanNode, impala::ExchangeNode, impala::TopNNode, impala::SortNode, impala::UnionNode, and impala::SelectNode.

Definition at line 166 of file exec-node.cc.

References children_, impala::Expr::Close(), conjunct_ctxs_, COUNTER_SET, impala::RuntimeState::instance_mem_tracker(), is_closed_, impala::MemTracker::LogUsage(), mem_tracker(), num_rows_returned_, impala::RuntimeState::query_id(), and rows_returned_counter_.

Referenced by impala::BlockingJoinNode::BuildSideThread(), impala::SelectNode::Close(), impala::SortNode::Close(), impala::UnionNode::Close(), impala::TopNNode::Close(), impala::ExchangeNode::Close(), impala::HBaseScanNode::Close(), impala::AggregationNode::Close(), impala::BlockingJoinNode::Close(), impala::AnalyticEvalNode::Close(), impala::PartitionedAggregationNode::Close(), impala::HdfsScanNode::Close(), impala::PlanFragmentExecutor::Close(), impala::UnionNode::GetNext(), impala::SortNode::Open(), impala::TopNNode::Open(), impala::AggregationNode::Open(), and impala::PartitionedAggregationNode::Open().

void impala::ExecNode::CollectNodes ( TPlanNodeType::type  node_type,
std::vector< ExecNode * > *  nodes 
)

Collect all nodes of given 'node_type' that are part of this subtree, and return in 'nodes'.

Definition at line 359 of file exec-node.cc.

References children_, and type_.

Referenced by CollectScanNodes(), and impala::PlanFragmentExecutor::Prepare().

void impala::ExecNode::CollectScanNodes ( std::vector< ExecNode * > *  nodes)

Collect all scan node types.

Definition at line 366 of file exec-node.cc.

References CollectNodes().

Referenced by impala::PlanFragmentExecutor::Prepare().

Status impala::ExecNode::CreateNode ( ObjectPool pool,
const TPlanNode &  tnode,
const DescriptorTbl descs,
ExecNode **  node 
)
staticprotected

Create a single exec node derived from thrift node; place exec node in 'pool'.

Definition at line 260 of file exec-node.cc.

References impala::ObjectPool::Add(), impala::Status::OK, and RETURN_IF_ERROR.

Referenced by CreateTreeHelper().

Status impala::ExecNode::CreateTree ( ObjectPool pool,
const TPlan &  plan,
const DescriptorTbl descs,
ExecNode **  root 
)
static

Creates exec node tree from list of nodes contained in plan via depth-first traversal. All nodes are placed in pool. Returns error if 'plan' is corrupted, otherwise success.

Definition at line 199 of file exec-node.cc.

References CreateTreeHelper(), impala::Status::OK, and impala::Status::ok().

Referenced by impala::PlanFragmentExecutor::Prepare().

Status impala::ExecNode::CreateTreeHelper ( ObjectPool pool,
const std::vector< TPlanNode > &  tnodes,
const DescriptorTbl descs,
ExecNode parent,
int *  node_idx,
ExecNode **  root 
)
staticprotected
virtual void impala::ExecNode::DebugString ( int  indentation_level,
std::stringstream *  out 
) const
virtual

Recursive helper method for generating a string for DebugString(). Implementations should call DebugString(int, std::stringstream) on their children. Input parameters: indentation_level: Current level in plan tree. Output parameters: out: Stream to accumulate debug string.

Reimplemented in impala::BlockingJoinNode, impala::PartitionedAggregationNode, impala::AnalyticEvalNode, impala::AggregationNode, impala::DataSourceScanNode, impala::ExchangeNode, impala::HBaseScanNode, impala::TopNNode, and impala::SortNode.

virtual Status impala::ExecNode::GetNext ( RuntimeState state,
RowBatch row_batch,
bool eos 
)
pure virtual

Retrieves rows and returns them via row_batch. Sets eos to true if subsequent calls will not retrieve any more rows. Data referenced by any tuples returned in row_batch must not be overwritten by the callee until Close() is called. The memory holding that data can be returned via row_batch's tuple_data_pool (in which case it may be deleted by the caller) or held on to by the callee. The row_batch, including its tuple_data_pool, will be destroyed by the caller at some point prior to the final Close() call. In other words, if the memory holding the tuple data will be referenced by the callee in subsequent GetNext() calls, it must not be attached to the row_batch's tuple_data_pool. Caller must not be holding any io buffers. This will cause deadlock. TODO: AggregationNode and HashJoinNode cannot be "re-opened" yet.

Implemented in impala::HdfsScanNode, impala::PartitionedAggregationNode, impala::AnalyticEvalNode, impala::PartitionedHashJoinNode, impala::AggregationNode, impala::HashJoinNode, impala::DataSourceScanNode, impala::CrossJoinNode, impala::ExchangeNode, impala::TopNNode, impala::HBaseScanNode, impala::SortNode, impala::UnionNode, impala::SelectNode, and impala::EmptySetNode.

Referenced by impala::PlanFragmentExecutor::GetNextInternal(), and impala::BlockingJoinNode::Open().

int impala::ExecNode::GetNodeIdFromProfile ( RuntimeProfile p)
static

Extract node id from p->name().

Definition at line 62 of file exec-node.cc.

References impala::RuntimeProfile::metadata().

void impala::ExecNode::InitRuntimeProfile ( const std::string &  name)
protected

Definition at line 371 of file exec-node.cc.

References id_, pool_, and runtime_profile_.

Referenced by ExecNode().

virtual bool impala::ExecNode::IsScanNode ( ) const
inlineprotectedvirtual

Reimplemented in impala::ScanNode.

Definition at line 251 of file exec-node.h.

int64_t impala::ExecNode::limit ( ) const
inline

Definition at line 158 of file exec-node.h.

References limit_.

Referenced by impala::CrossJoinNode::GetNext(), and impala::HashJoinNode::LeftJoinGetNext().

Status impala::ExecNode::Prepare ( RuntimeState state)
virtual

Sets up internal structures, etc., without doing any actual work. Must be called prior to Open(). Will only be called once in this node's lifetime. All code generation (adding functions to the LlvmCodeGen object) must happen in Prepare(). Retrieving the jit compiled function pointer must happen in Open(). If overridden in subclass, must first call superclass's Prepare().

Reimplemented in impala::HdfsScanNode, impala::PartitionedAggregationNode, impala::ScanNode, impala::AnalyticEvalNode, impala::PartitionedHashJoinNode, impala::AggregationNode, impala::HashJoinNode, impala::BlockingJoinNode, impala::CrossJoinNode, impala::DataSourceScanNode, impala::ExchangeNode, impala::TopNNode, impala::SortNode, impala::UnionNode, impala::HBaseScanNode, and impala::SelectNode.

Definition at line 130 of file exec-node.cc.

References ADD_COUNTER, impala::RuntimeProfile::AddDerivedCounter(), AddExprCtxsToFree(), children_, conjunct_ctxs_, ExecDebugAction(), expr_mem_tracker(), expr_mem_tracker_, impala::RuntimeState::instance_mem_tracker(), mem_tracker_, impala::Status::OK, impala::Expr::Prepare(), RETURN_IF_ERROR, row_desc(), ROW_THROUGHPUT_COUNTER, rows_returned_counter_, rows_returned_rate_, runtime_profile(), runtime_profile_, and impala::RuntimeProfile::UnitsPerSecond().

Referenced by impala::SelectNode::Prepare(), impala::SortNode::Prepare(), impala::UnionNode::Prepare(), impala::TopNNode::Prepare(), impala::ExchangeNode::Prepare(), impala::BlockingJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::AnalyticEvalNode::Prepare(), impala::ScanNode::Prepare(), impala::PlanFragmentExecutor::Prepare(), and impala::PartitionedAggregationNode::Prepare().

Status impala::ExecNode::QueryMaintenance ( RuntimeState state)
protectedvirtual
Status impala::ExecNode::Reset ( RuntimeState state)
virtual

Resets all data-specific state, returning this node to the state it was in after calling Prepare() and before calling Open(). Prepare() must have already been called before calling Reset(). Open() and GetNext() may have optionally been called. Close() must not have been called. If overridden in a subclass, must call superclass's Reset() at the end. The default implementation calls Reset() on children. Note that this function may be called many times, so should be fast. For example, accumulated memory does not need to be freed on every call if it's expensive.

Reimplemented in impala::HdfsScanNode, impala::PartitionedAggregationNode, impala::AnalyticEvalNode, impala::PartitionedHashJoinNode, impala::AggregationNode, impala::BlockingJoinNode, impala::HashJoinNode, impala::DataSourceScanNode, impala::CrossJoinNode, impala::ExchangeNode, impala::HBaseScanNode, impala::TopNNode, impala::SortNode, impala::UnionNode, and impala::SelectNode.

Definition at line 159 of file exec-node.cc.

References children_, impala::Status::OK, and RETURN_IF_ERROR.

Referenced by impala::BlockingJoinNode::Reset().

void impala::ExecNode::SetDebugOptions ( int  node_id,
TExecNodePhase::type  phase,
TDebugAction::type  action,
ExecNode tree 
)
static

Set debug action for node with given id in 'tree'.

Definition at line 332 of file exec-node.cc.

References children_, debug_action_, debug_phase_, and id_.

Referenced by impala::PlanFragmentExecutor::Prepare().

TPlanNodeType::type impala::ExecNode::type ( ) const
inline

Friends And Related Function Documentation

friend class DataSink
friend

Definition at line 172 of file exec-node.h.

Member Data Documentation

TDebugAction::type impala::ExecNode::debug_action_
protected

Definition at line 220 of file exec-node.h.

Referenced by ExecDebugAction(), and SetDebugOptions().

TExecNodePhase::type impala::ExecNode::debug_phase_
protected

debug-only: if debug_action_ is not INVALID, node will perform action in debug_phase_

Definition at line 219 of file exec-node.h.

Referenced by ExecDebugAction(), and SetDebugOptions().

boost::mutex impala::ExecNode::exec_options_lock_
protected

Execution options that are determined at runtime. This is added to the runtime profile at Close(). Examples for options logged here would be "Codegen Enabled"

Definition at line 238 of file exec-node.h.

Referenced by AddRuntimeExecOption().

std::vector<ExprContext*> impala::ExecNode::expr_ctxs_to_free_
private

Expr contexts whose local allocations are safe to free in the main execution thread.

Definition at line 286 of file exec-node.h.

Referenced by AddExprCtxToFree(), and QueryMaintenance().

boost::scoped_ptr<MemTracker> impala::ExecNode::expr_mem_tracker_
protected

MemTracker that should be used for ExprContexts.

Definition at line 233 of file exec-node.h.

Referenced by expr_mem_tracker(), and Prepare().

bool impala::ExecNode::is_closed_
private

Set in ExecNode::Close(). Used to make Close() idempotent. This is not protected by a lock, it assumes all calls to Close() are made by the same thread.

Definition at line 283 of file exec-node.h.

Referenced by Close(), impala::PartitionedHashJoinNode::Partition::Close(), and is_closed().

boost::scoped_ptr<MemTracker> impala::ExecNode::mem_tracker_
protected

Account for peak memory used by this node.

Definition at line 230 of file exec-node.h.

Referenced by mem_tracker(), and Prepare().

RowDescriptor impala::ExecNode::row_descriptor_
protected
const string impala::ExecNode::ROW_THROUGHPUT_COUNTER = "RowsReturnedRate"
static

Names of counters shared by all exec nodes.

Definition at line 169 of file exec-node.h.

Referenced by Prepare().

RuntimeProfile::Counter* impala::ExecNode::rows_returned_rate_
protected

Definition at line 227 of file exec-node.h.

Referenced by Prepare().

std::string impala::ExecNode::runtime_exec_options_
protected

Definition at line 239 of file exec-node.h.

Referenced by AddRuntimeExecOption().

TPlanNodeType::type impala::ExecNode::type_
protected

Definition at line 210 of file exec-node.h.

Referenced by CollectNodes(), and type().


The documentation for this class was generated from the following files: