Impala
Impalaistheopensource,nativeanalyticdatabaseforApacheHadoop.
|
#include <scalar-fn-call.h>
Public Member Functions | |
virtual std::string | DebugString () const |
int | output_scale () const |
void | AddChild (Expr *expr) |
Expr * | GetChild (int i) const |
int | GetNumChildren () const |
const ColumnType & | type () const |
bool | is_slotref () const |
const std::vector< Expr * > & | children () const |
virtual int | GetSlotIds (std::vector< SlotId > *slot_ids) const |
virtual AnyVal * | GetConstVal (ExprContext *context) |
Static Public Member Functions | |
static Status | CreateExprTree (ObjectPool *pool, const TExpr &texpr, ExprContext **ctx) |
static Status | CreateExprTrees (ObjectPool *pool, const std::vector< TExpr > &texprs, std::vector< ExprContext * > *ctxs) |
static Status | Prepare (const std::vector< ExprContext * > &ctxs, RuntimeState *state, const RowDescriptor &row_desc, MemTracker *tracker) |
static Status | Open (const std::vector< ExprContext * > &ctxs, RuntimeState *state) |
Convenience function for opening multiple expr trees. More... | |
static Status | Clone (const std::vector< ExprContext * > &ctxs, RuntimeState *state, std::vector< ExprContext * > *new_ctxs) |
static void | Close (const std::vector< ExprContext * > &ctxs, RuntimeState *state) |
Convenience function for closing multiple expr trees. More... | |
static Expr * | CreateLiteral (ObjectPool *pool, const ColumnType &type, void *data) |
static Expr * | CreateLiteral (ObjectPool *pool, const ColumnType &type, const std::string &) |
static int | ComputeResultsLayout (const std::vector< Expr * > &exprs, std::vector< int > *offsets, int *var_result_begin) |
static int | ComputeResultsLayout (const std::vector< ExprContext * > &ctxs, std::vector< int > *offsets, int *var_result_begin) |
static std::string | DebugString (const std::vector< Expr * > &exprs) |
static std::string | DebugString (const std::vector< ExprContext * > &ctxs) |
static void | InitBuiltinsDummy () |
Static Public Attributes | |
static const char * | LLVM_CLASS_NAME = "class.impala::Expr" |
Protected Member Functions | |
ScalarFnCall (const TExprNode &node) | |
virtual Status | Prepare (RuntimeState *state, const RowDescriptor &desc, ExprContext *context) |
virtual Status | Open (RuntimeState *state, ExprContext *context, FunctionContext::FunctionStateScope scope=FunctionContext::FRAGMENT_LOCAL) |
virtual Status | GetCodegendComputeFn (RuntimeState *state, llvm::Function **fn) |
virtual void | Close (RuntimeState *state, ExprContext *context, FunctionContext::FunctionStateScope scope=FunctionContext::FRAGMENT_LOCAL) |
Subclasses overriding this function should call Expr::Close(). More... | |
virtual bool | IsConstant () const |
virtual BooleanVal | GetBooleanVal (ExprContext *context, TupleRow *) |
virtual TinyIntVal | GetTinyIntVal (ExprContext *context, TupleRow *) |
virtual SmallIntVal | GetSmallIntVal (ExprContext *context, TupleRow *) |
virtual IntVal | GetIntVal (ExprContext *context, TupleRow *) |
virtual BigIntVal | GetBigIntVal (ExprContext *context, TupleRow *) |
virtual FloatVal | GetFloatVal (ExprContext *context, TupleRow *) |
virtual DoubleVal | GetDoubleVal (ExprContext *context, TupleRow *) |
virtual StringVal | GetStringVal (ExprContext *context, TupleRow *) |
virtual TimestampVal | GetTimestampVal (ExprContext *context, TupleRow *) |
virtual DecimalVal | GetDecimalVal (ExprContext *context, TupleRow *) |
std::string | DebugString (const std::string &expr_name) const |
Simple debug string that provides no expr subclass-specific information. More... | |
FunctionContext * | RegisterFunctionContext (ExprContext *ctx, RuntimeState *state, int varargs_buffer_size=0) |
llvm::Function * | CreateIrFunctionPrototype (LlvmCodeGen *codegen, const std::string &name, llvm::Value *(*args)[2]) |
Status | GetCodegendComputeFnWrapper (RuntimeState *state, llvm::Function **fn) |
llvm::Function * | GetStaticGetValWrapper (ColumnType type, LlvmCodeGen *codegen) |
Protected Attributes | |
LibCache::LibCacheEntry * | cache_entry_ |
Cache entry for the library implementing this function. More... | |
TFunction | fn_ |
Function description. More... | |
const bool | is_slotref_ |
recognize if this node is a slotref in order to speed up GetValue() More... | |
const ColumnType | type_ |
analysis is done, types are fixed at this point More... | |
std::vector< Expr * > | children_ |
int | output_scale_ |
int | context_index_ |
llvm::Function * | ir_compute_fn_ |
Cached codegened compute function. Exprs should set this in GetCodegendComputeFn(). More... | |
boost::scoped_ptr< AnyVal > | constant_val_ |
Private Member Functions | |
int | NumFixedArgs () const |
Returns the number of non-vararg arguments. More... | |
Status | GetUdf (RuntimeState *state, llvm::Function **udf) |
Loads the native or IR function from HDFS and puts the result in *udf. More... | |
Status | GetFunction (RuntimeState *state, const std::string &symbol, void **fn) |
void | EvaluateChildren (ExprContext *context, TupleRow *row, std::vector< impala_udf::AnyVal * > *input_vals) |
template<typename RETURN_TYPE > | |
RETURN_TYPE | InterpretEval (ExprContext *context, TupleRow *row) |
Function to call scalar_fn_. Used in the interpreted path. More... | |
Private Attributes | |
int | vararg_start_idx_ |
void * | scalar_fn_wrapper_ |
UdfPrepare | prepare_fn_ |
UdfClose | close_fn_ |
void * | scalar_fn_ |
Friends | |
class | Expr |
Expr for evaluating a pre-compiled native or LLVM IR function that uses the UDF interface (i.e. a scalar function). This class overrides GetCodegendComputeFn() to return a function that calls any child exprs and passes the results as arguments to the specified scalar function. If codegen is enabled, ScalarFnCall's Get*Val() compute functions are wrappers around this codegen'd function. If codegen is disabled, some native functions can be called without codegen, depending on the native function's signature. However, since we can't write static code to call every possible function signature, codegen may be required to generate the call to the function even if codegen is disabled. Codegen will also be used for IR UDFs (note that there is no way to specify both a native and IR library for a single UDF). TODO:
Definition at line 48 of file scalar-fn-call.h.
|
protected |
Definition at line 41 of file scalar-fn-call.cc.
References impala::Expr::fn_.
|
inlineinherited |
Definition at line 141 of file expr.h.
Referenced by impala::Expr::CreateTreeFromThrift().
|
inlineinherited |
Definition at line 148 of file expr.h.
Referenced by impala::CaseExpr::GetChildVal(), and impala::CaseExpr::GetCodegendComputeFn().
|
staticinherited |
Clones each ExprContext for multiple expr trees. 'new_ctxs' should be an empty vector, and a clone of each context in 'ctxs' will be added to it. The new ExprContexts are created in state->obj_pool().
Definition at line 374 of file expr.cc.
References impala::Status::OK, and RETURN_IF_ERROR.
Referenced by impala::HdfsScanNode::GetConjunctCtxs(), and impala::SortExecExprs::Open().
|
protectedvirtual |
Subclasses overriding this function should call Expr::Close().
If scope if FRAGMENT_LOCAL, both fragment- and thread-local state should be torn down. Otherwise, if scope is THREAD_LOCAL, only thread-local state should be torn down.
Reimplemented from impala::Expr.
Definition at line 198 of file scalar-fn-call.cc.
References impala::Expr::Close(), close_fn_, impala::Expr::context_index_, and impala::ExprContext::fn_context().
|
staticinherited |
Convenience function for closing multiple expr trees.
Referenced by impala::UnionNode::Close(), impala::SortExecExprs::Close(), impala::CaseExpr::Close(), impala::HashJoinNode::Close(), impala::AggregationNode::Close(), Close(), impala::ExprContext::Close(), impala::HiveUdfCall::Close(), impala::PartitionedHashJoinNode::Close(), impala::DataStreamSender::Close(), impala::PartitionedAggregationNode::Close(), impala::AggFnEvaluator::Close(), impala::HdfsScanner::Close(), impala::ExecNode::Close(), impala::HdfsScanNode::Close(), impala::HdfsTableSink::Close(), impala::HdfsPartitionDescriptor::CloseExprs(), impala::ImpalaServer::QueryExecState::Done(), impala::OldHashTableTest::TearDown(), and impala::HashTableTest::TearDown().
|
staticinherited |
Computes a memory efficient layout for storing the results of evaluating 'exprs' Returns the number of bytes necessary to store all the results and offsets where the result for each expr should be stored. Variable length types are guaranteed to be at the end and 'var_result_begin' will be set the beginning byte offset where variable length results begin. 'var_result_begin' will be set to -1 if there are no variable len types.
Referenced by impala::HashTableCtx::HashTableCtx(), and impala::OldHashTable::OldHashTable().
|
staticinherited |
|
staticinherited |
Create expression tree from the list of nodes contained in texpr within 'pool'. Returns the root of expression tree in 'expr' and the corresponding ExprContext in 'ctx'.
Definition at line 129 of file expr.cc.
References impala::Expr::CreateTreeFromThrift(), impala::Status::GetDetail(), impala::Status::OK, and impala::Status::ok().
Referenced by impala::Expr::CreateExprTrees(), impala::HdfsPartitionDescriptor::HdfsPartitionDescriptor(), impala::HashJoinNode::Init(), impala::AnalyticEvalNode::Init(), impala::PartitionedHashJoinNode::Init(), and Java_com_cloudera_impala_service_FeSupport_NativeEvalConstExprs().
|
staticinherited |
Creates vector of ExprContexts containing exprs from the given vector of TExprs within 'pool'. Returns an error if any of the individual conversions caused an error, otherwise OK.
Definition at line 149 of file expr.cc.
References impala::Expr::CreateExprTree(), impala::Status::OK, and RETURN_IF_ERROR.
Referenced by impala::DataStreamSender::DataStreamSender(), impala::UnionNode::Init(), impala::HashJoinNode::Init(), impala::AggregationNode::Init(), impala::ExecNode::Init(), impala::PartitionedHashJoinNode::Init(), impala::PartitionedAggregationNode::Init(), impala::HdfsScanNode::Prepare(), and impala::HdfsTableSink::Prepare().
|
protectedinherited |
Helper function to create an empty Function* with the appropriate signature to be returned by GetCodegendComputeFn(). 'name' is the name of the returned Function*. The arguments to the function are returned in 'args'.
Definition at line 456 of file expr.cc.
References impala::LlvmCodeGen::FnPrototype::AddArgument(), impala::LlvmCodeGen::FnPrototype::GeneratePrototype(), impala::CodegenAnyVal::GetLoweredType(), impala::LlvmCodeGen::GetPtrType(), impala::TupleRow::LLVM_CLASS_NAME, impala::ExprContext::LLVM_CLASS_NAME, and impala::Expr::type().
Referenced by impala::CaseExpr::GetCodegendComputeFn(), impala::SlotRef::GetCodegendComputeFn(), impala::Literal::GetCodegendComputeFn(), GetCodegendComputeFn(), and impala::Expr::GetCodegendComputeFnWrapper().
|
staticinherited |
Create a new literal expr of 'type' with initial 'data'. data should match the ColumnType (i.e. type == TYPE_INT, data is a int*) The new Expr will be allocated from the pool.
|
staticinherited |
Create a new literal expr of 'type' by parsing the string. NULL will be returned if the string and type are not compatible. The new Expr will be allocated from the pool.
|
virtual |
Reimplemented from impala::Expr.
Definition at line 733 of file scalar-fn-call.cc.
References impala::Expr::DebugString(), and impala::Expr::fn_.
|
staticinherited |
|
staticinherited |
|
inlineprotectedinherited |
Simple debug string that provides no expr subclass-specific information.
Definition at line 332 of file expr.h.
References impala_udf::DebugString().
|
private |
Evaluates the children exprs and stores the results in input_vals. Used in the interpreted path.
Definition at line 493 of file scalar-fn-call.cc.
References impala::AnyValUtil::AnyValSize(), impala::Expr::children_, impala::Expr::context_index_, impala::ExprContext::fn_context(), impala::ExprContext::GetValue(), impala_udf::FunctionContext::impl(), NumFixedArgs(), impala::AnyValUtil::SetAnyVal(), impala::Expr::type(), vararg_start_idx_, and impala::FunctionContextImpl::varargs_buffer().
Referenced by InterpretEval().
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 685 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_BIGINT.
|
protectedvirtual |
Virtual compute functions for each *Val type. Each Expr subclass should implement the functions for the return type(s) it supports. For example, a boolean function will only implement GetBooleanVal(). Some Exprs, like Literal, have many possible return types and will implement multiple Get*Val() functions.
Reimplemented from impala::Expr.
Definition at line 653 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_BOOLEAN.
|
inlineinherited |
Definition at line 142 of file expr.h.
Referenced by impala::HiveUdfCall::Evaluate(), and impala::HiveUdfCall::Prepare().
|
protectedvirtual |
Returns an llvm::Function* with signature: <subclass of="" anyval>=""> ComputeFn(ExprContext* context, TupleRow* row) The function should evaluate this expr over 'row' and return the result as the appropriate type of AnyVal.
Implements impala::Expr.
Definition at line 243 of file scalar-fn-call.cc.
References impala::AnyValUtil::AnyValSize(), impala::LlvmCodeGen::CastPtrToLlvmPtr(), impala::Expr::children_, impala::RuntimeState::codegen_enabled(), impala::LlvmCodeGen::context(), impala::Expr::context_index_, impala::CodegenAnyVal::CreateCall(), impala::LlvmCodeGen::CreateEntryBlockAlloca(), impala::Expr::CreateIrFunctionPrototype(), impala::SymbolsUtil::DemangleNameOnly(), impala::LlvmCodeGen::FinalizeFunction(), gen_ir_descriptions::fn_name, impala::RuntimeState::GetCodegen(), impala::LlvmCodeGen::GetIntConstant(), impala::CodegenAnyVal::GetLoweredPtrType(), impala::Expr::GetNumChildren(), impala::LlvmCodeGen::GetPtrType(), impala::Expr::GetStaticGetValWrapper(), GetUdf(), impala::CodegenAnyVal::GetUnloweredType(), impala::Expr::ir_compute_fn_, impala::Expr::LLVM_CLASS_NAME, impala::Status::OK, RETURN_IF_ERROR, impala::ColumnType::type, impala::Expr::type(), impala::TYPE_CHAR, impala::TYPE_DECIMAL, impala::TYPE_INT, and vararg_start_idx_.
Referenced by Prepare().
|
protectedinherited |
Generates an IR compute function that calls the appropriate interpreted Get*Val() compute function. This is useful for builtins that can't be implemented with the UDF interface (e.g. functions that need short-circuiting) and that don't have custom codegen functions that use the IRBuilder. It doesn't provide any performance benefit over the interpreted path. TODO: this should be replaced with fancier xcompiling infrastructure
Definition at line 546 of file expr.cc.
References impala::LlvmCodeGen::CastPtrToLlvmPtr(), impala::LlvmCodeGen::context(), impala::CodegenAnyVal::CreateCall(), impala::Expr::CreateIrFunctionPrototype(), impala::LlvmCodeGen::FinalizeFunction(), impala::RuntimeState::GetCodegen(), impala::LlvmCodeGen::GetPtrType(), impala::Expr::GetStaticGetValWrapper(), impala::Expr::ir_compute_fn_, impala::Expr::LLVM_CLASS_NAME, impala::Status::OK, RETURN_IF_ERROR, and impala::Expr::type().
Referenced by impala::TupleIsNullPredicate::GetCodegendComputeFn(), and impala::HiveUdfCall::GetCodegendComputeFn().
|
virtualinherited |
If this expr is constant, evaluates the expr with no input row argument and returns the output. Returns NULL if the argument is not constant. The returned AnyVal* is owned by this expr. This should only be called after Open() has been called on this expr.
Definition at line 491 of file expr.cc.
References impala::Expr::constant_val_, impala::Expr::GetBigIntVal(), impala::Expr::GetBooleanVal(), impala::Expr::GetDecimalVal(), impala::Expr::GetDoubleVal(), impala::Expr::GetFloatVal(), impala::Expr::GetIntVal(), impala::Expr::GetSmallIntVal(), impala::Expr::GetStringVal(), impala::Expr::GetTimestampVal(), impala::Expr::GetTinyIntVal(), impala::Expr::IsConstant(), impala::ExprContext::opened_, impala::ColumnType::type, impala::Expr::type(), impala::Expr::type_, impala::TYPE_BIGINT, impala::TYPE_BOOLEAN, impala::TYPE_CHAR, impala::TYPE_DECIMAL, impala::TYPE_DOUBLE, impala::TYPE_FLOAT, impala::TYPE_INT, impala::TYPE_SMALLINT, impala::TYPE_STRING, impala::TYPE_TIMESTAMP, impala::TYPE_TINYINT, and impala::TYPE_VARCHAR.
Referenced by Open().
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 725 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_DECIMAL.
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 701 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_DOUBLE.
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 693 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_FLOAT.
|
private |
Loads the native or IR function 'symbol' from HDFS and puts the result in *fn. If the function is loaded from an IR module, it cannot be called until the module has been JIT'd (i.e. after Prepare() has completed).
Definition at line 472 of file scalar-fn-call.cc.
References impala::LlvmCodeGen::AddFunctionToJit(), impala::Expr::cache_entry_, impala::Expr::fn_, impala::RuntimeState::GetCodegen(), impala::LibCache::GetSoFunctionPtr(), impala::LibCache::instance(), impala::LlvmCodeGen::module(), impala::Status::OK, and RETURN_IF_ERROR.
Referenced by Prepare().
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 677 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_INT.
|
inlineinherited |
Definition at line 143 of file expr.h.
Referenced by impala::HiveUdfCall::Evaluate(), impala::CaseExpr::GetCodegendComputeFn(), impala::SlotRef::GetCodegendComputeFn(), impala::Literal::GetCodegendComputeFn(), GetCodegendComputeFn(), impala::HiveUdfCall::Open(), Prepare(), and impala::HiveUdfCall::Prepare().
|
virtualinherited |
Returns the slots that are referenced by this expr tree in 'slot_ids'. Returns the number of slots added to the vector
Reimplemented in impala::SlotRef.
Definition at line 418 of file expr.cc.
References impala::Expr::children_.
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 669 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_SMALLINT.
|
protectedinherited |
Returns the IR version of the static Get*Val() wrapper function corresponding to 'type'. This is used for calling interpreted Get*Val() functions from codegen'd functions (e.g. in ScalarFnCall() when codegen is disabled).
Definition at line 426 of file expr.cc.
References impala::ColumnType::DebugString(), impala::LlvmCodeGen::GetFunction(), impala::ColumnType::type, impala::TYPE_BIGINT, impala::TYPE_BOOLEAN, impala::TYPE_CHAR, impala::TYPE_DECIMAL, impala::TYPE_DOUBLE, impala::TYPE_FLOAT, impala::TYPE_INT, impala::TYPE_SMALLINT, impala::TYPE_STRING, impala::TYPE_TIMESTAMP, impala::TYPE_TINYINT, and impala::TYPE_VARCHAR.
Referenced by GetCodegendComputeFn(), and impala::Expr::GetCodegendComputeFnWrapper().
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 709 of file scalar-fn-call.cc.
References impala::ColumnType::IsStringType(), scalar_fn_wrapper_, and impala::Expr::type_.
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 717 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_TIMESTAMP.
|
protectedvirtual |
Reimplemented from impala::Expr.
Definition at line 661 of file scalar-fn-call.cc.
References scalar_fn_wrapper_, impala::ColumnType::type, impala::Expr::type_, and impala::TYPE_TINYINT.
|
private |
Loads the native or IR function from HDFS and puts the result in *udf.
Definition at line 377 of file scalar-fn-call.cc.
References impala::Status::AddDetail(), impala::Expr::cache_entry_, impala::Expr::children_, impala::RuntimeState::codegen_enabled(), impala::LlvmCodeGen::execution_engine(), impala::Expr::fn_, impala::RuntimeState::GetCodegen(), impala::CodegenAnyVal::GetLoweredType(), impala::LlvmCodeGen::GetPtrType(), impala::LibCache::GetSoFunctionPtr(), impala::LlvmCodeGen::GetType(), impala::CodegenAnyVal::GetUnloweredPtrType(), impala::CodegenAnyVal::GetUnloweredType(), impala::LibCache::instance(), impala::LlvmCodeGen::module(), impala::ErrorMsg::msg(), NumFixedArgs(), impala::Status::OK, impala::Status::ok(), RETURN_IF_ERROR, impala::Expr::type(), impala::TYPE_DECIMAL, impala::TYPE_INT, vararg_start_idx_, and impala::LlvmCodeGen::void_type().
Referenced by GetCodegendComputeFn().
|
staticinherited |
The builtin functions are not called from anywhere in the code and the symbols are therefore not included in the binary. We call these functions by using dlsym. The compiler must think this function is callable to not strip these symbols.
Definition at line 470 of file expr.cc.
References impala::Operators::Add_IntVal_IntVal(), impala::CastFunctions::CastToBooleanVal(), impala::DecimalOperators::CastToDecimalVal(), impala::InPredicate::InIterate(), impala::AggregateFunctions::InitNull(), impala::IsNullPredicate::IsNull(), impala::StringFunctions::Length(), impala::LikePredicate::Like(), impala::CompoundPredicate::Not(), impala::ConditionalFunctions::NullIfZero(), impala::UdfBuiltins::Pi(), impala::MathFunctions::Pi(), impala::UtilityFunctions::Pid(), impala::DecimalFunctions::Precision(), and impala::TimestampFunctions::Year().
|
private |
Function to call scalar_fn_. Used in the interpreted path.
Definition at line 512 of file scalar-fn-call.cc.
References impala::Expr::children_, impala::Expr::context_index_, EvaluateChildren(), impala::ExprContext::fn_context(), impala_udf::FunctionContext::impl(), NumFixedArgs(), scalar_fn_, impala::FunctionContextImpl::staging_input_vals(), vararg_start_idx_, and impala::FunctionContextImpl::varargs_buffer().
|
protectedvirtual |
Returns true if GetValue(NULL) can be called on this expr and always returns the same result (e.g., exprs that don't contain slotrefs). The default implementation returns true if all children are constant.
Reimplemented from impala::Expr.
Definition at line 211 of file scalar-fn-call.cc.
References impala::Expr::fn_, and impala::Expr::IsConstant().
Referenced by Open().
|
inlineprivate |
Returns the number of non-vararg arguments.
Definition at line 102 of file scalar-fn-call.h.
Referenced by EvaluateChildren(), GetUdf(), InterpretEval(), Open(), and Prepare().
|
protectedvirtual |
Initializes 'context' for execution. If scope if FRAGMENT_LOCAL, both fragment- and thread-local state should be initialized. Otherwise, if scope is THREAD_LOCAL, only thread-local state should be initialized. Subclasses overriding this function should call Expr::Open() to recursively call Open() on the expr tree.
Reimplemented from impala::Expr.
Definition at line 148 of file scalar-fn-call.cc.
References impala::Expr::children_, impala::Expr::context_index_, impala::CreateAnyVal(), impala_udf::FunctionContext::error_msg(), impala::Expr::fn_, impala::ExprContext::fn_context(), impala::Expr::GetConstVal(), impala_udf::FunctionContext::has_error(), impala_udf::FunctionContext::impl(), IsConstant(), NumFixedArgs(), impala::RuntimeState::obj_pool(), impala::obj_pool(), impala::Status::OK, impala::Expr::Open(), impala::Expr::output_scale_, prepare_fn_, RETURN_IF_ERROR, scalar_fn_, scalar_fn_wrapper_, impala::FunctionContextImpl::SetConstantArgs(), impala::FunctionContextImpl::staging_input_vals(), impala::ColumnType::type, impala::Expr::type(), impala::Expr::type_, impala::TYPE_DOUBLE, and impala_udf::IntVal::val.
|
staticinherited |
Convenience function for opening multiple expr trees.
Referenced by impala::ExprContext::Clone(), impala::HashJoinNode::ConstructBuildSide(), impala::PartitionedHashJoinNode::ConstructBuildSide(), impala::UnionNode::Open(), impala::SortExecExprs::Open(), impala::ExprContext::Open(), impala::CaseExpr::Open(), impala::AggregationNode::Open(), Open(), impala::HiveUdfCall::Open(), impala::DataStreamSender::Open(), impala::ExecNode::Open(), impala::PartitionedAggregationNode::Open(), impala::AggFnEvaluator::Open(), impala::HdfsScanNode::Open(), impala::HdfsTableSink::Open(), impala::Expr::Open(), impala::HdfsPartitionDescriptor::OpenExprs(), impala::HashTableTest::SetUp(), impala::OldHashTableTest::SetUp(), and impala::ImpalaServer::QueryExecState::WaitInternal().
|
inlineinherited |
Get the number of digits after the decimal that should be displayed for this value. Returns -1 if no scale has been specified (currently the scale is only set for doubles set by RoundUpTo). GetValue() must have already been called. TODO: is this still necessary?
|
protectedvirtual |
Initializes this expr instance for execution. This does not include initializing state in the ExprContext; 'context' should only be used to register a FunctionContext via RegisterFunctionContext(). Any IR functions must be generated here. Subclasses overriding this function should call Expr::Prepare() to recursively call Prepare() on the expr tree.
Reimplemented from impala::Expr.
Definition at line 52 of file scalar-fn-call.cc.
References impala::LlvmCodeGen::AddFunctionToJit(), impala::AnyValUtil::AnyValSize(), impala::Expr::cache_entry_, impala::Expr::children_, close_fn_, impala::RuntimeState::codegen_created(), impala::AnyValUtil::ColumnTypeToTypeDesc(), impala::Expr::context_index_, impala::Expr::fn_, impala::RuntimeState::GetCodegen(), GetCodegendComputeFn(), impala::Status::GetDetail(), GetFunction(), impala::Expr::GetNumChildren(), impala::LibCache::GetSoFunctionPtr(), impala::LibCache::instance(), impala::LlvmCodeGen::LinkModule(), NumFixedArgs(), impala::Status::OK, impala::Status::ok(), impala::Expr::Prepare(), prepare_fn_, impala::ExprContext::Register(), RETURN_IF_ERROR, scalar_fn_, scalar_fn_wrapper_, impala::Status::SetErrorMsg(), impala::Expr::type(), impala::Expr::type_, impala::TYPE_CHAR, impala::LibCache::TYPE_IR, and vararg_start_idx_.
|
staticinherited |
Convenience function for preparing multiple expr trees. Allocations from 'ctxs' will be counted against 'tracker'.
Referenced by impala::TupleIsNullPredicate::Prepare(), impala::UnionNode::Prepare(), impala::SortExecExprs::Prepare(), impala::ExprContext::Prepare(), impala::CaseExpr::Prepare(), impala::HashJoinNode::Prepare(), impala::AggregationNode::Prepare(), Prepare(), impala::HiveUdfCall::Prepare(), impala::PartitionedHashJoinNode::Prepare(), impala::DataStreamSender::Prepare(), impala::ExecNode::Prepare(), impala::AggFnEvaluator::Prepare(), impala::PartitionedAggregationNode::Prepare(), impala::HdfsScanNode::Prepare(), impala::Expr::Prepare(), impala::HdfsTableSink::PrepareExprs(), impala::HdfsPartitionDescriptor::PrepareExprs(), impala::HashTableTest::SetUp(), and impala::OldHashTableTest::SetUp().
|
protectedinherited |
Helper function that calls ctx->Register(), sets context_index_, and returns the registered FunctionContext.
Definition at line 80 of file expr.cc.
References impala::AnyValUtil::ColumnTypeToTypeDesc(), impala::ExprContext::fn_context(), and impala::ExprContext::Register().
Referenced by impala::CaseExpr::Prepare(), and impala::HiveUdfCall::Prepare().
|
inlineinherited |
Definition at line 145 of file expr.h.
Referenced by impala::CaseExpr::AnyValEq(), impala::HdfsParquetTableWriter::BoolColumnWriter::BoolColumnWriter(), impala::AggregationNode::CodegenUpdateSlot(), impala::PartitionedAggregationNode::CodegenUpdateSlot(), impala::HdfsParquetTableWriter::ColumnWriter< T >::ColumnWriter(), impala::Expr::CreateIrFunctionPrototype(), impala::Literal::DebugString(), impala::HiveUdfCall::Evaluate(), EvaluateChildren(), impala::CaseExpr::GetChildVal(), impala::CaseExpr::GetCodegendComputeFn(), impala::SlotRef::GetCodegendComputeFn(), impala::Literal::GetCodegendComputeFn(), GetCodegendComputeFn(), impala::Expr::GetCodegendComputeFnWrapper(), impala::Expr::GetConstVal(), impala::Literal::GetDecimalVal(), GetUdf(), impala::Literal::Literal(), Open(), impala::HiveUdfCall::Open(), Prepare(), impala::HiveUdfCall::Prepare(), and impala::DataStreamSender::Send().
|
friend |
Definition at line 53 of file scalar-fn-call.h.
|
protectedinherited |
Cache entry for the library implementing this function.
Definition at line 281 of file expr.h.
Referenced by impala::Expr::Close(), GetFunction(), GetUdf(), Prepare(), and impala::Expr::~Expr().
|
protectedinherited |
Definition at line 290 of file expr.h.
Referenced by impala::Expr::Close(), impala::Expr::DebugString(), EvaluateChildren(), GetCodegendComputeFn(), impala::Expr::GetSlotIds(), GetUdf(), InterpretEval(), impala::Expr::IsConstant(), impala::CaseExpr::Open(), Open(), impala::Expr::Open(), impala::SlotRef::Prepare(), impala::TupleIsNullPredicate::Prepare(), Prepare(), and impala::Expr::Prepare().
|
private |
|
protectedinherited |
If this expr is constant, this will store and cache the value generated by GetConstVal().
Definition at line 303 of file expr.h.
Referenced by impala::Expr::GetConstVal().
|
protectedinherited |
Index to pass to ExprContext::fn_context() to retrieve this expr's FunctionContext. Set in RegisterFunctionContext(). -1 if this expr does not need a FunctionContext and doesn't call RegisterFunctionContext().
Definition at line 296 of file expr.h.
Referenced by impala::CaseExpr::Close(), Close(), impala::HiveUdfCall::Close(), impala::HiveUdfCall::Evaluate(), EvaluateChildren(), GetCodegendComputeFn(), InterpretEval(), impala::CaseExpr::Open(), Open(), impala::HiveUdfCall::Open(), and Prepare().
|
protectedinherited |
Function description.
Definition at line 284 of file expr.h.
Referenced by DebugString(), impala::HiveUdfCall::DebugString(), impala::HiveUdfCall::Evaluate(), impala::Expr::Expr(), GetFunction(), GetUdf(), IsConstant(), Open(), impala::HiveUdfCall::Open(), Prepare(), impala::HiveUdfCall::Prepare(), and ScalarFnCall().
|
protectedinherited |
Cached codegened compute function. Exprs should set this in GetCodegendComputeFn().
Definition at line 299 of file expr.h.
Referenced by impala::CaseExpr::GetCodegendComputeFn(), impala::SlotRef::GetCodegendComputeFn(), impala::Literal::GetCodegendComputeFn(), GetCodegendComputeFn(), and impala::Expr::GetCodegendComputeFnWrapper().
|
protectedinherited |
|
staticinherited |
Definition at line 232 of file expr.h.
Referenced by GetCodegendComputeFn(), and impala::Expr::GetCodegendComputeFnWrapper().
|
protectedinherited |
Definition at line 291 of file expr.h.
Referenced by impala::ExprContext::GetValue(), and Open().
|
private |
|
private |
If running with codegen disabled, scalar_fn_ will be a pointer to the non-JIT'd scalar function.
Definition at line 99 of file scalar-fn-call.h.
Referenced by InterpretEval(), Open(), and Prepare().
|
private |
Function pointer to the JIT'd function produced by GetCodegendComputeFn(). Has signature Val (ExprContext, TupleRow*), and calls the scalar function with signature like Val (FunctionContext, const *Val& arg1, ...)
Definition at line 86 of file scalar-fn-call.h.
Referenced by GetBigIntVal(), GetBooleanVal(), GetDecimalVal(), GetDoubleVal(), GetFloatVal(), GetIntVal(), GetSmallIntVal(), GetStringVal(), GetTimestampVal(), GetTinyIntVal(), Open(), and Prepare().
|
protectedinherited |
analysis is done, types are fixed at this point
Definition at line 289 of file expr.h.
Referenced by impala::Literal::DebugString(), impala::Expr::DebugString(), impala::SlotRef::GetBigIntVal(), impala::Literal::GetBigIntVal(), GetBigIntVal(), impala::HiveUdfCall::GetBigIntVal(), impala::SlotRef::GetBooleanVal(), impala::Literal::GetBooleanVal(), impala::HiveUdfCall::GetBooleanVal(), GetBooleanVal(), impala::SlotRef::GetCodegendComputeFn(), impala::Literal::GetCodegendComputeFn(), impala::Expr::GetConstVal(), impala::SlotRef::GetDecimalVal(), impala::Literal::GetDecimalVal(), GetDecimalVal(), impala::HiveUdfCall::GetDecimalVal(), impala::SlotRef::GetDoubleVal(), impala::Literal::GetDoubleVal(), GetDoubleVal(), impala::HiveUdfCall::GetDoubleVal(), impala::SlotRef::GetFloatVal(), impala::Literal::GetFloatVal(), GetFloatVal(), impala::HiveUdfCall::GetFloatVal(), impala::SlotRef::GetIntVal(), impala::Literal::GetIntVal(), GetIntVal(), impala::HiveUdfCall::GetIntVal(), impala::SlotRef::GetSmallIntVal(), impala::Literal::GetSmallIntVal(), GetSmallIntVal(), impala::HiveUdfCall::GetSmallIntVal(), impala::SlotRef::GetStringVal(), impala::Literal::GetStringVal(), impala::HiveUdfCall::GetStringVal(), GetStringVal(), impala::SlotRef::GetTimestampVal(), impala::HiveUdfCall::GetTimestampVal(), GetTimestampVal(), impala::SlotRef::GetTinyIntVal(), impala::Literal::GetTinyIntVal(), GetTinyIntVal(), impala::HiveUdfCall::GetTinyIntVal(), impala::ExprContext::GetValue(), impala::Literal::Literal(), Open(), impala::HiveUdfCall::Open(), Prepare(), and impala::Expr::Prepare().
|
private |
If this function has var args, children()[vararg_start_idx_] is the first vararg argument. If this function does not have varargs, it is set to -1.
Definition at line 81 of file scalar-fn-call.h.
Referenced by EvaluateChildren(), GetCodegendComputeFn(), GetUdf(), InterpretEval(), and Prepare().