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

#include <scalar-fn-call.h>

Inheritance diagram for impala::ScalarFnCall:
Collaboration diagram for impala::ScalarFnCall:

Public Member Functions

virtual std::string DebugString () const
 
int output_scale () const
 
void AddChild (Expr *expr)
 
ExprGetChild (int i) const
 
int GetNumChildren () const
 
const ColumnTypetype () const
 
bool is_slotref () const
 
const std::vector< Expr * > & children () const
 
virtual int GetSlotIds (std::vector< SlotId > *slot_ids) const
 
virtual AnyValGetConstVal (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 ExprCreateLiteral (ObjectPool *pool, const ColumnType &type, void *data)
 
static ExprCreateLiteral (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...
 
FunctionContextRegisterFunctionContext (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::LibCacheEntrycache_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< AnyValconstant_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
 

Detailed Description

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:

  • Fix error reporting, e.g. reporting leaks
  • Testing
    • Test cancellation
    • Type descs in UDA test harness
    • Allow more functions to be NULL in UDA test harness

Definition at line 48 of file scalar-fn-call.h.

Constructor & Destructor Documentation

ScalarFnCall::ScalarFnCall ( const TExprNode &  node)
protected

Definition at line 41 of file scalar-fn-call.cc.

References impala::Expr::fn_.

Member Function Documentation

void impala::Expr::AddChild ( Expr expr)
inlineinherited

Definition at line 141 of file expr.h.

Referenced by impala::Expr::CreateTreeFromThrift().

const std::vector<Expr*>& impala::Expr::children ( ) const
inlineinherited
Status Expr::Clone ( const std::vector< ExprContext * > &  ctxs,
RuntimeState state,
std::vector< ExprContext * > *  new_ctxs 
)
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().

void ScalarFnCall::Close ( RuntimeState state,
ExprContext context,
FunctionContext::FunctionStateScope  scope = FunctionContext::FRAGMENT_LOCAL 
)
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().

static int impala::Expr::ComputeResultsLayout ( const std::vector< Expr * > &  exprs,
std::vector< int > *  offsets,
int *  var_result_begin 
)
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().

static int impala::Expr::ComputeResultsLayout ( const std::vector< ExprContext * > &  ctxs,
std::vector< int > *  offsets,
int *  var_result_begin 
)
staticinherited
Status Expr::CreateExprTree ( ObjectPool pool,
const TExpr &  texpr,
ExprContext **  ctx 
)
staticinherited
Status Expr::CreateExprTrees ( ObjectPool pool,
const std::vector< TExpr > &  texprs,
std::vector< ExprContext * > *  ctxs 
)
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().

Function * Expr::CreateIrFunctionPrototype ( LlvmCodeGen codegen,
const std::string &  name,
llvm::Value *(*)  args[2] 
)
protectedinherited
static Expr* impala::Expr::CreateLiteral ( ObjectPool pool,
const ColumnType type,
void *  data 
)
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.

static Expr* impala::Expr::CreateLiteral ( ObjectPool pool,
const ColumnType type,
const std::string &   
)
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.

string ScalarFnCall::DebugString ( ) const
virtual

Reimplemented from impala::Expr.

Definition at line 733 of file scalar-fn-call.cc.

References impala::Expr::DebugString(), and impala::Expr::fn_.

static std::string impala::Expr::DebugString ( const std::vector< Expr * > &  exprs)
staticinherited
static std::string impala::Expr::DebugString ( const std::vector< ExprContext * > &  ctxs)
staticinherited
std::string impala::Expr::DebugString ( const std::string &  expr_name) const
inlineprotectedinherited

Simple debug string that provides no expr subclass-specific information.

Definition at line 332 of file expr.h.

References impala_udf::DebugString().

void ScalarFnCall::EvaluateChildren ( ExprContext context,
TupleRow row,
std::vector< impala_udf::AnyVal * > *  input_vals 
)
private
BigIntVal ScalarFnCall::GetBigIntVal ( ExprContext context,
TupleRow row 
)
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.

BooleanVal ScalarFnCall::GetBooleanVal ( ExprContext context,
TupleRow row 
)
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.

Expr* impala::Expr::GetChild ( int  i) const
inlineinherited

Definition at line 142 of file expr.h.

Referenced by impala::HiveUdfCall::Evaluate(), and impala::HiveUdfCall::Prepare().

Status Expr::GetCodegendComputeFnWrapper ( RuntimeState state,
llvm::Function **  fn 
)
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().

DecimalVal ScalarFnCall::GetDecimalVal ( ExprContext context,
TupleRow row 
)
protectedvirtual
DoubleVal ScalarFnCall::GetDoubleVal ( ExprContext context,
TupleRow row 
)
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.

FloatVal ScalarFnCall::GetFloatVal ( ExprContext context,
TupleRow row 
)
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.

Status ScalarFnCall::GetFunction ( RuntimeState state,
const std::string &  symbol,
void **  fn 
)
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().

IntVal ScalarFnCall::GetIntVal ( ExprContext context,
TupleRow row 
)
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.

int Expr::GetSlotIds ( std::vector< SlotId > *  slot_ids) const
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_.

SmallIntVal ScalarFnCall::GetSmallIntVal ( ExprContext context,
TupleRow row 
)
protectedvirtual
Function * Expr::GetStaticGetValWrapper ( ColumnType  type,
LlvmCodeGen codegen 
)
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().

StringVal ScalarFnCall::GetStringVal ( ExprContext context,
TupleRow row 
)
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_.

TimestampVal ScalarFnCall::GetTimestampVal ( ExprContext context,
TupleRow row 
)
protectedvirtual
TinyIntVal ScalarFnCall::GetTinyIntVal ( ExprContext context,
TupleRow row 
)
protectedvirtual
void Expr::InitBuiltinsDummy ( )
staticinherited
template<typename RETURN_TYPE >
RETURN_TYPE ScalarFnCall::InterpretEval ( ExprContext context,
TupleRow row 
)
private
bool impala::Expr::is_slotref ( ) const
inlineinherited

Definition at line 146 of file expr.h.

bool ScalarFnCall::IsConstant ( ) const
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().

int impala::ScalarFnCall::NumFixedArgs ( ) const
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().

Status ScalarFnCall::Open ( RuntimeState state,
ExprContext context,
FunctionContext::FunctionStateScope  scope = FunctionContext::FRAGMENT_LOCAL 
)
protectedvirtual
int impala::Expr::output_scale ( ) const
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?

Definition at line 139 of file expr.h.

FunctionContext * Expr::RegisterFunctionContext ( ExprContext ctx,
RuntimeState state,
int  varargs_buffer_size = 0 
)
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().

Friends And Related Function Documentation

friend class Expr
friend

Definition at line 53 of file scalar-fn-call.h.

Member Data Documentation

LibCache::LibCacheEntry* impala::Expr::cache_entry_
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().

UdfClose impala::ScalarFnCall::close_fn_
private

THe UDF's close function, if specified. This is initialized in Prepare() and called in Close().

Definition at line 95 of file scalar-fn-call.h.

Referenced by Close(), and Prepare().

boost::scoped_ptr<AnyVal> impala::Expr::constant_val_
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().

int impala::Expr::context_index_
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().

llvm::Function* impala::Expr::ir_compute_fn_
protectedinherited
const bool impala::Expr::is_slotref_
protectedinherited

recognize if this node is a slotref in order to speed up GetValue()

Definition at line 287 of file expr.h.

const char * Expr::LLVM_CLASS_NAME = "class.impala::Expr"
staticinherited

Definition at line 232 of file expr.h.

Referenced by GetCodegendComputeFn(), and impala::Expr::GetCodegendComputeFnWrapper().

int impala::Expr::output_scale_
protectedinherited

Definition at line 291 of file expr.h.

Referenced by impala::ExprContext::GetValue(), and Open().

UdfPrepare impala::ScalarFnCall::prepare_fn_
private

The UDF's prepare function, if specified. This is initialized in Prepare() and called in Open() (since we may have needed to codegen the function if it's from an IR module).

Definition at line 91 of file scalar-fn-call.h.

Referenced by Open(), and Prepare().

void* impala::ScalarFnCall::scalar_fn_
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().

void* impala::ScalarFnCall::scalar_fn_wrapper_
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().

const ColumnType impala::Expr::type_
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().

int impala::ScalarFnCall::vararg_start_idx_
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().


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