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

This is the superclass of all expr evaluation nodes. More...

#include <expr.h>

Inheritance diagram for impala::Expr:
Collaboration diagram for impala::Expr:

Public Member Functions

virtual ~Expr ()
 
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 *)
 
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 bool IsConstant () const
 
virtual int GetSlotIds (std::vector< SlotId > *slot_ids) const
 
virtual Status GetCodegendComputeFn (RuntimeState *state, llvm::Function **fn)=0
 
virtual AnyValGetConstVal (ExprContext *context)
 
virtual std::string DebugString () const
 

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

 Expr (const ColumnType &type, bool is_slotref=false)
 
 Expr (const TExprNode &node, bool is_slotref=false)
 
virtual Status Prepare (RuntimeState *state, const RowDescriptor &row_desc, ExprContext *context)
 
virtual Status Open (RuntimeState *state, ExprContext *context, FunctionContext::FunctionStateScope scope=FunctionContext::FRAGMENT_LOCAL)
 
virtual void Close (RuntimeState *state, ExprContext *context, FunctionContext::FunctionStateScope scope=FunctionContext::FRAGMENT_LOCAL)
 Subclasses overriding this function should call Expr::Close(). 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)
 
std::string DebugString (const std::string &expr_name) const
 Simple debug string that provides no expr subclass-specific information. More...
 

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_
 

Static Private Member Functions

static Status CreateExpr (ObjectPool *pool, const TExprNode &texpr_node, Expr **expr)
 Create a new Expr based on texpr_node.node_type within 'pool'. More...
 
static Status CreateTreeFromThrift (ObjectPool *pool, const std::vector< TExprNode > &nodes, Expr *parent, int *node_idx, Expr **root_expr, ExprContext **ctx)
 
static BooleanVal GetBooleanVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static TinyIntVal GetTinyIntVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static SmallIntVal GetSmallIntVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static IntVal GetIntVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static BigIntVal GetBigIntVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static FloatVal GetFloatVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static DoubleVal GetDoubleVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static StringVal GetStringVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static TimestampVal GetTimestampVal (Expr *expr, ExprContext *context, TupleRow *row)
 
static DecimalVal GetDecimalVal (Expr *expr, ExprContext *context, TupleRow *row)
 

Friends

class AggFnEvaluator
 
class CastExpr
 
class ComputeFunctions
 
class DecimalFunctions
 
class DecimalLliteral
 
class DecimalOperators
 
class MathFunctions
 
class StringFunctions
 
class TimestampFunctions
 
class ConditionalFunctions
 
class UtilityFunctions
 
class CaseExpr
 
class InPredicate
 
class FunctionCall
 
class ScalarFnCall
 
class ExprContext
 
class ExprTest
 

Detailed Description

This is the superclass of all expr evaluation nodes.

Definition at line 116 of file expr.h.

Constructor & Destructor Documentation

Expr::~Expr ( )
virtual

Definition at line 110 of file expr.cc.

References cache_entry_.

Expr::Expr ( const ColumnType type,
bool  is_slotref = false 
)
protected

Definition at line 91 of file expr.cc.

Expr::Expr ( const TExprNode &  node,
bool  is_slotref = false 
)
protected

Definition at line 100 of file expr.cc.

References fn_.

Member Function Documentation

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

Definition at line 141 of file expr.h.

Referenced by CreateTreeFromThrift().

const std::vector<Expr*>& impala::Expr::children ( ) const
inline
Status Expr::Clone ( const std::vector< ExprContext * > &  ctxs,
RuntimeState state,
std::vector< ExprContext * > *  new_ctxs 
)
static

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 Expr::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 in impala::HiveUdfCall, impala::ScalarFnCall, and impala::CaseExpr.

Definition at line 114 of file expr.cc.

References cache_entry_, children_, impala::LibCache::DecrementUseCount(), and impala::LibCache::instance().

static int impala::Expr::ComputeResultsLayout ( const std::vector< Expr * > &  exprs,
std::vector< int > *  offsets,
int *  var_result_begin 
)
static

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 
)
static
Status Expr::CreateExpr ( ObjectPool pool,
const TExprNode &  texpr_node,
Expr **  expr 
)
staticprivate

Create a new Expr based on texpr_node.node_type within 'pool'.

Definition at line 190 of file expr.cc.

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

Referenced by CreateTreeFromThrift().

Status Expr::CreateExprTree ( ObjectPool pool,
const TExpr &  texpr,
ExprContext **  ctx 
)
static

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 CreateTreeFromThrift(), impala::Status::GetDetail(), impala::Status::OK, and impala::Status::ok().

Referenced by CreateExprTrees(), impala::HdfsPartitionDescriptor::HdfsPartitionDescriptor(), impala::HashJoinNode::Init(), impala::AnalyticEvalNode::Init(), impala::PartitionedHashJoinNode::Init(), and Java_com_cloudera_impala_service_FeSupport_NativeEvalConstExprs().

Status Expr::CreateExprTrees ( ObjectPool pool,
const std::vector< TExpr > &  texprs,
std::vector< ExprContext * > *  ctxs 
)
static

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 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] 
)
protected
static Expr* impala::Expr::CreateLiteral ( ObjectPool pool,
const ColumnType type,
void *  data 
)
static

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 &   
)
static

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.

Status Expr::CreateTreeFromThrift ( ObjectPool pool,
const std::vector< TExprNode > &  nodes,
Expr parent,
int *  node_idx,
Expr **  root_expr,
ExprContext **  ctx 
)
staticprivate

Creates an expr tree for the node rooted at 'node_idx' via depth-first traversal. parameters nodes: vector of thrift expression nodes to be translated parent: parent of node at node_idx (or NULL for node_idx == 0) node_idx: in: root of TExprNode tree out: next node in 'nodes' that isn't part of tree root_expr: out: root of constructed expr tree ctx: out: context of constructed expr tree return status.ok() if successful !status.ok() if tree is inconsistent or corrupt

Definition at line 160 of file expr.cc.

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

Referenced by impala::AggFnEvaluator::Create(), and CreateExprTree().

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

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

Definition at line 332 of file expr.h.

References impala_udf::DebugString().

BigIntVal Expr::GetBigIntVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 54 of file expr-ir.cc.

References GetBigIntVal().

BooleanVal Expr::GetBooleanVal ( ExprContext context,
TupleRow row 
)
virtual

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 in impala::CoalesceExpr, impala::IfExpr, impala::NullIfExpr, impala::HiveUdfCall, impala::ScalarFnCall, impala::OrPredicate, impala::IsNullExpr, impala::Literal, impala::SlotRef, impala::TupleIsNullPredicate, impala::AndPredicate, impala::CaseExpr, and impala::NullLiteral.

Definition at line 573 of file expr.cc.

References DebugString().

Referenced by impala::ExprContext::GetBooleanVal(), GetBooleanVal(), GetConstVal(), and impala::ExprContext::GetValue().

BooleanVal Expr::GetBooleanVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Static wrappers around the virtual Get*Val() functions. Calls the appropriate Get*Val() function on expr, passing it the context and row arguments. These are used to call Get*Val() functions from generated functions, since I don't know how to call virtual functions directly. GetStaticGetValWrapper() returns the IR function of the appropriate wrapper function.

Definition at line 42 of file expr-ir.cc.

References GetBooleanVal().

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

Definition at line 142 of file expr.h.

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

virtual Status impala::Expr::GetCodegendComputeFn ( RuntimeState state,
llvm::Function **  fn 
)
pure virtual

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.

Implemented in impala::CoalesceExpr, impala::IfExpr, impala::NullIfExpr, impala::HiveUdfCall, impala::IsNullExpr, impala::OrPredicate, impala::ScalarFnCall, impala::Literal, impala::AndPredicate, impala::SlotRef, impala::TupleIsNullPredicate, impala::CaseExpr, and impala::NullLiteral.

Referenced by impala::AggregationNode::CodegenUpdateSlot(), and impala::PartitionedAggregationNode::CodegenUpdateSlot().

Status Expr::GetCodegendComputeFnWrapper ( RuntimeState state,
llvm::Function **  fn 
)
protected

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(), CreateIrFunctionPrototype(), impala::LlvmCodeGen::FinalizeFunction(), impala::RuntimeState::GetCodegen(), impala::LlvmCodeGen::GetPtrType(), GetStaticGetValWrapper(), ir_compute_fn_, LLVM_CLASS_NAME, impala::Status::OK, RETURN_IF_ERROR, and type().

Referenced by impala::TupleIsNullPredicate::GetCodegendComputeFn(), and impala::HiveUdfCall::GetCodegendComputeFn().

AnyVal * Expr::GetConstVal ( ExprContext context)
virtual

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 constant_val_, GetBigIntVal(), GetBooleanVal(), GetDecimalVal(), GetDoubleVal(), GetFloatVal(), GetIntVal(), GetSmallIntVal(), GetStringVal(), GetTimestampVal(), GetTinyIntVal(), IsConstant(), impala::ExprContext::opened_, impala::ColumnType::type, type(), 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 impala::ScalarFnCall::Open().

DecimalVal Expr::GetDecimalVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 69 of file expr-ir.cc.

References GetDecimalVal().

DoubleVal Expr::GetDoubleVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 60 of file expr-ir.cc.

References GetDoubleVal().

FloatVal Expr::GetFloatVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 57 of file expr-ir.cc.

References GetFloatVal().

IntVal Expr::GetIntVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 51 of file expr-ir.cc.

References GetIntVal().

int Expr::GetSlotIds ( std::vector< SlotId > *  slot_ids) const
virtual

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 children_.

SmallIntVal Expr::GetSmallIntVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 48 of file expr-ir.cc.

References GetSmallIntVal().

Function * Expr::GetStaticGetValWrapper ( ColumnType  type,
LlvmCodeGen codegen 
)
protected

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 impala::ScalarFnCall::GetCodegendComputeFn(), and GetCodegendComputeFnWrapper().

StringVal Expr::GetStringVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 63 of file expr-ir.cc.

References GetStringVal().

TimestampVal Expr::GetTimestampVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 66 of file expr-ir.cc.

References GetTimestampVal().

TinyIntVal Expr::GetTinyIntVal ( Expr expr,
ExprContext context,
TupleRow row 
)
staticprivate

Definition at line 45 of file expr-ir.cc.

References GetTinyIntVal().

void Expr::InitBuiltinsDummy ( )
static
bool impala::Expr::is_slotref ( ) const
inline

Definition at line 146 of file expr.h.

bool Expr::IsConstant ( ) const
virtual

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 in impala::ScalarFnCall, impala::TupleIsNullPredicate, and impala::SlotRef.

Definition at line 411 of file expr.cc.

References children_.

Referenced by GetConstVal(), and impala::ScalarFnCall::IsConstant().

Status Expr::Open ( RuntimeState state,
ExprContext context,
FunctionContext::FunctionStateScope  scope = FunctionContext::FRAGMENT_LOCAL 
)
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 in impala::HiveUdfCall, impala::ScalarFnCall, and impala::CaseExpr.

Definition at line 366 of file expr.cc.

References children_, impala::Status::OK, Open(), and RETURN_IF_ERROR.

int impala::Expr::output_scale ( ) const
inline

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.

Status Expr::Prepare ( RuntimeState state,
const RowDescriptor row_desc,
ExprContext context 
)
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 in impala::HiveUdfCall, impala::ScalarFnCall, impala::CaseExpr, impala::TupleIsNullPredicate, and impala::SlotRef.

Definition at line 350 of file expr.cc.

References children_, impala::INVALID_TYPE, impala::Status::OK, Prepare(), RETURN_IF_ERROR, impala::ColumnType::type, and type_.

FunctionContext * Expr::RegisterFunctionContext ( ExprContext ctx,
RuntimeState state,
int  varargs_buffer_size = 0 
)
protected

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 AggFnEvaluator
friend

Definition at line 235 of file expr.h.

friend class CaseExpr
friend

Definition at line 246 of file expr.h.

Referenced by CreateExpr().

friend class CastExpr
friend

Definition at line 236 of file expr.h.

friend class ComputeFunctions
friend

Definition at line 237 of file expr.h.

friend class ConditionalFunctions
friend

Definition at line 244 of file expr.h.

friend class DecimalFunctions
friend

Definition at line 238 of file expr.h.

friend class DecimalLliteral
friend

Definition at line 239 of file expr.h.

friend class DecimalOperators
friend

Definition at line 240 of file expr.h.

friend class ExprContext
friend

Definition at line 339 of file expr.h.

Referenced by CreateTreeFromThrift().

friend class ExprTest
friend

Definition at line 340 of file expr.h.

friend class FunctionCall
friend

Definition at line 248 of file expr.h.

friend class InPredicate
friend

Definition at line 247 of file expr.h.

friend class MathFunctions
friend

Definition at line 241 of file expr.h.

friend class ScalarFnCall
friend

Definition at line 249 of file expr.h.

Referenced by CreateExpr().

friend class StringFunctions
friend

Definition at line 242 of file expr.h.

friend class TimestampFunctions
friend

Definition at line 243 of file expr.h.

friend class UtilityFunctions
friend

Definition at line 245 of file expr.h.

Member Data Documentation

LibCache::LibCacheEntry* impala::Expr::cache_entry_
protected

Cache entry for the library implementing this function.

Definition at line 281 of file expr.h.

Referenced by Close(), impala::ScalarFnCall::GetFunction(), impala::ScalarFnCall::GetUdf(), impala::ScalarFnCall::Prepare(), and ~Expr().

boost::scoped_ptr<AnyVal> impala::Expr::constant_val_
protected

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 GetConstVal().

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

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"
static
int impala::Expr::output_scale_
protected

Definition at line 291 of file expr.h.

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

const ColumnType impala::Expr::type_
protected

analysis is done, types are fixed at this point

Definition at line 289 of file expr.h.

Referenced by impala::Literal::DebugString(), DebugString(), impala::SlotRef::GetBigIntVal(), impala::Literal::GetBigIntVal(), impala::ScalarFnCall::GetBigIntVal(), impala::HiveUdfCall::GetBigIntVal(), impala::SlotRef::GetBooleanVal(), impala::Literal::GetBooleanVal(), impala::HiveUdfCall::GetBooleanVal(), impala::ScalarFnCall::GetBooleanVal(), impala::SlotRef::GetCodegendComputeFn(), impala::Literal::GetCodegendComputeFn(), GetConstVal(), impala::SlotRef::GetDecimalVal(), impala::Literal::GetDecimalVal(), impala::ScalarFnCall::GetDecimalVal(), impala::HiveUdfCall::GetDecimalVal(), impala::SlotRef::GetDoubleVal(), impala::Literal::GetDoubleVal(), impala::ScalarFnCall::GetDoubleVal(), impala::HiveUdfCall::GetDoubleVal(), impala::SlotRef::GetFloatVal(), impala::Literal::GetFloatVal(), impala::ScalarFnCall::GetFloatVal(), impala::HiveUdfCall::GetFloatVal(), impala::SlotRef::GetIntVal(), impala::Literal::GetIntVal(), impala::ScalarFnCall::GetIntVal(), impala::HiveUdfCall::GetIntVal(), impala::SlotRef::GetSmallIntVal(), impala::Literal::GetSmallIntVal(), impala::ScalarFnCall::GetSmallIntVal(), impala::HiveUdfCall::GetSmallIntVal(), impala::SlotRef::GetStringVal(), impala::Literal::GetStringVal(), impala::HiveUdfCall::GetStringVal(), impala::ScalarFnCall::GetStringVal(), impala::SlotRef::GetTimestampVal(), impala::ScalarFnCall::GetTimestampVal(), impala::HiveUdfCall::GetTimestampVal(), impala::SlotRef::GetTinyIntVal(), impala::Literal::GetTinyIntVal(), impala::HiveUdfCall::GetTinyIntVal(), impala::ScalarFnCall::GetTinyIntVal(), impala::ExprContext::GetValue(), impala::Literal::Literal(), impala::ScalarFnCall::Open(), impala::HiveUdfCall::Open(), impala::ScalarFnCall::Prepare(), and Prepare().


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