Impala
Impalaistheopensource,nativeanalyticdatabaseforApacheHadoop.
|
#include <sort-node.h>
Public Member Functions | |
SortNode (ObjectPool *pool, const TPlanNode &tnode, const DescriptorTbl &descs) | |
~SortNode () | |
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) |
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... | |
const std::vector< ExprContext * > & | conjunct_ctxs () const |
int | id () const |
TPlanNodeType::type | type () const |
const RowDescriptor & | row_desc () const |
int64_t | rows_returned () const |
int64_t | limit () const |
bool | ReachedLimit () |
RuntimeProfile * | runtime_profile () |
MemTracker * | mem_tracker () |
MemTracker * | expr_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 | |
virtual void | DebugString (int indentation_level, std::stringstream *out) const |
ExecNode * | child (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_ |
ObjectPool * | pool_ |
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< RuntimeProfile > | runtime_profile_ |
RuntimeProfile::Counter * | rows_returned_counter_ |
RuntimeProfile::Counter * | rows_returned_rate_ |
boost::scoped_ptr< MemTracker > | mem_tracker_ |
Account for peak memory used by this node. More... | |
boost::scoped_ptr< MemTracker > | expr_mem_tracker_ |
MemTracker that should be used for ExprContexts. More... | |
boost::mutex | exec_options_lock_ |
std::string | runtime_exec_options_ |
Private Member Functions | |
Status | SortInput (RuntimeState *state) |
Fetch input rows and feed them to the sorter until the input is exhausted. More... | |
Status | CreateBlockMgr (RuntimeState *state) |
Private Attributes | |
int64_t | offset_ |
Number of rows to skip. More... | |
int64_t | num_rows_skipped_ |
boost::scoped_ptr< Sorter > | sorter_ |
Object used for external sorting. More... | |
SortExecExprs | sort_exec_exprs_ |
Expressions and parameters used for tuple materialization and tuple comparison. More... | |
std::vector< bool > | is_asc_order_ |
std::vector< bool > | nulls_first_ |
Node that implements a full sort of its input with a fixed memory budget, spilling to disk if the input is larger than available memory. Uses Sorter and BufferedBlockMgr for the external sort implementation. Input rows to SortNode are materialized by the Sorter into a single tuple using the expressions specified in sort_exec_exprs_. In GetNext(), SortNode passes in the output batch to the sorter instance created in Open() to fill it with sorted rows. If a merge phase was performed in the sort, sorted rows are deep copied into the output batch. Otherwise, the sorter instance owns the sorted data.
Definition at line 34 of file sort-node.h.
impala::SortNode::SortNode | ( | ObjectPool * | pool, |
const TPlanNode & | tnode, | ||
const DescriptorTbl & | descs | ||
) |
Definition at line 25 of file sort-node.cc.
impala::SortNode::~SortNode | ( | ) |
Definition at line 31 of file sort-node.cc.
|
protectedinherited |
Referenced by impala::ExecNode::AddExprCtxsToFree(), impala::UnionNode::Prepare(), Prepare(), impala::TopNNode::Prepare(), impala::ExchangeNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::ExecNode::Prepare(), impala::PartitionedAggregationNode::Prepare(), and impala::HdfsScanNode::Prepare().
|
protectedinherited |
Definition at line 410 of file exec-node.cc.
References impala::ExecNode::AddExprCtxsToFree(), impala::SortExecExprs::lhs_ordering_expr_ctxs(), impala::SortExecExprs::rhs_ordering_expr_ctxs(), and impala::SortExecExprs::sort_tuple_slot_expr_ctxs().
|
inlineprotectedinherited |
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 impala::ExecNode::expr_ctxs_to_free_.
Referenced by impala::AnalyticEvalNode::Prepare().
|
protectedinherited |
Appends option to 'runtime_exec_options_'.
Definition at line 188 of file exec-node.cc.
References impala::RuntimeProfile::AddInfoString(), impala::ExecNode::exec_options_lock_, impala::ExecNode::runtime_exec_options_, and impala::ExecNode::runtime_profile().
Referenced by impala::PartitionedHashJoinNode::AttachProbeFilters(), impala::HashJoinNode::ConstructBuildSide(), impala::BlockingJoinNode::Open(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::PartitionedAggregationNode::Prepare(), impala::HdfsScanNode::Prepare(), and impala::HdfsScanNode::StopAndFinalizeCounters().
|
inlineprotectedinherited |
Definition at line 241 of file exec-node.h.
References impala::ExecNode::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(), Open(), impala::TopNNode::Open(), impala::BlockingJoinNode::Open(), impala::AggregationNode::Open(), impala::AnalyticEvalNode::Open(), impala::PartitionedAggregationNode::Open(), impala::UnionNode::OpenCurrentChild(), impala::SelectNode::Prepare(), 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 SortInput().
|
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 from impala::ExecNode.
Definition at line 123 of file sort-node.cc.
References impala::SortExecExprs::Close(), impala::ExecNode::Close(), impala::ExecNode::is_closed(), sort_exec_exprs_, and sorter_.
|
staticinherited |
Returns a codegen'd version of EvalConjuncts(), or NULL if the function couldn't be codegen'd. The codegen'd version uses inlined, codegen'd GetBooleanVal() functions.
Definition at line 452 of file exec-node.cc.
References impala::LlvmCodeGen::FnPrototype::AddArgument(), impala::LlvmCodeGen::context(), impala::CodegenAnyVal::CreateCallWrapped(), impala::LlvmCodeGen::false_value(), impala::LlvmCodeGen::FinalizeFunction(), impala::RuntimeState::GetCodegen(), impala::Status::GetDetail(), impala::CodegenAnyVal::GetIsNull(), impala::LlvmCodeGen::GetType(), impala::CodegenAnyVal::GetVal(), impala::TupleRow::LLVM_CLASS_NAME, impala::ExprContext::LLVM_CLASS_NAME, impala::Status::ok(), impala::LlvmCodeGen::true_value(), impala::ExecNode::type(), impala::TYPE_BOOLEAN, impala::TYPE_INT, and VLOG_QUERY.
Referenced by impala::HdfsAvroScanner::CodegenDecodeAvroData(), impala::HashJoinNode::CodegenProcessProbeBatch(), and impala::PartitionedHashJoinNode::CodegenProcessProbeBatch().
|
inherited |
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 impala::ExecNode::children_, and impala::ExecNode::type_.
Referenced by impala::ExecNode::CollectScanNodes(), and impala::PlanFragmentExecutor::Prepare().
|
inherited |
Collect all scan node types.
Definition at line 366 of file exec-node.cc.
References impala::ExecNode::CollectNodes().
Referenced by impala::PlanFragmentExecutor::Prepare().
|
inlineinherited |
Definition at line 152 of file exec-node.h.
References impala::ExecNode::conjunct_ctxs_.
Referenced by impala::HdfsScanNode::ComputeSlotMaterializationOrder(), impala::SelectNode::CopyRows(), impala::UnionNode::EvalAndMaterializeExprs(), impala::HashJoinNode::GetNext(), impala::PartitionedHashJoinNode::OutputUnmatchedBuild(), impala::HashJoinNode::ProcessProbeBatch(), and impala::PartitionedHashJoinNode::ProcessProbeBatch().
|
private |
Create a block manager object and set it in block_mgr_. Returns and sets the query status to Status::MEM_LIMIT_EXCEEDED if there is not enough memory for the sort.
|
staticprotectedinherited |
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 impala::ExecNode::CreateTreeHelper().
|
staticinherited |
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 impala::ExecNode::CreateTreeHelper(), impala::Status::OK, and impala::Status::ok().
Referenced by impala::PlanFragmentExecutor::Prepare().
|
staticprotectedinherited |
Definition at line 218 of file exec-node.cc.
References impala::RuntimeProfile::AddChild(), impala::ExecNode::children_, impala::ExecNode::CreateNode(), impala::Status::OK, RETURN_IF_ERROR, and impala::ExecNode::runtime_profile().
Referenced by impala::ExecNode::CreateTree().
|
protectedvirtual |
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 from impala::ExecNode.
Definition at line 130 of file sort-node.cc.
References impala::ExecNode::DebugString(), impala::Expr::DebugString(), is_asc_order_, impala::SortExecExprs::lhs_ordering_expr_ctxs(), nulls_first_, and sort_exec_exprs_.
|
inherited |
Returns a string representation in DFS order of the plan rooted at this.
Definition at line 345 of file exec-node.cc.
Referenced by DebugString(), impala::TopNNode::DebugString(), impala::ExchangeNode::DebugString(), impala::AggregationNode::DebugString(), impala::AnalyticEvalNode::DebugString(), impala::PartitionedAggregationNode::DebugString(), impala::BlockingJoinNode::DebugString(), and impala::PlanFragmentExecutor::Prepare().
|
staticinherited |
Evaluate ExprContexts over row. Returns true if all exprs return true. TODO: This doesn't use the vector<Expr*> signature because I haven't figured out how to deal with declaring a templated std:vector type in IR
Definition at line 393 of file exec-node.cc.
References impala::ExprContext::GetBooleanVal(), impala_udf::AnyVal::is_null, and impala_udf::BooleanVal::val.
Referenced by impala::SelectNode::CopyRows(), impala::UnionNode::EvalAndMaterializeExprs(), impala::HdfsScanner::EvalConjuncts(), EvalOtherJoinConjuncts(), EvalOtherJoinConjuncts2(), impala::PartitionedHashJoinNode::EvaluateNullProbe(), impala::HBaseScanNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::PartitionedAggregationNode::GetNext(), impala::AnalyticEvalNode::GetNextOutputBatch(), impala::PartitionedHashJoinNode::OutputNullAwareProbeRows(), impala::PartitionedHashJoinNode::OutputUnmatchedBuild(), impala::CrossJoinNode::ProcessLeftChildBatch(), impala::HashJoinNode::ProcessProbeBatch(), and impala::PartitionedHashJoinNode::ProcessProbeBatch().
|
protectedinherited |
Executes debug_action_ if phase matches debug_phase_. 'phase' must not be INVALID.
Definition at line 378 of file exec-node.cc.
References impala::Status::CANCELLED, impala::ExecNode::debug_action_, impala::ExecNode::debug_phase_, impala::RuntimeState::is_cancelled(), and impala::Status::OK.
Referenced by impala::SelectNode::GetNext(), GetNext(), impala::UnionNode::GetNext(), impala::HBaseScanNode::GetNext(), impala::TopNNode::GetNext(), impala::ExchangeNode::GetNext(), impala::CrossJoinNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::AnalyticEvalNode::GetNext(), impala::PartitionedHashJoinNode::GetNext(), impala::PartitionedAggregationNode::GetNext(), impala::HdfsScanNode::GetNextInternal(), impala::ExecNode::Open(), and impala::ExecNode::Prepare().
|
inlineinherited |
Definition at line 163 of file exec-node.h.
References impala::ExecNode::expr_mem_tracker_.
Referenced by impala::PartitionedAggregationNode::Partition::InitStreams(), Prepare(), impala::UnionNode::Prepare(), impala::TopNNode::Prepare(), impala::ExchangeNode::Prepare(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::AnalyticEvalNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::ExecNode::Prepare(), impala::PartitionedAggregationNode::Prepare(), and impala::HdfsScanNode::Prepare().
|
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.
Implements impala::ExecNode.
Definition at line 78 of file sort-node.cc.
References impala::RowBatch::CopyRows(), COUNTER_SET, impala::ExecNode::ExecDebugAction(), impala::ExecNode::limit_, impala::RowBatch::num_rows(), impala::ExecNode::num_rows_returned_, num_rows_skipped_, offset_, impala::Status::OK, impala::ExecNode::QueryMaintenance(), impala::ExecNode::ReachedLimit(), RETURN_IF_CANCELLED, RETURN_IF_ERROR, impala::ExecNode::rows_returned_counter_, impala::ExecNode::runtime_profile_, SCOPED_TIMER, impala::RowBatch::set_num_rows(), and sorter_.
Referenced by SortInput().
|
staticinherited |
Extract node id from p->name().
Definition at line 62 of file exec-node.cc.
References impala::RuntimeProfile::metadata().
|
inlineinherited |
Definition at line 154 of file exec-node.h.
References impala::ExecNode::id_.
Referenced by impala::AnalyticEvalNode::AddResultTuple(), impala::AnalyticEvalNode::AddRow(), impala::AnalyticEvalNode::AnalyticEvalNode(), impala::AnalyticEvalNode::GetNext(), impala::AnalyticEvalNode::GetNextOutputBatch(), impala::HdfsScanner::InitializeWriteTuplesFn(), impala::HdfsAvroScanner::InitNewRange(), impala::AnalyticEvalNode::InitNextPartition(), impala::PartitionedAggregationNode::MoveHashPartitions(), impala::PartitionedHashJoinNode::NodeDebugString(), impala::AnalyticEvalNode::Open(), impala::HdfsScanNode::Open(), impala::PlanFragmentExecutor::Prepare(), impala::PartitionedHashJoinNode::ProcessBuildInput(), impala::AnalyticEvalNode::ProcessChildBatch(), impala::HdfsScanNode::ScannerThread(), impala::AnalyticEvalNode::TryAddRemainingResults(), impala::AnalyticEvalNode::TryAddResultTupleForCurrRow(), impala::AnalyticEvalNode::TryAddResultTupleForPrevRow(), and impala::AnalyticEvalNode::TryRemoveRowsBeforeWindow().
|
virtual |
Initializes this object from the thrift tnode desc. The subclass should do any initialization that can fail in Init() rather than the ctor. If overridden in subclass, must first call superclass's Init().
Reimplemented from impala::ExecNode.
Definition at line 34 of file sort-node.cc.
References impala::SortExecExprs::Init(), impala::ExecNode::Init(), is_asc_order_, nulls_first_, impala::Status::OK, impala::ExecNode::pool_, RETURN_IF_ERROR, and sort_exec_exprs_.
|
protectedinherited |
Definition at line 371 of file exec-node.cc.
References impala::ExecNode::id_, impala::ExecNode::pool_, and impala::ExecNode::runtime_profile_.
Referenced by impala::ExecNode::ExecNode().
|
inlineprotectedinherited |
Definition at line 242 of file exec-node.h.
References impala::ExecNode::is_closed_.
Referenced by impala::SelectNode::Close(), Close(), impala::UnionNode::Close(), impala::TopNNode::Close(), impala::ExchangeNode::Close(), impala::HBaseScanNode::Close(), impala::CrossJoinNode::Close(), impala::HashJoinNode::Close(), impala::AggregationNode::Close(), impala::BlockingJoinNode::Close(), impala::AnalyticEvalNode::Close(), impala::PartitionedHashJoinNode::Close(), impala::PartitionedAggregationNode::Close(), impala::HdfsScanNode::Close(), impala::PartitionedAggregationNode::Partition::Close(), impala::PartitionedHashJoinNode::Partition::Close(), impala::PartitionedHashJoinNode::ReserveTupleStreamBlocks(), impala::PartitionedHashJoinNode::SpillPartition(), impala::PartitionedAggregationNode::SpillPartition(), and impala::PartitionedHashJoinNode::Partition::~Partition().
|
inlineprotectedvirtualinherited |
Reimplemented in impala::ScanNode.
Definition at line 251 of file exec-node.h.
|
inlineinherited |
Definition at line 158 of file exec-node.h.
References impala::ExecNode::limit_.
Referenced by impala::CrossJoinNode::GetNext(), and impala::HashJoinNode::LeftJoinGetNext().
|
inlineinherited |
Definition at line 162 of file exec-node.h.
References impala::ExecNode::mem_tracker_.
Referenced by impala::ExecNode::Close(), impala::CrossJoinNode::ConstructBuildSide(), impala::HashJoinNode::ConstructBuildSide(), impala::HdfsScanNode::EnoughMemoryForScannerThread(), impala::AnalyticEvalNode::GetNextOutputBatch(), Open(), impala::TopNNode::Open(), impala::AggregationNode::Open(), impala::AnalyticEvalNode::Open(), impala::PartitionedAggregationNode::Open(), impala::HdfsScanNode::Open(), impala::UnionNode::OpenCurrentChild(), impala::SelectNode::Prepare(), impala::HBaseScanNode::Prepare(), impala::TopNNode::Prepare(), impala::BlockingJoinNode::Prepare(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::AnalyticEvalNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::PartitionedAggregationNode::Prepare(), impala::HdfsScanNode::Prepare(), impala::PartitionedHashJoinNode::PrepareNextPartition(), impala::PartitionedHashJoinNode::ProcessBuildInput(), impala::PartitionedAggregationNode::ProcessStream(), impala::HdfsRCFileScanner::ReadRowGroup(), impala::HdfsAvroScanner::ResolveSchemas(), SortInput(), and impala::HdfsScanner::StartNewRowBatch().
|
virtual |
Performs any preparatory work prior to calling GetNext(). Caller must not be holding any io buffers. This will cause deadlock. If overridden in subclass, must first call superclass's Open(). If a parent exec node adds slot filters (see RuntimeState::AddBitmapFilter()), they need to be added before calling Open() on the child that will consume them.
Reimplemented from impala::ExecNode.
Definition at line 51 of file sort-node.cc.
References impala::ExecNode::child(), impala::ExecNode::Close(), is_asc_order_, impala::SortExecExprs::lhs_ordering_expr_ctxs(), impala::ExecNode::mem_tracker(), nulls_first_, impala::Status::OK, impala::SortExecExprs::Open(), impala::ExecNode::Open(), impala::ExecNode::QueryMaintenance(), RETURN_IF_CANCELLED, RETURN_IF_ERROR, impala::SortExecExprs::rhs_ordering_expr_ctxs(), impala::ExecNode::row_descriptor_, impala::ExecNode::runtime_profile(), impala::ExecNode::runtime_profile_, SCOPED_TIMER, sort_exec_exprs_, impala::SortExecExprs::sort_tuple_slot_expr_ctxs(), sorter_, and SortInput().
|
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 from impala::ExecNode.
Definition at line 42 of file sort-node.cc.
References impala::ExecNode::AddExprCtxsToFree(), impala::ExecNode::child(), impala::ExecNode::expr_mem_tracker(), impala::Status::OK, impala::SortExecExprs::Prepare(), impala::ExecNode::Prepare(), RETURN_IF_ERROR, impala::ExecNode::row_desc(), impala::ExecNode::row_descriptor_, impala::ExecNode::runtime_profile_, SCOPED_TIMER, and sort_exec_exprs_.
|
protectedvirtualinherited |
Frees any local allocations made by expr_ctxs_to_free_ and returns the result of state->CheckQueryState(). Nodes should call this periodically, e.g. once per input row batch. This should not be called outside the main execution thread. Nodes may override this to add extra periodic cleanup, e.g. freeing other local allocations. ExecNodes overriding this function should return ExecNode::QueryMaintenance().
Reimplemented in impala::PartitionedAggregationNode, and impala::AnalyticEvalNode.
Definition at line 401 of file exec-node.cc.
References impala::RuntimeState::CheckQueryState(), impala::ExecNode::expr_ctxs_to_free_, and impala::ExprContext::FreeLocalAllocations().
Referenced by impala::CrossJoinNode::ConstructBuildSide(), impala::HashJoinNode::ConstructBuildSide(), impala::SelectNode::GetNext(), GetNext(), impala::UnionNode::GetNext(), impala::HBaseScanNode::GetNext(), impala::TopNNode::GetNext(), impala::ExchangeNode::GetNext(), impala::CrossJoinNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::PartitionedHashJoinNode::GetNext(), impala::HdfsScanNode::GetNextInternal(), impala::HBaseScanNode::Open(), Open(), impala::TopNNode::Open(), impala::BlockingJoinNode::Open(), impala::AggregationNode::Open(), impala::PartitionedHashJoinNode::ProcessBuildInput(), impala::AnalyticEvalNode::QueryMaintenance(), impala::PartitionedAggregationNode::QueryMaintenance(), and SortInput().
|
inlineinherited |
Definition at line 159 of file exec-node.h.
References impala::ExecNode::limit_, and impala::ExecNode::num_rows_returned_.
Referenced by impala::HdfsParquetScanner::AssembleRows(), impala::SelectNode::CopyRows(), impala::UnionNode::EvalAndMaterializeExprs(), impala::HdfsTextScanner::FinishScanRange(), impala::SelectNode::GetNext(), impala::UnionNode::GetNext(), GetNext(), impala::HBaseScanNode::GetNext(), impala::ExchangeNode::GetNext(), impala::CrossJoinNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::PartitionedHashJoinNode::GetNext(), impala::AnalyticEvalNode::GetNext(), impala::PartitionedAggregationNode::GetNext(), impala::HdfsScanNode::GetNextInternal(), impala::ExchangeNode::GetNextMerging(), impala::AnalyticEvalNode::GetNextOutputBatch(), impala::HashJoinNode::LeftJoinGetNext(), impala::HdfsSequenceScanner::ProcessBlockCompressedScanRange(), impala::HdfsTextScanner::ProcessRange(), impala::HdfsAvroScanner::ProcessRange(), impala::HdfsSequenceScanner::ProcessRange(), impala::HdfsRCFileScanner::ProcessRange(), and impala::PlanFragmentExecutor::ReachedLimit().
|
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 from impala::ExecNode.
Definition at line 118 of file sort-node.cc.
|
inlineinherited |
Definition at line 156 of file exec-node.h.
References impala::ExecNode::row_descriptor_.
Referenced by impala::CrossJoinNode::BuildListDebugString(), impala::HashJoinNode::CodegenCreateOutputRow(), impala::PartitionedHashJoinNode::CodegenCreateOutputRow(), impala::CrossJoinNode::ConstructBuildSide(), impala::HashJoinNode::ConstructBuildSide(), impala::BlockingJoinNode::GetLeftChildRowString(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::AnalyticEvalNode::GetNextOutputBatch(), impala::PartitionedAggregationNode::Partition::InitStreams(), impala::TopNNode::Open(), impala::AggregationNode::Open(), impala::AnalyticEvalNode::Open(), impala::PartitionedAggregationNode::Open(), impala::UnionNode::OpenCurrentChild(), impala::SelectNode::Prepare(), Prepare(), impala::UnionNode::Prepare(), impala::TopNNode::Prepare(), impala::BlockingJoinNode::Prepare(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::AnalyticEvalNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::ExecNode::Prepare(), impala::PlanFragmentExecutor::Prepare(), impala::PartitionedAggregationNode::Prepare(), impala::HdfsScanNode::Prepare(), impala::PartitionedHashJoinNode::ProcessBuildInput(), impala::PartitionedAggregationNode::ProcessStream(), impala::PlanFragmentExecutor::row_desc(), SortInput(), and impala::HdfsScanner::StartNewRowBatch().
|
inlineinherited |
Definition at line 157 of file exec-node.h.
References impala::ExecNode::num_rows_returned_.
Referenced by impala::CrossJoinNode::GetNext(), impala::AnalyticEvalNode::GetNext(), impala::HashJoinNode::LeftJoinGetNext(), impala::PartitionedAggregationNode::Open(), impala::HdfsSequenceScanner::ProcessDecompressedBlock(), impala::CrossJoinNode::ProcessLeftChildBatch(), impala::HashJoinNode::ProcessProbeBatch(), and impala::HdfsTextScanner::WriteFields().
|
inlineinherited |
Definition at line 161 of file exec-node.h.
References impala::ExecNode::runtime_profile_.
Referenced by impala::ExecNode::AddRuntimeExecOption(), impala::BlockingJoinNode::BuildSideThread(), impala::ExecNode::CreateTreeHelper(), impala::PartitionedAggregationNode::Partition::InitStreams(), Open(), impala::AnalyticEvalNode::Open(), impala::PartitionedAggregationNode::Open(), impala::HdfsScanNode::Open(), impala::HdfsTextScanner::Prepare(), impala::HBaseScanNode::Prepare(), impala::BaseSequenceScanner::Prepare(), impala::ExchangeNode::Prepare(), impala::HdfsParquetScanner::Prepare(), impala::BlockingJoinNode::Prepare(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::AnalyticEvalNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::ExecNode::Prepare(), impala::ScanNode::Prepare(), impala::PlanFragmentExecutor::Prepare(), impala::PartitionedAggregationNode::Prepare(), impala::HdfsScanner::Prepare(), impala::HdfsScanNode::Prepare(), and impala::PartitionedHashJoinNode::ProcessBuildInput().
|
staticinherited |
Set debug action for node with given id in 'tree'.
Definition at line 332 of file exec-node.cc.
References impala::ExecNode::children_, impala::ExecNode::debug_action_, impala::ExecNode::debug_phase_, and impala::ExecNode::id_.
Referenced by impala::PlanFragmentExecutor::Prepare().
|
private |
Fetch input rows and feed them to the sorter until the input is exhausted.
Definition at line 143 of file sort-node.cc.
References impala::RuntimeState::batch_size(), impala::ExecNode::child(), GetNext(), impala::ExecNode::mem_tracker(), impala::Status::OK, impala::ExecNode::QueryMaintenance(), impala::RowBatch::Reset(), RETURN_IF_CANCELLED, RETURN_IF_ERROR, impala::ExecNode::row_desc(), and sorter_.
Referenced by Open().
|
inlineinherited |
Definition at line 155 of file exec-node.h.
References impala::ExecNode::type_.
Referenced by impala::ExecNode::CodegenEvalConjuncts(), impala::PartitionedAggregationNode::CodegenUpdateTuple(), and impala::PlanFragmentExecutor::Prepare().
|
protectedinherited |
Definition at line 214 of file exec-node.h.
Referenced by impala::ExecNode::child(), impala::ExecNode::Close(), impala::ExecNode::CollectNodes(), impala::ExecNode::CreateTreeHelper(), impala::HBaseScanNode::DebugString(), impala::UnionNode::GetNext(), impala::UnionNode::Open(), impala::AggregationNode::Open(), impala::PartitionedAggregationNode::Open(), impala::UnionNode::OpenCurrentChild(), impala::ExecNode::Prepare(), impala::PartitionedAggregationNode::ProcessStream(), impala::ExecNode::Reset(), and impala::ExecNode::SetDebugOptions().
|
protectedinherited |
Definition at line 212 of file exec-node.h.
Referenced by impala::ExecNode::Close(), impala::HashJoinNode::CodegenProcessProbeBatch(), impala::PartitionedHashJoinNode::CodegenProcessProbeBatch(), impala::ExecNode::conjunct_ctxs(), impala::SelectNode::CopyRows(), impala::UnionNode::EvalAndMaterializeExprs(), impala::HBaseScanNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::PartitionedAggregationNode::GetNext(), impala::AnalyticEvalNode::GetNextOutputBatch(), impala::TopNNode::Init(), impala::ExecNode::Init(), impala::PartitionedHashJoinNode::Init(), impala::ExecNode::Open(), impala::PartitionedHashJoinNode::OutputUnmatchedBuild(), impala::ExecNode::Prepare(), impala::CrossJoinNode::ProcessLeftChildBatch(), impala::HashJoinNode::ProcessProbeBatch(), and impala::PartitionedHashJoinNode::ProcessProbeBatch().
|
protectedinherited |
Definition at line 220 of file exec-node.h.
Referenced by impala::ExecNode::ExecDebugAction(), and impala::ExecNode::SetDebugOptions().
|
protectedinherited |
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 impala::ExecNode::ExecDebugAction(), and impala::ExecNode::SetDebugOptions().
|
protectedinherited |
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 impala::ExecNode::AddRuntimeExecOption().
|
protectedinherited |
MemTracker that should be used for ExprContexts.
Definition at line 233 of file exec-node.h.
Referenced by impala::ExecNode::expr_mem_tracker(), and impala::ExecNode::Prepare().
|
protectedinherited |
Definition at line 209 of file exec-node.h.
Referenced by impala::PartitionedAggregationNode::CreateHashPartitions(), impala::ExecNode::id(), impala::ExecNode::InitRuntimeProfile(), impala::PartitionedAggregationNode::NextPartition(), impala::ExchangeNode::Prepare(), impala::PartitionedHashJoinNode::PrepareNextPartition(), impala::PartitionedHashJoinNode::ProcessBuildInput(), and impala::ExecNode::SetDebugOptions().
|
private |
Definition at line 67 of file sort-node.h.
Referenced by DebugString(), Init(), and Open().
|
protectedinherited |
Definition at line 222 of file exec-node.h.
Referenced by GetNext(), impala::HdfsScanNode::GetNextInternal(), impala::ExchangeNode::GetNextMerging(), impala::TopNNode::InsertTupleRow(), impala::HdfsScanNode::limit(), impala::ExecNode::limit(), impala::TopNNode::Open(), and impala::ExecNode::ReachedLimit().
|
protectedinherited |
Account for peak memory used by this node.
Definition at line 230 of file exec-node.h.
Referenced by impala::ExecNode::mem_tracker(), and impala::ExecNode::Prepare().
|
private |
Definition at line 68 of file sort-node.h.
Referenced by DebugString(), Init(), and Open().
|
protectedinherited |
Definition at line 223 of file exec-node.h.
Referenced by impala::ExecNode::Close(), impala::SelectNode::CopyRows(), impala::UnionNode::EvalAndMaterializeExprs(), GetNext(), impala::HBaseScanNode::GetNext(), impala::TopNNode::GetNext(), impala::ExchangeNode::GetNext(), impala::CrossJoinNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::PartitionedHashJoinNode::GetNext(), impala::AnalyticEvalNode::GetNext(), impala::PartitionedAggregationNode::GetNext(), impala::HdfsScanNode::GetNextInternal(), impala::ExchangeNode::GetNextMerging(), impala::AnalyticEvalNode::GetNextOutputBatch(), impala::HashJoinNode::LeftJoinGetNext(), impala::PartitionedHashJoinNode::OutputUnmatchedBuild(), impala::ExecNode::ReachedLimit(), and impala::ExecNode::rows_returned().
|
private |
Definition at line 60 of file sort-node.h.
Referenced by GetNext().
|
private |
|
protectedinherited |
Definition at line 211 of file exec-node.h.
Referenced by Init(), impala::UnionNode::Init(), impala::TopNNode::Init(), impala::ExchangeNode::Init(), impala::HashJoinNode::Init(), impala::AggregationNode::Init(), impala::ExecNode::Init(), impala::PartitionedHashJoinNode::Init(), impala::AnalyticEvalNode::Init(), impala::PartitionedAggregationNode::Init(), impala::ExecNode::InitRuntimeProfile(), impala::HdfsScanNode::Open(), impala::PartitionedHashJoinNode::Prepare(), and impala::PartitionedHashJoinNode::ProcessBuildInput().
|
protectedinherited |
Definition at line 215 of file exec-node.h.
Referenced by Open(), Prepare(), impala::TopNNode::Prepare(), impala::ExchangeNode::Prepare(), and impala::ExecNode::row_desc().
|
staticinherited |
Names of counters shared by all exec nodes.
Definition at line 169 of file exec-node.h.
Referenced by impala::ExecNode::Prepare().
|
protectedinherited |
Definition at line 226 of file exec-node.h.
Referenced by impala::ExecNode::Close(), impala::SelectNode::CopyRows(), impala::UnionNode::EvalAndMaterializeExprs(), GetNext(), impala::HBaseScanNode::GetNext(), impala::TopNNode::GetNext(), impala::ExchangeNode::GetNext(), impala::CrossJoinNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::AnalyticEvalNode::GetNext(), impala::PartitionedHashJoinNode::GetNext(), impala::PartitionedAggregationNode::GetNext(), impala::HdfsScanNode::GetNextInternal(), impala::ExchangeNode::GetNextMerging(), impala::HashJoinNode::LeftJoinGetNext(), impala::PartitionedHashJoinNode::OutputUnmatchedBuild(), and impala::ExecNode::Prepare().
|
protectedinherited |
Definition at line 227 of file exec-node.h.
Referenced by impala::ExecNode::Prepare().
|
protectedinherited |
Definition at line 239 of file exec-node.h.
Referenced by impala::ExecNode::AddRuntimeExecOption().
|
protectedinherited |
Definition at line 225 of file exec-node.h.
Referenced by impala::HBaseScanNode::Close(), impala::SelectNode::GetNext(), impala::UnionNode::GetNext(), GetNext(), impala::HBaseScanNode::GetNext(), impala::TopNNode::GetNext(), impala::ExchangeNode::GetNext(), impala::CrossJoinNode::GetNext(), impala::HashJoinNode::GetNext(), impala::AggregationNode::GetNext(), impala::AnalyticEvalNode::GetNext(), impala::PartitionedHashJoinNode::GetNext(), impala::PartitionedAggregationNode::GetNext(), impala::HdfsScanNode::GetNext(), impala::ExecNode::InitRuntimeProfile(), impala::SelectNode::Open(), impala::HBaseScanNode::Open(), impala::UnionNode::Open(), Open(), impala::TopNNode::Open(), impala::ExchangeNode::Open(), impala::DataSourceScanNode::Open(), impala::BlockingJoinNode::Open(), impala::AggregationNode::Open(), impala::AnalyticEvalNode::Open(), impala::PartitionedAggregationNode::Open(), impala::SelectNode::Prepare(), impala::UnionNode::Prepare(), Prepare(), impala::TopNNode::Prepare(), impala::BlockingJoinNode::Prepare(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), impala::AnalyticEvalNode::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::ExecNode::Prepare(), impala::ScanNode::Prepare(), impala::PartitionedAggregationNode::Prepare(), impala::HdfsScanNode::Prepare(), impala::ExecNode::runtime_profile(), and impala::HdfsScanNode::StopAndFinalizeCounters().
|
private |
Expressions and parameters used for tuple materialization and tuple comparison.
Definition at line 66 of file sort-node.h.
Referenced by Close(), DebugString(), Init(), Open(), and Prepare().
|
private |
Object used for external sorting.
Definition at line 63 of file sort-node.h.
Referenced by Close(), GetNext(), Open(), and SortInput().
|
protectedinherited |
Definition at line 210 of file exec-node.h.
Referenced by impala::ExecNode::CollectNodes(), and impala::ExecNode::type().