diff --git a/be/generated-sources/gen-cpp/CMakeLists.txt b/be/generated-sources/gen-cpp/CMakeLists.txt index d75f6e16f..85ae6b762 100644 --- a/be/generated-sources/gen-cpp/CMakeLists.txt +++ b/be/generated-sources/gen-cpp/CMakeLists.txt @@ -62,8 +62,6 @@ set(SRC_FILES NetworkTest_constants.cpp NetworkTest_types.cpp NetworkTestService.cpp - Opcodes_constants.cpp - Opcodes_types.cpp PlanNodes_constants.cpp PlanNodes_types.cpp ResourceBrokerNotificationService.cpp diff --git a/be/generated-sources/opcode/CMakeLists.txt b/be/generated-sources/opcode/CMakeLists.txt index 3b783fa14..163311d00 100644 --- a/be/generated-sources/opcode/CMakeLists.txt +++ b/be/generated-sources/opcode/CMakeLists.txt @@ -21,7 +21,6 @@ set(EXECUTABLE_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/opcode") set(SRC_FILES functions.cc - opcode-registry-init.cc ) # keep everything in one library, the object files reference @@ -31,4 +30,4 @@ add_library(Opcode STATIC ${SRC_FILES}) # Setting these files as code-generated lets make clean and incremental builds work correctly set_source_files_properties(${SRC_FILES} PROPERTIES GENERATED TRUE) -add_dependencies(Opcode function-registry) \ No newline at end of file +add_dependencies(Opcode function-registry) diff --git a/be/src/common/init.cc b/be/src/common/init.cc index 1b409675d..850921f05 100644 --- a/be/src/common/init.cc +++ b/be/src/common/init.cc @@ -18,6 +18,7 @@ #include "common/logging.h" #include "common/status.h" +#include "exprs/expr.h" #include "util/cpu-info.h" #include "util/debug-util.h" #include "util/disk-info.h" @@ -93,6 +94,13 @@ void impala::InitCommonRuntime(int argc, char** argv, bool init_jvm) { InitJvmLoggingSupport(); } + if (argc == -1) { + // Should not be called. We need BuiltinsInit() so the builtin symbols are + // not stripped. + DCHECK(false); + Expr::InitBuiltinsDummy(); + } + #ifndef ADDRESS_SANITIZER // tcmalloc and address sanitizer can not be used together if (FLAGS_enable_process_lifetime_heap_profiling) { diff --git a/be/src/exec/aggregation-node.cc b/be/src/exec/aggregation-node.cc index efb3ecf91..43956ae4f 100644 --- a/be/src/exec/aggregation-node.cc +++ b/be/src/exec/aggregation-node.cc @@ -271,11 +271,11 @@ Tuple* AggregationNode::ConstructAggTuple() { ExprValue default_value; void* default_value_ptr = NULL; switch (evaluator->agg_op()) { - case TAggregationOp::MIN: + case AggFnEvaluator::MIN: default_value_ptr = default_value.SetToMax((*slot_desc)->type()); RawValue::Write(default_value_ptr, agg_tuple, *slot_desc, NULL); break; - case TAggregationOp::MAX: + case AggFnEvaluator::MAX: default_value_ptr = default_value.SetToMin((*slot_desc)->type()); RawValue::Write(default_value_ptr, agg_tuple, *slot_desc, NULL); break; @@ -395,23 +395,23 @@ llvm::Function* AggregationNode::CodegenUpdateSlot( // Update the slot Value* dst_value = builder.CreateLoad(dst_ptr, "dst_val"); switch (evaluator->agg_op()) { - case TAggregationOp::COUNT: + case AggFnEvaluator::COUNT: result = builder.CreateAdd(dst_value, codegen->GetIntConstant(TYPE_BIGINT, 1), "count_inc"); break; - case TAggregationOp::MIN: { + case AggFnEvaluator::MIN: { Function* min_fn = codegen->CodegenMinMax(slot_desc->type(), true); Value* min_args[] = { dst_value, src_value }; result = builder.CreateCall(min_fn, min_args, "min_value"); break; } - case TAggregationOp::MAX: { + case AggFnEvaluator::MAX: { Function* max_fn = codegen->CodegenMinMax(slot_desc->type(), false); Value* max_args[] = { dst_value, src_value }; result = builder.CreateCall(max_fn, max_args, "max_value"); break; } - case TAggregationOp::SUM: + case AggFnEvaluator::SUM: if (slot_desc->type() == TYPE_FLOAT || slot_desc->type() == TYPE_DOUBLE) { result = builder.CreateFAdd(dst_value, src_value); } else { @@ -491,12 +491,6 @@ Function* AggregationNode::CodegenUpdateAggTuple(LlvmCodeGen* codegen) { // Don't codegen things that aren't builtins (for now) if (!evaluator->is_builtin()) return NULL; - - // Don't code gen distinct estimate - if (evaluator->agg_op() == TAggregationOp::DISTINCT_PC - || evaluator->agg_op() == TAggregationOp::DISTINCT_PCSA) { - return NULL; - } } if (agg_tuple_desc_->GenerateLlvmStruct(codegen) == NULL) { diff --git a/be/src/exec/hdfs-lzo-text-scanner.cc b/be/src/exec/hdfs-lzo-text-scanner.cc index 34078ae57..217f5409c 100644 --- a/be/src/exec/hdfs-lzo-text-scanner.cc +++ b/be/src/exec/hdfs-lzo-text-scanner.cc @@ -81,7 +81,7 @@ Status HdfsLzoTextScanner::IssueInitialRanges(RuntimeState* state, Status HdfsLzoTextScanner::LoadLzoLibrary(RuntimeState* state) { void* handle; - RETURN_IF_ERROR(DynamicOpen(LIB_IMPALA_LZO, &handle)); + RETURN_IF_ERROR(DynamicOpen(LIB_IMPALA_LZO.c_str(), &handle)); RETURN_IF_ERROR(DynamicLookup(handle, "GetImpalaBuildVersion", reinterpret_cast(&GetImpalaBuildVersion))); diff --git a/be/src/exprs/CMakeLists.txt b/be/src/exprs/CMakeLists.txt index adf8fba8f..3a4b356f0 100644 --- a/be/src/exprs/CMakeLists.txt +++ b/be/src/exprs/CMakeLists.txt @@ -42,7 +42,6 @@ add_library(Exprs like-predicate.cc math-functions.cc null-literal.cc - opcode-registry.cc slot-ref.cc string-literal.cc string-functions.cc diff --git a/be/src/exprs/agg-fn-evaluator.cc b/be/src/exprs/agg-fn-evaluator.cc index 5bcddff5c..5606abf55 100644 --- a/be/src/exprs/agg-fn-evaluator.cc +++ b/be/src/exprs/agg-fn-evaluator.cc @@ -26,6 +26,7 @@ #include "udf/udf-internal.h" #include "util/debug-util.h" +#include using namespace impala; using namespace impala_udf; using namespace llvm; @@ -56,37 +57,39 @@ typedef void (*UpdateFn8)(FunctionContext*, const AnyVal&, const AnyVal&, typedef StringVal (*SerializeFn)(FunctionContext*, const StringVal&); typedef AnyVal (*FinalizeFn)(FunctionContext*, const AnyVal&); -Status AggFnEvaluator::Create(ObjectPool* pool, const TAggregateFunctionCall& desc, +Status AggFnEvaluator::Create(ObjectPool* pool, const TExpr& desc, AggFnEvaluator** result) { - *result = pool->Add(new AggFnEvaluator(desc)); - RETURN_IF_ERROR( - Expr::CreateExprTrees(pool, desc.input_exprs, &(*result)->input_exprs_)); + DCHECK_GT(desc.nodes.size(), 0); + *result = pool->Add(new AggFnEvaluator(desc.nodes[0])); + int node_idx = 0; + for (int i = 0; i < desc.nodes[0].num_children; ++i) { + ++node_idx; + Expr* expr = NULL; + RETURN_IF_ERROR(Expr::CreateTreeFromThrift( + pool, desc.nodes, NULL, &node_idx, &expr)); + (*result)->input_exprs_.push_back(expr); + } return Status::OK; } -AggFnEvaluator::AggFnEvaluator(const TAggregateFunctionCall& desc) - : return_type_(desc.fn.ret_type), +AggFnEvaluator::AggFnEvaluator(const TExprNode& desc) + : fn_(desc.fn), + return_type_(desc.fn.ret_type), intermediate_type_(desc.fn.aggregate_fn.intermediate_type), - function_type_(desc.fn.binary_type), output_slot_desc_(NULL) { - if (function_type_ == TFunctionBinaryType::BUILTIN) { - agg_op_ = static_cast(desc.fn.id); - DCHECK_NE(agg_op_, TAggregationOp::INVALID); + DCHECK(desc.fn.__isset.aggregate_fn); + DCHECK(desc.node_type == TExprNodeType::AGGREGATE_EXPR); + // TODO: remove. See comment with AggregationOp + if (fn_.name.function_name == "count") { + agg_op_ = COUNT; + } else if (fn_.name.function_name == "min") { + agg_op_ = MIN; + } else if (fn_.name.function_name == "max") { + agg_op_ = MAX; + } else if (fn_.name.function_name == "sum") { + agg_op_ = SUM; } else { - DCHECK_EQ(function_type_, TFunctionBinaryType::NATIVE); - DCHECK(desc.fn.__isset.aggregate_fn); - - hdfs_location_ = desc.fn.hdfs_location; - init_fn_symbol_ = desc.fn.aggregate_fn.init_fn_symbol; - update_fn_symbol_ = desc.fn.aggregate_fn.update_fn_symbol; - merge_fn_symbol_ = desc.fn.aggregate_fn.merge_fn_symbol; - serialize_fn_symbol_ = desc.fn.aggregate_fn.serialize_fn_symbol; - finalize_fn_symbol_ = desc.fn.aggregate_fn.finalize_fn_symbol; - - DCHECK(!hdfs_location_.empty()); - DCHECK(!init_fn_symbol_.empty()); - DCHECK(!update_fn_symbol_.empty()); - DCHECK(!merge_fn_symbol_.empty()); + agg_op_ = OTHER; } } @@ -106,40 +109,41 @@ Status AggFnEvaluator::Prepare(RuntimeState* state, const RowDescriptor& desc, } staging_output_val_ = CreateAnyVal(obj_pool, output_slot_desc_->type()); - // TODO: this should be made identical for the builtin and UDA case by - // putting all this logic in an improved opcode registry. - if (function_type_ == TFunctionBinaryType::BUILTIN) { - pair key; - if (is_count_star()) { - key = make_pair(agg_op_, INVALID_TYPE); - } else { - DCHECK_GE(input_exprs().size(), 1); - key = make_pair(agg_op_, input_exprs()[0]->type()); - } - const OpcodeRegistry::AggFnDescriptor* fn_desc = - OpcodeRegistry::Instance()->GetBuiltinAggFnDescriptor(key); - DCHECK(fn_desc != NULL); - fn_ptrs_ = *fn_desc; - } else { - DCHECK_EQ(function_type_, TFunctionBinaryType::NATIVE); - // Load the function pointers. - RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( - state->fs_cache(), hdfs_location_, init_fn_symbol_, &fn_ptrs_.init_fn)); - RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( - state->fs_cache(), hdfs_location_, update_fn_symbol_, &fn_ptrs_.update_fn)); - RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( - state->fs_cache(), hdfs_location_, merge_fn_symbol_, &fn_ptrs_.merge_fn)); + // Load the function pointers. + if (fn_.aggregate_fn.init_fn_symbol.empty() || + fn_.aggregate_fn.update_fn_symbol.empty() || + fn_.aggregate_fn.merge_fn_symbol.empty()) { + // This path is only for partially implemented builtins. + DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::BUILTIN); + stringstream ss; + ss << "Function " << fn_.name.function_name << " is not implemented."; + return Status(ss.str()); + } - // Serialize and Finalize are optional - if (!serialize_fn_symbol_.empty()) { - RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( - state->fs_cache(), hdfs_location_, serialize_fn_symbol_, - &fn_ptrs_.serialize_fn)); - } - if (!finalize_fn_symbol_.empty()) { - RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( - state->fs_cache(), hdfs_location_, finalize_fn_symbol_, &fn_ptrs_.finalize_fn)); - } + RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( + state->fs_cache(), fn_.hdfs_location, fn_.aggregate_fn.init_fn_symbol, + &init_fn_)); + RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( + state->fs_cache(), fn_.hdfs_location, fn_.aggregate_fn.update_fn_symbol, + &update_fn_)); + RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( + state->fs_cache(), fn_.hdfs_location, fn_.aggregate_fn.merge_fn_symbol, + &merge_fn_)); + + // Serialize and Finalize are optional + if (!fn_.aggregate_fn.serialize_fn_symbol.empty()) { + RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( + state->fs_cache(), fn_.hdfs_location, fn_.aggregate_fn.serialize_fn_symbol, + &serialize_fn_)); + } else { + serialize_fn_ = NULL; + } + if (!fn_.aggregate_fn.finalize_fn_symbol.empty()) { + RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( + state->fs_cache(), fn_.hdfs_location, fn_.aggregate_fn.finalize_fn_symbol, + &finalize_fn_)); + } else { + finalize_fn_ = NULL; } return Status::OK; } @@ -236,8 +240,8 @@ inline void AggFnEvaluator::SetOutputSlot(const AnyVal* src, Tuple* dst) { // This function would be replaced in codegen. void AggFnEvaluator::Init(Tuple* dst) { - DCHECK(fn_ptrs_.init_fn != NULL); - reinterpret_cast(fn_ptrs_.init_fn)(ctx_.get(), staging_output_val_); + DCHECK(init_fn_ != NULL); + reinterpret_cast(init_fn_)(ctx_.get(), staging_output_val_); SetOutputSlot(staging_output_val_, dst); } @@ -313,11 +317,11 @@ void AggFnEvaluator::UpdateOrMerge(TupleRow* row, Tuple* dst, void* fn) { } void AggFnEvaluator::Update(TupleRow* row, Tuple* dst) { - return UpdateOrMerge(row, dst, fn_ptrs_.update_fn); + return UpdateOrMerge(row, dst, update_fn_); } void AggFnEvaluator::Merge(TupleRow* row, Tuple* dst) { - return UpdateOrMerge(row, dst, fn_ptrs_.merge_fn); + return UpdateOrMerge(row, dst, merge_fn_); } void AggFnEvaluator::SerializeOrFinalize(Tuple* tuple, void* fn) { @@ -384,11 +388,11 @@ void AggFnEvaluator::SerializeOrFinalize(Tuple* tuple, void* fn) { } void AggFnEvaluator::Serialize(Tuple* tuple) { - SerializeOrFinalize(tuple, fn_ptrs_.serialize_fn); + SerializeOrFinalize(tuple, serialize_fn_); } void AggFnEvaluator::Finalize(Tuple* tuple) { - SerializeOrFinalize(tuple, fn_ptrs_.finalize_fn); + SerializeOrFinalize(tuple, finalize_fn_); } string AggFnEvaluator::DebugString(const vector& exprs) { diff --git a/be/src/exprs/agg-fn-evaluator.h b/be/src/exprs/agg-fn-evaluator.h index 752e79bc8..f257e1adf 100644 --- a/be/src/exprs/agg-fn-evaluator.h +++ b/be/src/exprs/agg-fn-evaluator.h @@ -18,17 +18,29 @@ #include -#include "exprs/opcode-registry.h" +#include +#include "common/status.h" +#include "runtime/primitive-type.h" #include "udf/udf.h" #include "gen-cpp/Exprs_types.h" +#include "gen-cpp/PlanNodes_types.h" +#include "gen-cpp/Types_types.h" namespace impala { class AggregationNode; +class Expr; +class MemPool; +class ObjectPool; +class RowDescriptor; +class RuntimeState; +class SlotDescriptor; +class Tuple; +class TupleRow; class TExprNode; -// This class evaluates aggregate functions. Aggregate funtions can either be +// This class evaluates aggregate functions. Aggregate functions can either be // builtins or external UDAs. For both of types types, they can either use codegen // or not. // This class provides an interface that's 1:1 with the UDA interface and serves @@ -37,29 +49,39 @@ class TExprNode; // slots from TupleRows and aggregating the result to the result tuple. class AggFnEvaluator { public: + // TODO: The aggregation node has custom codegen paths for a few of the builtins. + // That logic needs to be removed. For now, add some enums for those builtins. + enum AggregationOp { + COUNT, + MIN, + MAX, + SUM, + OTHER, + }; + // Creates an AggFnEvaluator object from desc. The object is added to 'pool' // and returned in *result. This constructs the input Expr trees for // this aggregate function as specified in desc. The result is returned in // *result. - static Status Create(ObjectPool* pool, const TAggregateFunctionCall& desc, + static Status Create(ObjectPool* pool, const TExpr& desc, AggFnEvaluator** result); // Initializes the agg expr. 'desc' must be the row descriptor for the input TupleRow. // It is used to get the input values in the Update() and Merge() functions. - // 'output_slot_desc' is the slot that this aggregator should write to. + // 'output_slot_desc' is the slot that this evaluator should write to. // The underlying aggregate function allocates memory from the 'pool'. This is - // either string data for intemerdiate results or whatever memory the UDA might + // either string data for intermediate results or whatever memory the UDA might // need. // TODO: should we give them their own pool? Status Prepare(RuntimeState* state, const RowDescriptor& desc, MemPool* pool, const SlotDescriptor* output_slot_desc); - TAggregationOp::type agg_op() const { return agg_op_; } + AggregationOp agg_op() const { return agg_op_; } const std::vector& input_exprs() const { return input_exprs_; } bool is_count_star() const { - return agg_op_ == TAggregationOp::COUNT && input_exprs_.empty(); + return agg_op_ == COUNT && input_exprs_.empty(); } - bool is_builtin() const { return function_type_ == TFunctionBinaryType::BUILTIN; } + bool is_builtin() const { return fn_.binary_type == TFunctionBinaryType::BUILTIN; } static std::string DebugString(const std::vector& exprs); std::string DebugString() const; @@ -80,23 +102,14 @@ class AggFnEvaluator { // Function* GetIrFinalizeFn(); private: + TFunction fn_; + const ColumnType return_type_; const ColumnType intermediate_type_; std::vector input_exprs_; - // Native (.so), IR (.ll) or builtin - TFunctionBinaryType::type function_type_; - - // If it's a builtin, the opcode. - TAggregationOp::type agg_op_; - - // HDFS path and function names for UDAs. - std::string hdfs_location_; - std::string init_fn_symbol_; - std::string update_fn_symbol_; - std::string merge_fn_symbol_; - std::string serialize_fn_symbol_; - std::string finalize_fn_symbol_; + // The enum for some of the builtins that still require special cased logic. + AggregationOp agg_op_; // Unowned const SlotDescriptor* output_slot_desc_; @@ -113,14 +126,17 @@ class AggFnEvaluator { std::vector staging_input_vals_; impala_udf::AnyVal* staging_output_val_; - // Function ptrs to the aggregate function. This is either populated from the - // opcode registry for builtins or from the external binary for native UDAs. - OpcodeRegistry::AggFnDescriptor fn_ptrs_; + // Function ptrs for the different phases of the aggregate function. + void* init_fn_; + void* update_fn_; + void* merge_fn_; + void* serialize_fn_; + void* finalize_fn_; // Use Create() instead. - AggFnEvaluator(const TAggregateFunctionCall& desc); + AggFnEvaluator(const TExprNode& desc); - // TODO: these funtions below are not extensible and we need to use codegen to + // TODO: these functions below are not extensible and we need to use codegen to // generate the calls into the UDA functions (like for UDFs). // Remove these functions when this is supported. diff --git a/be/src/exprs/aggregate-functions.cc b/be/src/exprs/aggregate-functions.cc index 8ec5a5ea5..6a69f5847 100644 --- a/be/src/exprs/aggregate-functions.cc +++ b/be/src/exprs/aggregate-functions.cc @@ -132,6 +132,11 @@ void AggregateFunctions::Max(FunctionContext*, if (src_tv > dst_tv) *dst = src; } +void AggregateFunctions::StringConcat(FunctionContext* ctx, const StringVal& src, + StringVal* result) { + StringConcat(ctx, src, DEFAULT_STRING_CONCAT_DELIM, result); +} + void AggregateFunctions::StringConcat(FunctionContext* ctx, const StringVal& src, const StringVal& separator, StringVal* result) { if (src.is_null) return; @@ -423,8 +428,6 @@ StringVal AggregateFunctions::HllFinalize(FunctionContext* ctx, const StringVal& // Stamp out the templates for the types we need. template void AggregateFunctions::InitZero(FunctionContext*, BigIntVal* dst); -template void AggregateFunctions::Sum( - FunctionContext*, const BooleanVal& src, BigIntVal* dst); template void AggregateFunctions::Sum( FunctionContext*, const TinyIntVal& src, BigIntVal* dst); template void AggregateFunctions::Sum( diff --git a/be/src/exprs/aggregate-functions.h b/be/src/exprs/aggregate-functions.h index 91965de45..8b9e814d4 100644 --- a/be/src/exprs/aggregate-functions.h +++ b/be/src/exprs/aggregate-functions.h @@ -16,7 +16,6 @@ #ifndef IMPALA_EXPRS_AGGREGATE_FUNCTIONS_H #define IMPALA_EXPRS_AGGREGATE_FUNCTIONS_H -#include "exprs/opcode-registry.h" #include "udf/udf.h" using namespace impala_udf; @@ -56,6 +55,7 @@ class AggregateFunctions { static void Max(FunctionContext*, const T& src, T* dst); // String concat + static void StringConcat(FunctionContext*, const StringVal& src, StringVal* result); static void StringConcat(FunctionContext*, const StringVal& src, const StringVal& separator, StringVal* result); diff --git a/be/src/exprs/arithmetic-expr.cc b/be/src/exprs/arithmetic-expr.cc index d697abd0f..ec4b67255 100644 --- a/be/src/exprs/arithmetic-expr.cc +++ b/be/src/exprs/arithmetic-expr.cc @@ -46,16 +46,16 @@ string ArithmeticExpr::DebugString() const { // %child_result = call i64 @IntLiteral(i8** %row, i8* %state_data, i1* %is_null) // %child_null = load i1* %is_null // br i1 %child_null, label %ret, label %compute_rhs -// +// // compute_rhs: ; preds = %entry // %child_result1 = call i64 @IntLiteral1(i8** %row, i8* %state_data, i1* %is_null) // %child_null2 = load i1* %is_null // br i1 %child_null2, label %ret, label %arith -// +// // arith: ; preds = %compute_rhs // %tmp_add = add i64 %child_result, %child_result1 // br label %ret -// +// // ret: ; preds = %arith, %compute_rhs, %entry // %tmp_phi = phi i64 [ 0, %entry ], [ 0, %compute_rhs ], [ %tmp_add, %arith ] // ret i64 %tmp_phi @@ -70,7 +70,7 @@ Function* ArithmeticExpr::Codegen(LlvmCodeGen* codegen) { rhs_function = children()[1]->Codegen(codegen); if (rhs_function == NULL) return NULL; } - + LLVMContext& context = codegen->context(); LlvmCodeGen::LlvmBuilder builder(context); Type* return_type = codegen->GetType(type()); @@ -78,7 +78,7 @@ Function* ArithmeticExpr::Codegen(LlvmCodeGen* codegen) { BasicBlock* entry_block = BasicBlock::Create(context, "entry", function); builder.SetInsertPoint(entry_block); - + BasicBlock* compute_rhs_block = NULL; BasicBlock* compute_arith_block = NULL; Value* lhs_value = NULL; @@ -97,7 +97,7 @@ Function* ArithmeticExpr::Codegen(LlvmCodeGen* codegen) { // Lhs not null, eval rhs if (GetNumChildren() == 2) { builder.SetInsertPoint(compute_rhs_block); - rhs_value = children()[1]->CodegenGetValue(codegen, compute_rhs_block, + rhs_value = children()[1]->CodegenGetValue(codegen, compute_rhs_block, ret_block, compute_arith_block); } @@ -105,90 +105,71 @@ Function* ArithmeticExpr::Codegen(LlvmCodeGen* codegen) { builder.SetInsertPoint(compute_arith_block); Value* result = NULL; - switch (op()) { - case TExprOpcode::BITNOT_CHAR: - case TExprOpcode::BITNOT_SHORT: - case TExprOpcode::BITNOT_INT: - case TExprOpcode::BITNOT_LONG: - result = builder.CreateNot(lhs_value, "tmp_not"); - break; - case TExprOpcode::ADD_CHAR_CHAR: - case TExprOpcode::ADD_SHORT_SHORT: - case TExprOpcode::ADD_INT_INT: - case TExprOpcode::ADD_LONG_LONG: - result = builder.CreateAdd(lhs_value, rhs_value, "tmp_add"); - break; - case TExprOpcode::ADD_FLOAT_FLOAT: - case TExprOpcode::ADD_DOUBLE_DOUBLE: - result = builder.CreateFAdd(lhs_value, rhs_value, "tmp_add"); - break; - case TExprOpcode::SUBTRACT_CHAR_CHAR: - case TExprOpcode::SUBTRACT_SHORT_SHORT: - case TExprOpcode::SUBTRACT_INT_INT: - case TExprOpcode::SUBTRACT_LONG_LONG: - result = builder.CreateSub(lhs_value, rhs_value, "tmp_sub"); - break; - case TExprOpcode::SUBTRACT_FLOAT_FLOAT: - case TExprOpcode::SUBTRACT_DOUBLE_DOUBLE: - result = builder.CreateFSub(lhs_value, rhs_value, "tmp_sub"); - break; - case TExprOpcode::MULTIPLY_CHAR_CHAR: - case TExprOpcode::MULTIPLY_SHORT_SHORT: - case TExprOpcode::MULTIPLY_INT_INT: - case TExprOpcode::MULTIPLY_LONG_LONG: - result = builder.CreateMul(lhs_value, rhs_value, "tmp_mul"); - break; - case TExprOpcode::MULTIPLY_FLOAT_FLOAT: - case TExprOpcode::MULTIPLY_DOUBLE_DOUBLE: - result = builder.CreateFMul(lhs_value, rhs_value, "tmp_mul"); - break; - case TExprOpcode::DIVIDE: - result = builder.CreateFDiv(lhs_value, rhs_value, "tmp_div"); - break; - - case TExprOpcode::INT_DIVIDE_CHAR_CHAR: - case TExprOpcode::INT_DIVIDE_SHORT_SHORT: - case TExprOpcode::INT_DIVIDE_INT_INT: - case TExprOpcode::INT_DIVIDE_LONG_LONG: + // TODO: this is a temporary hack until the expr refactoring goes in and + // all this code is removed. + if (fn_.name.function_name == "bitnot") { + result = builder.CreateNot(lhs_value, "tmp_not"); + } else if (fn_.name.function_name == "bitand") { + result = builder.CreateAnd(lhs_value, rhs_value, "tmp_and"); + } else if (fn_.name.function_name == "bitor") { + result = builder.CreateOr(lhs_value, rhs_value, "tmp_or"); + } else if (fn_.name.function_name == "bitxor") { + result = builder.CreateXor(lhs_value, rhs_value, "tmp_xor"); + } else if (fn_.name.function_name == "add") { + switch(type()) { + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateAdd(lhs_value, rhs_value, "tmp_add"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFAdd(lhs_value, rhs_value, "tmp_add"); + break; + default: + DCHECK(false) << "Shouldn't get here."; + } + } else if (fn_.name.function_name == "subtract") { + switch(type()) { + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateSub(lhs_value, rhs_value, "tmp_sub"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFSub(lhs_value, rhs_value, "tmp_sub"); + break; + default: + DCHECK(false) << "Shouldn't get here."; + } + } else if (fn_.name.function_name == "multiply") { + switch(type()) { + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateMul(lhs_value, rhs_value, "tmp_mul"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFMul(lhs_value, rhs_value, "tmp_div"); + break; + default: + DCHECK(false) << "Shouldn't get here."; + } + } else if (fn_.name.function_name == "divide") { + result = builder.CreateFDiv(lhs_value, rhs_value, "tmp_div"); + } else if (fn_.name.function_name == "int_divide") { result = builder.CreateSDiv(lhs_value, rhs_value, "tmp_div"); - break; - - case TExprOpcode::MOD_CHAR_CHAR: - case TExprOpcode::MOD_SHORT_SHORT: - case TExprOpcode::MOD_INT_INT: - case TExprOpcode::MOD_LONG_LONG: - result = builder.CreateSRem(lhs_value, rhs_value, "tmp_mod"); - break; - - case TExprOpcode::MATH_FMOD_FLOAT_FLOAT: - case TExprOpcode::MATH_FMOD_DOUBLE_DOUBLE: - result = builder.CreateFRem(lhs_value, rhs_value, "tmp_fmod"); - break; - - case TExprOpcode::BITAND_CHAR_CHAR: - case TExprOpcode::BITAND_SHORT_SHORT: - case TExprOpcode::BITAND_INT_INT: - case TExprOpcode::BITAND_LONG_LONG: - result = builder.CreateAnd(lhs_value, rhs_value, "tmp_and"); - break; - - case TExprOpcode::BITOR_CHAR_CHAR: - case TExprOpcode::BITOR_SHORT_SHORT: - case TExprOpcode::BITOR_INT_INT: - case TExprOpcode::BITOR_LONG_LONG: - result = builder.CreateOr(lhs_value, rhs_value, "tmp_or"); - break; - - case TExprOpcode::BITXOR_CHAR_CHAR: - case TExprOpcode::BITXOR_SHORT_SHORT: - case TExprOpcode::BITXOR_INT_INT: - case TExprOpcode::BITXOR_LONG_LONG: - result = builder.CreateXor(lhs_value, rhs_value, "tmp_xor"); - break; - - default: - DCHECK(false) << "Unknown op: " << op(); - return NULL; + } else if (fn_.name.function_name == "mod") { + result = builder.CreateSRem(lhs_value, rhs_value, "tmp_mul"); + } else if (fn_.name.function_name == "fmod") { + result = builder.CreateFRem(lhs_value, rhs_value, "tmp_fmod"); + } else { + DCHECK(false) << "Unknown arithmetic function: " << fn_.name.function_name; } builder.CreateBr(ret_block); diff --git a/be/src/exprs/binary-predicate.cc b/be/src/exprs/binary-predicate.cc index 034756016..5ca8c962b 100644 --- a/be/src/exprs/binary-predicate.cc +++ b/be/src/exprs/binary-predicate.cc @@ -46,19 +46,19 @@ string BinaryPredicate::DebugString() const { // %child_result = call i8 @IntLiteral(i8** %row, i8* %state_data, i1* %is_null) // %child_null = load i1* %is_null // br i1 %child_null, label %ret_block, label %lhs_not_null -// +// // lhs_not_null: ; preds = %entry // %child_result1 = call i8 @IntLiteral1(i8** %row, i8* %state_data, i1* %is_null) // %child_null2 = load i1* %is_null // br i1 %child_null2, label %ret_block, label %rhs_not_null -// +// // rhs_not_null: ; preds = %lhs_not_null // %tmp_lt = icmp slt i8 %child_result, %child_result1 // br label %ret_block -// -// ret_block: -// %tmp_phi = phi i1 [ false, %entry ], -// [ false, %lhs_not_null ], +// +// ret_block: +// %tmp_phi = phi i1 [ false, %entry ], +// [ false, %lhs_not_null ], // [ %tmp_lt, %rhs_not_null ] // ret i1 %tmp_phi // } @@ -67,7 +67,7 @@ Function* BinaryPredicate::Codegen(LlvmCodeGen* codegen) { Function* lhs_function = children()[0]->Codegen(codegen); Function* rhs_function = children()[1]->Codegen(codegen); if (lhs_function == NULL || rhs_function == NULL) return NULL; - + LLVMContext& context = codegen->context(); LlvmCodeGen::LlvmBuilder builder(context); Type* return_type = codegen->GetType(type()); @@ -80,121 +80,143 @@ Function* BinaryPredicate::Codegen(LlvmCodeGen* codegen) { builder.SetInsertPoint(entry_block); // Call lhs function - Value* lhs_value = children()[0]->CodegenGetValue(codegen, entry_block, + Value* lhs_value = children()[0]->CodegenGetValue(codegen, entry_block, ret_block, lhs_not_null_block); // Lhs not null, eval rhs builder.SetInsertPoint(lhs_not_null_block); - Value* rhs_value = children()[1]->CodegenGetValue(codegen, lhs_not_null_block, + Value* rhs_value = children()[1]->CodegenGetValue(codegen, lhs_not_null_block, ret_block, rhs_not_null_block); // rhs not null, do arithmetic op builder.SetInsertPoint(rhs_not_null_block); + PrimitiveType t = children_[0]->type(); + DCHECK_EQ(t, children_[1]->type()); + Value* result = NULL; - switch (op()) { - case TExprOpcode::EQ_BOOL_BOOL: - case TExprOpcode::EQ_CHAR_CHAR: - case TExprOpcode::EQ_SHORT_SHORT: - case TExprOpcode::EQ_INT_INT: - case TExprOpcode::EQ_LONG_LONG: - case TExprOpcode::EQ_FLOAT_FLOAT: - case TExprOpcode::EQ_DOUBLE_DOUBLE: - case TExprOpcode::EQ_STRINGVALUE_STRINGVALUE: - result = codegen->CodegenEquals(&builder, lhs_value, rhs_value, - children()[0]->type()); - break; - - case TExprOpcode::NE_BOOL_BOOL: - case TExprOpcode::NE_CHAR_CHAR: - case TExprOpcode::NE_SHORT_SHORT: - case TExprOpcode::NE_INT_INT: - case TExprOpcode::NE_LONG_LONG: - result = builder.CreateICmpNE(lhs_value, rhs_value, "tmp_neq"); - break; - case TExprOpcode::NE_FLOAT_FLOAT: - case TExprOpcode::NE_DOUBLE_DOUBLE: - result = builder.CreateFCmpUNE(lhs_value, rhs_value, "tmp_neq"); - break; - - case TExprOpcode::LE_BOOL_BOOL: // LLVM defines false > true - case TExprOpcode::GE_CHAR_CHAR: - case TExprOpcode::GE_SHORT_SHORT: - case TExprOpcode::GE_INT_INT: - case TExprOpcode::GE_LONG_LONG: - result = builder.CreateICmpSGE(lhs_value, rhs_value, "tmp_ge"); - break; - case TExprOpcode::GE_FLOAT_FLOAT: - case TExprOpcode::GE_DOUBLE_DOUBLE: - result = builder.CreateFCmpUGE(lhs_value, rhs_value, "tmp_ge"); - break; - - case TExprOpcode::LT_BOOL_BOOL: // LLVM defines false > true - case TExprOpcode::GT_CHAR_CHAR: - case TExprOpcode::GT_SHORT_SHORT: - case TExprOpcode::GT_INT_INT: - case TExprOpcode::GT_LONG_LONG: - result = builder.CreateICmpSGT(lhs_value, rhs_value, "tmp_gt"); - break; - case TExprOpcode::GT_FLOAT_FLOAT: - case TExprOpcode::GT_DOUBLE_DOUBLE: - result = builder.CreateFCmpUGT(lhs_value, rhs_value, "tmp_gt"); - break; - - case TExprOpcode::GE_BOOL_BOOL: // LLVM defines false > true - case TExprOpcode::LE_CHAR_CHAR: - case TExprOpcode::LE_SHORT_SHORT: - case TExprOpcode::LE_INT_INT: - case TExprOpcode::LE_LONG_LONG: - result = builder.CreateICmpSLE(lhs_value, rhs_value, "tmp_le"); - break; - case TExprOpcode::LE_FLOAT_FLOAT: - case TExprOpcode::LE_DOUBLE_DOUBLE: - result = builder.CreateFCmpULE(lhs_value, rhs_value, "tmp_le"); - break; - - case TExprOpcode::GT_BOOL_BOOL: // LLVM defines false > true - case TExprOpcode::LT_CHAR_CHAR: - case TExprOpcode::LT_SHORT_SHORT: - case TExprOpcode::LT_INT_INT: - case TExprOpcode::LT_LONG_LONG: - result = builder.CreateICmpSLT(lhs_value, rhs_value, "tmp_lt"); - break; - case TExprOpcode::LT_FLOAT_FLOAT: - case TExprOpcode::LT_DOUBLE_DOUBLE: - result = builder.CreateFCmpULT(lhs_value, rhs_value, "tmp_lt"); - break; - - // String comparison functions - case TExprOpcode::NE_STRINGVALUE_STRINGVALUE: { - Function* str_fn = codegen->GetFunction(IRFunction::STRING_VALUE_NE); - result = builder.CreateCall2(str_fn, lhs_value, rhs_value, "tmp_ne"); - break; + if (fn_.name.function_name == "eq") { + result = codegen->CodegenEquals(&builder, lhs_value, rhs_value, + children()[0]->type()); + } else if (fn_.name.function_name == "ne") { + switch (t) { + case TYPE_BOOLEAN: + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateICmpNE(lhs_value, rhs_value, "tmp_ne"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFCmpUNE(lhs_value, rhs_value, "tmp_ne"); + break; + case TYPE_STRING: { + Function* call_fn = codegen->GetFunction(IRFunction::STRING_VALUE_NE); + result = builder.CreateCall2(call_fn, lhs_value, rhs_value, "tmp_ne"); + break; + } + default: + DCHECK(false) << "Shouldn't get here."; } - case TExprOpcode::GE_STRINGVALUE_STRINGVALUE: { - Function* str_fn = codegen->GetFunction(IRFunction::STRING_VALUE_GE); - result = builder.CreateCall2(str_fn, lhs_value, rhs_value, "tmp_ge"); - break; + } else if (fn_.name.function_name == "ge") { + switch (t) { + case TYPE_BOOLEAN: + // LLVM defines false > true + result = builder.CreateICmpSLE(lhs_value, rhs_value, "tmp_ge"); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateICmpSGE(lhs_value, rhs_value, "tmp_ge"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFCmpUGE(lhs_value, rhs_value, "tmp_ge"); + break; + case TYPE_STRING: { + Function* call_fn = codegen->GetFunction(IRFunction::STRING_VALUE_GE); + result = builder.CreateCall2(call_fn, lhs_value, rhs_value, "tmp_ge"); + break; + } + default: + DCHECK(false) << "Shouldn't get here."; } - case TExprOpcode::GT_STRINGVALUE_STRINGVALUE: { - Function* str_fn = codegen->GetFunction(IRFunction::STRING_VALUE_GT); - result = builder.CreateCall2(str_fn, lhs_value, rhs_value, "tmp_gt"); - break; + } else if (fn_.name.function_name == "gt") { + switch (t) { + case TYPE_BOOLEAN: + // LLVM defines false > true + result = builder.CreateICmpSLT(lhs_value, rhs_value, "tmp_gt"); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateICmpSGT(lhs_value, rhs_value, "tmp_gt"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFCmpUGT(lhs_value, rhs_value, "tmp_gt"); + break; + case TYPE_STRING: { + Function* call_fn = codegen->GetFunction(IRFunction::STRING_VALUE_GT); + result = builder.CreateCall2(call_fn, lhs_value, rhs_value, "tmp_gt"); + break; + } + default: + DCHECK(false) << "Shouldn't get here."; } - case TExprOpcode::LE_STRINGVALUE_STRINGVALUE: { - Function* str_fn = codegen->GetFunction(IRFunction::STRING_VALUE_LE); - result = builder.CreateCall2(str_fn, lhs_value, rhs_value, "tmp_le"); - break; + } else if (fn_.name.function_name == "le") { + switch (t) { + case TYPE_BOOLEAN: + // LLVM defines false > true + result = builder.CreateICmpSGE(lhs_value, rhs_value, "tmp_le"); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateICmpSLE(lhs_value, rhs_value, "tmp_le"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFCmpULE(lhs_value, rhs_value, "tmp_le"); + break; + case TYPE_STRING: { + Function* call_fn = codegen->GetFunction(IRFunction::STRING_VALUE_LE); + result = builder.CreateCall2(call_fn, lhs_value, rhs_value, "tmp_le"); + break; + } + default: + DCHECK(false) << "Shouldn't get here."; } - case TExprOpcode::LT_STRINGVALUE_STRINGVALUE: { - Function* call_fn = codegen->GetFunction(IRFunction::STRING_VALUE_LT); - result = builder.CreateCall2(call_fn, lhs_value, rhs_value, "tmp_lt"); - break; + } else if (fn_.name.function_name == "lt") { + switch (t) { + case TYPE_BOOLEAN: + // LLVM defines false > true + result = builder.CreateICmpSGT(lhs_value, rhs_value, "tmp_lt"); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateICmpSLT(lhs_value, rhs_value, "tmp_lt"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateFCmpULT(lhs_value, rhs_value, "tmp_lt"); + break; + case TYPE_STRING: { + Function* call_fn = codegen->GetFunction(IRFunction::STRING_VALUE_LT); + result = builder.CreateCall2(call_fn, lhs_value, rhs_value, "tmp_lt"); + break; + } + default: + DCHECK(false) << "Shouldn't get here."; } - - default: - DCHECK(false) << "Unknown op: " << op(); - return NULL; + } else { + DCHECK(false) << "Unknown binary predicate function: " << fn_.name.function_name; } builder.CreateBr(ret_block); diff --git a/be/src/exprs/cast-expr.cc b/be/src/exprs/cast-expr.cc index e6ed783e7..7d4a3b9ef 100644 --- a/be/src/exprs/cast-expr.cc +++ b/be/src/exprs/cast-expr.cc @@ -54,135 +54,184 @@ bool CastExpr::IsJittable(LlvmCodeGen* codegen) const { // %child_result = call i64 @ArithmeticExpr(i8** %row, i8* %state_data, i1* %is_null) // %child_null = load i1* %is_null // br i1 %child_null, label %ret_block, label %child_not_null -// +// // child_not_null: ; preds = %entry // %tmp_cast = sitofp i64 %child_result to double // br label %ret_block -// +// // ret_block: ; preds = %child_not_null, %entry // %tmp_phi = phi double [ 0.000000e+00, %entry ], [ %tmp_cast, %child_not_null ] // ret double %tmp_phi // } Function* CastExpr::Codegen(LlvmCodeGen* codegen) { DCHECK_EQ(GetNumChildren(), 1); + DCHECK_EQ(fn_.arg_types.size(), 2); Function* child_function = children()[0]->Codegen(codegen); if (child_function == NULL) return NULL; + PrimitiveType t1 = ThriftToType(fn_.arg_types[0].type); + PrimitiveType t2 = ThriftToType(fn_.arg_types[1].type); + // No op cast. Just return the child function. - switch (op()) { - case TExprOpcode::CAST_BOOL_BOOL: - case TExprOpcode::CAST_CHAR_CHAR: - case TExprOpcode::CAST_SHORT_SHORT: - case TExprOpcode::CAST_INT_INT: - case TExprOpcode::CAST_LONG_LONG: - case TExprOpcode::CAST_FLOAT_FLOAT: - case TExprOpcode::CAST_DOUBLE_DOUBLE: - codegen_fn_ = child_function; - scratch_buffer_size_ = children()[0]->scratch_buffer_size(); - return codegen_fn_; - default: - break; + if (t1 == t2) { + codegen_fn_ = child_function; + scratch_buffer_size_ = children()[0]->scratch_buffer_size(); + return codegen_fn_; } LLVMContext& context = codegen->context(); LlvmCodeGen::LlvmBuilder builder(context); Type* return_type = codegen->GetType(type()); Function* function = CreateComputeFnPrototype(codegen, "CastExpr"); - + BasicBlock* entry_block = BasicBlock::Create(context, "entry", function); - BasicBlock* child_not_null_block = + BasicBlock* child_not_null_block = BasicBlock::Create(context, "child_not_null", function); BasicBlock* ret_block = BasicBlock::Create(context, "ret_block", function); - + builder.SetInsertPoint(entry_block); // Call child function Value* child_value = children()[0]->CodegenGetValue(codegen, entry_block, ret_block, child_not_null_block); - + // Do the cast builder.SetInsertPoint(child_not_null_block); Value* result = NULL; - switch (op()) { - case TExprOpcode::CAST_BOOL_CHAR: - case TExprOpcode::CAST_BOOL_SHORT: - case TExprOpcode::CAST_BOOL_INT: - case TExprOpcode::CAST_BOOL_LONG: - result = builder.CreateZExt(child_value, return_type, "tmp_cast"); + switch (t1) { + case TYPE_BOOLEAN: + switch (t2) { + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateZExt(child_value, return_type, "tmp_cast"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateUIToFP(child_value, return_type, "tmp_cast"); + break; + default: + DCHECK(false); + } break; - - case TExprOpcode::CAST_CHAR_SHORT: - case TExprOpcode::CAST_CHAR_INT: - case TExprOpcode::CAST_CHAR_LONG: - case TExprOpcode::CAST_SHORT_INT: - case TExprOpcode::CAST_SHORT_LONG: - case TExprOpcode::CAST_INT_LONG: - result = builder.CreateSExt(child_value, return_type, "tmp_cast"); + case TYPE_TINYINT: + switch (t2) { + case TYPE_BOOLEAN: + result = builder.CreateICmpNE(child_value, + codegen->GetIntConstant(children()[0]->type(), 0)); + break; + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateSExt(child_value, return_type, "tmp_cast"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateSIToFP(child_value, return_type, "tmp_cast"); + break; + default: DCHECK(false); + } break; - - case TExprOpcode::CAST_SHORT_CHAR: - case TExprOpcode::CAST_INT_CHAR: - case TExprOpcode::CAST_LONG_CHAR: - case TExprOpcode::CAST_INT_SHORT: - case TExprOpcode::CAST_LONG_SHORT: - case TExprOpcode::CAST_LONG_INT: - result = builder.CreateTrunc(child_value, return_type, "tmp_cast"); + case TYPE_SMALLINT: + switch (t2) { + case TYPE_BOOLEAN: + result = builder.CreateICmpNE(child_value, + codegen->GetIntConstant(children()[0]->type(), 0)); + break; + case TYPE_TINYINT: + result = builder.CreateTrunc(child_value, return_type, "tmp_cast"); + break; + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateSExt(child_value, return_type, "tmp_cast"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateSIToFP(child_value, return_type, "tmp_cast"); + break; + default: DCHECK(false); + } break; - - case TExprOpcode::CAST_BOOL_FLOAT: - case TExprOpcode::CAST_BOOL_DOUBLE: - result = builder.CreateUIToFP(child_value, return_type, "tmp_cast"); + case TYPE_INT: + switch (t2) { + case TYPE_BOOLEAN: + result = builder.CreateICmpNE(child_value, + codegen->GetIntConstant(children()[0]->type(), 0)); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + result = builder.CreateTrunc(child_value, return_type, "tmp_cast"); + break; + case TYPE_BIGINT: + result = builder.CreateSExt(child_value, return_type, "tmp_cast"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateSIToFP(child_value, return_type, "tmp_cast"); + break; + default: DCHECK(false); + } break; - - case TExprOpcode::CAST_CHAR_FLOAT: - case TExprOpcode::CAST_SHORT_FLOAT: - case TExprOpcode::CAST_INT_FLOAT: - case TExprOpcode::CAST_LONG_FLOAT: - case TExprOpcode::CAST_CHAR_DOUBLE: - case TExprOpcode::CAST_SHORT_DOUBLE: - case TExprOpcode::CAST_INT_DOUBLE: - case TExprOpcode::CAST_LONG_DOUBLE: - result = builder.CreateSIToFP(child_value, return_type, "tmp_cast"); + case TYPE_BIGINT: + switch (t2) { + case TYPE_BOOLEAN: + result = builder.CreateICmpNE(child_value, + codegen->GetIntConstant(children()[0]->type(), 0)); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + result = builder.CreateTrunc(child_value, return_type, "tmp_cast"); + break; + case TYPE_FLOAT: + case TYPE_DOUBLE: + result = builder.CreateSIToFP(child_value, return_type, "tmp_cast"); + break; + default: DCHECK(false); + } break; - - // llvm will treat bool types like tinyint types. true is any non-zero value, - // not 'one'. We'll fix this by explicitly checking against 0. - // TODO: is this a problem? How should we deal with this? - case TExprOpcode::CAST_CHAR_BOOL: - case TExprOpcode::CAST_SHORT_BOOL: - case TExprOpcode::CAST_INT_BOOL: - case TExprOpcode::CAST_LONG_BOOL: - result = builder.CreateICmpNE(child_value, - codegen->GetIntConstant(children()[0]->type(), 0)); + case TYPE_FLOAT: + switch (t2) { + case TYPE_BOOLEAN: + result = builder.CreateFPToSI( + child_value, codegen->GetType(TYPE_INT), "tmp_cast"); + result = builder.CreateICmpNE(result, codegen->GetIntConstant(TYPE_INT, 0)); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateFPToSI(child_value, return_type, "tmp_cast"); + break; + case TYPE_DOUBLE: + result = builder.CreateFPExt(child_value, return_type, "tmp_cast"); + break; + default: DCHECK(false); + } break; - case TExprOpcode::CAST_DOUBLE_BOOL: - case TExprOpcode::CAST_FLOAT_BOOL: - result = builder.CreateFPToSI(child_value, codegen->GetType(TYPE_INT), "tmp_cast"); - result = builder.CreateICmpNE(result, codegen->GetIntConstant(TYPE_INT, 0)); + case TYPE_DOUBLE: + switch (t2) { + case TYPE_BOOLEAN: + result = builder.CreateFPToSI( + child_value, codegen->GetType(TYPE_INT), "tmp_cast"); + result = builder.CreateICmpNE(result, codegen->GetIntConstant(TYPE_INT, 0)); + break; + case TYPE_TINYINT: + case TYPE_SMALLINT: + case TYPE_INT: + case TYPE_BIGINT: + result = builder.CreateFPToSI(child_value, return_type, "tmp_cast"); + break; + case TYPE_FLOAT: + result = builder.CreateFPTrunc(child_value, return_type, "tmp_cast"); + break; + default: DCHECK(false); + } break; - - case TExprOpcode::CAST_FLOAT_CHAR: - case TExprOpcode::CAST_FLOAT_SHORT: - case TExprOpcode::CAST_FLOAT_INT: - case TExprOpcode::CAST_FLOAT_LONG: - case TExprOpcode::CAST_DOUBLE_CHAR: - case TExprOpcode::CAST_DOUBLE_SHORT: - case TExprOpcode::CAST_DOUBLE_INT: - case TExprOpcode::CAST_DOUBLE_LONG: - result = builder.CreateFPToSI(child_value, return_type, "tmp_cast"); - break; - - case TExprOpcode::CAST_FLOAT_DOUBLE: - result = builder.CreateFPExt(child_value, return_type, "tmp_cast"); - break; - - case TExprOpcode::CAST_DOUBLE_FLOAT: - result = builder.CreateFPTrunc(child_value, return_type, "tmp_cast"); - break; - default: - DCHECK(false) << "Unknown op: " << op(); - return NULL; + DCHECK(false); } + builder.CreateBr(ret_block); // Return block. is_null return does not have to set explicitly, propagated from child diff --git a/be/src/exprs/compound-predicate.cc b/be/src/exprs/compound-predicate.cc index 0be2f6284..30a055b9c 100644 --- a/be/src/exprs/compound-predicate.cc +++ b/be/src/exprs/compound-predicate.cc @@ -35,7 +35,6 @@ Status CompoundPredicate::Prepare(RuntimeState* state, const RowDescriptor& desc void* CompoundPredicate::AndComputeFn(Expr* e, TupleRow* row) { CompoundPredicate* p = static_cast(e); DCHECK_EQ(p->children_.size(), 2); - DCHECK_EQ(p->opcode_, TExprOpcode::COMPOUND_AND); Expr* op1 = e->children()[0]; bool* val1 = reinterpret_cast(op1->GetValue(row)); Expr* op2 = e->children()[1]; @@ -56,7 +55,6 @@ void* CompoundPredicate::AndComputeFn(Expr* e, TupleRow* row) { void* CompoundPredicate::OrComputeFn(Expr* e, TupleRow* row) { CompoundPredicate* p = static_cast(e); DCHECK_EQ(p->children_.size(), 2); - DCHECK_EQ(p->opcode_, TExprOpcode::COMPOUND_OR); Expr* op1 = e->children()[0]; bool* val1 = reinterpret_cast(op1->GetValue(row)); Expr* op2 = e->children()[1]; @@ -77,7 +75,6 @@ void* CompoundPredicate::OrComputeFn(Expr* e, TupleRow* row) { void* CompoundPredicate::NotComputeFn(Expr* e, TupleRow* row) { CompoundPredicate* p = static_cast(e); DCHECK_EQ(p->children_.size(), 1); - DCHECK_EQ(p->opcode_, TExprOpcode::COMPOUND_NOT); Expr* op = e->children()[0]; bool* val = reinterpret_cast(op->GetValue(row)); if (val == NULL) return NULL; @@ -97,19 +94,18 @@ string CompoundPredicate::DebugString() const { // %child_result = call i1 @BinaryPredicate(i8** %row, i8* %state_data, i1* %is_null) // %child_null = load i1* %is_null // br i1 %child_null, label %ret, label %child_not_null -// +// // child_not_null: ; preds = %entry // %0 = xor i1 %child_result, true // br label %ret -// +// // ret: ; preds = %child_not_null, %entry // %tmp_phi = phi i1 [ false, %entry ], [ %0, %child_not_null ] // ret i1 %tmp_phi // } Function* CompoundPredicate::CodegenNot(LlvmCodeGen* codegen) { DCHECK_EQ(GetNumChildren(), 1); - DCHECK_EQ(op(), TExprOpcode::COMPOUND_NOT); - + Function* child_function = children()[0]->Codegen(codegen); if (child_function == NULL) return NULL; @@ -121,7 +117,7 @@ Function* CompoundPredicate::CodegenNot(LlvmCodeGen* codegen) { BasicBlock* entry_block = BasicBlock::Create(context, "entry", function); builder.SetInsertPoint(entry_block); - BasicBlock* child_not_null_block = + BasicBlock* child_not_null_block = BasicBlock::Create(context, "child_not_null", function); BasicBlock* ret_block = BasicBlock::Create(context, "ret", function); @@ -145,8 +141,8 @@ Function* CompoundPredicate::CodegenNot(LlvmCodeGen* codegen) { return function; } -// IR codegen for compound and/or predicates. Compound predicate has non trivial -// null handling as well as many branches so this is pretty complicated. The IR +// IR codegen for compound and/or predicates. Compound predicate has non trivial +// null handling as well as many branches so this is pretty complicated. The IR // for x && y is: // // define i1 @CompoundPredicate(i8** %row, i8* %state_data, i1* %is_null) { @@ -159,30 +155,30 @@ Function* CompoundPredicate::CodegenNot(LlvmCodeGen* codegen) { // %rhs_null3 = load i1* %rhs_null // %tmp_and = and i1 %lhs_call, %rhs_call // br i1 %lhs_null2, label %lhs_null1, label %lhs_not_null -// +// // lhs_null1: ; preds = %entry // br i1 %rhs_null3, label %null_block, label %lhs_null_rhs_not_null -// +// // lhs_not_null: ; preds = %entry // br i1 %rhs_null3, label %lhs_not_null_rhs_null, label %not_null_block -// +// // lhs_null_rhs_not_null: ; preds = %lhs_null1 // br i1 %rhs_call, label %null_block, label %not_null_block -// +// // lhs_not_null_rhs_null: ; preds = %lhs_not_null // br i1 %lhs_call, label %null_block, label %not_null_block -// -// null_block: +// +// null_block: // store i1 true, i1* %is_null // br label %ret -// -// not_null_block: -// %0 = phi i1 [ false, %lhs_null_rhs_not_null ], -// [ false, %lhs_not_null_rhs_null ], +// +// not_null_block: +// %0 = phi i1 [ false, %lhs_null_rhs_not_null ], +// [ false, %lhs_not_null_rhs_null ], // [ %tmp_and, %lhs_not_null ] // store i1 false, i1* %is_null // br label %ret -// +// // ret: ; preds = %not_null_block, %null_block // %tmp_phi = phi i1 [ false, %null_block ], [ %0, %not_null_block ] // ret i1 %tmp_phi @@ -194,7 +190,7 @@ Function* CompoundPredicate::CodegenBinary(LlvmCodeGen* codegen) { if (lhs_function == NULL) return NULL; Function* rhs_function = children()[1]->Codegen(codegen); if (rhs_function == NULL) return NULL; - + LLVMContext& context = codegen->context(); LlvmCodeGen::LlvmBuilder builder(context); Type* return_type = codegen->GetType(type()); @@ -205,18 +201,18 @@ Function* CompoundPredicate::CodegenBinary(LlvmCodeGen* codegen) { LlvmCodeGen::NamedVariable lhs_null_var("lhs_null_ptr", codegen->boolean_type()); LlvmCodeGen::NamedVariable rhs_null_var("rhs_null_ptr", codegen->boolean_type()); - + // Create stack variables for lhs is_null result and rhs is_null result Value* lhs_is_null = codegen->CreateEntryBlockAlloca(function, lhs_null_var); Value* rhs_is_null = codegen->CreateEntryBlockAlloca(function, rhs_null_var); // Control blocks for aggregating results BasicBlock* lhs_null_block = BasicBlock::Create(context, "lhs_null", function); - BasicBlock* lhs_not_null_block = + BasicBlock* lhs_not_null_block = BasicBlock::Create(context, "lhs_not_null", function); - BasicBlock* lhs_null_rhs_not_null_block = + BasicBlock* lhs_null_rhs_not_null_block = BasicBlock::Create(context, "lhs_null_rhs_not_null", function); - BasicBlock* lhs_not_null_rhs_null_block = + BasicBlock* lhs_not_null_rhs_null_block = BasicBlock::Create(context, "lhs_not_null_rhs_null", function); BasicBlock* null_block = BasicBlock::Create(context, "null_block", function); BasicBlock* not_null_block = BasicBlock::Create(context, "not_null_block", function); @@ -234,11 +230,13 @@ Function* CompoundPredicate::CodegenBinary(LlvmCodeGen* codegen) { // Call rhs args[2] = rhs_is_null; Value* rhs_value = builder.CreateCall(rhs_function, args, "rhs_call"); - + Value* lhs_is_null_val = builder.CreateLoad(lhs_is_null, "lhs_null"); Value* rhs_is_null_val = builder.CreateLoad(rhs_is_null, "rhs_null"); Value* compare = NULL; - if (op() == TExprOpcode::COMPOUND_AND) { + + bool is_and = fn_.name.function_name == "and"; + if (is_and) { compare = builder.CreateAnd(lhs_value, rhs_value, "tmp_and"); } else { compare = builder.CreateOr(lhs_value, rhs_value, "tmp_or"); @@ -257,7 +255,7 @@ Function* CompoundPredicate::CodegenBinary(LlvmCodeGen* codegen) { // lhs_not_null rhs_null block builder.SetInsertPoint(lhs_not_null_rhs_null_block); - if (op() == TExprOpcode::COMPOUND_AND) { + if (is_and) { // false && null -> false; true && null -> null builder.CreateCondBr(lhs_value, null_block, not_null_block); } else { @@ -267,7 +265,7 @@ Function* CompoundPredicate::CodegenBinary(LlvmCodeGen* codegen) { // lhs_null rhs_not_null block builder.SetInsertPoint(lhs_null_rhs_not_null_block); - if (op() == TExprOpcode::COMPOUND_AND) { + if (is_and) { // null && false -> false; null && true -> null builder.CreateCondBr(rhs_value, null_block, not_null_block); } else { @@ -283,7 +281,7 @@ Function* CompoundPredicate::CodegenBinary(LlvmCodeGen* codegen) { // not-NULL block builder.SetInsertPoint(not_null_block); PHINode* not_null_phi = builder.CreatePHI(codegen->boolean_type(), 3); - if (op() == TExprOpcode::COMPOUND_AND) { + if (is_and) { not_null_phi->addIncoming(codegen->false_value(), lhs_null_rhs_not_null_block); not_null_phi->addIncoming(codegen->false_value(), lhs_not_null_rhs_null_block); not_null_phi->addIncoming(compare, lhs_not_null_block); @@ -305,19 +303,14 @@ Function* CompoundPredicate::CodegenBinary(LlvmCodeGen* codegen) { return function; } -Function* CompoundPredicate::Codegen(LlvmCodeGen* codegen) { +Function* CompoundPredicate::Codegen(LlvmCodeGen* codegen) { Function* function = NULL; - switch (op()) { - case TExprOpcode::COMPOUND_AND: - case TExprOpcode::COMPOUND_OR: - function = CodegenBinary(codegen); - break; - case TExprOpcode::COMPOUND_NOT: - function = CodegenNot(codegen); - break; - default: - DCHECK(false); - break; + if (fn_.name.function_name == "and" || fn_.name.function_name == "or") { + function = CodegenBinary(codegen); + } else if (fn_.name.function_name == "not") { + function = CodegenNot(codegen); + } else { + DCHECK(false) << fn_.name.function_name; } if (function == NULL) return NULL; return codegen->FinalizeFunction(function); diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc index 59392250e..718ac2c34 100644 --- a/be/src/exprs/expr-test.cc +++ b/be/src/exprs/expr-test.cc @@ -136,8 +136,8 @@ class ExprTest : public testing::Test { Status status = executor_->Exec(stmt, &result_types); ASSERT_TRUE(status.ok()) << "stmt: " << stmt << "\nerror: " << status.GetErrorMsg(); string result_row; - ASSERT_TRUE(executor_->FetchResult(&result_row).ok()); - EXPECT_EQ(TypeToOdbcString(expr_type), result_types[0].type); + ASSERT_TRUE(executor_->FetchResult(&result_row).ok()) << expr; + EXPECT_EQ(TypeToOdbcString(expr_type), result_types[0].type) << expr; *interpreted_value = ConvertValue(expr_type, result_row); } @@ -176,7 +176,7 @@ class ExprTest : public testing::Test { expr_value_.timestamp_val = TimestampValue(&value[0], value.size()); return &expr_value_.timestamp_val; default: - DCHECK(type); + DCHECK(false) << type; } return NULL; @@ -1783,11 +1783,9 @@ TEST_F(ExprTest, MathTrigonometricFunctions) { TestIsNull("atan(NULL)", TYPE_DOUBLE); TestIsNull("radians(NULL)", TYPE_DOUBLE); TestIsNull("degrees(NULL)", TYPE_DOUBLE); - } TEST_F(ExprTest, MathConversionFunctions) { - TestStringValue("bin(0)", "0"); TestStringValue("bin(1)", "1"); TestStringValue("bin(12)", "1100"); @@ -1817,7 +1815,8 @@ TEST_F(ExprTest, MathConversionFunctions) { // Uneven number of chars results in empty string. TestStringValue("unhex('30A')", ""); - // Run the test suite twice, once with a bigint parameter, and once with string parameters. + // Run the test suite twice, once with a bigint parameter, and once with + // string parameters. for (int i = 0; i < 2; ++i) { // First iteration is with bigint, second with string parameter. string q = (i == 0) ? "" : "'"; @@ -2154,7 +2153,7 @@ TEST_F(ExprTest, MathFunctions) { TestIsNull("quotient(NULL, 1.0)", TYPE_BIGINT); TestIsNull("quotient(1.0, NULL)", TYPE_BIGINT); TestIsNull("quotient(NULL, NULL)", TYPE_BIGINT); - TestIsNull("least(NULL)", TYPE_STRING); + TestIsNull("least(NULL)", TYPE_TINYINT); TestIsNull("least(cast(NULL as tinyint))", TYPE_TINYINT); TestIsNull("least(cast(NULL as smallint))", TYPE_SMALLINT); TestIsNull("least(cast(NULL as int))", TYPE_INT); @@ -2162,7 +2161,7 @@ TEST_F(ExprTest, MathFunctions) { TestIsNull("least(cast(NULL as float))", TYPE_FLOAT); TestIsNull("least(cast(NULL as double))", TYPE_DOUBLE); TestIsNull("least(cast(NULL as timestamp))", TYPE_TIMESTAMP); - TestIsNull("greatest(NULL)", TYPE_STRING); + TestIsNull("greatest(NULL)", TYPE_TINYINT); TestIsNull("greatest(cast(NULL as tinyint))", TYPE_TINYINT); TestIsNull("greatest(cast(NULL as smallint))", TYPE_SMALLINT); TestIsNull("greatest(cast(NULL as int))", TYPE_INT); @@ -2469,8 +2468,10 @@ TEST_F(ExprTest, TimestampFunctions) { TestStringValue( "to_date(cast('2011-12-22 09:10:11.12345678' as timestamp))", "2011-12-22"); - TestValue("datediff('2011-12-22 09:10:11.12345678', '2012-12-22')", TYPE_INT, -366); - TestValue("datediff('2012-12-22', '2011-12-22 09:10:11.12345678')", TYPE_INT, 366); + TestValue("datediff(cast('2011-12-22 09:10:11.12345678' as timestamp), \ + cast('2012-12-22' as timestamp))", TYPE_INT, -366); + TestValue("datediff(cast('2012-12-22' as timestamp), \ + cast('2011-12-22 09:10:11.12345678' as timestamp))", TYPE_INT, 366); TestIsNull("year(cast('09:10:11.000000' as timestamp))", TYPE_INT); TestIsNull("month(cast('09:10:11.000000' as timestamp))", TYPE_INT); @@ -2489,8 +2490,9 @@ TEST_F(ExprTest, TimestampFunctions) { TestIsNull("dayofweek(NULL)", TYPE_INT); TestIsNull("dayofyear(NULL)", TYPE_INT); TestIsNull("weekofyear(NULL)", TYPE_INT); - TestIsNull("datediff(NULL, '2011-12-22 09:10:11.12345678')", TYPE_INT); - TestIsNull("datediff('2012-12-22', NULL)", TYPE_INT); + TestIsNull("datediff(NULL, cast('2011-12-22 09:10:11.12345678' as timestamp))", + TYPE_INT); + TestIsNull("datediff(cast('2012-12-22' as timestamp), NULL)", TYPE_INT); TestIsNull("datediff(NULL, NULL)", TYPE_INT); TestStringValue("dayname(cast('2011-12-18 09:10:11.000000' as timestamp))", "Sunday"); @@ -2664,8 +2666,8 @@ TEST_F(ExprTest, ConditionalFunctions) { TestIsNull(f + "(NULL, NULL)", TYPE_BOOLEAN); } - TestIsNull("coalesce(NULL)", TYPE_FLOAT); - TestIsNull("coalesce(NULL, NULL)", TYPE_FLOAT); + TestIsNull("coalesce(NULL)", TYPE_BOOLEAN); + TestIsNull("coalesce(NULL, NULL)", TYPE_BOOLEAN); TestValue("coalesce(TRUE)", TYPE_BOOLEAN, true); TestValue("coalesce(NULL, TRUE, NULL)", TYPE_BOOLEAN, true); TestValue("coalesce(FALSE, NULL, TRUE, NULL)", TYPE_BOOLEAN, false); @@ -2938,6 +2940,7 @@ int main(int argc, char **argv) { EXIT_IF_ERROR( impala_server->StartWithClientServers(FLAGS_beeswax_port, FLAGS_beeswax_port + 1, false)); + impala_server->SetCatalogInitialized(); executor_ = new ImpaladQueryExecutor(); EXIT_IF_ERROR(executor_->Setup()); diff --git a/be/src/exprs/expr.cc b/be/src/exprs/expr.cc index 65bc168bc..9e1314adc 100644 --- a/be/src/exprs/expr.cc +++ b/be/src/exprs/expr.cc @@ -20,6 +20,7 @@ #include "common/object-pool.h" #include "common/status.h" #include "exprs/expr.h" +#include "exprs/aggregate-functions.h" #include "exprs/anyval-util.h" #include "exprs/arithmetic-expr.h" #include "exprs/binary-predicate.h" @@ -28,6 +29,7 @@ #include "exprs/cast-expr.h" #include "exprs/char-literal.h" #include "exprs/compound-predicate.h" +#include "exprs/conditional-functions.h" #include "exprs/date-literal.h" #include "exprs/float-literal.h" #include "exprs/function-call.h" @@ -36,19 +38,25 @@ #include "exprs/int-literal.h" #include "exprs/is-null-predicate.h" #include "exprs/like-predicate.h" +#include "exprs/math-functions.h" +#include "exprs/native-udf-expr.h" #include "exprs/null-literal.h" -#include "exprs/opcode-registry.h" +#include "exprs/string-functions.h" #include "exprs/string-literal.h" +#include "exprs/timestamp-functions.h" #include "exprs/timestamp-literal.h" #include "exprs/tuple-is-null-predicate.h" -#include "exprs/native-udf-expr.h" +#include "exprs/udf-builtins.h" +#include "exprs/utility-functions.h" #include "gen-cpp/Exprs_types.h" #include "gen-cpp/Data_types.h" +#include "runtime/lib-cache.h" #include "runtime/runtime-state.h" #include "runtime/raw-value.h" #include "gen-cpp/Exprs_types.h" #include "gen-cpp/ImpalaService_types.h" +#include "opcode/functions.h" using namespace std; using namespace impala; @@ -65,43 +73,9 @@ bool ParseString(const string& str, T* val) { return !stream.fail(); } -void* ExprValue::TryParse(const string& str, PrimitiveType type) { - switch(type) { - case TYPE_NULL: - return NULL; - case TYPE_BOOLEAN: - if (ParseString(str, &bool_val)) return &bool_val; - break; - case TYPE_TINYINT: - if (ParseString(str, &tinyint_val)) return &tinyint_val; - break; - case TYPE_SMALLINT: - if (ParseString(str, &smallint_val)) return &smallint_val; - break; - case TYPE_INT: - if (ParseString(str, &int_val)) return &int_val; - break; - case TYPE_BIGINT: - if (ParseString(str, &bigint_val)) return &bigint_val; - break; - case TYPE_FLOAT: - if (ParseString(str, &float_val)) return &float_val; - break; - case TYPE_DOUBLE: - if (ParseString(str, &double_val)) return &double_val; - break; - case TYPE_STRING: - SetStringVal(str); - return &string_val; - default: - DCHECK(false) << "Invalid type."; - } - return NULL; -} - Expr::Expr(const ColumnType& type, bool is_slotref) - : compute_fn_(NULL), - opcode_(TExprOpcode::INVALID_OPCODE), + : is_udf_call_(false), + compute_fn_(NULL), is_slotref_(is_slotref), type_(type), output_scale_(-1), @@ -112,8 +86,8 @@ Expr::Expr(const ColumnType& type, bool is_slotref) } Expr::Expr(const TExprNode& node, bool is_slotref) - : compute_fn_(NULL), - opcode_(node.__isset.opcode ? node.opcode : TExprOpcode::INVALID_OPCODE), + : is_udf_call_(false), + compute_fn_(NULL), is_slotref_(is_slotref), type_(ColumnType(node.type)), output_scale_(-1), @@ -121,6 +95,7 @@ Expr::Expr(const TExprNode& node, bool is_slotref) adapter_fn_used_(false), scratch_buffer_size_(0), jitted_compute_fn_(NULL) { + if (node.__isset.fn) fn_ = node.fn; } Status Expr::CreateExprTree(ObjectPool* pool, const TExpr& texpr, Expr** root_expr) { @@ -308,7 +283,6 @@ Status Expr::CreateExpr(ObjectPool* pool, const TExprNode& texpr_node, Expr** ex *expr = pool->Add(new FloatLiteral(texpr_node)); return Status::OK; case TExprNodeType::COMPUTE_FUNCTION_CALL: - DCHECK(texpr_node.__isset.opcode); *expr = pool->Add(new FunctionCall(texpr_node)); return Status::OK; case TExprNodeType::INT_LITERAL: @@ -351,10 +325,10 @@ Status Expr::CreateExpr(ObjectPool* pool, const TExprNode& texpr_node, Expr** ex *expr = pool->Add(new TupleIsNullPredicate(texpr_node)); return Status::OK; case TExprNodeType::FUNCTION_CALL: - if (!texpr_node.__isset.fn_call_expr) { - return Status("Udf call not set in thrift node"); + if (!texpr_node.__isset.fn) { + return Status("Function not set in thrift node"); } - if (texpr_node.fn_call_expr.fn.binary_type == TFunctionBinaryType::HIVE) { + if (texpr_node.fn.binary_type == TFunctionBinaryType::HIVE) { *expr = pool->Add(new HiveUdfCall(texpr_node)); } else { *expr = pool->Add(new NativeUdfExpr(texpr_node)); @@ -522,17 +496,33 @@ bool Expr::codegend_fn_thread_safe() const { } Status Expr::Prepare(RuntimeState* state, const RowDescriptor& row_desc) { - PrepareChildren(state, row_desc); - // Not all exprs have opcodes (i.e. literals) - DCHECK(opcode_ != TExprOpcode::INVALID_OPCODE); - void* compute_fn_ptr = - OpcodeRegistry::Instance()->GetFunctionPtr(opcode_); - if (compute_fn_ptr == NULL) { - stringstream out; - out << "Expr::Prepare(): Opcode: " << opcode_ << " does not have a registry entry. "; - return Status(out.str()); + RETURN_IF_ERROR(PrepareChildren(state, row_desc)); + if (is_udf_call_) return Status::OK; + + // Not all exprs have a function set (i.e. literals) + if (fn_.__isset.scalar_fn) { + if (!fn_.scalar_fn.symbol.empty()) { + DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::BUILTIN); + void* fn_ptr; + Status status = state->lib_cache()->GetSoFunctionPtr( + state->fs_cache(), "", fn_.scalar_fn.symbol, &fn_ptr); + if (!status.ok()) { + // Builtins symbols should exist unless there is a version mismatch. + stringstream ss; + ss << "Builtin '" << fn_.name.function_name << "' with symbol '" + << fn_.scalar_fn.symbol << "' does not exist. " + << "Verify that all your impalads are the same version."; + status.AddErrorMsg(ss.str()); + return status; + } + DCHECK(fn_ptr != NULL); + compute_fn_ = reinterpret_cast(fn_ptr); + } else { + stringstream ss; + ss << "Function " << fn_.name.function_name << " is not implemented."; + return Status(ss.str()); + } } - compute_fn_ = reinterpret_cast(compute_fn_ptr); return Status::OK; } @@ -556,9 +546,6 @@ string Expr::DebugString() const { // TODO: implement partial debug string for member vars stringstream out; out << " type=" << type_.DebugString(); - if (opcode_ != TExprOpcode::INVALID_OPCODE) { - out << " opcode=" << opcode_; - } out << " codegen=" << (codegen_fn_ == NULL ? "false" : "true"); if (!children_.empty()) { out << " children=" << DebugString(children_); @@ -953,4 +940,19 @@ Function* Expr::CreateIrFunctionPrototype(LlvmCodeGen* codegen, const string& na return function; } +void Expr::InitBuiltinsDummy() { + // Call one function from each of the classes to pull all the symbols + // from that class in. + // TODO: is there a better way to do this? + AggregateFunctions::InitNull(NULL, NULL); + ComputeFunctions::Add_char_char(NULL, NULL); + ConditionalFunctions::IsNull(NULL, NULL); + MathFunctions::Pi(NULL, NULL); + StringFunctions::Length(NULL, NULL); + TimestampFunctions::Year(NULL, NULL); + UdfBuiltins::Pi(NULL); + UtilityFunctions::Pid(NULL, NULL); +} + + } diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h index 018af6c24..3a550ad0e 100644 --- a/be/src/exprs/expr.h +++ b/be/src/exprs/expr.h @@ -20,7 +20,6 @@ #include #include "common/status.h" -#include "gen-cpp/Opcodes_types.h" #include "runtime/descriptors.h" #include "runtime/raw-value.h" #include "runtime/tuple.h" @@ -91,10 +90,6 @@ struct ExprValue { string_val(const_cast(string_data.data()), string_data.size()) { } - // Update this ExprValue by parsing the string and return a pointer to the result. - // NULL will be returned if the string and type are not compatible. - void* TryParse(const std::string& string, PrimitiveType type); - // Set string value to copy of str void SetStringVal(const StringValue& str) { string_data = std::string(str.ptr, str.len); @@ -290,8 +285,6 @@ class Expr { PrimitiveType type() const { return type_.type; } const std::vector& children() const { return children_; } - TExprOpcode::type op() const { return opcode_; } - // Returns true if expr doesn't contain slotrefs, ie, can be evaluated // with GetValue(NULL). The default implementation returns true if all of // the children are constant. @@ -398,9 +391,16 @@ class Expr { virtual std::string DebugString() const; static std::string DebugString(const std::vector& exprs); + // 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. + static void InitBuiltinsDummy(); + static const char* LLVM_CLASS_NAME; protected: + friend class AggFnEvaluator; friend class ComputeFunctions; friend class MathFunctions; friend class StringFunctions; @@ -424,12 +424,13 @@ class Expr { // Return OK if successful, otherwise return error status. Status PrepareChildren(RuntimeState* state, const RowDescriptor& row_desc); + // Function description. + TFunction fn_; + bool is_udf_call_; + // function to evaluate expr; typically set in Prepare() ComputeFn compute_fn_; - // function opcode - TExprOpcode::type opcode_; - // recognize if this node is a slotref in order to speed up GetValue() const bool is_slotref_; // analysis is done, types are fixed at this point @@ -585,7 +586,7 @@ inline void* Expr::GetValue(TupleRow* row) { if (is_slotref_) { return SlotRef::ComputeFn(this, row); } else { - DCHECK(compute_fn_ != NULL); + DCHECK(compute_fn_ != NULL) << DebugString(); return compute_fn_(this, row); } } diff --git a/be/src/exprs/function-call.cc b/be/src/exprs/function-call.cc index 5a7dc85e0..09f8a80c3 100644 --- a/be/src/exprs/function-call.cc +++ b/be/src/exprs/function-call.cc @@ -32,52 +32,44 @@ FunctionCall::FunctionCall(const TExprNode& node) Status FunctionCall::Prepare(RuntimeState* state, const RowDescriptor& row_desc) { RETURN_IF_ERROR(Expr::Prepare(state, row_desc)); - switch (opcode_) { - case TExprOpcode::TIMESTAMP_NOW: - case TExprOpcode::UNIX_TIMESTAMP: { - DCHECK(state != NULL); - DCHECK(!state->now()->NotADateTime()); - result_.timestamp_val = *(state->now()); - break; + + // TODO: remove when UDF allow for an Init() function. + const string& name = fn_.name.function_name; + if (name == "now" || + name == "current_timestamp" || + (name == "unix_timestamp" && fn_.arg_types.empty())) { + DCHECK(state != NULL); + DCHECK(!state->now()->NotADateTime()); + result_.timestamp_val = *(state->now()); + } else if (name == "pid") { + DCHECK(state != NULL); + result_.int_val = state->query_ctxt().pid; + } else if (name == "user") { + // Set username from runtime state. + DCHECK(state != NULL); + result_.SetStringVal(state->connected_user()); + } else if (name == "current_database") { + // Set current database from the session. + DCHECK(state != NULL); + result_.SetStringVal(state->query_ctxt().session.database); + } else if (name == "unix_timestamp" || name == "from_unixtime") { + if (children_.size() < 2) return Status::OK; + if (children_[1]->IsConstant()) { + StringValue* fmt = reinterpret_cast(children_[1]->GetValue(NULL)); + if (fmt != NULL && fmt->len > 0) SetDateTimeFormatCtx(fmt); + } else { + SetDateTimeFormatCtx(NULL); } - case TExprOpcode::UNIX_TIMESTAMP_STRINGVALUE_STRINGVALUE: - case TExprOpcode::FROM_UNIXTIME_INT_STRINGVALUE: - case TExprOpcode::FROM_UNIXTIME_LONG_STRINGVALUE: { - if (children_.size() < 2) return Status::OK; - if (children_[1]->IsConstant()) { - StringValue* fmt = reinterpret_cast(children_[1]->GetValue(NULL)); - if (fmt != NULL && fmt->len > 0) SetDateTimeFormatCtx(fmt); - } else { - SetDateTimeFormatCtx(NULL); - } - break; - } - case TExprOpcode::UTILITY_PID: { - // Set pid from runtime state. - DCHECK(state != NULL); - result_.int_val = state->query_ctxt().pid; - break; - } - case TExprOpcode::UTILITY_USER: { - // Set username from runtime state. - DCHECK(state != NULL); - result_.SetStringVal(state->connected_user()); - break; - } - case TExprOpcode::UTILITY_CURRENT_DATABASE: { - // Set current database from the session. - DCHECK(state != NULL); - result_.SetStringVal(state->query_ctxt().session.database); - break; - } - default: break; } return Status::OK; } string FunctionCall::DebugString() const { stringstream out; - out << "FunctionCall(" << Expr::DebugString() << ")"; + out << "FunctionCall(" + << "name=" << fn_.name.function_name << " " + << "symbol=" << fn_.scalar_fn.symbol << " " + << Expr::DebugString() << ")"; return out.str(); } diff --git a/be/src/exprs/hive-udf-call.cc b/be/src/exprs/hive-udf-call.cc index 18117a612..5e78bf0ad 100644 --- a/be/src/exprs/hive-udf-call.cc +++ b/be/src/exprs/hive-udf-call.cc @@ -59,11 +59,11 @@ struct HiveUdfCall::JniContext { }; HiveUdfCall::HiveUdfCall(const TExprNode& node) - : Expr(node), udf_(node.fn_call_expr.fn), + : Expr(node), jni_context_(new JniContext) { - DCHECK(node.__isset.fn_call_expr); + is_udf_call_ = true; DCHECK_EQ(node.node_type, TExprNodeType::FUNCTION_CALL); - DCHECK_EQ(node.fn_call_expr.fn.binary_type, TFunctionBinaryType::HIVE); + DCHECK_EQ(node.fn.binary_type, TFunctionBinaryType::HIVE); } HiveUdfCall::~HiveUdfCall() { @@ -133,8 +133,8 @@ void* HiveUdfCall::Evaluate(Expr* e, TupleRow* row) { Status status = JniUtil::GetJniExceptionMsg(env); if (!status.ok()) { stringstream ss; - ss << "Hive UDF path=" << udf->udf_.hdfs_location << " class=" - << udf->udf_.scalar_fn.symbol + ss << "Hive UDF path=" << udf->fn_.hdfs_location << " class=" + << udf->fn_.scalar_fn.symbol << " failed due to: " << status.GetErrorMsg(); udf->state_->LogError(ss.str()); return NULL; @@ -150,7 +150,7 @@ Status HiveUdfCall::Prepare(RuntimeState* state, const RowDescriptor& row_desc) // Copy the Hive Jar from hdfs to local file system. string local_path; RETURN_IF_ERROR(state->lib_cache()->GetLocalLibPath( - state->fs_cache(), udf_.hdfs_location, LibCache::TYPE_JAR, &local_path)); + state->fs_cache(), fn_.hdfs_location, LibCache::TYPE_JAR, &local_path)); JNIEnv* env = getJNIEnv(); if (env == NULL) return Status("Failed to get/create JVM"); @@ -169,7 +169,7 @@ Status HiveUdfCall::Prepare(RuntimeState* state, const RowDescriptor& row_desc) int input_buffer_size = 0; THiveUdfExecutorCtorParams ctor_params; - ctor_params.fn = udf_; + ctor_params.fn = fn_; ctor_params.local_location = local_path; for (int i = 0; i < GetNumChildren(); ++i) { ctor_params.input_byte_offsets.push_back(input_buffer_size); @@ -208,8 +208,8 @@ Status HiveUdfCall::Prepare(RuntimeState* state, const RowDescriptor& row_desc) string HiveUdfCall::DebugString() const { stringstream out; - out << "HiveUdfCall(hdfs_location=" << udf_.hdfs_location - << " classname=" << udf_.scalar_fn.symbol << " " + out << "HiveUdfCall(hdfs_location=" << fn_.hdfs_location + << " classname=" << fn_.scalar_fn.symbol << " " << Expr::DebugString() << ")"; return out.str(); } diff --git a/be/src/exprs/hive-udf-call.h b/be/src/exprs/hive-udf-call.h index 7afb59c2f..9074c3937 100644 --- a/be/src/exprs/hive-udf-call.h +++ b/be/src/exprs/hive-udf-call.h @@ -68,7 +68,6 @@ class HiveUdfCall : public Expr { static void* Evaluate(Expr* e, TupleRow* row); RuntimeState* state_; - const TFunction udf_; struct JniContext; boost::scoped_ptr jni_context_; diff --git a/be/src/exprs/like-predicate.cc b/be/src/exprs/like-predicate.cc index 92e38feea..03c5b961a 100644 --- a/be/src/exprs/like-predicate.cc +++ b/be/src/exprs/like-predicate.cc @@ -110,7 +110,7 @@ void* LikePredicate::RegexMatch(Expr* e, TupleRow* row, bool is_like_pattern) { } re2::RE2 re(re_pattern); if (re.ok()) { - p->result_.bool_val = + p->result_.bool_val = RE2::FullMatch(re2::StringPiece(operand_value->ptr, operand_value->len), re); return &p->result_.bool_val; } else { @@ -131,25 +131,24 @@ void* LikePredicate::RegexFn(Expr* e, TupleRow* row) { Status LikePredicate::Prepare(RuntimeState* state, const RowDescriptor& row_desc) { RETURN_IF_ERROR(Expr::PrepareChildren(state, row_desc)); DCHECK_EQ(children_.size(), 2); - switch (opcode_) { - case TExprOpcode::LIKE: - compute_fn_ = LikeFn; - break; - case TExprOpcode::REGEX: - compute_fn_ = RegexFn; - break; - default: - stringstream error; - error << "Invalid LIKE operator: " << opcode_; - return Status(error.str()); + bool is_like = fn_.name.function_name == "like"; + if (is_like) { + compute_fn_ = LikeFn; + } else if (fn_.name.function_name == "regexp" || fn_.name.function_name == "rlike") { + compute_fn_ = RegexFn; + } else { + stringstream error; + error << "Invalid LIKE operator: " << fn_.name.function_name; + return Status(error.str()); } + if (GetChild(1)->IsConstant()) { // determine pattern and decide on eval fn StringValue* pattern = static_cast(GetChild(1)->GetValue(NULL)); if (pattern == NULL) return Status::OK; string pattern_str(pattern->ptr, pattern->len); - // Generate a regex search to look for simple patterns: - // - "%anything%": This maps to a fast substring search implementation. + // Generate a regex search to look for simple patterns: + // - "%anything%": This maps to a fast substring search implementation. // - anything%: This maps to a strncmp implementation // - %anything: This maps to a strncmp implementation // - anything: This maps to a strncmp implementation @@ -165,7 +164,7 @@ Status LikePredicate::Prepare(RuntimeState* state, const RowDescriptor& row_desc DCHECK(equals_re.ok()); void* no_arg = NULL; - if (opcode_ == TExprOpcode::LIKE) { + if (is_like) { if (RE2::FullMatch(pattern_str, substring_re, no_arg, &search_string_, no_arg)) { search_string_sv_ = StringValue(search_string_); substring_pattern_ = StringSearch(&search_string_sv_); @@ -184,9 +183,9 @@ Status LikePredicate::Prepare(RuntimeState* state, const RowDescriptor& row_desc compute_fn_ = ConstantEqualsFn; return Status::OK; } - } + } string re_pattern; - if (opcode_ == TExprOpcode::LIKE) { + if (is_like) { ConvertLikePattern(pattern, &re_pattern); } else { re_pattern = pattern_str; diff --git a/be/src/exprs/math-functions.cc b/be/src/exprs/math-functions.cc index 17eb8b209..068b56ddf 100644 --- a/be/src/exprs/math-functions.cc +++ b/be/src/exprs/math-functions.cc @@ -26,7 +26,7 @@ using namespace std; -namespace impala { +namespace impala { const char* MathFunctions::ALPHANUMERIC_CHARS = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ"; @@ -544,7 +544,7 @@ void* MathFunctions::QuotientDouble(Expr* e, TupleRow* row) { } void* MathFunctions::QuotientBigInt(Expr* e, TupleRow* row) { - return ComputeFunctions::Int_Divide_long_long(e, row); + return ComputeFunctions::Int_divide_long_long(e, row); } template diff --git a/be/src/exprs/math-functions.h b/be/src/exprs/math-functions.h index 520cc312a..85639a273 100644 --- a/be/src/exprs/math-functions.h +++ b/be/src/exprs/math-functions.h @@ -94,7 +94,6 @@ class MathFunctions { // Returns false otherwise, indicating some other error condition. static bool HandleParseResult(int8_t dest_base, int64_t* num, StringParser::ParseResult parse_res); - }; } diff --git a/be/src/exprs/native-udf-expr.cc b/be/src/exprs/native-udf-expr.cc index c52f4baab..05c0925e9 100644 --- a/be/src/exprs/native-udf-expr.cc +++ b/be/src/exprs/native-udf-expr.cc @@ -19,7 +19,6 @@ #include #include "codegen/llvm-codegen.h" #include "exprs/anyval-util.h" -#include "exprs/opcode-registry.h" #include "runtime/hdfs-fs-cache.h" #include "runtime/lib-cache.h" #include "runtime/runtime-state.h" @@ -33,15 +32,13 @@ using namespace std; NativeUdfExpr::NativeUdfExpr(const TExprNode& node) : Expr(node), - udf_type_(node.fn_call_expr.fn.binary_type), - hdfs_location_(node.fn_call_expr.fn.hdfs_location), - symbol_name_(node.fn_call_expr.fn.scalar_fn.symbol), - vararg_start_idx_(node.fn_call_expr.__isset.vararg_start_idx ? - node.fn_call_expr.vararg_start_idx : -1), + vararg_start_idx_(node.__isset.vararg_start_idx ? + node.vararg_start_idx : -1), udf_wrapper_(NULL), varargs_input_(NULL) { + is_udf_call_ = true; DCHECK_EQ(node.node_type, TExprNodeType::FUNCTION_CALL); - DCHECK_NE(udf_type_, TFunctionBinaryType::HIVE); + DCHECK_NE(fn_.binary_type, TFunctionBinaryType::HIVE); } NativeUdfExpr::~NativeUdfExpr() { @@ -135,6 +132,16 @@ void* NativeUdfExpr::ComputeFn(Expr* e, TupleRow* row) { Status NativeUdfExpr::Prepare(RuntimeState* state, const RowDescriptor& desc) { RETURN_IF_ERROR(Expr::PrepareChildren(state, desc)); + if (fn_.scalar_fn.symbol.empty()) { + // This path is intended to only be used during development to test FE + // code before the BE has implemented the function. + // Having the failure in the BE (rather than during analysis) allows for + // better FE testing. + DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::BUILTIN); + stringstream ss; + ss << "Function " << fn_.name.function_name << " is not implemented."; + return Status(ss.str()); + } // TODO: this should come from the ExprContext udf_context_.reset(FunctionContextImpl::CreateContext(state, state->udf_pool())); @@ -271,19 +278,24 @@ Status NativeUdfExpr::GetIrComputeFn(RuntimeState* state, llvm::Function** fn) { Status NativeUdfExpr::GetUdf(RuntimeState* state, llvm::Function** udf) { LlvmCodeGen* codegen = state->codegen(); DCHECK(codegen != NULL); - if (udf_type_ == TFunctionBinaryType::NATIVE || - (udf_type_ == TFunctionBinaryType::BUILTIN && !state->codegen_enabled())) { + if (fn_.binary_type == TFunctionBinaryType::NATIVE || + (fn_.binary_type == TFunctionBinaryType::BUILTIN && !state->codegen_enabled())) { // In this path, we are code that has been statically compiled to assembly. // This can either be a UDF implemented in a .so or a builtin using the UDF // interface with the code in impalad. - void* udf_ptr; - if (udf_type_ == TFunctionBinaryType::NATIVE) { - RETURN_IF_ERROR(state->lib_cache()->GetSoFunctionPtr( - state->fs_cache(), hdfs_location_, symbol_name_, &udf_ptr)); - } else { - udf_ptr = OpcodeRegistry::Instance()->GetFunctionPtr(opcode_); + void* fn_ptr; + Status status = state->lib_cache()->GetSoFunctionPtr( + state->fs_cache(), fn_.hdfs_location, fn_.scalar_fn.symbol, &fn_ptr); + if (!status.ok() && fn_.binary_type == TFunctionBinaryType::BUILTIN) { + // Builtins symbols should exist unless there is a version mismatch. + stringstream ss; + ss << "Builtin '" << fn_.name.function_name << "' with symbol '" + << fn_.scalar_fn.symbol << "' does not exist. " + << "Verify that all your impalads are the same version."; + status.AddErrorMsg(ss.str()); } - DCHECK(udf_ptr != NULL); + RETURN_IF_ERROR(status); + DCHECK(fn_ptr != NULL); // Convert UDF function pointer to llvm::Function* // First generate the llvm::FunctionType* corresponding to the UDF. @@ -309,39 +321,42 @@ Status NativeUdfExpr::GetUdf(RuntimeState* state, llvm::Function** udf) { // declaration, not a definition, since we do not create any basic blocks or // instructions in it. *udf = llvm::Function::Create( - udf_type, llvm::GlobalValue::ExternalLinkage, symbol_name_, codegen->module()); + udf_type, llvm::GlobalValue::ExternalLinkage, + fn_.scalar_fn.symbol, codegen->module()); // Associate the dynamically loaded function pointer with the Function* we // defined. This tells LLVM where the compiled function definition is located in // memory. - codegen->execution_engine()->addGlobalMapping(*udf, udf_ptr); - } else if (udf_type_ == TFunctionBinaryType::BUILTIN) { + codegen->execution_engine()->addGlobalMapping(*udf, fn_ptr); + } else if (fn_.binary_type == TFunctionBinaryType::BUILTIN) { // In this path, we're running a builtin with the UDF interface. The IR is // in the llvm module. DCHECK(state->codegen_enabled()); - const string& symbol = OpcodeRegistry::Instance()->GetFunctionSymbol(opcode_); - *udf = codegen->module()->getFunction(symbol); + *udf = codegen->module()->getFunction(fn_.scalar_fn.symbol); if (*udf == NULL) { + // Builtins symbols should exist unless there is a version mismatch. stringstream ss; - ss << "Could not load builtin " << opcode_ << " with symbol: " << symbol; + ss << "Builtin '" << fn_.name.function_name << "' with symbol '" + << fn_.scalar_fn.symbol << "' does not exist. " + << "Verify that all your impalads are the same version."; return Status(ss.str()); } } else { // We're running a IR UDF. - DCHECK_EQ(udf_type_, TFunctionBinaryType::IR); + DCHECK_EQ(fn_.binary_type, TFunctionBinaryType::IR); string local_path; RETURN_IF_ERROR(state->lib_cache()->GetLocalLibPath( - state->fs_cache(), hdfs_location_, LibCache::TYPE_IR, &local_path)); + state->fs_cache(), fn_.hdfs_location, LibCache::TYPE_IR, &local_path)); // Link the UDF module into this query's main module (essentially copy the UDF module // into the main module) so the UDF function is available for inlining in the main // module. RETURN_IF_ERROR(codegen->LinkModule(local_path)); - *udf = codegen->module()->getFunction(symbol_name_); + *udf = codegen->module()->getFunction(fn_.scalar_fn.symbol); if (*udf == NULL) { stringstream ss; - ss << "Unable to locate function " << symbol_name_ - << " from LLVM module " << hdfs_location_; + ss << "Unable to locate function " << fn_.scalar_fn.symbol + << " from LLVM module " << fn_.hdfs_location; return Status(ss.str()); } } @@ -350,7 +365,8 @@ Status NativeUdfExpr::GetUdf(RuntimeState* state, llvm::Function** udf) { string NativeUdfExpr::DebugString() const { stringstream out; - out << "NativeUdfExpr(udf_type=" << udf_type_ << " location=" << hdfs_location_ - << " symbol_name=" << symbol_name_ << Expr::DebugString() << ")"; + out << "NativeUdfExpr(udf_type=" << fn_.binary_type + << " location=" << fn_.hdfs_location + << " symbol_name=" << fn_.scalar_fn.symbol << Expr::DebugString() << ")"; return out.str(); } diff --git a/be/src/exprs/native-udf-expr.h b/be/src/exprs/native-udf-expr.h index ccf9c87e5..bbd96408f 100644 --- a/be/src/exprs/native-udf-expr.h +++ b/be/src/exprs/native-udf-expr.h @@ -68,13 +68,6 @@ class NativeUdfExpr: public Expr { // TODO: Get this from the to-be-implemented ExprContext instead boost::scoped_ptr udf_context_; - // Native (.so), IR (.ll) or builtin - TFunctionBinaryType::type udf_type_; - - // HDFS path and name of the compiled UDF binary - std::string hdfs_location_; - std::string symbol_name_; - // 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. diff --git a/be/src/exprs/opcode-registry.cc b/be/src/exprs/opcode-registry.cc deleted file mode 100644 index a8f8df7ba..000000000 --- a/be/src/exprs/opcode-registry.cc +++ /dev/null @@ -1,263 +0,0 @@ -// Copyright 2012 Cloudera Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include "exprs/opcode-registry.h" -#include "exprs/aggregate-functions.h" - -using namespace boost; -using namespace std; - -namespace impala { - -OpcodeRegistry* OpcodeRegistry::instance_ = NULL; -mutex OpcodeRegistry::instance_lock_; - -static void InitAggregateBuiltins(OpcodeRegistry::AggregateBuiltins* fns); - -OpcodeRegistry::OpcodeRegistry() { - int num_opcodes = static_cast(TExprOpcode::LAST_OPCODE); - scalar_builtins_.resize(num_opcodes); - symbols_.resize(num_opcodes); - Init(); - InitAggregateBuiltins(&aggregate_builtins_); -} - -// TODO: Older versions of gcc have problems resolving template types so use this -// nasty macro to help it. -// This needs to be used if the function is templated. -#define COERCE_CAST1(FUNC, TYPE) ((void*)(void(*)(FunctionContext*, TYPE*))FUNC) -#define COERCE_CAST2_SAME(FUNC, TYPE)\ - ((void*)(void(*)(FunctionContext*, const TYPE&, TYPE*))FUNC) -#define COERCE_CAST2(FUNC, TYPE1, TYPE2)\ - ((void*)(void(*)(FunctionContext*, const TYPE1&, TYPE2*))FUNC) - -static OpcodeRegistry::AggFnDescriptor NullDesc() { - return OpcodeRegistry::AggFnDescriptor((void*)AggregateFunctions::InitNull); -} - -static OpcodeRegistry::AggFnDescriptor CountStarDesc() { - return OpcodeRegistry::AggFnDescriptor( - COERCE_CAST1(AggregateFunctions::InitZero, BigIntVal), - (void*)AggregateFunctions::CountStarUpdate, - // TODO: this should be Sum but there needs to be a planner - // change. The planner currently manually splits count to count/sum. - // For count(distinct col1, col2), the plan relies on count on this step. - (void*)AggregateFunctions::CountStarUpdate); -} - -static OpcodeRegistry::AggFnDescriptor CountDesc() { - return OpcodeRegistry::AggFnDescriptor( - COERCE_CAST1(AggregateFunctions::InitZero, BigIntVal), - (void*)AggregateFunctions::CountUpdate, - (void*)AggregateFunctions::CountUpdate); -} - -template -static OpcodeRegistry::AggFnDescriptor SumDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::InitNull, - COERCE_CAST2(AggregateFunctions::Sum, INPUT, OUTPUT), - COERCE_CAST2(AggregateFunctions::Sum, OUTPUT, OUTPUT)); -} - -template -static OpcodeRegistry::AggFnDescriptor MinDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::InitNull, - COERCE_CAST2_SAME(AggregateFunctions::Min, T), - COERCE_CAST2_SAME(AggregateFunctions::Min, T)); -} - -template -static OpcodeRegistry::AggFnDescriptor MaxDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::InitNull, - COERCE_CAST2_SAME(AggregateFunctions::Max, T), - COERCE_CAST2_SAME(AggregateFunctions::Max, T)); -} - -template<> inline -OpcodeRegistry::AggFnDescriptor MinDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::InitNullString, - COERCE_CAST2_SAME(AggregateFunctions::Min, StringVal), - COERCE_CAST2_SAME(AggregateFunctions::Min, StringVal)); -} - -template<> inline -OpcodeRegistry::AggFnDescriptor MaxDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::InitNullString, - COERCE_CAST2_SAME(AggregateFunctions::Max, StringVal), - COERCE_CAST2_SAME(AggregateFunctions::Max, StringVal)); -} - -static OpcodeRegistry::AggFnDescriptor StringConcatDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::InitNullString, - (void*)AggregateFunctions::StringConcat, - (void*)AggregateFunctions::StringConcat); -} - -template -static OpcodeRegistry::AggFnDescriptor DistinctPcDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::PcInit, - (void*) (void(*)(FunctionContext*, const T&, StringVal*)) - AggregateFunctions::PcUpdate, - (void*)AggregateFunctions::PcMerge, - NULL, - (void*)AggregateFunctions::PcFinalize); -} - -template -static OpcodeRegistry::AggFnDescriptor DistinctPcsaDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::PcInit, - (void*) (void(*)(FunctionContext*, const T&, StringVal*)) - AggregateFunctions::PcsaUpdate, - (void*)AggregateFunctions::PcMerge, - NULL, - (void*)AggregateFunctions::PcsaFinalize); -} - -template -static OpcodeRegistry::AggFnDescriptor HllDesc() { - return OpcodeRegistry::AggFnDescriptor( - (void*)AggregateFunctions::HllInit, - (void*) (void(*)(FunctionContext*, const T&, StringVal*)) - AggregateFunctions::HllUpdate, - (void*)AggregateFunctions::HllMerge, - NULL, - (void*)AggregateFunctions::HllFinalize); -} - -void InitAggregateBuiltins(OpcodeRegistry::AggregateBuiltins* fns) { - // Count(*) - (*fns)[make_pair(TAggregationOp::COUNT, INVALID_TYPE)] = CountStarDesc(); - - // Count - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_BOOLEAN)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_TINYINT)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_SMALLINT)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_INT)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_BIGINT)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_FLOAT)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_DOUBLE)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_TIMESTAMP)] = CountDesc(); - (*fns)[make_pair(TAggregationOp::COUNT, TYPE_STRING)] = CountDesc(); - - // Sum - (*fns)[make_pair(TAggregationOp::SUM, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::SUM, TYPE_BOOLEAN)] = - SumDesc(); - (*fns)[make_pair(TAggregationOp::SUM, TYPE_TINYINT)] = - SumDesc(); - (*fns)[make_pair(TAggregationOp::SUM, TYPE_SMALLINT)] = - SumDesc(); - (*fns)[make_pair(TAggregationOp::SUM, TYPE_INT)] = - SumDesc(); - (*fns)[make_pair(TAggregationOp::SUM, TYPE_BIGINT)] = - SumDesc(); - (*fns)[make_pair(TAggregationOp::SUM, TYPE_FLOAT)] = - SumDesc(); - (*fns)[make_pair(TAggregationOp::SUM, TYPE_DOUBLE)] = - SumDesc(); - - // Min - (*fns)[make_pair(TAggregationOp::MIN, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_BOOLEAN)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_TINYINT)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_SMALLINT)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_INT)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_BIGINT)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_FLOAT)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_DOUBLE)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_TIMESTAMP)] = MinDesc(); - (*fns)[make_pair(TAggregationOp::MIN, TYPE_STRING)] = MinDesc(); - - // Max - (*fns)[make_pair(TAggregationOp::MAX, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_BOOLEAN)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_TINYINT)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_SMALLINT)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_INT)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_BIGINT)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_FLOAT)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_DOUBLE)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_TIMESTAMP)] = MaxDesc(); - (*fns)[make_pair(TAggregationOp::MAX, TYPE_STRING)] = MaxDesc(); - - // Group Concat - (*fns)[make_pair(TAggregationOp::GROUP_CONCAT, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::GROUP_CONCAT, TYPE_STRING)] = StringConcatDesc(); - - // Distinct PC - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_BOOLEAN)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_TINYINT)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_SMALLINT)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_INT)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_BIGINT)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_FLOAT)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_DOUBLE)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_TIMESTAMP)] = - DistinctPcDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PC, TYPE_STRING)] = - DistinctPcDesc(); - - // Distinct PCSA - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_BOOLEAN)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_TINYINT)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_SMALLINT)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_INT)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_BIGINT)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_FLOAT)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_DOUBLE)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_TIMESTAMP)] = - DistinctPcsaDesc(); - (*fns)[make_pair(TAggregationOp::DISTINCT_PCSA, TYPE_STRING)] = - DistinctPcsaDesc(); - - // HLL - (*fns)[make_pair(TAggregationOp::HLL, TYPE_NULL)] = NullDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_BOOLEAN)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_TINYINT)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_SMALLINT)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_INT)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_BIGINT)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_FLOAT)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_DOUBLE)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_TIMESTAMP)] = HllDesc(); - (*fns)[make_pair(TAggregationOp::HLL, TYPE_STRING)] = HllDesc(); -} - -} - diff --git a/be/src/exprs/opcode-registry.h b/be/src/exprs/opcode-registry.h deleted file mode 100644 index a13b84ad5..000000000 --- a/be/src/exprs/opcode-registry.h +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2012 Cloudera Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - - -#ifndef IMPALA_EXPRS_OPCODE_REGISTRY_H -#define IMPALA_EXPRS_OPCODE_REGISTRY_H - -#include -#include -#include -#include - -#include "common/logging.h" -#include "exprs/expr.h" // For ComputeFn typedef - -#include "gen-cpp/Opcodes_types.h" -#include "gen-cpp/PlanNodes_types.h" - -namespace impala { - -class Expr; -class TupleRow; - -// TODO: use this as a function cache mechanism for UDFs/UDAs -class OpcodeRegistry { - public: - // Struct that contains function ptrs for each phase of the aggregation as - // defined by the UDA interface. - // Nulls indicates that the function does not implement that phase. - // These are not typedefed, since the actual signature is variable. - // TODO: for cross-compiled IR aggregate functions, we'll either add the - // symbol name (string) or the llvm::Function*. - struct AggFnDescriptor { - void* init_fn; - void* update_fn; - void* merge_fn; - void* serialize_fn; - void* finalize_fn; - - AggFnDescriptor(void* init = NULL, void* update = NULL, void* merge = NULL, - void* serialize = NULL, void* finalize = NULL) - : init_fn(init), update_fn(update), merge_fn(merge), serialize_fn(serialize), - finalize_fn(finalize) { - } - }; - - // Returns the function ptr for this opcode. - void* GetFunctionPtr(TExprOpcode::type opcode) { - int index = static_cast(opcode); - DCHECK_GE(index, 0); - DCHECK_LT(index, scalar_builtins_.size()); - return scalar_builtins_[index]; - } - - // Returns the builtin function ptrs for an aggregate function. - const AggFnDescriptor* GetBuiltinAggFnDescriptor( - const std::pair& arg_type) const { - AggregateBuiltins::const_iterator it = aggregate_builtins_.find(arg_type); - if (it == aggregate_builtins_.end()) return NULL; - return &it->second; - } - - // Returns the function symbol for this opcode (used for loading IR functions). - const std::string& GetFunctionSymbol(TExprOpcode::type opcode) { - int index = static_cast(opcode); - DCHECK_GE(index, 0); - DCHECK_LT(index, symbols_.size()); - return symbols_[index]; - } - - // Registry is a singleton - static OpcodeRegistry* Instance() { - if (instance_ == NULL) { - boost::lock_guard l(instance_lock_); - if (instance_ == NULL) { - // Make sure not to assign instance_ (and make it visible to other threads) - // until it is fully initialized. Note the fast path does not lock. - instance_ = new OpcodeRegistry(); - } - } - return instance_; - } - - // Mapping of builtin aggregate function op and input type to function desc. - typedef boost::unordered_map< - std::pair, AggFnDescriptor> AggregateBuiltins; - - private: - // Private ctor. Singleton interface. - OpcodeRegistry(); - - // Populates all of the registered functions. Implemented in - // opcode-registry-init.cc which is an auto-generated file - void Init(); - - void Add(TExprOpcode::type opcode, void* fn, const char* symbol) { - int index = static_cast(opcode); - DCHECK_LT(index, scalar_builtins_.size()); - DCHECK_GE(index, 0); - scalar_builtins_[index] = fn; - symbols_[index] = symbol; - } - - static OpcodeRegistry* instance_; - static boost::mutex instance_lock_; - std::vector scalar_builtins_; - std::vector symbols_; - AggregateBuiltins aggregate_builtins_; -}; - -} - -#endif - diff --git a/be/src/exprs/timestamp-functions.cc b/be/src/exprs/timestamp-functions.cc index b53cc1a32..baf98684f 100644 --- a/be/src/exprs/timestamp-functions.cc +++ b/be/src/exprs/timestamp-functions.cc @@ -50,7 +50,7 @@ template void* TimestampFunctions::FromUnix(Expr* e, TupleRow* row) { DCHECK_LE(e->GetNumChildren(), 2); DCHECK_NE(e->GetNumChildren(), 0); - + Expr* op = e->children()[0]; TIME* intp = reinterpret_cast(op->GetValue(row)); if (intp == NULL) return NULL; @@ -125,6 +125,19 @@ void* TimestampFunctions::Unix(Expr* e, TupleRow* row) { return &e->result_.int_val; } +void* TimestampFunctions::UnixFromString(Expr* e, TupleRow* row) { + DCHECK_EQ(e->GetNumChildren(), 1); + Expr* op = e->children()[0]; + StringValue* sv = reinterpret_cast(op->GetValue(row)); + if (sv == NULL) return NULL; + TimestampValue tv(sv->ptr, sv->len); + if (tv.date().is_special()) return NULL; + ptime temp; + tv.ToPtime(&temp); + e->result_.int_val = static_cast(to_time_t(temp)); + return &e->result_.int_val; +} + void TimestampFunctions::ReportBadFormat(StringValue* format) { LOG(WARNING) << "Bad date/time conversion format: " << format->DebugString(); } diff --git a/be/src/exprs/timestamp-functions.h b/be/src/exprs/timestamp-functions.h index ed419b127..cdfaf01ba 100644 --- a/be/src/exprs/timestamp-functions.h +++ b/be/src/exprs/timestamp-functions.h @@ -32,10 +32,11 @@ class TupleRow; class TimestampFunctions { public: // Return the unix time_t, seconds from 1970 - // With 0 argments, returns the current time. - // With 1 arument, converts it to a unix time_t - // With 2 aruments, the second argument is the format of the timestamp string. + // With 0 arguments, returns the current time. + // With 1 argument, converts it to a unix time_t + // With 2 arguments, the second argument is the format of the timestamp string. static void* Unix(Expr* e, TupleRow* row); + static void* UnixFromString(Expr* e, TupleRow* row); // Return a timestamp string from a unix time_t // Optional second argument is the format of the string. diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc index 84011d241..f9382387d 100644 --- a/be/src/runtime/exec-env.cc +++ b/be/src/runtime/exec-env.cc @@ -205,6 +205,7 @@ ExecEnv::~ExecEnv() { Status ExecEnv::InitForFeTests() { mem_tracker_.reset(new MemTracker(-1, "Process")); + RETURN_IF_ERROR(lib_cache_->Init(true)); is_fe_tests_ = true; return Status::OK; } @@ -295,6 +296,8 @@ Status ExecEnv::StartServices() { } } + RETURN_IF_ERROR(lib_cache_->Init()); + return Status::OK; } diff --git a/be/src/runtime/lib-cache.cc b/be/src/runtime/lib-cache.cc index e8d83c915..572294bff 100644 --- a/be/src/runtime/lib-cache.cc +++ b/be/src/runtime/lib-cache.cc @@ -24,6 +24,7 @@ #include "util/dynamic-util.h" #include "util/hash-util.h" #include "util/hdfs-util.h" +#include "util/path-builder.h" using namespace boost; using namespace std; @@ -32,8 +33,27 @@ using namespace impala; DEFINE_string(local_library_dir, "/tmp", "Local directory to copy UDF libraries from HDFS into"); +LibCache::LibCache() :current_process_handle_(NULL) { +} + LibCache::~LibCache() { DropCache(); + if (current_process_handle_ != NULL) dlclose(current_process_handle_); +} + +Status LibCache::Init(bool is_fe_tests) { + if (is_fe_tests) { + // In the FE tests, NULL gives the handle to the java process. + // Explicitly load the fe-support shared object. + string fe_support_path; + PathBuilder::GetFullBuildPath("service/libfesupport.so", &fe_support_path); + RETURN_IF_ERROR(DynamicOpen(fe_support_path.c_str(), ¤t_process_handle_)); + } else { + RETURN_IF_ERROR(DynamicOpen(NULL, ¤t_process_handle_)); + } + DCHECK(current_process_handle_ != NULL) + << "We should always be able to get current process handle."; + return Status::OK; } LibCache::LibCacheEntry::~LibCacheEntry() { @@ -48,6 +68,13 @@ LibCache::LibCacheEntry::~LibCacheEntry() { Status LibCache::GetSoFunctionPtr(HdfsFsCache* hdfs_cache, const string& hdfs_lib_file, const string& symbol, void** fn_ptr) { + if (hdfs_lib_file.empty()) { + // Just loading a function ptr in the current process. No need to take any locks. + DCHECK(current_process_handle_ != NULL); + RETURN_IF_ERROR(DynamicLookup(current_process_handle_, symbol.c_str(), fn_ptr)); + return Status::OK; + } + unique_lock lock; LibCacheEntry* entry = NULL; RETURN_IF_ERROR(GetCacheEntry(hdfs_cache, hdfs_lib_file, TYPE_SO, &lock, &entry)); @@ -194,7 +221,8 @@ Status LibCache::GetCacheEntry(HdfsFsCache* hdfs_cache, const string& hdfs_lib_f RETURN_IF_ERROR((*entry)->copy_file_status); if (type == TYPE_SO) { // dlopen the local library - RETURN_IF_ERROR(DynamicOpen((*entry)->local_path, &(*entry)->shared_object_handle)); + RETURN_IF_ERROR( + DynamicOpen((*entry)->local_path.c_str(), &(*entry)->shared_object_handle)); } else if (type == TYPE_IR) { // Load the module and populate all symbols. ObjectPool pool; diff --git a/be/src/runtime/lib-cache.h b/be/src/runtime/lib-cache.h index 81b2165cc..37781e8a8 100644 --- a/be/src/runtime/lib-cache.h +++ b/be/src/runtime/lib-cache.h @@ -53,9 +53,14 @@ class LibCache { TYPE_JAR, // Java jar file. We don't care about the contents in the BE. }; + LibCache(); + // Calls dlclose on all cached handles. ~LibCache(); + // Initializes the libcache. Must be called before any other APIs. + Status Init(bool is_fe_tests = false); + // Gets the local file system path for the library at 'hdfs_lib_file'. If // this file is not already on the local fs, it copies it and caches the // result. Returns an error if 'hdfs_lib_file' cannot be copied to the local fs. @@ -72,9 +77,10 @@ class LibCache { Status CheckSymbolExists(HdfsFsCache* hdfs_cache, const std::string& hdfs_lib_file, LibType type, const std::string& symbol); - // Returns a pointer to the function for the given library and symbol. 'hdfs_lib_file' - // should be the HDFS path to a shared library (.so) file and 'symbol' should be a - // symbol within that library. dlopen handles and symbols are cached. + // Returns a pointer to the function for the given library and symbol. + // If 'hdfs_lib_file' is empty, the symbol is looked up in the impalad process. + // Otherwise, 'hdfs_lib_file' should be the HDFS path to a shared library (.so) file. + // dlopen handles and symbols are cached. // Only usable if 'hdfs_lib_file' refers to a shared object. Status GetSoFunctionPtr(HdfsFsCache* hdfs_cache, const std::string& hdfs_lib_file, const std::string& symbol, void** fn_ptr); @@ -86,6 +92,9 @@ class LibCache { void DropCache(); private: + // dlopen() handle for the current process (i.e. impalad). + void* current_process_handle_; + // Protects lib_cache_. For lock ordering, this lock must always be taken before // the per entry lock. boost::mutex lock_; diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc index 012fda273..4004b9fe8 100644 --- a/be/src/service/frontend.cc +++ b/be/src/service/frontend.cc @@ -62,6 +62,7 @@ Frontend::Frontend() { {"getFunctions", "([B)[B", &get_functions_id_}, {"getCatalogObject", "([B)[B", &get_catalog_object_id_}, {"execHiveServer2MetadataOp", "([B)[B", &exec_hs2_metadata_op_id_}, + {"setCatalogInitialized", "()V", &set_catalog_initialized_id_}, {"loadTableData", "([B)[B", &load_table_data_id_}}; JNIEnv* jni_env = getJNIEnv(); @@ -200,3 +201,12 @@ Status Frontend::LoadData(const TLoadDataReq& request, TLoadDataResp* response) bool Frontend::IsAuthorizationError(const Status& status) { return !status.ok() && status.GetErrorMsg().find("AuthorizationException") == 0; } + +Status Frontend::SetCatalogInitialized() { + JNIEnv* jni_env = getJNIEnv(); + JniLocalFrame jni_frame; + RETURN_IF_ERROR(jni_frame.push(jni_env)); + jni_env->CallObjectMethod(fe_, set_catalog_initialized_id_); + RETURN_ERROR_IF_EXC(jni_env); + return Status::OK; +} diff --git a/be/src/service/frontend.h b/be/src/service/frontend.h index d05dd564b..0aaf61e70 100644 --- a/be/src/service/frontend.h +++ b/be/src/service/frontend.h @@ -126,6 +126,12 @@ class Frontend { // Returns true if the error returned by the FE was due to an AuthorizationException. static bool IsAuthorizationError(const Status& status); + // Sets the FE catalog to be initialized. This is only used for testing in + // conjunction with InProcessImpalaServer. This sets the FE catalog to + // be initialized, ready to receive queries without needing a catalog + // server. + Status SetCatalogInitialized(); + private: // Descriptor of Java Frontend class itself, used to create a new instance. jclass fe_class_; @@ -145,6 +151,7 @@ class Frontend { jmethodID get_catalog_object_id_; // JniFrontend.getCatalogObject jmethodID exec_hs2_metadata_op_id_; // JniFrontend.execHiveServer2MetadataOp jmethodID load_table_data_id_; // JniFrontend.loadTableData + jmethodID set_catalog_initialized_id_; // JniFrontend.setCatalogInitialized jmethodID fe_ctor_; }; diff --git a/be/src/testutil/in-process-servers.cc b/be/src/testutil/in-process-servers.cc index 77bceccf8..6f6e1b60b 100644 --- a/be/src/testutil/in-process-servers.cc +++ b/be/src/testutil/in-process-servers.cc @@ -40,6 +40,11 @@ InProcessImpalaServer::InProcessImpalaServer(const string& hostname, int backend statestore_host, statestore_port)) { } +void InProcessImpalaServer::SetCatalogInitialized() { + DCHECK(impala_server_ != NULL) << "Call Start*() first."; + impala_server_->frontend()->SetCatalogInitialized(); +} + Status InProcessImpalaServer::StartWithClientServers(int beeswax_port, int hs2_port, bool use_statestore) { RETURN_IF_ERROR(exec_env_->StartServices()); diff --git a/be/src/testutil/in-process-servers.h b/be/src/testutil/in-process-servers.h index a4cae7658..02221e67f 100644 --- a/be/src/testutil/in-process-servers.h +++ b/be/src/testutil/in-process-servers.h @@ -61,6 +61,10 @@ class InProcessImpalaServer { Metrics* metrics() { return exec_env_->metrics(); } + // Sets the catalog on this impalad to be initialized. If we don't + // start up a catalogd, then there is no one to initialize it otherwise. + void SetCatalogInitialized(); + private: // Hostname for this server, usually FLAGS_hostname const std::string hostname_; diff --git a/be/src/util/debug-util.cc b/be/src/util/debug-util.cc index 603c3913b..55c7fcd32 100644 --- a/be/src/util/debug-util.cc +++ b/be/src/util/debug-util.cc @@ -28,7 +28,6 @@ #include "runtime/row-batch.h" #include "util/cpu-info.h" #include "util/string-parser.h" -#include "gen-cpp/Opcodes_types.h" // For DumpStackTraceToString #include @@ -74,8 +73,6 @@ namespace impala { return ss.str();\ } -THRIFT_ENUM_OUTPUT_FN(TExprOpcode); -THRIFT_ENUM_OUTPUT_FN(TAggregationOp); THRIFT_ENUM_OUTPUT_FN(TFunctionBinaryType); THRIFT_ENUM_OUTPUT_FN(TCatalogObjectType); THRIFT_ENUM_OUTPUT_FN(TDdlType); diff --git a/be/src/util/debug-util.h b/be/src/util/debug-util.h index b059c4539..02c625495 100644 --- a/be/src/util/debug-util.h +++ b/be/src/util/debug-util.h @@ -24,7 +24,6 @@ #include "gen-cpp/Descriptors_types.h" #include "gen-cpp/Exprs_types.h" #include "gen-cpp/Frontend_types.h" -#include "gen-cpp/Opcodes_types.h" #include "gen-cpp/PlanNodes_types.h" #include "gen-cpp/RuntimeProfile_types.h" #include "gen-cpp/ImpalaService_types.h" @@ -38,8 +37,6 @@ class Tuple; class TupleRow; class RowBatch; -std::ostream& operator<<(std::ostream& os, const TExprOpcode::type& op); -std::ostream& operator<<(std::ostream& os, const TAggregationOp::type& op); std::ostream& operator<<(std::ostream& os, const TFunctionBinaryType::type& op); std::ostream& operator<<(std::ostream& os, const TUniqueId& id); std::ostream& operator<<(std::ostream& os, const THdfsFileFormat::type& type); diff --git a/be/src/util/dynamic-util.cc b/be/src/util/dynamic-util.cc index fb3089e07..318ec9891 100644 --- a/be/src/util/dynamic-util.cc +++ b/be/src/util/dynamic-util.cc @@ -27,13 +27,13 @@ Status DynamicLookup(void* handle, const char* symbol, void** fn_ptr) { char* error = dlerror(); if (error != NULL) { stringstream ss; - ss << "Unable to find " << symbol << " dlerror: " << error; + ss << "Unable to find " << symbol << "\ndlerror: " << error; return Status(ss.str()); } return Status::OK; } -Status DynamicOpen(const string& library, void** handle) { +Status DynamicOpen(const char* library, void** handle) { int flags = RTLD_NOW; // If we are loading shared libraries from the FE tests, where the Java // side loads the initial impala binary (libfesupport.so), we are unable @@ -45,10 +45,10 @@ Status DynamicOpen(const string& library, void** handle) { // the symbols (e.g. planner tests with some UDFs). // TODO: this is to work around some build breaks. We need to fix this better. if (ExecEnv::GetInstance()->is_fe_tests()) flags = RTLD_LAZY; - *handle = dlopen(library.c_str(), flags); + *handle = dlopen(library, flags); if (*handle == NULL) { stringstream ss; - ss << "Unable to load " << library << " dlerror: " << dlerror(); + ss << "Unable to load " << library << "\ndlerror: " << dlerror(); return Status(ss.str()); } return Status::OK; diff --git a/be/src/util/dynamic-util.h b/be/src/util/dynamic-util.h index 99be91fa2..6b0fc631c 100644 --- a/be/src/util/dynamic-util.h +++ b/be/src/util/dynamic-util.h @@ -20,15 +20,16 @@ namespace impala { // Look up smybols in a dynamically linked library. -// handle -- handle to the library. +// handle -- handle to the library. NULL if loading from the current process. // symbol -- symbol to lookup. // fn_ptr -- pointer tor retun addres of function. Status DynamicLookup(void* handle, const char* symbol, void** fn_ptr); // Open a dynamicly loaded library. // library -- name of the library. The default paths will be searched. +// library can be NULL to get the handle for the current process. // handle -- returned handle to the library. -Status DynamicOpen(const std::string& library, void** handle); +Status DynamicOpen(const char* library, void** handle); } diff --git a/common/function-registry/CMakeLists.txt b/common/function-registry/CMakeLists.txt index 7c01df293..d7c511ed5 100644 --- a/common/function-registry/CMakeLists.txt +++ b/common/function-registry/CMakeLists.txt @@ -22,26 +22,23 @@ set(FE_OUTPUT_DIR ${CMAKE_SOURCE_DIR}/fe/generated-sources/gen-java/com/cloudera set(CODE_GEN_OUTPUT ${BE_OUTPUT_DIR}/opcode/functions.cc ${BE_OUTPUT_DIR}/opcode/functions.h - ${BE_OUTPUT_DIR}/opcode/opcode-registry-init.cc - ${FE_OUTPUT_DIR}/opcode/FunctionOperator.java - ${FE_OUTPUT_DIR}/opcode/FunctionRegistry.java - ${CMAKE_CURRENT_SOURCE_DIR}/../thrift/Opcodes.thrift + ${FE_OUTPUT_DIR}/builtins/ScalarBuiltins.java ) # Source python files set(FUNCTION_REGISTRY_INPUT - gen_functions.py - gen_opcodes.py + gen_operators.py + gen_builtins_catalog.py impala_functions.py ) # Run the python scripts add_custom_command( OUTPUT ${CODE_GEN_OUTPUT} - COMMAND ./gen_functions.py - COMMAND ./gen_opcodes.py + COMMAND ./gen_operators.py + COMMAND ./gen_builtins_catalog.py DEPENDS ${FUNCTION_REGISTRY_INPUT} - COMMENT "Generating Opcode Registry files." + COMMENT "Generating files for builtins." VERBATIM ) diff --git a/common/function-registry/gen_builtins_catalog.py b/common/function-registry/gen_builtins_catalog.py new file mode 100755 index 000000000..2dc0c315e --- /dev/null +++ b/common/function-registry/gen_builtins_catalog.py @@ -0,0 +1,123 @@ +#!/usr/bin/env python +# Copyright 2012 Cloudera Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# This script generates the FE calls to populate the builtins. +# To add a builtin, add an entry to impala_functions.py. + +import sys +import os +from string import Template +import impala_functions + +java_registry_preamble = '\ +// Copyright 2012 Cloudera Inc.\n\ +// \n\ +// Licensed under the Apache License, Version 2.0 (the "License");\n\ +// you may not use this file except in compliance with the License.\n\ +// You may obtain a copy of the License at\n\ +// \n\ +// http://www.apache.org/licenses/LICENSE-2.0\n\ +// \n\ +// Unless required by applicable law or agreed to in writing, software\n\ +// distributed under the License is distributed on an "AS IS" BASIS,\n\ +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n\ +// See the License for the specific language governing permissions and\n\ +// limitations under the License.\n\ +\n\ +// This is a generated file, DO NOT EDIT.\n\ +// To add new functions, see impala/common/function-registry/gen_opcodes.py\n\ +\n\ +package com.cloudera.impala.builtins;\n\ +\n\ +import com.cloudera.impala.analysis.ColumnType;\n\ +import com.cloudera.impala.catalog.Db;\n\ +\n\ +public class ScalarBuiltins { \n\ + public static void initBuiltins(Db db) { \ +\n' + +java_registry_epilogue = '\ + }\n\ +}\n' + +FE_PATH = "../../fe/generated-sources/gen-java/com/cloudera/impala/builtins/" + +# This contains all the metadata to describe all the builtins. +# Each meta data entry is itself a map to store all the meta data +# - fn_name, ret_type, args, symbol, sql_names +meta_data_entries = [] + +# Read in the function and add it to the meta_data_entries map +def add_function(fn_meta_data, udf_interface): + entry = {} + entry["sql_names"] = fn_meta_data[0] + entry["ret_type"] = fn_meta_data[1] + entry["args"] = fn_meta_data[2] + entry["symbol"] = fn_meta_data[3] + entry["udf_interface"] = udf_interface + meta_data_entries.append(entry) + +def generate_fe_entry(entry, name): + java_output = "" + if entry["udf_interface"]: + java_output += "true" + else: + java_output += "false" + java_output += ", \"" + name + "\"" + java_output += ", \"" + entry["symbol"] + "\"" + + # Check the last entry for varargs indicator. + if entry["args"] and entry["args"][-1] == "...": + entry["args"].pop() + java_output += ", true" + else: + java_output += ", false" + + java_output += ", ColumnType." + entry["ret_type"] + for arg in entry["args"]: + java_output += ", ColumnType." + arg + return java_output + +# Generates the FE builtins init file that registers all the builtins. +def generate_fe_registry_init(filename): + java_registry_file = open(filename, "w") + java_registry_file.write(java_registry_preamble) + + for entry in meta_data_entries: + for name in entry["sql_names"]: + java_output = generate_fe_entry(entry, name) + java_registry_file.write(" db.addScalarBuiltin(%s);\n" % java_output) + + java_registry_file.write("\n") + java_registry_file.write(java_registry_epilogue) + java_registry_file.close() + +# Read the function metadata inputs +for function in impala_functions.functions: + if len(function) != 4: + print "Invalid function entry in impala_functions.py:\n\t" + repr(function) + sys.exit(1) + add_function(function, False) + +for function in impala_functions.udf_functions: + assert len(function) == 4, \ + "Invalid function entry in impala_functions.py:\n\t" + repr(function) + add_function(function, True) + +if not os.path.exists(FE_PATH): + os.makedirs(FE_PATH) + +generate_fe_registry_init(FE_PATH + "ScalarBuiltins.java") + diff --git a/common/function-registry/gen_opcodes.py b/common/function-registry/gen_opcodes.py deleted file mode 100755 index 4a89d1c76..000000000 --- a/common/function-registry/gen_opcodes.py +++ /dev/null @@ -1,383 +0,0 @@ -#!/usr/bin/env python -# Copyright 2012 Cloudera Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# This script generates the necessary files to coordinate function calls between the FE -# and BE. In the FE, this creates a mapping between function signature (Operation & -# Arguments) to an opcode. The opcode is a thrift enum which is passed to the backend. -# The backend has all the information from just the opcode and does not need to worry -# about type checking. -# -# This scripts pulls function metadata input from -# - impala/common/function-registry/impala_functions.py (manually maintained) -# - impala/common/function-registry/generated_functions.py (auto-generated metadata) -# -# This script will generate 4 outputs -# 1. Thrift enum for all the opcodes -# - impala/fe/src/thrift/Opcodes.thrift -# 2. FE java operators (one per function, ignoring overloading) -# - impala/fe/generated-sources/gen-java/com/cloudera/impala/opcode/FunctionOperater.java -# 3 Java registry setup (registering all the functions with signatures) -# - impala/fe/generated-sources/gen-java/com/cloudera/impala/opcode/FunctionRegistry.java -# 4. BE registry setup (mapping opcodes to ComputeFunctions) -# - impala/be/generated-sources/opcode/opcode-registry-init.cc -# -# TODO: version the registry on the FE and BE so we can identify if they are out of sync - -import sys -import os -from string import Template -import impala_functions -import generated_functions - -native_types = { - 'BOOLEAN' : 'bool', - 'TINYINT' : 'char', - 'SMALLINT' : 'short', - 'INT' : 'int', - 'BIGINT' : 'long', - 'FLOAT' : 'float', - 'DOUBLE' : 'double', - 'STRING' : 'StringValue', - 'TIMESTAMP' : 'TimestampValue', -} - -thrift_preamble = '\ -// Copyright 2012 Cloudera Inc.\n\ -//\n\ -// Licensed under the Apache License, Version 2.0 (the "License");\n\ -// you may not use this file except in compliance with the License.\n\ -// You may obtain a copy of the License at\n\ -//\n\ -// http://www.apache.org/licenses/LICENSE-2.0\n\ -//\n\ -// Unless required by applicable law or agreed to in writing, software\n\ -// distributed under the License is distributed on an "AS IS" BASIS,\n\ -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n\ -// See the License for the specific language governing permissions and\n\ -// limitations under the License.\n\ -\n\ -// This is a generated file, DO NOT EDIT.\n\ -// To add new functions, see impala/common/function-registry/gen_opcodes.py\n\ -\n\ -namespace cpp impala\n\ -namespace java com.cloudera.impala.thrift\n\ -\n\ -enum TExprOpcode {\n' - -thrift_epilogue = '\ -}\n\ -\n' - -cc_registry_preamble = '\ -// Copyright 2012 Cloudera Inc.\n\ -//\n\ -// Licensed under the Apache License, Version 2.0 (the "License");\n\ -// you may not use this file except in compliance with the License.\n\ -// You may obtain a copy of the License at\n\ -//\n\ -// http://www.apache.org/licenses/LICENSE-2.0\n\ -//\n\ -// Unless required by applicable law or agreed to in writing, software\n\ -// distributed under the License is distributed on an "AS IS" BASIS,\n\ -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n\ -// See the License for the specific language governing permissions and\n\ -// limitations under the License.\n\ -\n\ -// This is a generated file, DO NOT EDIT.\n\ -// To add new functions, see impala/common/function-registry/gen_opcodes.py\n\ -\n\ -#include "exprs/opcode-registry.h"\n\ -#include "exprs/expr.h"\n\ -#include "exprs/compound-predicate.h"\n\ -#include "exprs/like-predicate.h"\n\ -#include "exprs/math-functions.h"\n\ -#include "exprs/string-functions.h"\n\ -#include "exprs/timestamp-functions.h"\n\ -#include "exprs/conditional-functions.h"\n\ -#include "exprs/udf-builtins.h"\n\ -#include "exprs/utility-functions.h"\n\ -#include "opcode/functions.h"\n\ -\n\ -using namespace boost::posix_time;\n\ -using namespace boost::gregorian;\n\ -\n\ -namespace impala {\n\ -\n\ -void OpcodeRegistry::Init() {\n' - -cc_registry_epilogue = '\ -}\n\ -\n\ -}\n' - -operator_file_preamble = '\ -// Copyright 2012 Cloudera Inc.\n\ -// \n\ -// Licensed under the Apache License, Version 2.0 (the "License");\n\ -// you may not use this file except in compliance with the License.\n\ -// You may obtain a copy of the License at\n\ -// \n\ -// http://www.apache.org/licenses/LICENSE-2.0\n\ -// \n\ -// Unless required by applicable law or agreed to in writing, software\n\ -// distributed under the License is distributed on an "AS IS" BASIS,\n\ -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n\ -// See the License for the specific language governing permissions and\n\ -// limitations under the License.\n\ -\n\ -// This is a generated file, DO NOT EDIT.\n\ -// To add new functions, see impala/common/function-registry/gen_opcodes.py\n\ -\n\ -package com.cloudera.impala.opcode;\n\ -\n\ -public enum FunctionOperator {\n' - -operator_file_epilogue = '\ -}\n' - -java_registry_preamble = '\ -// Copyright 2012 Cloudera Inc.\n\ -// \n\ -// Licensed under the Apache License, Version 2.0 (the "License");\n\ -// you may not use this file except in compliance with the License.\n\ -// You may obtain a copy of the License at\n\ -// \n\ -// http://www.apache.org/licenses/LICENSE-2.0\n\ -// \n\ -// Unless required by applicable law or agreed to in writing, software\n\ -// distributed under the License is distributed on an "AS IS" BASIS,\n\ -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n\ -// See the License for the specific language governing permissions and\n\ -// limitations under the License.\n\ -\n\ -// This is a generated file, DO NOT EDIT.\n\ -// To add new functions, see impala/common/function-registry/gen_opcodes.py\n\ -\n\ -package com.cloudera.impala.opcode;\n\ -\n\ -import com.cloudera.impala.analysis.OpcodeRegistry;\n\ -import com.cloudera.impala.analysis.ColumnType;\n\ -import com.cloudera.impala.thrift.TExprOpcode;\n\ -import com.google.common.base.Preconditions;\n\ -\n\ -public class FunctionRegistry { \n\ - public static void InitFunctions(OpcodeRegistry registry) { \n\ - boolean result = true;\n\ -\n' - -java_registry_epilogue = '\ - Preconditions.checkState(result); \n\ - }\n\ -}\n' - - -def initialize_sub(op, return_type, arg_types): - sub = {} - java_args = "ColumnType." + return_type - sub["fn_class"] = "GetValueFunctions" - sub["fn_signature"] = op - sub["num_args"] = len(arg_types) - for idx in range(0, len(arg_types)): - arg = arg_types[idx] - sub["fn_signature"] += "_" + native_types[arg] - sub["native_type" + repr(idx + 1)] = native_types[arg] - java_args += ", ColumnType." + arg - sub["thrift_enum"] = sub["fn_signature"].upper() - sub["java_output"] = "FunctionOperator." + op.upper() + ", TExprOpcode." + sub["thrift_enum"] - sub["java_output"] += ", " + java_args - return sub - -FE_PATH = "../../fe/generated-sources/gen-java/com/cloudera/impala/opcode/" -BE_PATH = "../../be/generated-sources/opcode/" -THRIFT_PATH = "../thrift/" - -# This contains a list of all the opcodes that are built base on the -# function name from the input. Inputs can have multiple signatures -# with the same function name and the opcode is mangled using the -# arg types. -opcodes = [] - -# This contains a list of all the function names (no overloading/mangling) -operators = [] - -# This is a mapping of operators to a list of function meta data entries -# Each meta data entry is itself a map to store all the meta data -# - fn_name, ret_type, args, be_fn, sql_names -meta_data_entries = {} - -# Read in the function and add it to the meta_data_entries map -def add_function(fn_meta_data, udf_interface): - fn_name = fn_meta_data[0] - - entry = {} - entry["fn_name"] = fn_name - entry["ret_type"] = fn_meta_data[1] - entry["args"] = fn_meta_data[2] - entry["be_fn"] = fn_meta_data[3] - entry["sql_names"] = fn_meta_data[4] - if udf_interface: - entry["symbol"] = fn_meta_data[5] - else: - entry["symbol"] = "" - entry["udf_interface"] = udf_interface - - if fn_name in meta_data_entries: - meta_data_entries[fn_name].append(entry) - else: - fn_list = [entry] - meta_data_entries[fn_name] = fn_list - operators.append(fn_name.upper()) - -# Iterate over entries in the meta_data_entries map and generate opcodes. Some -# entries will have the same name at this stage, quality the name withe the -# signature to generate unique enums. -# Resulting opcode list is sorted with INVALID_OPCODE at beginning and LAST_OPCODE -# at end. -def generate_opcodes(): - for fn in meta_data_entries: - entries = meta_data_entries[fn] - if len(entries) > 1: - for entry in entries: - opcode = fn.upper() - for arg in entry["args"]: - if arg == "...": - opcode += "_" + 'VARARGS' - else: - opcode += "_" + native_types[arg].upper() - opcodes.append(opcode) - entry["opcode"] = opcode - else: - opcodes.append(fn.upper()) - entries[0]["opcode"] = fn.upper() - opcodes.sort() - opcodes.insert(0, 'INVALID_OPCODE') - opcodes.append('LAST_OPCODE') - -# Generates the BE registry init file that will add all the compute functions -# to the registry. Outputs the generated-file to 'filename' -def generate_be_registry_init(filename): - cc_registry_file = open(filename, "w") - cc_registry_file.write(cc_registry_preamble) - - for fn in meta_data_entries: - entries = meta_data_entries[fn] - for entry in entries: - opcode = entry["opcode"] - be_fn = entry["be_fn"] - symbol = entry["symbol"] - # We generate two casts to work around GCC Bug 11407 - cc_output = 'TExprOpcode::%s, (void*)(Expr::ComputeFn)%s, "%s"' \ - % (opcode, be_fn, symbol) - cc_registry_file.write(" this->Add(%s);\n" % (cc_output)) - - cc_registry_file.write(cc_registry_epilogue) - cc_registry_file.close() - -# Generates the FE registry init file that registers all the functions. This file -# contains all the opcode->function signature mappings and all of the string->operator -# mappings for sql functions -def generate_fe_registry_init(filename): - java_registry_file = open(filename, "w") - java_registry_file.write(java_registry_preamble) - - for fn in meta_data_entries: - entries = meta_data_entries[fn] - for entry in entries: - java_output = "" - if entry["udf_interface"]: - java_output += "true" - else: - java_output += "false" - java_output += ", FunctionOperator." + fn.upper() - java_output += ", TExprOpcode." + entry["opcode"] - # Check the last entry for varargs indicator. - if entry["args"] and entry["args"][-1] == "...": - entry["args"].pop() - java_output += ", true" - else: - java_output += ", false" - java_output += ", ColumnType." + entry["ret_type"] - for arg in entry["args"]: - java_output += ", ColumnType." + arg - java_registry_file.write(" result &= registry.add(%s);\n" % java_output) - java_registry_file.write("\n") - - mappings = {} - - for fn in meta_data_entries: - entries = meta_data_entries[fn] - for entry in entries: - for name in entry["sql_names"]: - if name in mappings: - if mappings[name] != fn.upper(): - print "Invalid mapping \"%s\" -> FunctionOperator.%s." % (name, mappings[name]) - print "There is already a mapping \"%s\" -> FunctionOperator.%s.\n" % (name, fn.upper()) - sys.exit(1) - continue - mappings[name] = fn.upper() - java_output = "\"%s\", FunctionOperator.%s" % (name, fn.upper()) - java_registry_file.write(" result &= registry.addFunctionMapping(%s);\n" % java_output) - java_registry_file.write("\n") - - java_registry_file.write(java_registry_epilogue) - java_registry_file.close() - -# Read the function metadata inputs -for function in impala_functions.functions: - if len(function) != 5: - print "Invalid function entry in impala_functions.py:\n\t" + repr(function) - sys.exit(1) - add_function(function, False) - -for function in impala_functions.udf_functions: - assert len(function) == 6, \ - "Invalid function entry in impala_functions.py:\n\t" + repr(function) - add_function(function, True) - -for function in generated_functions.functions: - if len(function) != 5: - print "Invalid function entry in generated_functions.py:\n\t" + repr(function) - sys.exit(1) - add_function(function, False) - -generate_opcodes() - -if not os.path.exists(BE_PATH): - os.makedirs(BE_PATH) -if not os.path.exists(FE_PATH): - os.makedirs(FE_PATH) -if not os.path.exists(THRIFT_PATH): - os.makedirs(THRIFT_PATH) - -generate_be_registry_init(BE_PATH + "opcode-registry-init.cc") -generate_fe_registry_init(FE_PATH + "FunctionRegistry.java") - -# Output the opcodes to thrift -thrift_file = open(THRIFT_PATH + "Opcodes.thrift", "w") -thrift_file.write(thrift_preamble) -for opcode in opcodes: - thrift_file.write(" %s,\n" % opcode) -thrift_file.write(thrift_epilogue) -thrift_file.close() - -# Output the operators to java -operators.sort() -operators.insert(0, "INVALID_OPERATOR") -operator_java_file = open(FE_PATH + "FunctionOperator.java", "w") -operator_java_file.write(operator_file_preamble) -for op in operators: - operator_java_file.write(" %s,\n" % op) -operator_java_file.write(operator_file_epilogue) -operator_java_file.close() diff --git a/common/function-registry/gen_functions.py b/common/function-registry/gen_operators.py similarity index 90% rename from common/function-registry/gen_functions.py rename to common/function-registry/gen_operators.py index 476c2bdb2..1160dcf0b 100755 --- a/common/function-registry/gen_functions.py +++ b/common/function-registry/gen_operators.py @@ -16,7 +16,7 @@ from string import Template import os -# This script will generate the implementation of the simple functions for the BE. +# This script will generate the implementation of the simple operators for the BE. # These include: # - Arithmetic functions # - Binary functions @@ -25,8 +25,6 @@ import os # The script outputs (run: 'impala/common/function-registry/gen_functions.py') # - header and implemention for above functions: # - impala/be/src/generated-sources/opcode/functions.[h/cc] -# - python file that contains the metadata for those functions: -# - impala/common/function-registry/generated_functions.py unary_op = Template("\ void* ComputeFunctions::${fn_signature}(Expr* e, TupleRow* row) {\n\ @@ -182,21 +180,18 @@ void* ComputeFunctions::${fn_signature}(Expr* e, TupleRow* row) {\n\ return NULL;\n\ }\n\n") -python_template = Template("\ - ['${fn_name}', '${return_type}', [${args}], 'ComputeFunctions::${fn_signature}', []], \n") - # Mapping of function to template templates = { 'Add' : binary_op, 'Subtract' : binary_op, 'Multiply' : binary_op, 'Divide' : binary_op, - 'Int_Divide' : binary_op_check_zero, + 'Int_divide' : binary_op_check_zero, 'Mod' : binary_op_check_zero, - 'BitAnd' : binary_op, - 'BitXor' : binary_op, - 'BitOr' : binary_op, - 'BitNot' : unary_op, + 'Bitand' : binary_op, + 'Bitxor' : binary_op, + 'Bitor' : binary_op, + 'Bitnot' : unary_op, 'Eq' : binary_op, 'Ne' : binary_op, 'Ge' : binary_op, @@ -234,12 +229,12 @@ functions = [ ['Subtract', ['NUMERIC_TYPES'], [['NUMERIC_TYPES'], ['NUMERIC_TYPES']] ], ['Multiply', ['NUMERIC_TYPES'], [['NUMERIC_TYPES'], ['NUMERIC_TYPES']] ], ['Divide', ['DOUBLE'], [['DOUBLE'], ['DOUBLE']] ], - ['Int_Divide', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], + ['Int_divide', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], ['Mod', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], - ['BitAnd', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], - ['BitXor', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], - ['BitOr', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], - ['BitNot', ['INT_TYPES'], [['INT_TYPES']] ], + ['Bitand', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], + ['Bitxor', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], + ['Bitor', ['INT_TYPES'], [['INT_TYPES'], ['INT_TYPES']] ], + ['Bitnot', ['INT_TYPES'], [['INT_TYPES']] ], # BinaryPredicates ['Eq', ['BOOLEAN'], [['NATIVE_TYPES'], ['NATIVE_TYPES']] ], @@ -422,29 +417,6 @@ h_epilogue = '\ \n\ #endif\n' -python_preamble = '\ -#!/usr/bin/env python\n\ -# Copyright 2012 Cloudera Inc.\n\ -#\n\ -# Licensed under the Apache License, Version 2.0 (the "License");\n\ -# you may not use this file except in compliance with the License.\n\ -# You may obtain a copy of the License at\n\ -#\n\ -# http://www.apache.org/licenses/LICENSE-2.0\n\ -#\n\ -# Unless required by applicable law or agreed to in writing, software\n\ -# distributed under the License is distributed on an "AS IS" BASIS,\n\ -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n\ -# See the License for the specific language governing permissions and\n\ -# limitations under the License.\n\ -\n\ -# This is a generated file, DO NOT EDIT IT.\n\ -# To add new functions, see impala/common/function-registry/gen_opcodes.py\n\ -\n\ -functions = [\n' - -python_epilogue = ']' - header_template = Template("\ static void* ${fn_signature}(Expr* e, TupleRow* row);\n") @@ -473,10 +445,8 @@ def initialize_sub(op, return_type, arg_types): if __name__ == "__main__": h_file = open(BE_PATH + 'functions.h', 'w') cc_file = open(BE_PATH + 'functions.cc', 'w') - python_file = open('generated_functions.py', 'w') h_file.write(h_preamble) cc_file.write(cc_preamble) - python_file.write(python_preamble) # Generate functions and headers for func_data in functions: @@ -543,11 +513,8 @@ if __name__ == "__main__": h_file.write(header_template.substitute(sub)) cc_file.write(template.substitute(sub)) - python_file.write(python_template.substitute(sub)) h_file.write(h_epilogue) cc_file.write(cc_epilogue) - python_file.write(python_epilogue) h_file.close() cc_file.close() - python_file.close() diff --git a/common/function-registry/impala_functions.py b/common/function-registry/impala_functions.py index 052edffdd..fb724233b 100755 --- a/common/function-registry/impala_functions.py +++ b/common/function-registry/impala_functions.py @@ -13,438 +13,432 @@ # See the License for the specific language governing permissions and # limitations under the License. - # This is a list of all the functions that are not auto-generated. -# It contains all the meta data that describes the function. The format is: -# , , [], , [] -# -# 'function name' is the base of what the opcode enum will be generated from. It does not -# have to be unique, the script will mangle the name with the signature if necessary. -# -# 'sql aliases' are the function names that can be used from sql. They are -# optional and there can be multiple aliases for a function. -# -# This is combined with the list in generated_functions to code-gen the opcode -# registry in the FE and BE. +# It contains all the meta data that describes the function. +templated_type_symbol_map = { + 'bool' : 'b', + 'int8_t' : 'a', + 'int16_t' : 's', + 'int32_t' : 'i', + 'int64_t' : 'l', + 'float' : 'f', + 'double' : 'd', + 'string' : 'NS_11StringValueE', + 'timestamp' : 'NS_14TimestampValueE' +} +# Generates the BE symbol for the Compute Function class_name::fn_name. +# Does not handle varargs. +# TODO: this is a stopgap. ComputeFunctions are being removed and we can use the +# symbol lookup code in the BE. +def symbol(class_name, fn_name, templated_type = None): + sym = '_ZN6impala' + sym += str(len(class_name)) + class_name + sym += str(len(fn_name)) + fn_name + if templated_type == None: + sym += 'EPNS_4ExprEPNS_8TupleRowE' + else: + sym += 'I' + sym += templated_type_symbol_map[templated_type] + sym += 'EEPvPNS_4ExprEPNS_8TupleRowE' + return sym + +# The format is: [sql aliases], , [], +# +# 'sql aliases' are the function names that can be used from sql. There must be +# at least one per function. +# +# The symbol can be empty for functions that are not yet implemented. functions = [ - ['Compound_And', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], - 'CompoundPredicate::AndComputeFn', []], - ['Compound_Or', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], - 'CompoundPredicate::OrComputeFn', []], - ['Compound_Not', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], - 'CompoundPredicate::NotComputeFn', []], + # Math builtin functions + [['pi'], 'DOUBLE', [], symbol('MathFunctions', 'Pi')], + [['e'], 'DOUBLE', [], symbol('MathFunctions', 'E')], + [['abs'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Abs')], + [['sign'], 'FLOAT', ['DOUBLE'], symbol('MathFunctions', 'Sign')], + [['sin'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Sin')], + [['asin'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Asin')], + [['cos'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Cos')], + [['acos'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Acos')], + [['tan'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Tan')], + [['atan'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Atan')], + [['radians'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Radians')], + [['degrees'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Degrees')], + [['ceil', 'ceiling'], 'BIGINT', ['DOUBLE'], symbol('MathFunctions', 'Ceil')], + [['floor'], 'BIGINT', ['DOUBLE'], symbol('MathFunctions', 'Floor')], + [['round'], 'BIGINT', ['DOUBLE'], symbol('MathFunctions', 'Round')], + [['round'], 'DOUBLE', ['DOUBLE', 'INT'], symbol('MathFunctions', 'RoundUpTo')], + [['exp'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Exp')], + [['ln'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Ln')], + [['log10'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Log10')], + [['log2'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Log2')], + [['log'], 'DOUBLE', ['DOUBLE', 'DOUBLE'], symbol('MathFunctions', 'Log')], + [['pow', 'power'], 'DOUBLE', ['DOUBLE', 'DOUBLE'], symbol('MathFunctions', 'Pow')], + [['sqrt'], 'DOUBLE', ['DOUBLE'], symbol('MathFunctions', 'Sqrt')], + [['rand'], 'DOUBLE', [], symbol('MathFunctions', 'Rand')], + [['rand'], 'DOUBLE', ['INT'], symbol('MathFunctions', 'RandSeed')], + [['bin'], 'STRING', ['BIGINT'], symbol('MathFunctions', 'Bin')], + [['hex'], 'STRING', ['BIGINT'], symbol('MathFunctions', 'HexInt')], + [['hex'], 'STRING', ['STRING'], symbol('MathFunctions', 'HexString')], + [['unhex'], 'STRING', ['STRING'], symbol('MathFunctions', 'Unhex')], + [['conv'], 'STRING', ['BIGINT', 'TINYINT', 'TINYINT'], + symbol('MathFunctions', 'ConvInt')], + [['conv'], 'STRING', ['STRING', 'TINYINT', 'TINYINT'], + symbol('MathFunctions', 'ConvString')], + [['pmod'], 'BIGINT', ['BIGINT', 'BIGINT'], symbol('MathFunctions', 'PmodBigInt')], + [['pmod'], 'DOUBLE', ['DOUBLE', 'DOUBLE'], symbol('MathFunctions', 'PmodDouble')], + [['fmod'], 'FLOAT', ['FLOAT', 'FLOAT'], symbol('MathFunctions', 'FmodFloat')], + [['fmod'], 'DOUBLE', ['DOUBLE', 'DOUBLE'], symbol('MathFunctions', 'FmodDouble')], + [['positive'], 'TINYINT', ['TINYINT'], + symbol('MathFunctions', 'Positive', 'int8_t')], + [['positive'], 'SMALLINT', ['SMALLINT'], + symbol('MathFunctions', 'Positive', 'int16_t')], + [['positive'], 'INT', ['INT'], + symbol('MathFunctions', 'Positive', 'int32_t')], + [['positive'], 'BIGINT', ['BIGINT'], + symbol('MathFunctions', 'Positive', 'int64_t')], + [['positive'], 'FLOAT', ['FLOAT'], + symbol('MathFunctions', 'Positive', 'float')], + [['positive'], 'DOUBLE', ['DOUBLE'], + symbol('MathFunctions', 'Positive', 'double')], + [['negative'], 'TINYINT', ['TINYINT'], + symbol('MathFunctions', 'Negative', 'int8_t')], + [['negative'], 'SMALLINT', ['SMALLINT'], + symbol('MathFunctions', 'Negative', 'int16_t')], + [['negative'], 'INT', ['INT'], + symbol('MathFunctions', 'Negative', 'int32_t')], + [['negative'], 'BIGINT', ['BIGINT'], + symbol('MathFunctions', 'Negative', 'int64_t')], + [['negative'], 'FLOAT', ['FLOAT'], + symbol('MathFunctions', 'Negative', 'float')], + [['negative'], 'DOUBLE', ['DOUBLE'], + symbol('MathFunctions', 'Negative', 'double')], + [['quotient'], 'BIGINT', ['BIGINT', 'BIGINT'], + symbol('MathFunctions', 'QuotientBigInt')], + [['quotient'], 'BIGINT', ['DOUBLE', 'DOUBLE'], + symbol('MathFunctions', 'QuotientDouble')], + [['least'], 'TINYINT', ['TINYINT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIaLb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['least'], 'SMALLINT', ['SMALLINT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIsLb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['least'], 'INT', ['INT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIiLb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['least'], 'BIGINT', ['BIGINT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIlLb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['least'], 'FLOAT', ['FLOAT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIfLb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['least'], 'DOUBLE', ['DOUBLE', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIdLb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['least'], 'TIMESTAMP', ['TIMESTAMP', '...'], + '_ZN6impala13MathFunctions13LeastGreatestINS_14TimestampValueELb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['least'], 'STRING', ['STRING', '...'], + '_ZN6impala13MathFunctions19LeastGreatestStringILb1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'TINYINT', ['TINYINT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIaLb0EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'SMALLINT', ['SMALLINT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIsLb0EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'INT', ['INT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIiLb0EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'BIGINT', ['BIGINT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIlLb0EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'FLOAT', ['FLOAT', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIfLb0EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'DOUBLE', ['DOUBLE', '...'], + '_ZN6impala13MathFunctions13LeastGreatestIdLb0EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'TIMESTAMP', ['TIMESTAMP', '...'], + '_ZN6impala13MathFunctions13LeastGreatestINS_14TimestampValueELb0EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['greatest'], 'STRING', ['STRING', '...'], + '_ZN6impala13MathFunctions19LeastGreatestStringILb0EEEPvPNS_4ExprEPNS_8TupleRowE'], - ['Constant_Regex', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], - 'LikePredicate::ConstantRegexFn', []], - ['Constant_Substring', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], - 'LikePredicate::ConstantSubstringFn', []], - ['Like', 'BOOLEAN', ['STRING', 'STRING'], 'LikePredicate::LikeFn', []], - ['Regex', 'BOOLEAN', ['STRING', 'STRING'], 'LikePredicate::RegexFn', []], - - ['Math_Pi', 'DOUBLE', [], 'MathFunctions::Pi', ['pi']], - ['Math_E', 'DOUBLE', [], 'MathFunctions::E', ['e']], - ['Math_Abs', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Abs', ['abs']], - ['Math_Sign', 'FLOAT', ['DOUBLE'], 'MathFunctions::Sign', ['sign']], - ['Math_Sin', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Sin', ['sin']], - ['Math_Asin', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Asin', ['asin']], - ['Math_Cos', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Cos', ['cos']], - ['Math_Acos', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Acos', ['acos']], - ['Math_Tan', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Tan', ['tan']], - ['Math_Atan', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Atan', ['atan']], - ['Math_Radians', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Radians', ['radians']], - ['Math_Degrees', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Degrees', ['degrees']], - ['Math_Ceil', 'BIGINT', ['DOUBLE'], 'MathFunctions::Ceil', ['ceil', 'ceiling']], - ['Math_Floor', 'BIGINT', ['DOUBLE'], 'MathFunctions::Floor', ['floor']], - ['Math_Round', 'BIGINT', ['DOUBLE'], 'MathFunctions::Round', ['round']], - ['Math_Round', 'DOUBLE', ['DOUBLE', 'INT'], 'MathFunctions::RoundUpTo', ['round']], - ['Math_Exp', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Exp', ['exp']], - ['Math_Ln', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Ln', ['ln']], - ['Math_Log10', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Log10', ['log10']], - ['Math_Log2', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Log2', ['log2']], - ['Math_Log', 'DOUBLE', ['DOUBLE', 'DOUBLE'], 'MathFunctions::Log', ['log']], - ['Math_Pow', 'DOUBLE', ['DOUBLE', 'DOUBLE'], 'MathFunctions::Pow', ['pow', 'power']], - ['Math_Sqrt', 'DOUBLE', ['DOUBLE'], 'MathFunctions::Sqrt', ['sqrt']], - ['Math_Rand', 'DOUBLE', [], 'MathFunctions::Rand', ['rand']], - ['Math_Rand', 'DOUBLE', ['INT'], 'MathFunctions::RandSeed', ['rand']], - ['Math_Bin', 'STRING', ['BIGINT'], 'MathFunctions::Bin', ['bin']], - ['Math_Hex', 'STRING', ['BIGINT'], 'MathFunctions::HexInt', ['hex']], - ['Math_Hex', 'STRING', ['STRING'], 'MathFunctions::HexString', ['hex']], - ['Math_Unhex', 'STRING', ['STRING'], 'MathFunctions::Unhex', ['unhex']], - ['Math_Conv', 'STRING', ['BIGINT', 'TINYINT', 'TINYINT'], - 'MathFunctions::ConvInt', ['conv']], - ['Math_Conv', 'STRING', ['STRING', 'TINYINT', 'TINYINT'], - 'MathFunctions::ConvString', ['conv']], - ['Math_Pmod', 'BIGINT', ['BIGINT', 'BIGINT'], 'MathFunctions::PmodBigInt', ['pmod']], - ['Math_Pmod', 'DOUBLE', ['DOUBLE', 'DOUBLE'], 'MathFunctions::PmodDouble', ['pmod']], - ['Math_Fmod', 'FLOAT', ['FLOAT', 'FLOAT'], - 'MathFunctions::FmodFloat', ['fmod']], - ['Math_Fmod', 'DOUBLE', ['DOUBLE', 'DOUBLE'], - 'MathFunctions::FmodDouble', ['fmod']], - ['Math_Positive', 'TINYINT', ['TINYINT'], - 'MathFunctions::Positive', ['positive']], - ['Math_Positive', 'SMALLINT', ['SMALLINT'], - 'MathFunctions::Positive', ['positive']], - ['Math_Positive', 'INT', ['INT'],\ - 'MathFunctions::Positive', ['positive']], - ['Math_Positive', 'BIGINT', ['BIGINT'], - 'MathFunctions::Positive', ['positive']], - ['Math_Positive', 'FLOAT', ['FLOAT'], - 'MathFunctions::Positive', ['positive']], - ['Math_Positive', 'DOUBLE', ['DOUBLE'], - 'MathFunctions::Positive', ['positive']], - ['Math_Negative', 'TINYINT', ['TINYINT'], - 'MathFunctions::Negative', ['negative']], - ['Math_Negative', 'SMALLINT', ['SMALLINT'], - 'MathFunctions::Negative', ['negative']], - ['Math_Negative', 'INT', ['INT'], - 'MathFunctions::Negative', ['negative']], - ['Math_Negative', 'BIGINT', ['BIGINT'], - 'MathFunctions::Negative', ['negative']], - ['Math_Negative', 'FLOAT', ['FLOAT'], - 'MathFunctions::Negative', ['negative']], - ['Math_Negative', 'DOUBLE', ['DOUBLE'], - 'MathFunctions::Negative', ['negative']], - ['Math_Quotient', 'BIGINT', ['BIGINT', 'BIGINT'], - 'MathFunctions::QuotientBigInt', ['quotient']], - ['Math_Quotient', 'BIGINT', ['DOUBLE', 'DOUBLE'], - 'MathFunctions::QuotientDouble', ['quotient']], - ['Math_Least', 'TINYINT', ['TINYINT', '...'], - 'MathFunctions::LeastGreatest', ['least']], - ['Math_Least', 'SMALLINT', ['SMALLINT', '...'], - 'MathFunctions::LeastGreatest', ['least']], - ['Math_Least', 'INT', ['INT', '...'], - 'MathFunctions::LeastGreatest', ['least']], - ['Math_Least', 'BIGINT', ['BIGINT', '...'], - 'MathFunctions::LeastGreatest', ['least']], - ['Math_Least', 'FLOAT', ['FLOAT', '...'], - 'MathFunctions::LeastGreatest', ['least']], - ['Math_Least', 'DOUBLE', ['DOUBLE', '...'], - 'MathFunctions::LeastGreatest', ['least']], - ['Math_Least', 'TIMESTAMP', ['TIMESTAMP', '...'], - 'MathFunctions::LeastGreatest', ['least']], - ['Math_Least', 'STRING', ['STRING', '...'], - 'MathFunctions::LeastGreatestString', ['least']], - ['Math_Greatest', 'TINYINT', ['TINYINT', '...'], - 'MathFunctions::LeastGreatest', ['greatest']], - ['Math_Greatest', 'SMALLINT', ['SMALLINT', '...'], - 'MathFunctions::LeastGreatest', ['greatest']], - ['Math_Greatest', 'INT', ['INT', '...'], - 'MathFunctions::LeastGreatest', ['greatest']], - ['Math_Greatest', 'BIGINT', ['BIGINT', '...'], - 'MathFunctions::LeastGreatest', ['greatest']], - ['Math_Greatest', 'FLOAT', ['FLOAT', '...'], - 'MathFunctions::LeastGreatest', ['greatest']], - ['Math_Greatest', 'DOUBLE', ['DOUBLE', '...'], - 'MathFunctions::LeastGreatest', ['greatest']], - ['Math_Greatest', 'TIMESTAMP', ['TIMESTAMP', '...'], - 'MathFunctions::LeastGreatest', ['greatest']], - ['Math_Greatest', 'STRING', ['STRING', '...'], - 'MathFunctions::LeastGreatestString', ['greatest']], - - ['String_Substring', 'STRING', ['STRING', 'INT'], - 'StringFunctions::Substring', ['substr', 'substring']], - ['String_Substring', 'STRING', ['STRING', 'BIGINT'], - 'StringFunctions::Substring', ['substr', 'substring']], - ['String_Substring', 'STRING', ['STRING', 'INT', 'INT'], - 'StringFunctions::Substring', ['substr', 'substring']], - ['String_Substring', 'STRING', ['STRING', 'BIGINT', 'BIGINT'], - 'StringFunctions::Substring', ['substr', 'substring']], + # String builtin functions + [['substr', 'substring'], 'STRING', ['STRING', 'INT'], + symbol('StringFunctions', 'Substring', 'int32_t')], + [['substr', 'substring'], 'STRING', ['STRING', 'BIGINT'], + symbol('StringFunctions', 'Substring', 'int64_t')], + [['substr', 'substring'], 'STRING', ['STRING', 'INT', 'INT'], + symbol('StringFunctions', 'Substring', 'int32_t')], + [['substr', 'substring'], 'STRING', ['STRING', 'BIGINT', 'BIGINT'], + symbol('StringFunctions', 'Substring', 'int64_t')], # left and right are key words, leave them out for now. - ['String_Left', 'STRING', ['STRING', 'INT'], - 'StringFunctions::Left', ['strleft']], - ['String_Left', 'STRING', ['STRING', 'BIGINT'], - 'StringFunctions::Left', ['strleft']], - ['String_Right', 'STRING', ['STRING', 'INT'], - 'StringFunctions::Right', ['strright']], - ['String_Right', 'STRING', ['STRING', 'BIGINT'], - 'StringFunctions::Right', ['strright']], - ['String_Space', 'STRING', ['INT'], 'StringFunctions::Space', ['space']], - ['String_Space', 'STRING', ['BIGINT'], 'StringFunctions::Space', ['space']], - ['String_Repeat', 'STRING', ['STRING', 'INT'], - 'StringFunctions::Repeat', ['repeat']], - ['String_Repeat', 'STRING', ['STRING', 'BIGINT'], - 'StringFunctions::Repeat', ['repeat']], - ['String_Lpad', 'STRING', ['STRING', 'INT', 'STRING'], - 'StringFunctions::Lpad', ['lpad']], - ['String_Lpad', 'STRING', ['STRING', 'BIGINT', 'STRING'], - 'StringFunctions::Lpad', ['lpad']], - ['String_Rpad', 'STRING', ['STRING', 'INT', 'STRING'], - 'StringFunctions::Rpad', ['rpad']], - ['String_Rpad', 'STRING', ['STRING', 'BIGINT', 'STRING'], - 'StringFunctions::Rpad', ['rpad']], - ['String_Length', 'INT', ['STRING'], 'StringFunctions::Length', ['length']], - ['String_Lower', 'STRING', ['STRING'], 'StringFunctions::Lower', ['lower', 'lcase']], - ['String_Upper', 'STRING', ['STRING'], 'StringFunctions::Upper', ['upper', 'ucase']], - ['String_InitCap', 'STRING', ['STRING'], 'StringFunctions::InitCap', ['initcap']], - ['String_Reverse', 'STRING', ['STRING'], 'StringFunctions::Reverse', ['reverse']], - ['String_Translate', 'STRING', ['STRING', 'STRING', 'STRING'], - 'StringFunctions::Translate', ['translate']], - ['String_Trim', 'STRING', ['STRING'], 'StringFunctions::Trim', ['trim']], - ['String_Ltrim', 'STRING', ['STRING'], 'StringFunctions::Ltrim', ['ltrim']], - ['String_Rtrim', 'STRING', ['STRING'], 'StringFunctions::Rtrim', ['rtrim']], - ['String_Ascii', 'INT', ['STRING'], 'StringFunctions::Ascii', ['ascii']], - ['String_Instr', 'INT', ['STRING', 'STRING'], 'StringFunctions::Instr', ['instr']], - ['String_Locate', 'INT', ['STRING', 'STRING'], 'StringFunctions::Locate', ['locate']], - ['String_Locate', 'INT', ['STRING', 'STRING', 'INT'], - 'StringFunctions::LocatePos', ['locate']], - ['String_Locate', 'INT', ['STRING', 'STRING', 'BIGINT'], - 'StringFunctions::LocatePos', ['locate']], - ['String_Regexp_Extract', 'STRING', ['STRING', 'STRING', 'INT'], - 'StringFunctions::RegexpExtract', ['regexp_extract']], - ['String_Regexp_Extract', 'STRING', ['STRING', 'STRING', 'BIGINT'], - 'StringFunctions::RegexpExtract', ['regexp_extract']], - ['String_Regexp_Replace', 'STRING', ['STRING', 'STRING', 'STRING'], - 'StringFunctions::RegexpReplace', ['regexp_replace']], - ['String_Concat', 'STRING', ['STRING', '...'], 'StringFunctions::Concat', ['concat']], - ['String_Concat_Ws', 'STRING', ['STRING', 'STRING', '...'], - 'StringFunctions::ConcatWs', ['concat_ws']], - ['String_Find_In_Set', 'INT', ['STRING', 'STRING'], - 'StringFunctions::FindInSet', ['find_in_set']], - ['String_Parse_Url', 'STRING', ['STRING', 'STRING'], - 'StringFunctions::ParseUrl', ['parse_url']], - ['String_Parse_Url', 'STRING', ['STRING', 'STRING', 'STRING'], - 'StringFunctions::ParseUrlKey', ['parse_url']], - ['Utility_Current_Database', 'STRING', [], 'UtilityFunctions::CurrentDatabase', - ['current_database']], - ['Utility_User', 'STRING', [], 'UtilityFunctions::User', ['user']], - ['Utility_Sleep', 'BOOLEAN', ['INT'], 'UtilityFunctions::Sleep', ['sleep']], - ['Utility_Pid', 'INT', [], 'UtilityFunctions::Pid', ['pid']], - ['Utility_Version', 'STRING', [], 'UtilityFunctions::Version', ['version']], - ['FnvHash', 'BIGINT', ['TINYINT'], 'UtilityFunctions::FnvHash<1>', ['fnv_hash']], - ['FnvHash', 'BIGINT', ['SMALLINT'], 'UtilityFunctions::FnvHash<2>', ['fnv_hash']], - ['FnvHash', 'BIGINT', ['INT'], 'UtilityFunctions::FnvHash<4>', ['fnv_hash']], - ['FnvHash', 'BIGINT', ['BIGINT'], 'UtilityFunctions::FnvHash<8>', ['fnv_hash']], - ['FnvHash', 'BIGINT', ['FLOAT'], 'UtilityFunctions::FnvHash<4>', ['fnv_hash']], - ['FnvHash', 'BIGINT', ['DOUBLE'], 'UtilityFunctions::FnvHash<8>', ['fnv_hash']], - ['FnvHash', 'BIGINT', ['TIMESTAMP'], 'UtilityFunctions::FnvHash<12>', ['fnv_hash']], - ['FnvHash', 'BIGINT', ['STRING'], 'UtilityFunctions::FnvHashString', ['fnv_hash']], + [['strleft'], 'STRING', ['STRING', 'INT'], + symbol('StringFunctions', 'Left', 'int32_t')], + [['strleft'], 'STRING', ['STRING', 'BIGINT'], + symbol('StringFunctions', 'Left', 'int64_t')], + [['strright'], 'STRING', ['STRING', 'INT'], + symbol('StringFunctions', 'Right', 'int32_t')], + [['strright'], 'STRING', ['STRING', 'BIGINT'], + symbol('StringFunctions', 'Right', 'int64_t')], + [['space'], 'STRING', ['INT'], symbol('StringFunctions', 'Space', 'int32_t')], + [['space'], 'STRING', ['BIGINT'], symbol('StringFunctions', 'Space', 'int64_t')], + [['repeat'], 'STRING', ['STRING', 'INT'], + symbol('StringFunctions', 'Repeat', 'int32_t')], + [['repeat'], 'STRING', ['STRING', 'BIGINT'], + symbol('StringFunctions', 'Repeat', 'int64_t')], + [['lpad'], 'STRING', ['STRING', 'INT', 'STRING'], + symbol('StringFunctions', 'Lpad', 'int32_t')], + [['lpad'], 'STRING', ['STRING', 'BIGINT', 'STRING'], + symbol('StringFunctions', 'Lpad', 'int64_t')], + [['rpad'], 'STRING', ['STRING', 'INT', 'STRING'], + symbol('StringFunctions', 'Rpad', 'int32_t')], + [['rpad'], 'STRING', ['STRING', 'BIGINT', 'STRING'], + symbol('StringFunctions', 'Rpad', 'int64_t')], + [['length'], 'INT', ['STRING'], symbol('StringFunctions', 'Length')], + [['lower', 'lcase'], 'STRING', ['STRING'], symbol('StringFunctions', 'Lower')], + [['upper', 'ucase'], 'STRING', ['STRING'], symbol('StringFunctions', 'Upper')], + [['initcap'], 'STRING', ['STRING'], symbol('StringFunctions', 'InitCap')], + [['reverse'], 'STRING', ['STRING'], symbol('StringFunctions', 'Reverse')], + [['translate'], 'STRING', ['STRING', 'STRING', 'STRING'], + symbol('StringFunctions', 'Translate')], + [['trim'], 'STRING', ['STRING'], symbol('StringFunctions', 'Trim')], + [['ltrim'], 'STRING', ['STRING'], symbol('StringFunctions', 'Ltrim')], + [['rtrim'], 'STRING', ['STRING'], symbol('StringFunctions', 'Rtrim')], + [['ascii'], 'INT', ['STRING'], symbol('StringFunctions', 'Ascii')], + [['instr'], 'INT', ['STRING', 'STRING'], symbol('StringFunctions', 'Instr')], + [['locate'], 'INT', ['STRING', 'STRING'], symbol('StringFunctions', 'Locate')], + [['locate'], 'INT', ['STRING', 'STRING', 'INT'], + symbol('StringFunctions', 'LocatePos', 'int32_t')], + [['locate'], 'INT', ['STRING', 'STRING', 'BIGINT'], + symbol('StringFunctions', 'LocatePos', 'int64_t')], + [['regexp_extract'], 'STRING', ['STRING', 'STRING', 'INT'], + symbol('StringFunctions', 'RegexpExtract', 'int32_t')], + [['regexp_extract'], 'STRING', ['STRING', 'STRING', 'BIGINT'], + symbol('StringFunctions', 'RegexpExtract', 'int64_t')], + [['regexp_replace'], 'STRING', ['STRING', 'STRING', 'STRING'], + symbol('StringFunctions', 'RegexpReplace')], + [['concat'], 'STRING', ['STRING', '...'], symbol('StringFunctions', 'Concat')], + [['concat_ws'], 'STRING', ['STRING', 'STRING', '...'], + symbol('StringFunctions', 'ConcatWs')], + [['find_in_set'], 'INT', ['STRING', 'STRING'], symbol('StringFunctions', 'FindInSet')], + [['parse_url'], 'STRING', ['STRING', 'STRING'], symbol('StringFunctions', 'ParseUrl')], + [['parse_url'], 'STRING', ['STRING', 'STRING', 'STRING'], + symbol('StringFunctions', 'ParseUrlKey')], + # Utility functions + [['current_database'], 'STRING', [], symbol('UtilityFunctions', 'CurrentDatabase')], + [['user'], 'STRING', [], symbol('UtilityFunctions', 'User')], + [['sleep'], 'BOOLEAN', ['INT'], symbol('UtilityFunctions', 'Sleep')], + [['pid'], 'INT', [], symbol('UtilityFunctions', 'Pid')], + [['version'], 'STRING', [], symbol('UtilityFunctions', 'Version')], -# Timestamp Functions - ['Unix_Timestamp', 'INT', [], - 'TimestampFunctions::Unix', ['unix_timestamp']], - ['Unix_Timestamp', 'INT', ['TIMESTAMP'], - 'TimestampFunctions::Unix', ['unix_timestamp']], - ['Unix_Timestamp', 'INT', ['STRING', 'STRING'], - 'TimestampFunctions::Unix', ['unix_timestamp']], - ['From_UnixTime', 'STRING', ['INT'], - 'TimestampFunctions::FromUnix', ['from_unixtime']], - ['From_UnixTime', 'STRING', ['INT', 'STRING'], - 'TimestampFunctions::FromUnix', ['from_unixtime']], - ['From_UnixTime', 'STRING', ['BIGINT'], - 'TimestampFunctions::FromUnix', ['from_unixtime']], - ['From_UnixTime', 'STRING', ['BIGINT', 'STRING'], - 'TimestampFunctions::FromUnix', ['from_unixtime']], - ['Timestamp_Dayname', 'STRING', ['TIMESTAMP'], - 'TimestampFunctions::DayName', ['dayname']], - ['Timestamp_year', 'INT', ['TIMESTAMP'], 'TimestampFunctions::Year', ['year']], - ['Timestamp_month', 'INT', ['TIMESTAMP'], 'TimestampFunctions::Month', ['month']], - ['Timestamp_dayofmonth', 'INT', ['TIMESTAMP'], - 'TimestampFunctions::DayOfMonth', ['day', 'dayofmonth']], - ['Timestamp_dayofweek', 'INT', ['TIMESTAMP'], - 'TimestampFunctions::DayOfWeek', ['dayofweek']], - ['Timestamp_dayofyear', 'INT', ['TIMESTAMP'], - 'TimestampFunctions::DayOfYear', ['dayofyear']], - ['Timestamp_weekofyear', 'INT', ['TIMESTAMP'], - 'TimestampFunctions::WeekOfYear', ['weekofyear']], - ['Timestamp_hour', 'INT', ['TIMESTAMP'], 'TimestampFunctions::Hour', ['hour']], - ['Timestamp_minute', 'INT', ['TIMESTAMP'], 'TimestampFunctions::Minute', ['minute']], - ['Timestamp_second', 'INT', ['TIMESTAMP'], 'TimestampFunctions::Second', ['second']], - ['Timestamp_now', 'TIMESTAMP', [], - 'TimestampFunctions::Now', ['now', 'current_timestamp']], - ['Timestamp_to_date', 'STRING', ['TIMESTAMP'], - 'TimestampFunctions::ToDate', ['to_date']], - ['Timestamp_diff', 'INT', ['TIMESTAMP', 'TIMESTAMP'], - 'TimestampFunctions::DateDiff', ['datediff']], - ['From_utc_timestamp', 'TIMESTAMP', ['TIMESTAMP', 'STRING'], - 'TimestampFunctions::FromUtc', ['from_utc_timestamp']], - ['To_utc_timestamp', 'TIMESTAMP', ['TIMESTAMP', 'STRING'], - 'TimestampFunctions::ToUtc', ['to_utc_timestamp']], + [['fnv_hash'], 'BIGINT', ['TINYINT'], + '_ZN6impala16UtilityFunctions7FnvHashILi1EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['fnv_hash'], 'BIGINT', ['SMALLINT'], + '_ZN6impala16UtilityFunctions7FnvHashILi2EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['fnv_hash'], 'BIGINT', ['INT'], + '_ZN6impala16UtilityFunctions7FnvHashILi4EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['fnv_hash'], 'BIGINT', ['BIGINT'], + '_ZN6impala16UtilityFunctions7FnvHashILi8EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['fnv_hash'], 'BIGINT', ['FLOAT'], + '_ZN6impala16UtilityFunctions7FnvHashILi4EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['fnv_hash'], 'BIGINT', ['DOUBLE'], + '_ZN6impala16UtilityFunctions7FnvHashILi8EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['fnv_hash'], 'BIGINT', ['TIMESTAMP'], + '_ZN6impala16UtilityFunctions7FnvHashILi12EEEPvPNS_4ExprEPNS_8TupleRowE'], + [['fnv_hash'], 'BIGINT', ['STRING'], + '_ZN6impala16UtilityFunctions13FnvHashStringEPNS_4ExprEPNS_8TupleRowE'], -# Date and time add/sub functions. - ['Timestamp_years_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', ['years_add']], - ['Timestamp_years_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', ['years_add']], - ['Timestamp_years_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', ['years_sub']], - ['Timestamp_years_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', ['years_sub']], - ['Timestamp_months_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', ['months_add']], - ['Timestamp_months_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', ['months_add']], - ['Timestamp_months_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', ['months_sub']], - ['Timestamp_months_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', ['months_sub']], - ['Timestamp_weeks_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', ['weeks_add']], - ['Timestamp_weeks_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', ['weeks_add']], - ['Timestamp_weeks_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', ['weeks_sub']], - ['Timestamp_weeks_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', ['weeks_sub']], - ['Timestamp_days_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', - ['days_add', 'date_add', 'adddate']], - ['Timestamp_days_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', - ['days_add', 'date_add', 'adddate']], - ['Timestamp_days_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::DateAddSub', - ['days_sub', 'date_sub', 'subdate']], - ['Timestamp_days_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::DateAddSub', - ['days_sub', 'date_sub', 'subdate']], - ['Timestamp_hours_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', ['hours_add']], - ['Timestamp_hours_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', ['hours_add']], - ['Timestamp_hours_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', ['hours_sub']], - ['Timestamp_hours_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', ['hours_sub']], - ['Timestamp_minutes_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', ['minutes_add']], - ['Timestamp_minutes_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', ['minutes_add']], - ['Timestamp_minutes_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', ['minutes_sub']], - ['Timestamp_minutes_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', ['minutes_sub']], - ['Timestamp_seconds_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', ['seconds_add']], - ['Timestamp_seconds_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', ['seconds_add']], - ['Timestamp_seconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', ['seconds_sub']], - ['Timestamp_seconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', ['seconds_sub']], - ['Timestamp_milliseconds_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', - ['milliseconds_add']], - ['Timestamp_milliseconds_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', - ['milliseconds_add']], - ['Timestamp_milliseconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', - ['milliseconds_sub']], - ['Timestamp_milliseconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', - ['milliseconds_sub']], - ['Timestamp_microseconds_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', - ['microseconds_add']], - ['Timestamp_microseconds_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', - ['microseconds_add']], - ['Timestamp_microseconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', - ['microseconds_sub']], - ['Timestamp_microseconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', - ['microseconds_sub']], - ['Timestamp_nanoseconds_add', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', - ['nanoseconds_add']], - ['Timestamp_nanoseconds_add', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', - ['nanoseconds_add']], - ['Timestamp_nanoseconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'INT'], - 'TimestampFunctions::TimeAddSub', - ['nanoseconds_sub']], - ['Timestamp_nanoseconds_sub', 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], - 'TimestampFunctions::TimeAddSub', - ['nanoseconds_sub']], + # Timestamp Functions + [['unix_timestamp'], 'INT', [], symbol('TimestampFunctions', 'Unix')], + [['unix_timestamp'], 'INT', ['STRING'], symbol('TimestampFunctions', 'UnixFromString')], + [['unix_timestamp'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'Unix')], + [['unix_timestamp'], 'INT', ['STRING', 'STRING'], symbol('TimestampFunctions', 'Unix')], + [['from_unixtime'], 'STRING', ['INT'], + symbol('TimestampFunctions', 'FromUnix', 'int32_t')], + [['from_unixtime'], 'STRING', ['INT', 'STRING'], + symbol('TimestampFunctions', 'FromUnix', 'int32_t')], + [['from_unixtime'], 'STRING', ['BIGINT'], + symbol('TimestampFunctions', 'FromUnix', 'int64_t')], + [['from_unixtime'], 'STRING', ['BIGINT', 'STRING'], + symbol('TimestampFunctions', 'FromUnix', 'int64_t')], + [['dayname'], 'STRING', ['TIMESTAMP'], symbol('TimestampFunctions', 'DayName')], + [['year'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'Year')], + [['month'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'Month')], + [['day', 'dayofmonth'], 'INT', ['TIMESTAMP'], + symbol('TimestampFunctions', 'DayOfMonth')], + [['dayofweek'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'DayOfWeek')], + [['dayofyear'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'DayOfYear')], + [['weekofyear'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'WeekOfYear')], + [['hour'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'Hour')], + [['minute'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'Minute')], + [['second'], 'INT', ['TIMESTAMP'], symbol('TimestampFunctions', 'Second')], + [['now', 'current_timestamp'], 'TIMESTAMP', [], symbol('TimestampFunctions', 'Now')], + [['to_date'], 'STRING', ['TIMESTAMP'], symbol('TimestampFunctions', 'ToDate')], + [['datediff'], 'INT', ['TIMESTAMP', 'TIMESTAMP'], + symbol('TimestampFunctions', 'DateDiff')], + [['from_utc_timestamp'], 'TIMESTAMP', ['TIMESTAMP', 'STRING'], + symbol('TimestampFunctions', 'FromUtc')], + [['to_utc_timestamp'], 'TIMESTAMP', ['TIMESTAMP', 'STRING'], + symbol('TimestampFunctions', 'ToUtc')], -# Conditional Functions - ['Conditional_If', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN', 'BOOLEAN'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'TINYINT', ['BOOLEAN', 'TINYINT', 'TINYINT'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'SMALLINT', ['BOOLEAN', 'SMALLINT', 'SMALLINT'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'INT', ['BOOLEAN', 'INT', 'INT'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'BIGINT', ['BOOLEAN', 'BIGINT', 'BIGINT'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'FLOAT', ['BOOLEAN', 'FLOAT', 'FLOAT'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'DOUBLE', ['BOOLEAN', 'DOUBLE', 'DOUBLE'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'STRING', ['BOOLEAN', 'STRING', 'STRING'], - 'ConditionalFunctions::IfFn', ['if']], - ['Conditional_If', 'TIMESTAMP', ['BOOLEAN', 'TIMESTAMP', 'TIMESTAMP'], - 'ConditionalFunctions::IfFn', ['if']], + # Date and time add/sub functions. + # TODO: there must be a better way to deal with this symbols. + [['years_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1EiN5boost9date_time14years_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['years_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1ElN5boost9date_time14years_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['years_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0EiN5boost9date_time14years_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['years_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0ElN5boost9date_time14years_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['months_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1EiN5boost9date_time15months_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['months_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1ElN5boost9date_time15months_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['months_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0EiN5boost9date_time15months_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['months_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0ElN5boost9date_time15months_durationINS2_9gregorian21greg_durations_configEEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['weeks_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1EiN5boost9gregorian14weeks_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['weeks_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1ElN5boost9gregorian14weeks_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['weeks_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0EiN5boost9gregorian14weeks_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['weeks_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0ElN5boost9gregorian14weeks_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['days_add', 'date_add', 'adddate'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1EiN5boost9gregorian13date_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['days_add', 'date_add', 'adddate'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb1ElN5boost9gregorian13date_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['days_sub', 'date_sub', 'subdate'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0EiN5boost9gregorian13date_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['days_sub', 'date_sub', 'subdate'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10DateAddSubILb0ElN5boost9gregorian13date_durationEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['hours_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1EiN5boost10posix_time5hoursEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['hours_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1ElN5boost10posix_time5hoursEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['hours_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0EiN5boost10posix_time5hoursEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['hours_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0ElN5boost10posix_time5hoursEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['minutes_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1EiN5boost10posix_time7minutesEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['minutes_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1ElN5boost10posix_time7minutesEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['minutes_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0EiN5boost10posix_time7minutesEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['minutes_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0ElN5boost10posix_time7minutesEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['seconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1EiN5boost10posix_time7secondsEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['seconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1ElN5boost10posix_time7secondsEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['seconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0EiN5boost10posix_time7secondsEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['seconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0ElN5boost10posix_time7secondsEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['milliseconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1EiN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['milliseconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1ElN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['milliseconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0EiN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['milliseconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0ElN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], - ['Conditional_NullIf', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'TINYINT', ['TINYINT', 'TINYINT'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'SMALLINT', ['SMALLINT', 'SMALLINT'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'INT', ['INT', 'INT'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'BIGINT', ['BIGINT', 'BIGINT'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'FLOAT', ['FLOAT', 'FLOAT'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'DOUBLE', ['DOUBLE', 'DOUBLE'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'STRING', ['STRING', 'STRING'], - 'ConditionalFunctions::NullIf', ['nullif']], - ['Conditional_NullIf', 'TIMESTAMP', ['TIMESTAMP', 'TIMESTAMP'], - 'ConditionalFunctions::NullIf', ['nullif']], + [['microseconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1EiN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['microseconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1ElN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['microseconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0EiN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['microseconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0ElN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], - ['Conditional_IsNull', 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], - 'ConditionalFunctions::IsNull', ['isnull', 'ifnull', 'nvl']], - ['Conditional_IsNull', 'TINYINT', ['TINYINT', 'TINYINT'], - 'ConditionalFunctions::IsNull', ['isnull', 'ifnull', 'nvl']], - ['Conditional_IsNull', 'SMALLINT', ['SMALLINT', 'SMALLINT'], - 'ConditionalFunctions::IsNull', ['isnull', 'ifnull', 'nvl']], - ['Conditional_IsNull', 'INT', ['INT', 'INT'], - 'ConditionalFunctions::IsNull', ['isnull', 'ifnull', 'nvl']], - ['Conditional_IsNull', 'BIGINT', ['BIGINT', 'BIGINT'], - 'ConditionalFunctions::IsNull', ['isnull', 'ifnull', 'nvl']], - ['Conditional_IsNull', 'FLOAT', ['FLOAT', 'FLOAT'], - 'ConditionalFunctions::IsNull', ['isnull','ifnull', 'nvl']], - ['Conditional_IsNull', 'DOUBLE', ['DOUBLE', 'DOUBLE'], - 'ConditionalFunctions::IsNull', ['isnull','ifnull', 'nvl']], - ['Conditional_IsNull', 'STRING', ['STRING', 'STRING'], - 'ConditionalFunctions::IsNull', ['isnull', 'ifnull', 'nvl']], - ['Conditional_IsNull', 'TIMESTAMP', ['TIMESTAMP', 'TIMESTAMP'], - 'ConditionalFunctions::IsNull', ['isnull', 'ifnull', 'nvl']], + [['nanoseconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1EiN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['nanoseconds_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb1ElN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['nanoseconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'INT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0EiN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['nanoseconds_sub'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'], + '_ZN6impala18TimestampFunctions10TimeAddSubILb0ElN5boost9date_time18subsecond_durationINS2_10posix_time13time_durationELl1000000000EEEEEPvPNS_4ExprEPNS_8TupleRowE'], - ['Conditional_Coalesce', 'BOOLEAN', ['BOOLEAN', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'TINYINT', ['TINYINT', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'SMALLINT', ['SMALLINT', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'INT', ['INT', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'BIGINT', ['BIGINT', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'FLOAT', ['FLOAT', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'DOUBLE', ['DOUBLE', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'STRING', ['STRING', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], - ['Conditional_Coalesce', 'TIMESTAMP', ['TIMESTAMP', '...'], - 'ConditionalFunctions::Coalesce', ['coalesce']], + # Conditional Functions + [['if'], 'BOOLEAN', ['BOOLEAN', 'BOOLEAN', 'BOOLEAN'], + symbol('ConditionalFunctions', 'IfFn', 'bool')], + [['if'], 'TINYINT', ['BOOLEAN', 'TINYINT', 'TINYINT'], + symbol('ConditionalFunctions', 'IfFn', 'int8_t')], + [['if'], 'SMALLINT', ['BOOLEAN', 'SMALLINT', 'SMALLINT'], + symbol('ConditionalFunctions', 'IfFn', 'int16_t')], + [['if'], 'INT', ['BOOLEAN', 'INT', 'INT'], + symbol('ConditionalFunctions', 'IfFn', 'int32_t')], + [['if'], 'BIGINT', ['BOOLEAN', 'BIGINT', 'BIGINT'], + symbol('ConditionalFunctions', 'IfFn', 'int64_t')], + [['if'], 'FLOAT', ['BOOLEAN', 'FLOAT', 'FLOAT'], + symbol('ConditionalFunctions', 'IfFn', 'float')], + [['if'], 'DOUBLE', ['BOOLEAN', 'DOUBLE', 'DOUBLE'], + symbol('ConditionalFunctions', 'IfFn', 'double')], + [['if'], 'STRING', ['BOOLEAN', 'STRING', 'STRING'], + symbol('ConditionalFunctions', 'IfFn', 'string')], + [['if'], 'TIMESTAMP', ['BOOLEAN', 'TIMESTAMP', 'TIMESTAMP'], + symbol('ConditionalFunctions', 'IfFn', 'timestamp')], + + [['nullif'], 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], + symbol('ConditionalFunctions', 'NullIf', 'bool')], + [['nullif'], 'TINYINT', ['TINYINT', 'TINYINT'], + symbol('ConditionalFunctions', 'NullIf', 'int8_t')], + [['nullif'], 'SMALLINT', ['SMALLINT', 'SMALLINT'], + symbol('ConditionalFunctions', 'NullIf', 'int16_t')], + [['nullif'], 'INT', ['INT', 'INT'], + symbol('ConditionalFunctions', 'NullIf', 'int32_t')], + [['nullif'], 'BIGINT', ['BIGINT', 'BIGINT'], + symbol('ConditionalFunctions', 'NullIf', 'int64_t')], + [['nullif'], 'FLOAT', ['FLOAT', 'FLOAT'], + symbol('ConditionalFunctions', 'NullIf', 'float')], + [['nullif'], 'DOUBLE', ['DOUBLE', 'DOUBLE'], + symbol('ConditionalFunctions', 'NullIf', 'double')], + [['nullif'], 'STRING', ['STRING', 'STRING'], + symbol('ConditionalFunctions', 'NullIf', 'string')], + [['nullif'], 'TIMESTAMP', ['TIMESTAMP', 'TIMESTAMP'], + symbol('ConditionalFunctions', 'NullIf', 'timestamp')], + + [['isnull', 'ifnull', 'nvl'], 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'TINYINT', ['TINYINT', 'TINYINT'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'SMALLINT', ['SMALLINT', 'SMALLINT'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'INT', ['INT', 'INT'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'BIGINT', ['BIGINT', 'BIGINT'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'FLOAT', ['FLOAT', 'FLOAT'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'DOUBLE', ['DOUBLE', 'DOUBLE'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'STRING', ['STRING', 'STRING'], + symbol('ConditionalFunctions', 'IsNull')], + [['isnull', 'ifnull', 'nvl'], 'TIMESTAMP', ['TIMESTAMP', 'TIMESTAMP'], + symbol('ConditionalFunctions', 'IsNull')], + + [['coalesce'], 'BOOLEAN', ['BOOLEAN', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceIbEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'TINYINT', ['TINYINT', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceIaEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'SMALLINT', ['SMALLINT', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceIsEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'INT', ['INT', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceIiEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'BIGINT', ['BIGINT', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceIlEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'FLOAT', ['FLOAT', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceIfEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'DOUBLE', ['DOUBLE', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceIdEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'STRING', ['STRING', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceINS_11StringValueEEEPvPNS_4ExprEPNS_8TupleRowE'], + [['coalesce'], 'TIMESTAMP', ['TIMESTAMP', '...'], + '_ZN6impala20ConditionalFunctions8CoalesceINS_14TimestampValueEEEPvPNS_4ExprEPNS_8TupleRowE'], ] # These functions are implemented against the UDF interface. # TODO: this list should subsume the one above when all builtins are migrated. udf_functions = [ - ['Udf_Math_Pi', 'DOUBLE', [], 'UdfBuiltins::Pi', ['udf_pi'], + [['udf_pi'], 'DOUBLE', [], '_ZN6impala11UdfBuiltins2PiEPN10impala_udf15FunctionContextE'], - ['Udf_Math_Abs', 'DOUBLE', ['DOUBLE'], 'UdfBuiltins::Abs', ['udf_abs'], + [['udf_abs'], 'DOUBLE', ['DOUBLE'], '_ZN6impala11UdfBuiltins3AbsEPN10impala_udf15FunctionContextERKNS1_9DoubleValE'], - ['Udf_String_Lower', 'STRING', ['STRING'], 'UdfBuiltins::Lower', ['udf_lower'], + [['udf_lower'], 'STRING', ['STRING'], '_ZN6impala11UdfBuiltins5LowerEPN10impala_udf15FunctionContextERKNS1_9StringValE'], ] diff --git a/common/thrift/CMakeLists.txt b/common/thrift/CMakeLists.txt index 9d57cba9b..c8652b913 100644 --- a/common/thrift/CMakeLists.txt +++ b/common/thrift/CMakeLists.txt @@ -101,10 +101,6 @@ file(MAKE_DIRECTORY ${PYTHON_OUTPUT_DIR}) set(JAVA_ARGS ${THRIFT_INCLUDE_DIR_OPTION} --gen java:hashcode -o ${FE_OUTPUT_DIR}) set(PYTHON_ARGS ${THRIFT_INCLUDE_DIR_OPTION} -r --gen py -o ${PYTHON_OUTPUT_DIR}) -set (GENERATES_SRC_FILES - Opcodes.thrift -) - set (SRC_FILES beeswax.thrift CatalogInternalService.thrift @@ -131,11 +127,8 @@ set (SRC_FILES StatestoreService.thrift Status.thrift Types.thrift - ${GENERATES_SRC_FILES} ) -set_source_files_properties(${GENERATES_SRC_FILES} PROPERTIES GENERATED TRUE) - # Create a build command for each of the thrift src files and generate # a list of files they produce THRIFT_GEN(THRIFT_FILES ${SRC_FILES}) diff --git a/common/thrift/CatalogService.thrift b/common/thrift/CatalogService.thrift index 8e9f5b2c7..5e0778ea0 100644 --- a/common/thrift/CatalogService.thrift +++ b/common/thrift/CatalogService.thrift @@ -74,7 +74,7 @@ struct TDdlExecRequest { // Parameters for CREATE FUNCTION 9: optional JniCatalog.TCreateFunctionParams create_fn_params - // Paramaters for DROP DATABASE + // Parameters for DROP DATABASE 10: optional JniCatalog.TDropDbParams drop_db_params // Parameters for DROP TABLE/VIEW diff --git a/common/thrift/Exprs.thrift b/common/thrift/Exprs.thrift index c2695be0a..85badb660 100644 --- a/common/thrift/Exprs.thrift +++ b/common/thrift/Exprs.thrift @@ -16,7 +16,6 @@ namespace cpp impala namespace java com.cloudera.impala.thrift include "Types.thrift" -include "Opcodes.thrift" enum TExprNodeType { ARITHMETIC_EXPR, @@ -37,10 +36,10 @@ enum TExprNodeType { STRING_LITERAL, TUPLE_IS_NULL_PRED, FUNCTION_CALL, + AGGREGATE_EXPR, // TODO: old style compute functions. this will be deprecated COMPUTE_FUNCTION_CALL, - } struct TBoolLiteral { @@ -93,35 +92,29 @@ struct TStringLiteral { 1: required string value; } -struct TFunctionCallExpr { - // The aggregate function to call. - 1: required Types.TFunction fn - - // If set, this aggregate function udf has varargs and this is the index for the - // first variable argument. - 2: optional i32 vararg_start_idx -} - // This is essentially a union over the subclasses of Expr. struct TExprNode { 1: required TExprNodeType node_type 2: required Types.TColumnType type - 3: optional Opcodes.TExprOpcode opcode - 4: required i32 num_children + 3: required i32 num_children - 5: optional TBoolLiteral bool_literal - 6: optional TCaseExpr case_expr - 7: optional TDateLiteral date_literal - 8: optional TFloatLiteral float_literal - 9: optional TIntLiteral int_literal - 10: optional TInPredicate in_predicate - 11: optional TIsNullPredicate is_null_pred - 12: optional TLikePredicate like_pred - 13: optional TLiteralPredicate literal_pred - 14: optional TSlotRef slot_ref - 15: optional TStringLiteral string_literal - 16: optional TTupleIsNullPredicate tuple_is_null_pred - 17: optional TFunctionCallExpr fn_call_expr + // The function to execute. Not set for SlotRefs and Literals. + 4: optional Types.TFunction fn + // If set, child[vararg_start_idx] is the first vararg child. + 5: optional i32 vararg_start_idx + + 6: optional TBoolLiteral bool_literal + 7: optional TCaseExpr case_expr + 8: optional TDateLiteral date_literal + 9: optional TFloatLiteral float_literal + 10: optional TIntLiteral int_literal + 11: optional TInPredicate in_predicate + 12: optional TIsNullPredicate is_null_pred + 13: optional TLikePredicate like_pred + 14: optional TLiteralPredicate literal_pred + 15: optional TSlotRef slot_ref + 16: optional TStringLiteral string_literal + 17: optional TTupleIsNullPredicate tuple_is_null_pred } // A flattened representation of a tree of Expr nodes, obtained by depth-first diff --git a/common/thrift/PlanNodes.thrift b/common/thrift/PlanNodes.thrift index f4ef8bdde..3bbf71425 100644 --- a/common/thrift/PlanNodes.thrift +++ b/common/thrift/PlanNodes.thrift @@ -150,33 +150,11 @@ struct THashJoinNode { 3: optional list other_join_conjuncts } -enum TAggregationOp { - INVALID, - COUNT, - MAX, - DISTINCT_PC, - DISTINCT_PCSA, - MIN, - SUM, - GROUP_CONCAT, - HLL, -} - -struct TAggregateFunctionCall { - // The aggregate function to call. - 1: required Types.TFunction fn - - // The input exprs to this aggregate function - 2: required list input_exprs - - // If set, this aggregate function udf has varargs and this is the index for the - // first variable argument. - 3: optional i32 vararg_start_idx -} - struct TAggregationNode { 1: optional list grouping_exprs - 2: required list aggregate_functions + // aggregate exprs. The root of each expr is the aggregate function. The + // other exprs are the inputs to the aggregate function. + 2: required list aggregate_functions 3: required Types.TTupleId agg_tuple_id // Set to true if this aggregation node needs to run the finalization step. diff --git a/common/thrift/Types.thrift b/common/thrift/Types.thrift index 216dc7edd..3152f57b1 100644 --- a/common/thrift/Types.thrift +++ b/common/thrift/Types.thrift @@ -110,50 +110,47 @@ struct TFunctionName { } struct TScalarFunction { - // Symbol for the function - 1: optional string symbol; + 1: required string symbol; } struct TAggregateFunction { 1: required TColumnType intermediate_type - 2: optional string update_fn_symbol - 3: optional string init_fn_symbol + 2: required string update_fn_symbol + 3: required string init_fn_symbol 4: optional string serialize_fn_symbol - 5: optional string merge_fn_symbol + 5: required string merge_fn_symbol 6: optional string finalize_fn_symbol + 7: optional bool ignores_distinct } // Represents a function in the Catalog. struct TFunction { - // Id that is unique across all functions. - 1: required i64 id - // Fully qualified function name. - 2: required TFunctionName name + 1: required TFunctionName name // Type of the udf. e.g. hive, native, ir - 3: required TFunctionBinaryType binary_type + 2: required TFunctionBinaryType binary_type // The types of the arguments to the function - 4: required list arg_types + 3: required list arg_types // Return type for the function. - 5: required TColumnType ret_type + 4: required TColumnType ret_type // If true, this function takes var args. - 6: required bool has_var_args + 5: required bool has_var_args // Optional comment to attach to the function - 7: optional string comment + 6: optional string comment - 8: optional string signature + 7: optional string signature // HDFS path for the function binary. This binary must exist at the time the // function is created. - 9: optional string hdfs_location + 8: optional string hdfs_location // One of these should be set. - 10: optional TScalarFunction scalar_fn - 11: optional TAggregateFunction aggregate_fn + 9: optional TScalarFunction scalar_fn + 10: optional TAggregateFunction aggregate_fn } diff --git a/fe/src/main/cup/sql-parser.y b/fe/src/main/cup/sql-parser.y index a72a251af..3123d989e 100644 --- a/fe/src/main/cup/sql-parser.y +++ b/fe/src/main/cup/sql-parser.y @@ -199,24 +199,24 @@ parser code {: // List of keywords. Please keep them sorted alphabetically. terminal - KW_ADD, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_AND, KW_AS, KW_ASC, KW_AVG, + KW_ADD, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_AND, KW_AS, KW_ASC, KW_AVRO, KW_BETWEEN, KW_BIGINT, KW_BOOLEAN, KW_BY, KW_CASE, KW_CAST, - KW_CHANGE, KW_CHAR, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMPUTE, KW_COUNT, KW_CREATE, + KW_CHANGE, KW_CHAR, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMPUTE, KW_CREATE, KW_CROSS, KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DECIMAL, - KW_DELIMITED, KW_DESC, KW_DESCRIBE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DIV, + KW_DELIMITED, KW_DESC, KW_DESCRIBE, KW_DISTINCT, KW_DIV, KW_DOUBLE, KW_DROP, KW_ELSE, KW_END, KW_ESCAPED, KW_EXISTS, KW_EXPLAIN, KW_EXTERNAL, KW_FALSE, KW_FIELDS, KW_FILEFORMAT, KW_FINALIZE_FN, KW_FIRST, KW_FLOAT, KW_FORMAT, KW_FORMATTED, KW_FROM, KW_FULL, KW_FUNCTION, KW_FUNCTIONS, - KW_GROUP, KW_GROUP_CONCAT, KW_HAVING, KW_IF, KW_IN, KW_INIT_FN, KW_INNER, + KW_GROUP, KW_HAVING, KW_IF, KW_IN, KW_INIT_FN, KW_INNER, KW_INPATH, KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERVAL, KW_INTO, KW_INVALIDATE, KW_IS, KW_JOIN, KW_LAST, KW_LEFT, KW_LIKE, KW_LIMIT, - KW_LINES, KW_LOAD, KW_LOCATION, KW_MAX, KW_MERGE_FN, KW_METADATA, KW_MIN, KW_NDV, + KW_LINES, KW_LOAD, KW_LOCATION, KW_MERGE_FN, KW_METADATA, KW_NOT, KW_NULL, KW_NULLS, KW_OFFSET, KW_ON, KW_OR, KW_ORDER, KW_OUTER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION, KW_PARTITIONED, KW_RCFILE, KW_REFRESH, KW_REGEXP, KW_RENAME, KW_REPLACE, KW_RETURNS, KW_RIGHT, KW_RLIKE, KW_ROW, KW_SCHEMA, KW_SCHEMAS, KW_SELECT, KW_SEMI, KW_SEQUENCEFILE, KW_SERDEPROPERTIES, KW_SERIALIZE_FN, KW_SET, KW_SHOW, KW_SMALLINT, KW_STORED, KW_STRAIGHT_JOIN, - KW_STRING, KW_SUM, KW_SYMBOL, KW_TABLE, KW_TABLES, KW_TBLPROPERTIES, KW_TERMINATED, + KW_STRING, KW_SYMBOL, KW_TABLE, KW_TABLES, KW_TBLPROPERTIES, KW_TERMINATED, KW_TEXTFILE, KW_THEN, KW_TIMESTAMP, KW_TINYINT, KW_STATS, KW_TO, KW_TRUE, KW_UNION, KW_UPDATE_FN, KW_USE, KW_USING, KW_VALUES, KW_VIEW, KW_WHEN, KW_WHERE, KW_WITH; @@ -281,12 +281,11 @@ nonterminal Boolean opt_order_param; nonterminal Boolean opt_nulls_order_param; nonterminal Expr opt_offset_param; nonterminal LimitElement opt_limit_clause; -nonterminal Expr cast_expr, case_else_clause, aggregate_expr; +nonterminal Expr cast_expr, case_else_clause; nonterminal LiteralExpr literal; nonterminal CaseExpr case_expr; nonterminal ArrayList case_when_clause_list; nonterminal FunctionParams function_params; -nonterminal BuiltinAggregateFunction.Operator aggregate_operator; nonterminal SlotRef column_ref; nonterminal ArrayList from_clause, table_ref_list; nonterminal ArrayList with_table_ref_list; @@ -1634,8 +1633,6 @@ non_pred_expr ::= {: RESULT = c; :} | case_expr:c {: RESULT = c; :} - | aggregate_expr:a - {: RESULT = a; :} | column_ref:c {: RESULT = c; :} | timestamp_arithmetic_expr:e @@ -1743,34 +1740,6 @@ literal ::= :} ; -aggregate_expr ::= - aggregate_operator:op LPAREN function_params:params RPAREN - {: - RESULT = new FunctionCallExpr(op, params); - :} - ; - -aggregate_operator ::= - KW_COUNT - {: RESULT = BuiltinAggregateFunction.Operator.COUNT; :} - | KW_MIN - {: RESULT = BuiltinAggregateFunction.Operator.MIN; :} - | KW_MAX - {: RESULT = BuiltinAggregateFunction.Operator.MAX; :} - | KW_DISTINCTPC - {: RESULT = BuiltinAggregateFunction.Operator.DISTINCT_PC; :} - | KW_DISTINCTPCSA - {: RESULT = BuiltinAggregateFunction.Operator.DISTINCT_PCSA; :} - | KW_NDV - {: RESULT = BuiltinAggregateFunction.Operator.NDV; :} - | KW_SUM - {: RESULT = BuiltinAggregateFunction.Operator.SUM; :} - | KW_AVG - {: RESULT = BuiltinAggregateFunction.Operator.AVG; :} - | KW_GROUP_CONCAT - {: RESULT = BuiltinAggregateFunction.Operator.GROUP_CONCAT; :} - ; - function_params ::= STAR {: RESULT = FunctionParams.createStarParam(); :} diff --git a/fe/src/main/java/com/cloudera/impala/analysis/AggregateInfo.java b/fe/src/main/java/com/cloudera/impala/analysis/AggregateInfo.java index 0b317022a..2d3496928 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/AggregateInfo.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/AggregateInfo.java @@ -323,9 +323,8 @@ public class AggregateInfo { new SlotRef(inputDesc.getSlots().get(i + getGroupingExprs().size())); List aggExprParamList = Lists.newArrayList(aggExprParam); FunctionCallExpr aggExpr = null; - if (inputExpr.getAggOp() == BuiltinAggregateFunction.Operator.COUNT) { - aggExpr = new FunctionCallExpr( - BuiltinAggregateFunction.Operator.SUM, new FunctionParams(aggExprParamList)); + if (inputExpr.getFnName().getFunction().equals("count")) { + aggExpr = new FunctionCallExpr("sum", new FunctionParams(aggExprParamList)); } else { aggExpr = new FunctionCallExpr(inputExpr, new FunctionParams(aggExprParamList)); } @@ -402,7 +401,7 @@ public class AggregateInfo { for (FunctionCallExpr inputExpr: distinctAggExprs) { Preconditions.checkState(inputExpr.isAggregateFunction()); FunctionCallExpr aggExpr = null; - if (inputExpr.getAggOp() == BuiltinAggregateFunction.Operator.COUNT) { + if (inputExpr.getFnName().getFunction().equals("count")) { // COUNT(DISTINCT ...) -> // COUNT(IF(IsNull(), NULL, IF(IsNull(), NULL, ...))) // We need the nested IF to make sure that we do not count @@ -418,17 +417,16 @@ public class AggregateInfo { throw new InternalException("Failed to analyze 'IF' function " + "in second phase count distinct aggregation.", e); } - aggExpr = new FunctionCallExpr(BuiltinAggregateFunction.Operator.COUNT, + aggExpr = new FunctionCallExpr("count", new FunctionParams(Lists.newArrayList(ifExpr))); } else { // SUM(DISTINCT ) -> SUM(); // (MIN(DISTINCT ...) and MAX(DISTINCT ...) have their DISTINCT turned // off during analysis, and AVG() is changed to SUM()/COUNT()) - Preconditions.checkState(inputExpr.getAggOp() == - BuiltinAggregateFunction.Operator.SUM); + Preconditions.checkState(inputExpr.getFnName().getFunction().equals("sum")); Expr aggExprParam = new SlotRef(inputDesc.getSlots().get(origGroupingExprs.size())); - aggExpr = new FunctionCallExpr(BuiltinAggregateFunction.Operator.SUM, + aggExpr = new FunctionCallExpr("sum", new FunctionParams(Lists.newArrayList(aggExprParam))); } secondPhaseAggExprs.add(aggExpr); @@ -443,8 +441,8 @@ public class AggregateInfo { new SlotRef(inputDesc.getSlots().get(i + getGroupingExprs().size())); List aggExprParamList = Lists.newArrayList(aggExprParam); FunctionCallExpr aggExpr = null; - if (inputExpr.getAggOp() == BuiltinAggregateFunction.Operator.COUNT) { - aggExpr = new FunctionCallExpr(BuiltinAggregateFunction.Operator.SUM, + if (inputExpr.getFnName().getFunction().equals("count")) { + aggExpr = new FunctionCallExpr("sum", new FunctionParams(aggExprParamList)); } else { // TODO: remap types here. The inserted agg expr doesn't need to be the same @@ -550,7 +548,7 @@ public class AggregateInfo { } else { Preconditions.checkArgument(expr instanceof FunctionCallExpr); FunctionCallExpr aggExpr = (FunctionCallExpr)expr; - if (aggExpr.getAggOp() == BuiltinAggregateFunction.Operator.COUNT) { + if (aggExpr.getFnName().getFunction().equals("count")) { outputSlotDesc.setIsNullable(false); } } diff --git a/fe/src/main/java/com/cloudera/impala/analysis/ArithmeticExpr.java b/fe/src/main/java/com/cloudera/impala/analysis/ArithmeticExpr.java index 2fac34204..eff3ac5d6 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/ArithmeticExpr.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/ArithmeticExpr.java @@ -15,42 +15,40 @@ package com.cloudera.impala.analysis; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Db; +import com.cloudera.impala.catalog.Function.CompareMode; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; -import com.cloudera.impala.opcode.FunctionOperator; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; import com.google.common.base.Objects; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; public class ArithmeticExpr extends Expr { enum Operator { - MULTIPLY("*", FunctionOperator.MULTIPLY), - DIVIDE("/", FunctionOperator.DIVIDE), - MOD("%", FunctionOperator.MOD), - INT_DIVIDE("DIV", FunctionOperator.INT_DIVIDE), - ADD("+", FunctionOperator.ADD), - SUBTRACT("-", FunctionOperator.SUBTRACT), - BITAND("&", FunctionOperator.BITAND), - BITOR("|", FunctionOperator.BITOR), - BITXOR("^", FunctionOperator.BITXOR), - BITNOT("~", FunctionOperator.BITNOT); + MULTIPLY("*", "multiply"), + DIVIDE("/", "divide"), + MOD("%", "mod"), + INT_DIVIDE("DIV", "int_divide"), + ADD("+", "add"), + SUBTRACT("-", "subtract"), + BITAND("&", "bitand"), + BITOR("|", "bitor"), + BITXOR("^", "bitxor"), + BITNOT("~", "bitnot"); - private final String description; - private final FunctionOperator functionOp; + private final String description_; + private final String name_; - private Operator(String description, FunctionOperator thriftOp) { - this.description = description; - this.functionOp = thriftOp; + private Operator(String description, String name) { + this.description_ = description; + this.name_ = name; } @Override - public String toString() { - return description; - } - - public FunctionOperator toFunctionOp() { - return functionOp; - } + public String toString() { return description_; } + public String getName() { return name_; } } private final Operator op_; @@ -67,6 +65,39 @@ public class ArithmeticExpr extends Expr { if (e2 != null) children_.add(e2); } + public static void initBuiltins(Db db) { + for (ColumnType t: ColumnType.getNumericTypes()) { + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.MULTIPLY.getName(), Lists.newArrayList(t, t), + t.getMaxResolutionType())); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.ADD.getName(), Lists.newArrayList(t, t), + t.getNextResolutionType())); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.SUBTRACT.getName(), Lists.newArrayList(t, t), + t.getNextResolutionType())); + } + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.DIVIDE.getName(), + Lists.newArrayList(ColumnType.DOUBLE, ColumnType.DOUBLE), + ColumnType.DOUBLE)); + + for (ColumnType t: ColumnType.getFixedPointTypes()) { + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.INT_DIVIDE.getName(), Lists.newArrayList(t, t), t)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.MOD.getName(), Lists.newArrayList(t, t), t)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.BITAND.getName(), Lists.newArrayList(t, t), t)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.BITOR.getName(), Lists.newArrayList(t, t), t)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.BITXOR.getName(), Lists.newArrayList(t, t), t)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.BITNOT.getName(), Lists.newArrayList(t), t)); + } + } + @Override public String debugString() { return Objects.toStringHelper(this) @@ -88,13 +119,6 @@ public class ArithmeticExpr extends Expr { @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.ARITHMETIC_EXPR; - msg.setOpcode(opcode_); - } - - @Override - public boolean equals(Object obj) { - if (!super.equals(obj)) return false; - return ((ArithmeticExpr) obj).opcode_ == opcode_; } @Override @@ -113,21 +137,20 @@ public class ArithmeticExpr extends Expr { // bitnot is the only unary op, deal with it here if (op_ == Operator.BITNOT) { type_ = getChild(0).getType(); - OpcodeRegistry.BuiltinFunction match = - OpcodeRegistry.instance().getFunctionInfo(op_.functionOp, true, type_); - if (match == null) { + fn_ = getBuiltinFunction(analyzer, op_.getName(), collectChildReturnTypes(), + CompareMode.IS_SUBTYPE); + if (fn_ == null) { throw new AnalysisException("Bitwise operations only allowed on fixed-point " + "types: " + toSql()); } - Preconditions.checkState(type_.equals(match.getReturnType()) || type_.isNull()); - opcode_ = match.opcode; + Preconditions.checkState(type_.equals(fn_.getReturnType()) || type_.isNull()); return; } ColumnType t1 = getChild(0).getType(); ColumnType t2 = getChild(1).getType(); // only bitnot is unary - FunctionOperator funcOp = op_.toFunctionOp(); + String fnName = op_.getName(); switch (op_) { case MULTIPLY: case ADD: @@ -145,7 +168,7 @@ public class ArithmeticExpr extends Expr { case MOD: type_ = ColumnType.getAssignmentCompatibleType(t1, t2); // Use MATH_MOD function operator for floating-point modulo. - if (type_.isFloatingPointType()) funcOp = FunctionOperator.MATH_FMOD; + if (type_.isFloatingPointType()) fnName = "fmod"; break; case DIVIDE: type_ = ColumnType.DOUBLE; @@ -171,12 +194,11 @@ public class ArithmeticExpr extends Expr { } type_ = castBinaryOp(type_); - OpcodeRegistry.BuiltinFunction match = - OpcodeRegistry.instance().getFunctionInfo(funcOp, true, type_, type_); - if (match == null) { + fn_ = getBuiltinFunction(analyzer, fnName, collectChildReturnTypes(), + CompareMode.IS_SUBTYPE); + if (fn_ == null) { Preconditions.checkState(false, String.format("No match in function registry " + "for '%s' with operand types %s and %s", toSql(), type_, type_)); } - this.opcode_ = match.opcode; } } diff --git a/fe/src/main/java/com/cloudera/impala/analysis/BetweenPredicate.java b/fe/src/main/java/com/cloudera/impala/analysis/BetweenPredicate.java index e0c467b3d..5baaf6379 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/BetweenPredicate.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/BetweenPredicate.java @@ -79,6 +79,7 @@ public class BetweenPredicate extends Predicate { try { rewrittenPredicate_.analyze(analyzer); + fn_ = rewrittenPredicate_.fn_; } catch (AnalysisException e) { // We should have already guaranteed that analysis will succeed. Preconditions.checkState(false, "Analysis failed in rewritten between predicate"); diff --git a/fe/src/main/java/com/cloudera/impala/analysis/BinaryPredicate.java b/fe/src/main/java/com/cloudera/impala/analysis/BinaryPredicate.java index 6c496edb0..55de9223e 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/BinaryPredicate.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/BinaryPredicate.java @@ -18,14 +18,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Db; +import com.cloudera.impala.catalog.Function.CompareMode; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.common.Pair; import com.cloudera.impala.common.Reference; -import com.cloudera.impala.opcode.FunctionOperator; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; import com.google.common.base.Objects; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; /** * Most predicates with two operands.. @@ -35,28 +38,41 @@ public class BinaryPredicate extends Predicate { private final static Logger LOG = LoggerFactory.getLogger(BinaryPredicate.class); public enum Operator { - EQ("=", FunctionOperator.EQ), - NE("!=", FunctionOperator.NE), - LE("<=", FunctionOperator.LE), - GE(">=", FunctionOperator.GE), - LT("<", FunctionOperator.LT), - GT(">", FunctionOperator.GT); + EQ("=", "eq"), + NE("!=", "ne"), + LE("<=", "le"), + GE(">=", "ge"), + LT("<", "lt"), + GT(">", "gt"); private final String description; - private final FunctionOperator functionOp; + private final String name; - private Operator(String description, FunctionOperator functionOp) { + private Operator(String description, String name) { this.description = description; - this.functionOp = functionOp; + this.name = name; } @Override - public String toString() { - return description; - } + public String toString() { return description; } + public String getName() { return name; } + } - public FunctionOperator toFunctionOp() { - return functionOp; + public static void initBuiltins(Db db) { + for (ColumnType t: ColumnType.getSupportedTypes()) { + if (t.isNull()) continue; // NULL is handled through type promotion. + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.EQ.getName(), Lists.newArrayList(t, t), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.NE.getName(), Lists.newArrayList(t, t), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.LE.getName(), Lists.newArrayList(t, t), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.GE.getName(), Lists.newArrayList(t, t), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.LT.getName(), Lists.newArrayList(t, t), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.GT.getName(), Lists.newArrayList(t, t), ColumnType.BOOLEAN)); } } @@ -73,12 +89,6 @@ public class BinaryPredicate extends Predicate { children_.add(e2); } - @Override - public boolean equals(Object obj) { - if (!super.equals(obj)) return false; - return ((BinaryPredicate) obj).opcode_ == this.opcode_; - } - @Override public String toSqlImpl() { return getChild(0).toSql() + " " + op_.toString() + " " + getChild(1).toSql(); @@ -87,7 +97,6 @@ public class BinaryPredicate extends Predicate { @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.BINARY_PRED; - msg.setOpcode(opcode_); } @Override @@ -104,23 +113,13 @@ public class BinaryPredicate extends Predicate { if (isAnalyzed_) return; super.analyze(analyzer); - ColumnType t1 = getChild(0).getType(); - ColumnType t2 = getChild(1).getType(); - ColumnType compatibleType = ColumnType.getAssignmentCompatibleType(t1, t2); - - if (!compatibleType.isValid()) { - // there is no type to which both are assignment-compatible -> we can't compare them - throw new AnalysisException("operands are not comparable: " + this.toSql()); + fn_ = getBuiltinFunction(analyzer, op_.getName(), collectChildReturnTypes(), + CompareMode.IS_SUBTYPE); + if (fn_ == null) { + throw new AnalysisException("operands are not comparable: " + toSql()); } - - // Ignore return value because type is always bool for predicates. - castBinaryOp(compatibleType); - - OpcodeRegistry.BuiltinFunction match = OpcodeRegistry.instance().getFunctionInfo( - op_.toFunctionOp(), true, compatibleType, compatibleType); - Preconditions.checkState(match != null); - Preconditions.checkState(match.getReturnType().isBoolean()); - this.opcode_ = match.opcode; + Preconditions.checkState(fn_.getReturnType().isBoolean()); + castForFunctionCall(); // determine selectivity Reference slotRefRef = new Reference(); diff --git a/fe/src/main/java/com/cloudera/impala/analysis/BuiltinAggregateFunction.java b/fe/src/main/java/com/cloudera/impala/analysis/BuiltinAggregateFunction.java deleted file mode 100644 index 26a331ddc..000000000 --- a/fe/src/main/java/com/cloudera/impala/analysis/BuiltinAggregateFunction.java +++ /dev/null @@ -1,105 +0,0 @@ -// Copyright 2012 Cloudera Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package com.cloudera.impala.analysis; - -import java.util.ArrayList; - -import com.cloudera.impala.catalog.Function; -import com.cloudera.impala.common.AnalysisException; -import com.cloudera.impala.thrift.TAggregateFunction; -import com.cloudera.impala.thrift.TAggregationOp; -import com.cloudera.impala.thrift.TFunction; -import com.cloudera.impala.thrift.TFunctionBinaryType; -import com.google.common.base.Preconditions; - -/** - * Internal representation of a builtin aggregate function. - */ -public class BuiltinAggregateFunction extends Function { - // TODO: this is effectively a catalog of builtin aggregate functions. - // We should move this to something in the catalog instead of having it - // here like this. - public enum Operator { - COUNT("COUNT", TAggregationOp.COUNT, ColumnType.BIGINT), - MIN("MIN", TAggregationOp.MIN, null), - MAX("MAX", TAggregationOp.MAX, null), - DISTINCT_PC("DISTINCT_PC", TAggregationOp.DISTINCT_PC, - // TODO: this needs to switch to CHAR(64) - ColumnType.STRING), - DISTINCT_PCSA("DISTINCT_PCSA", TAggregationOp.DISTINCT_PCSA, - // TODO: this needs to switch to CHAR(64) - ColumnType.STRING), - SUM("SUM", TAggregationOp.SUM, null), - AVG("AVG", TAggregationOp.INVALID, null), - GROUP_CONCAT("GROUP_CONCAT", TAggregationOp.GROUP_CONCAT, - // TODO: this needs to switch to CHAR(16) - ColumnType.STRING), - - // NDV is the external facing name (i.e. queries should always be written with NDV) - // The current implementation of NDV is hyperloglog (but we could change this without - // external query changes if we find a better algorithm). - NDV("NDV", TAggregationOp.HLL, - // TODO: this needs to switch to CHAR(64) - ColumnType.STRING); - - - private final String description; - private final TAggregationOp thriftOp; - - // The intermediate type for this function if it is constant regardless of - // input type. Set to null if it can only be determined during analysis. - private final ColumnType intermediateType; - - private Operator(String description, TAggregationOp thriftOp, - ColumnType intermediateType) { - this.description = description; - this.thriftOp = thriftOp; - this.intermediateType = intermediateType; - } - - @Override - public String toString() { return description; } - public TAggregationOp toThrift() { return thriftOp; } - public ColumnType intermediateType() { return intermediateType; } - } - - private final Operator op_; - // TODO: this is not used yet until the planner understand this. - private ColumnType intermediateType_; - - public BuiltinAggregateFunction(Operator op, ArrayList argTypes, - ColumnType retType, ColumnType intermediateType) throws AnalysisException { - super(FunctionName.CreateBuiltinName(op.toString()), argTypes, retType, false); - Preconditions.checkState(intermediateType != null); - Preconditions.checkState(op != null); - intermediateType.analyze(); - op_ = op; - intermediateType_ = intermediateType; - setBinaryType(TFunctionBinaryType.BUILTIN); - } - - @Override - public TFunction toThrift() { - TFunction fn = super.toThrift(); - // TODO: for now, just put the op_ enum as the id. - fn.setId(op_.thriftOp.ordinal()); - fn.setAggregate_fn(new TAggregateFunction(intermediateType_.toThrift())); - return fn; - } - - public Operator op() { return op_; } - public ColumnType getIntermediateType() { return intermediateType_; } - public void setIntermediateType(ColumnType t) { intermediateType_ = t; } -} diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CaseExpr.java b/fe/src/main/java/com/cloudera/impala/analysis/CaseExpr.java index ef31aa85d..22f18fed0 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/CaseExpr.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/CaseExpr.java @@ -17,12 +17,15 @@ package com.cloudera.impala.analysis; import java.util.List; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Db; +import com.cloudera.impala.catalog.Function.CompareMode; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; -import com.cloudera.impala.opcode.FunctionOperator; import com.cloudera.impala.thrift.TCaseExpr; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; /** * CaseExpr represents the SQL expression @@ -54,6 +57,19 @@ public class CaseExpr extends Expr { } } + public static void initBuiltins(Db db) { + for (ColumnType t: ColumnType.getSupportedTypes()) { + if (t.isNull()) continue; + // TODO: case is special and the signature cannot be represented. + // It is alternating varargs + // e.g. case(bool, type, bool type, bool type, etc). + // Instead we just add a version for each of the when types + // e.g. case(BOOLEAN), case(INT), etc + db.addBuiltin(ScalarFunction.createBuiltinOperator( + "case", Lists.newArrayList(t), t)); + } + } + @Override public boolean equals(Object obj) { if (!super.equals(obj)) return false; @@ -83,7 +99,6 @@ public class CaseExpr extends Expr { protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.CASE_EXPR; msg.case_expr = new TCaseExpr(hasCaseExpr_, hasElseExpr_); - msg.setOpcode(opcode_); } @Override @@ -176,15 +191,13 @@ public class CaseExpr extends Expr { } } - // Set opcode based on whenType. - OpcodeRegistry.BuiltinFunction match = OpcodeRegistry.instance().getFunctionInfo( - FunctionOperator.CASE, true, whenType); - if (match == null) { - throw new AnalysisException("Could not find match in function registry " + - "for CASE and arg type: " + whenType); + // Do the function lookup just based on the whenType. + ColumnType[] args = new ColumnType[1]; + args[0] = whenType; + fn_ = getBuiltinFunction(analyzer, "case", args, CompareMode.IS_SUBTYPE); + if (fn_ == null) { + throw new AnalysisException("CASE " + whenType + " is not supported."); } - opcode_ = match.opcode; type_ = returnType; - isAnalyzed_ = true; } } diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CastExpr.java b/fe/src/main/java/com/cloudera/impala/analysis/CastExpr.java index af9554242..c0230a147 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/CastExpr.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/CastExpr.java @@ -15,13 +15,18 @@ package com.cloudera.impala.analysis; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Catalog; +import com.cloudera.impala.catalog.Db; +import com.cloudera.impala.catalog.Function; +import com.cloudera.impala.catalog.Function.CompareMode; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; -import com.cloudera.impala.opcode.FunctionOperator; import com.cloudera.impala.thrift.TExpr; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; import com.google.common.base.Objects; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; public class CastExpr extends Expr { @@ -33,6 +38,8 @@ public class CastExpr extends Expr { // True if this cast does not change the type. private boolean noOp_ = false; + private static final String CAST_FN_NAME = "cast"; + public CastExpr(ColumnType targetType, Expr e, boolean isImplicit) { super(); Preconditions.checkArgument(targetType.isValid()); @@ -41,12 +48,31 @@ public class CastExpr extends Expr { Preconditions.checkNotNull(e); children_.add(e); if (isImplicit) { - type_ = targetType; - OpcodeRegistry.BuiltinFunction match = OpcodeRegistry.instance().getFunctionInfo( - FunctionOperator.CAST, true, getChild(0).getType(), type_); - Preconditions.checkState(match != null); - Preconditions.checkState(match.getReturnType().equals(type_)); - this.opcode_ = match.opcode; + // Implicit casts don't call analyze() + // TODO: this doesn't seem like the cleanest approach but there are places + // we generate these (e.g. table loading) where there is no analyzer object. + try { + analyze(); + computeNumDistinctValues(); + } catch (AnalysisException ex) { + Preconditions.checkState(false, + "Implicit casts should never throw analysis exception."); + } + isAnalyzed_ = true; + } + } + + public static void initBuiltins(Db db) { + for (ColumnType t1: ColumnType.getSupportedTypes()) { + if (t1.isNull()) continue; + for (ColumnType t2: ColumnType.getSupportedTypes()) { + if (t2.isNull()) continue; + // For some reason we don't allow string->bool. + // TODO: revisit + if (t1.isStringType() && t2.isBoolean()) continue; + db.addBuiltin(ScalarFunction.createBuiltinOperator( + CAST_FN_NAME, Lists.newArrayList(t1, t2), t2)); + } } } @@ -68,7 +94,6 @@ public class CastExpr extends Expr { @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.CAST_EXPR; - msg.setOpcode(opcode_); } @Override @@ -87,34 +112,39 @@ public class CastExpr extends Expr { AuthorizationException { if (isAnalyzed_) return; super.analyze(analyzer); + analyze(); + } - if (isImplicit_) return; - - // cast was asked for in the query, check for validity of cast - // this cast may result in loss of precision, but the user requested it - ColumnType childType = getChild(0).getType(); - this.type_ = targetType_; - - if (childType.equals(targetType_)) { + private void analyze() throws AnalysisException { + // Our cast fn currently takes two arguments. The first is the value to cast and the + // second is a dummy of the type to cast to. We need this to be able to resolve the + // proper function. + // e.g. to differentiate between cast(bool, int) and cast(bool, smallint). + // TODO: this is not very intuitive. We could also call the functions castToInt(*) + ColumnType[] args = new ColumnType[2]; + args[0] = children_.get(0).type_; + args[1] = targetType_; + if (args[0].equals(args[1])) { noOp_ = true; + type_ = targetType_; return; } - OpcodeRegistry.BuiltinFunction match = OpcodeRegistry.instance().getFunctionInfo( - FunctionOperator.CAST, childType.isNull(), getChild(0).getType(), type_); - if (match == null) { - throw new AnalysisException("Invalid type cast of " + getChild(0).toSql() + - " from " + childType + " to " + targetType_); + FunctionName fnName = new FunctionName(Catalog.BUILTINS_DB, CAST_FN_NAME); + Function searchDesc = new Function(fnName, args, ColumnType.INVALID, false); + if (isImplicit_) { + fn_ = Catalog.getBuiltin(searchDesc, CompareMode.IS_SUBTYPE); + Preconditions.checkState(fn_ != null); + } else { + fn_ = Catalog.getBuiltin(searchDesc, + args[0].isNull() ? CompareMode.IS_SUBTYPE : CompareMode.IS_IDENTICAL); + if (fn_ == null) { + throw new AnalysisException("Invalid type cast of " + getChild(0).toSql() + + " from " + args[0] + " to " + args[1]); + } } - Preconditions.checkState(match.getReturnType().equals(targetType_)); - this.opcode_ = match.opcode; - } - - @Override - public boolean equals(Object obj) { - if (!super.equals(obj)) return false; - CastExpr expr = (CastExpr) obj; - return this.opcode_ == expr.opcode_; + Preconditions.checkState(fn_.getReturnType().equals(targetType_)); + type_ = targetType_; } /** diff --git a/fe/src/main/java/com/cloudera/impala/analysis/ColumnType.java b/fe/src/main/java/com/cloudera/impala/analysis/ColumnType.java index e6229be61..7aa37d1b9 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/ColumnType.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/ColumnType.java @@ -63,6 +63,9 @@ public class ColumnType { private static ArrayList fixedSizeNumericTypes; private static ArrayList fixedPointTypes; + private static ArrayList numericTypes; + private static ArrayList nativeTypes; + private static ArrayList supportedTypes; static { fixedSizeNumericTypes = Lists.newArrayList(); @@ -78,6 +81,35 @@ public class ColumnType { fixedPointTypes.add(SMALLINT); fixedPointTypes.add(INT); fixedPointTypes.add(BIGINT); + + numericTypes = Lists.newArrayList(); + numericTypes.add(TINYINT); + numericTypes.add(SMALLINT); + numericTypes.add(INT); + numericTypes.add(BIGINT); + numericTypes.add(FLOAT); + numericTypes.add(DOUBLE); + + nativeTypes = Lists.newArrayList(); + nativeTypes.add(BOOLEAN); + nativeTypes.add(TINYINT); + nativeTypes.add(SMALLINT); + nativeTypes.add(INT); + nativeTypes.add(BIGINT); + nativeTypes.add(FLOAT); + nativeTypes.add(DOUBLE); + + supportedTypes = Lists.newArrayList(); + supportedTypes.add(NULL); + supportedTypes.add(BOOLEAN); + supportedTypes.add(TINYINT); + supportedTypes.add(SMALLINT); + supportedTypes.add(INT); + supportedTypes.add(BIGINT); + supportedTypes.add(FLOAT); + supportedTypes.add(DOUBLE); + supportedTypes.add(STRING); + supportedTypes.add(TIMESTAMP); } private ColumnType(PrimitiveType type) { @@ -134,6 +166,15 @@ public class ColumnType { public static ArrayList getFixedPointTypes() { return fixedPointTypes; } + public static ArrayList getNumericTypes() { + return numericTypes; + } + public static ArrayList getNativeTypes() { + return nativeTypes; + } + public static ArrayList getSupportedTypes() { + return supportedTypes; + } @Override public boolean equals(Object o) { @@ -575,7 +616,8 @@ public class ColumnType { compatibilityMatrix[TIMESTAMP.ordinal()][TIMESTAMP.ordinal()] = PrimitiveType.TIMESTAMP; - compatibilityMatrix[TIMESTAMP.ordinal()][STRING.ordinal()] = PrimitiveType.TIMESTAMP; + compatibilityMatrix[TIMESTAMP.ordinal()][STRING.ordinal()] = + PrimitiveType.TIMESTAMP; compatibilityMatrix[STRING.ordinal()][STRING.ordinal()] = PrimitiveType.STRING; } diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CompoundPredicate.java b/fe/src/main/java/com/cloudera/impala/analysis/CompoundPredicate.java index aa01d6e2e..14ede350e 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/CompoundPredicate.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/CompoundPredicate.java @@ -18,11 +18,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Db; +import com.cloudera.impala.catalog.Function.CompareMode; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; -import com.cloudera.impala.thrift.TExprOpcode; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; /** * &&, ||, ! predicates. @@ -32,29 +35,35 @@ public class CompoundPredicate extends Predicate { private final static Logger LOG = LoggerFactory.getLogger(CompoundPredicate.class); public enum Operator { - AND("AND", TExprOpcode.COMPOUND_AND), - OR("OR", TExprOpcode.COMPOUND_OR), - NOT("NOT", TExprOpcode.COMPOUND_NOT); + AND("AND"), + OR("OR"), + NOT("NOT"); private final String description; - private final TExprOpcode thriftOp; - private Operator(String description, TExprOpcode thriftOp) { + private Operator(String description) { this.description = description; - this.thriftOp = thriftOp; } @Override public String toString() { return description; } - - public TExprOpcode toThrift() { - return thriftOp; - } } private final Operator op_; + public static void initBuiltins(Db db) { + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.AND.name(), "CompoundPredicate", "AndComputeFn", + Lists.newArrayList(ColumnType.BOOLEAN, ColumnType.BOOLEAN), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.OR.name(), "CompoundPredicate", "OrComputeFn", + Lists.newArrayList(ColumnType.BOOLEAN, ColumnType.BOOLEAN), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.NOT.name(), "CompoundPredicate", "NotComputeFn", + Lists.newArrayList(ColumnType.BOOLEAN), ColumnType.BOOLEAN)); + } + public CompoundPredicate(Operator op, Expr e1, Expr e2) { super(); this.op_ = op; @@ -86,7 +95,6 @@ public class CompoundPredicate extends Predicate { @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.COMPOUND_PRED; - msg.setOpcode(op_.toThrift()); } @Override @@ -104,6 +112,11 @@ public class CompoundPredicate extends Predicate { } } + fn_ = getBuiltinFunction(analyzer, op_.toString(), collectChildReturnTypes(), + CompareMode.IS_SUBTYPE); + Preconditions.checkState(fn_ != null); + Preconditions.checkState(fn_.getReturnType().isBoolean()); + if (getChild(0).selectivity_ == -1 || children_.size() == 2 && getChild(1).selectivity_ == -1) { // give up if we're missing an input diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CreateFunctionStmtBase.java b/fe/src/main/java/com/cloudera/impala/analysis/CreateFunctionStmtBase.java index a0d72fa90..fe10c286b 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/CreateFunctionStmtBase.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/CreateFunctionStmtBase.java @@ -18,6 +18,8 @@ import java.util.HashMap; import com.cloudera.impala.authorization.Privilege; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Catalog; +import com.cloudera.impala.catalog.Db; import com.cloudera.impala.catalog.Function; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.common.InternalException; @@ -162,14 +164,17 @@ public class CreateFunctionStmtBase extends StatementBase { // Validate function name is legal fn_.getFunctionName().analyze(analyzer); - - // Validate DB is legal - String dbName = analyzer.getTargetDbName(fn_.getFunctionName()); - fn_.getFunctionName().setDb(dbName); - if (analyzer.getCatalog().getDb( - dbName, analyzer.getUser(), Privilege.CREATE) == null) { - throw new AnalysisException(Analyzer.DB_DOES_NOT_EXIST_ERROR_MSG + dbName); + Db builtinsDb = analyzer.getCatalog().getDb(Catalog.BUILTINS_DB); + if (builtinsDb.containsFunction(fn_.getName())) { + throw new AnalysisException("Function cannot have the same name as a builtin: " + + fn_.getFunctionName().getFunction()); } + + if (analyzer.getCatalog().getDb( + fn_.dbName(), analyzer.getUser(), Privilege.CREATE) == null) { + throw new AnalysisException(Analyzer.DB_DOES_NOT_EXIST_ERROR_MSG + fn_.dbName()); + } + Function existingFn = analyzer.getCatalog().getFunction( fn_, Function.CompareMode.IS_INDISTINGUISHABLE); if (existingFn != null && !ifNotExists_) { diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java b/fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java index f1dd1b7ed..3993fb1ab 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java @@ -167,6 +167,7 @@ public class CreateTableStmt extends StatementBase { throw new AnalysisException(Analyzer.TBL_ALREADY_EXISTS_ERROR_MSG + String.format("%s.%s", dbName_, getTbl())); } + analyzer.addAccessEvent(new TAccessEvent(dbName_ + "." + tableName_.getTbl(), TCatalogObjectType.TABLE, Privilege.CREATE.toString())); diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CreateUdaStmt.java b/fe/src/main/java/com/cloudera/impala/analysis/CreateUdaStmt.java index 9a39b7075..fb6b4c076 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/CreateUdaStmt.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/CreateUdaStmt.java @@ -17,7 +17,7 @@ package com.cloudera.impala.analysis; import java.util.HashMap; import com.cloudera.impala.catalog.AuthorizationException; -import com.cloudera.impala.catalog.Uda; +import com.cloudera.impala.catalog.AggregateFunction; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.thrift.TAggregateFunction; import com.cloudera.impala.thrift.TCreateFunctionParams; @@ -29,7 +29,7 @@ import com.google.common.base.Preconditions; */ public class CreateUdaStmt extends CreateFunctionStmtBase { // Same as super.fn_. Typed here for convenience. - private final Uda uda_; + private final AggregateFunction uda_; private ColumnType intermediateType_; /** @@ -47,8 +47,8 @@ public class CreateUdaStmt extends CreateFunctionStmtBase { ColumnType retType, ColumnType intermediateType, HdfsUri location, boolean ifNotExists, HashMap optArgs) { - super(new Uda(fnSymbol, args, retType), location, ifNotExists, optArgs); - uda_ = (Uda)super.fn_; + super(new AggregateFunction(fnSymbol, args, retType), location, ifNotExists, optArgs); + uda_ = (AggregateFunction)super.fn_; intermediateType_ = intermediateType; } diff --git a/fe/src/main/java/com/cloudera/impala/analysis/CreateUdfStmt.java b/fe/src/main/java/com/cloudera/impala/analysis/CreateUdfStmt.java index 5eb644561..35bc1d88d 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/CreateUdfStmt.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/CreateUdfStmt.java @@ -18,7 +18,7 @@ import java.util.HashMap; import com.cloudera.impala.catalog.AuthorizationException; import com.cloudera.impala.catalog.PrimitiveType; -import com.cloudera.impala.catalog.Udf; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.thrift.TCreateFunctionParams; import com.cloudera.impala.thrift.TFunctionBinaryType; @@ -29,7 +29,7 @@ import com.cloudera.impala.thrift.TScalarFunction; */ public class CreateUdfStmt extends CreateFunctionStmtBase { // Same as super.fn_. Typed here for convenience. - private final Udf udf_; + private final ScalarFunction udf_; /** * Builds a CREATE FUNCTION statement @@ -44,8 +44,8 @@ public class CreateUdfStmt extends CreateFunctionStmtBase { public CreateUdfStmt(FunctionName fnName, FunctionArgs args, ColumnType retType, HdfsUri location, boolean ifNotExists, HashMap optArgs) { - super(new Udf(fnName, args, retType), location, ifNotExists, optArgs); - udf_ = (Udf)fn_; + super(new ScalarFunction(fnName, args, retType), location, ifNotExists, optArgs); + udf_ = (ScalarFunction)fn_; } @Override diff --git a/fe/src/main/java/com/cloudera/impala/analysis/DropFunctionStmt.java b/fe/src/main/java/com/cloudera/impala/analysis/DropFunctionStmt.java index 9620d3564..468f95fd9 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/DropFunctionStmt.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/DropFunctionStmt.java @@ -69,11 +69,9 @@ public class DropFunctionStmt extends StatementBase { analyzer.getCatalog().checkCreateDropFunctionAccess(analyzer.getUser()); desc_.getFunctionName().analyze(analyzer); - String dbName = analyzer.getTargetDbName(desc_.getFunctionName()); - desc_.getFunctionName().setDb(dbName); - if (analyzer.getCatalog().getDb(dbName, analyzer.getUser(), Privilege.DROP) == null - && !ifExists_) { - throw new AnalysisException(Analyzer.DB_DOES_NOT_EXIST_ERROR_MSG + dbName); + if (analyzer.getCatalog().getDb( + desc_.dbName(), analyzer.getUser(), Privilege.DROP) == null && !ifExists_) { + throw new AnalysisException(Analyzer.DB_DOES_NOT_EXIST_ERROR_MSG + desc_.dbName()); } if (analyzer.getCatalog().getFunction( diff --git a/fe/src/main/java/com/cloudera/impala/analysis/Expr.java b/fe/src/main/java/com/cloudera/impala/analysis/Expr.java index c5b3074f7..862bb8a70 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/Expr.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/Expr.java @@ -27,11 +27,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Catalog; +import com.cloudera.impala.catalog.Function; +import com.cloudera.impala.catalog.Function.CompareMode; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.common.TreeNode; import com.cloudera.impala.thrift.TExpr; import com.cloudera.impala.thrift.TExprNode; -import com.cloudera.impala.thrift.TExprOpcode; import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Preconditions; @@ -64,7 +66,6 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl protected ColumnType type_; // result of analysis protected boolean isAnalyzed_; // true after analyze() has been called protected boolean isWhereClauseConjunct_; // set by Analyzer - protected TExprOpcode opcode_; // opcode for this expr // Flag to indicate whether to wrap this expr's toSql() in parenthesis. Set by parser. // Needed for properly capturing expr precedences in the SQL string. @@ -82,10 +83,13 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl // set during analysis protected long numDistinctValues_; + // The function to call. This can either be a scalar or aggregate function. + // Set in analyze(). + protected Function fn_; + protected Expr() { super(); type_ = ColumnType.INVALID; - opcode_ = TExprOpcode.INVALID_OPCODE; selectivity_ = -1.0; numDistinctValues_ = -1; } @@ -93,7 +97,6 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl public ExprId getId() { return id_; } protected void setId(ExprId id) { this.id_ = id; } public ColumnType getType() { return type_; } - public TExprOpcode getOpcode() { return opcode_; } public double getSelectivity() { return selectivity_; } public long getNumDistinctValues() { return numDistinctValues_; } public void setPrintSqlInParens(boolean b) { printSqlInParens_ = b; } @@ -143,7 +146,12 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl child.analyze(analyzer); } isAnalyzed_ = true; + computeNumDistinctValues(); + if (analyzer != null) analyzer.decrementCallDepth(); + } + + protected void computeNumDistinctValues() { if (isConstant()) { numDistinctValues_ = 1; } else { @@ -158,7 +166,45 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl numDistinctValues_ = Math.max(numDistinctValues_, slotRef.numDistinctValues_); } } - if (analyzer != null) analyzer.decrementCallDepth(); + } + + /** + * Collects the returns types of the child nodes in an array. + */ + protected ColumnType[] collectChildReturnTypes() { + ColumnType[] childTypes = new ColumnType[children_.size()]; + for (int i = 0; i < children_.size(); ++i) { + childTypes[i] = children_.get(i).type_; + } + return childTypes; + } + + /** + * Looks up in the catalog the builtin for 'name' and 'argTypes'. + * Returns null if the function is not found. + */ + protected Function getBuiltinFunction(Analyzer analyzer, String name, + ColumnType[] argTypes, CompareMode mode) throws AnalysisException { + FunctionName fnName = new FunctionName(Catalog.BUILTINS_DB, name); + Function searchDesc = new Function(fnName, argTypes, ColumnType.INVALID, false); + return analyzer.getCatalog().getFunction(searchDesc, mode); + } + + /** + * Generates the necessary casts for the children of this expr to call fn_. + * child(0) is cast to the functions first argument, child(1) to the second etc. + * This does not do any validation and the casts are assumed to be same. + */ + protected void castForFunctionCall() throws AnalysisException { + Preconditions.checkState(fn_ != null); + ColumnType[] fnArgs = fn_.getArgs(); + if (fnArgs.length > 0) { + for (int i = 0; i < children_.size(); ++i) { + // For varargs, we must compare with the last type in fnArgs.argTypes. + int ix = Math.min(fnArgs.length - 1, i); + if (!children_.get(i).type_.equals(fnArgs[ix])) castChild(fnArgs[ix], i); + } + } } /** @@ -210,9 +256,15 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl // Append a flattened version of this expr, including all children, to 'container'. protected void treeToThriftHelper(TExpr container) { + Preconditions.checkState(isAnalyzed_, + "Must be analyzed before serializing to thrift."); TExprNode msg = new TExprNode(); msg.type = type_.toThrift(); msg.num_children = children_.size(); + if (fn_ != null) { + msg.setFn(fn_.toThrift()); + if (fn_.hasVarArgs()) msg.setVararg_start_idx(fn_.getNumArgs() - 1); + } toThrift(msg); container.addToNodes(msg); for (Expr child: children_) { @@ -308,7 +360,10 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl for (int i = 0; i < children_.size(); ++i) { if (!children_.get(i).equals(expr.children_.get(i))) return false; } - return true; + if (fn_ == null && expr.fn_ == null) return true; + if (fn_ == null || expr.fn_ == null) return false; // One null, one not + // Both fn_'s are not null + return fn_.equals(expr.fn_); } /** @@ -793,8 +848,7 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl * @param childIndex * index of child to be cast */ - public void castChild(ColumnType targetType, - int childIndex) throws AnalysisException { + public void castChild(ColumnType targetType, int childIndex) throws AnalysisException { Expr child = getChild(childIndex); Expr newChild = child.castTo(targetType); setChild(childIndex, newChild); @@ -822,10 +876,8 @@ abstract public class Expr extends TreeNode implements ParseNode, Cloneabl * The possibly changed compatibleType * (if a string literal forced casting the other operand) */ - public ColumnType castBinaryOp(ColumnType compatibleType) - throws AnalysisException { - Preconditions.checkState( - this instanceof BinaryPredicate || this instanceof ArithmeticExpr); + public ColumnType castBinaryOp(ColumnType compatibleType) throws AnalysisException { + Preconditions.checkState(this instanceof ArithmeticExpr); ColumnType t1 = getChild(0).getType(); ColumnType t2 = getChild(1).getType(); // Convert string literals if the other operand is numeric, diff --git a/fe/src/main/java/com/cloudera/impala/analysis/FunctionCallExpr.java b/fe/src/main/java/com/cloudera/impala/analysis/FunctionCallExpr.java index cc3ba1f83..9a85bb65f 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/FunctionCallExpr.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/FunctionCallExpr.java @@ -14,40 +14,27 @@ package com.cloudera.impala.analysis; -import java.util.ArrayList; import java.util.List; import com.cloudera.impala.authorization.Privilege; +import com.cloudera.impala.catalog.AggregateFunction; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Catalog; +import com.cloudera.impala.catalog.Db; import com.cloudera.impala.catalog.Function; -import com.cloudera.impala.catalog.PrimitiveType; -import com.cloudera.impala.catalog.Uda; -import com.cloudera.impala.catalog.Udf; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; -import com.cloudera.impala.opcode.FunctionOperator; -import com.cloudera.impala.thrift.TAggregateFunctionCall; -import com.cloudera.impala.thrift.TExpr; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; -import com.cloudera.impala.thrift.TFunctionCallExpr; +import com.cloudera.impala.thrift.TFunctionBinaryType; import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -// TODO: for aggregations, we need to unify the code paths for builtins and UDAs. public class FunctionCallExpr extends Expr { private final FunctionName fnName_; - private final BuiltinAggregateFunction.Operator agg_op_; private final FunctionParams params_; - // The function to call. This can either be a builtin scalar, UDF or UDA. - // Set in analyze() except if this expr is created from creating the - // distributed plan for aggregations. In that case, fn_ is inherited from - // the FunctionCallExpr expr object that was created during the non-distributed - // planning phase. - private Function fn_; - public FunctionCallExpr(String functionName, List params) { this(new FunctionName(functionName), new FunctionParams(false, params)); } @@ -60,15 +47,6 @@ public class FunctionCallExpr extends Expr { super(); this.fnName_ = fnName; params_ = params; - agg_op_ = null; - if (params.exprs() != null) children_.addAll(params.exprs()); - } - - public FunctionCallExpr(BuiltinAggregateFunction.Operator op, FunctionParams params) { - Preconditions.checkState(op != null); - fnName_ = FunctionName.CreateBuiltinName(op.name()); - agg_op_ = op; - params_ = params; if (params.exprs() != null) children_.addAll(params.exprs()); } @@ -77,20 +55,26 @@ public class FunctionCallExpr extends Expr { Preconditions.checkState(e.isAnalyzed_); Preconditions.checkState(e.isAggregateFunction()); fnName_ = e.fnName_; - agg_op_ = e.agg_op_; params_ = params; // Just inherit the function object from 'e'. fn_ = e.fn_; if (params.exprs() != null) children_.addAll(params.exprs()); } + // This is a total hack because of how we remap count/avg aggregate functions. + // This needs to be removed when we stop doing the rewrites. + public FunctionCallExpr(String name, FunctionParams params) { + fnName_ = new FunctionName(Catalog.BUILTINS_DB, name); + params_ = params; + // Just inherit the function object from 'e'. + if (params.exprs() != null) children_.addAll(params.exprs()); + } + @Override public boolean equals(Object obj) { if (!super.equals(obj)) return false; FunctionCallExpr o = (FunctionCallExpr)obj; - return opcode_ == o.opcode_ && - agg_op_ == o.agg_op_ && - fnName_.equals(o.fnName_) && + return fnName_.equals(o.fnName_) && params_.isDistinct() == o.params_.isDistinct() && params_.isStar() == o.params_.isStar(); } @@ -108,7 +92,6 @@ public class FunctionCallExpr extends Expr { @Override public String debugString() { return Objects.toStringHelper(this) - .add("op", agg_op_) .add("name", fnName_) .add("isStar", params_.isStar()) .add("isDistinct", params_.isDistinct()) @@ -119,12 +102,12 @@ public class FunctionCallExpr extends Expr { public FunctionParams getParams() { return params_; } public boolean isScalarFunction() { Preconditions.checkState(fn_ != null); - return fn_ instanceof Udf || fn_ instanceof OpcodeRegistry.BuiltinFunction; + return fn_ instanceof ScalarFunction ; } public boolean isAggregateFunction() { Preconditions.checkState(fn_ != null); - return fn_ instanceof Uda || fn_ instanceof BuiltinAggregateFunction; + return fn_ instanceof AggregateFunction; } public boolean isDistinct() { @@ -132,206 +115,50 @@ public class FunctionCallExpr extends Expr { return params_.isDistinct(); } - public BuiltinAggregateFunction.Operator getAggOp() { return agg_op_; } + public FunctionName getFnName() { return fnName_; } @Override protected void toThrift(TExprNode msg) { - // TODO: we never serialize this to thrift if it's an aggregate function - // except in test cases that do it explicitly. - if (isAggregate()) return; - - if (fn_ instanceof Udf) { - Udf udf = (Udf)fn_; - msg.node_type = TExprNodeType.FUNCTION_CALL; - TFunctionCallExpr fnCall = new TFunctionCallExpr(); - fnCall.setFn(udf.toThrift()); - if (udf.hasVarArgs()) { - fnCall.setVararg_start_idx(udf.getNumArgs() - 1); - } - msg.setFn_call_expr(fnCall); + if (isAggregate()) { + msg.node_type = TExprNodeType.AGGREGATE_EXPR; } else { - Preconditions.checkState(fn_ instanceof OpcodeRegistry.BuiltinFunction); - OpcodeRegistry.BuiltinFunction builtin = (OpcodeRegistry.BuiltinFunction)fn_; - if (builtin.udfInterface) { - msg.node_type = TExprNodeType.FUNCTION_CALL; - TFunctionCallExpr fnCall = new TFunctionCallExpr(); - fnCall.setFn(fn_.toThrift()); - if (fn_.hasVarArgs()) { - fnCall.setVararg_start_idx(fn_.getNumArgs() - 1); - } - msg.setFn_call_expr(fnCall); - } else { - // TODO: remove. All builtins will go through UDF_CALL. + ScalarFunction fn = (ScalarFunction)fn_; + if (fn.getBinaryType() == TFunctionBinaryType.BUILTIN && + !fn.isUdfInterface()) { msg.node_type = TExprNodeType.COMPUTE_FUNCTION_CALL; - } - msg.setOpcode(opcode_); - } - } - - public TAggregateFunctionCall toTAggregateFunctionCall() { - Preconditions.checkState(isAggregateFunction()); - - List inputExprs = Lists.newArrayList(); - for (Expr e: children_) { - inputExprs.add(e.treeToThrift()); - } - - TAggregateFunctionCall f = new TAggregateFunctionCall(); - f.setFn(fn_.toThrift()); - f.setInput_exprs(inputExprs); - return f; - } - - private void analyzeBuiltinAggFunction(Analyzer analyzer) throws AnalysisException, - AuthorizationException { - Preconditions.checkState(agg_op_ != null); - - if (params_.isStar() && agg_op_ != BuiltinAggregateFunction.Operator.COUNT) { - throw new AnalysisException("'*' can only be used in conjunction with COUNT: " - + this.toSql()); - } - - if (agg_op_ == BuiltinAggregateFunction.Operator.COUNT) { - // for multiple exprs count must be qualified with distinct - if (children_.size() > 1 && !params_.isDistinct()) { - throw new AnalysisException( - "COUNT must have DISTINCT for multiple arguments: " + this.toSql()); - } - ArrayList childTypes = Lists.newArrayList(); - for (int i = 0; i < children_.size(); ++i) { - childTypes.add(getChild(i).type_); - } - fn_ = new BuiltinAggregateFunction(agg_op_, childTypes, - ColumnType.BIGINT, ColumnType.BIGINT); - return; - } - - if (agg_op_ == BuiltinAggregateFunction.Operator.GROUP_CONCAT) { - ArrayList argTypes = Lists.newArrayList(); - if (children_.size() > 2 || children_.isEmpty()) { - throw new AnalysisException( - agg_op_.toString() + " requires one or two parameters: " + this.toSql()); - } - - if (params_.isDistinct()) { - throw new AnalysisException(agg_op_.toString() + " does not support DISTINCT"); - } - - Expr arg0 = getChild(0); - if (!arg0.type_.isStringType() && !arg0.type_.isNull()) { - throw new AnalysisException( - agg_op_.toString() + " requires first parameter to be of type STRING: " - + this.toSql()); - } - argTypes.add(ColumnType.STRING); - - if (children_.size() == 2) { - Expr arg1 = getChild(1); - if (!arg1.type_.isStringType() && !arg1.type_.isNull()) { - throw new AnalysisException( - agg_op_.toString() + " requires second parameter to be of type STRING: " - + this.toSql()); - } - argTypes.add(ColumnType.STRING); } else { - // Add the default string so the BE always see two arguments. - Expr arg2 = new NullLiteral(); - arg2.analyze(analyzer); - addChild(arg2); - argTypes.add(ColumnType.NULL); + msg.node_type = TExprNodeType.FUNCTION_CALL; } - // TODO: we really want the intermediate type to be CHAR(16) - fn_ = new BuiltinAggregateFunction(agg_op_, argTypes, - ColumnType.STRING, agg_op_.intermediateType()); - return; } - - // only COUNT and GROUP_CONCAT can contain multiple exprs - if (children_.size() != 1) { - throw new AnalysisException( - agg_op_.toString() + " requires exactly one parameter: " + this.toSql()); - } - - // determine type - Expr arg = getChild(0); - - // SUM and AVG cannot be applied to non-numeric types - if (agg_op_ == BuiltinAggregateFunction.Operator.SUM && - !arg.type_.isNumericType() && !arg.type_.isNull()) { - throw new AnalysisException( - "SUM requires a numeric parameter: " + this.toSql()); - } - if (agg_op_ == BuiltinAggregateFunction.Operator.AVG && !arg.type_.isNumericType() && - arg.type_.getPrimitiveType() != PrimitiveType.TIMESTAMP && !arg.type_.isNull()) { - throw new AnalysisException( - "AVG requires a numeric or timestamp parameter: " + this.toSql()); - } - - ColumnType intermediateType = agg_op_.intermediateType(); - ArrayList argTypes = Lists.newArrayList(); - if (agg_op_ == BuiltinAggregateFunction.Operator.AVG) { - // division always results in a double value - type_ = ColumnType.DOUBLE; - intermediateType = type_; - } else if (agg_op_ == BuiltinAggregateFunction.Operator.SUM) { - // numeric types need to be accumulated at maximum precision - type_ = arg.type_.getMaxResolutionType(); - argTypes.add(type_); - intermediateType = type_; - } else if (agg_op_ == BuiltinAggregateFunction.Operator.MIN || - agg_op_ == BuiltinAggregateFunction.Operator.MAX) { - type_ = arg.type_; - params_.setIsDistinct(false); // DISTINCT is meaningless here - argTypes.add(type_); - intermediateType = type_; - } else if (agg_op_ == BuiltinAggregateFunction.Operator.DISTINCT_PC || - agg_op_ == BuiltinAggregateFunction.Operator.DISTINCT_PCSA || - agg_op_ == BuiltinAggregateFunction.Operator.NDV) { - type_ = ColumnType.STRING; - params_.setIsDistinct(false); - argTypes.add(arg.getType()); - } - fn_ = new BuiltinAggregateFunction(agg_op_, argTypes, type_, intermediateType); } - // Sets fn_ to the proper function object. - private void setFunction(Analyzer analyzer, ColumnType[] argTypes) - throws AnalysisException, AuthorizationException { - // First check if this is a builtin - FunctionOperator op = OpcodeRegistry.instance().getFunctionOperator( - fnName_.getFunction()); - if (op != FunctionOperator.INVALID_OPERATOR) { - OpcodeRegistry.BuiltinFunction match = - OpcodeRegistry.instance().getFunctionInfo(op, true, argTypes); - if (match != null) { - this.opcode_ = match.opcode; - fn_ = match; + // Provide better error message for some aggregate builtins. These can be + // a bit more user friendly than a generic function not found. + // TODO: should we bother to do this? We could also improve the general + // error messages. For example, listing the alternatives. + private String getFunctionNotFoundError(ColumnType[] argTypes) { + if (fnName_.isBuiltin_) { + // Some custom error message for builtins + if (params_.isStar()) { + return "'*' can only be used in conjunction with COUNT"; } - } else { - // Next check if it is a UDF/UDA - String dbName = analyzer.getTargetDbName(fnName_); - fnName_.setDb(dbName); - - // User needs DB access. - analyzer.getDb(dbName, Privilege.ANY); - - if (!analyzer.getCatalog().functionExists(fnName_)) { - throw new AnalysisException(fnName_ + "() unknown"); + if (fnName_.getFunction().equalsIgnoreCase("count")) { + if (!params_.isDistinct() && argTypes.length > 1) { + return "COUNT must have DISTINCT for multiple arguments: " + toSql(); + } + } + if (fnName_.getFunction().equalsIgnoreCase("sum")) { + return "SUM requires a numeric parameter: " + toSql(); + } + if (fnName_.getFunction().equalsIgnoreCase("avg")) { + return "AVG requires a numeric or timestamp parameter: " + toSql(); } - - Function searchDesc = - new Function(fnName_, argTypes, ColumnType.INVALID, false); - - fn_ = analyzer.getCatalog().getFunction( - searchDesc, Function.CompareMode.IS_SUBTYPE); } - if (fn_ == null) { - throw new AnalysisException(String.format( - "No matching function with signature: %s(%s).", - fnName_, Joiner.on(", ").join(argTypes))); - } + return String.format( + "No matching function with signature: %s(%s).", + fnName_, params_.isStar() ? "*" : Joiner.on(", ").join(argTypes)); } @Override @@ -339,41 +166,59 @@ public class FunctionCallExpr extends Expr { AuthorizationException { if (isAnalyzed_) return; super.analyze(analyzer); + fnName_.analyze(analyzer); if (fn_ != null && isAggregate()) { - ColumnType intermediateType = null; - if (fn_ instanceof Uda) { - intermediateType = ((Uda)fn_).getIntermediateType(); - } else { - Preconditions.checkState(fn_ instanceof BuiltinAggregateFunction); - intermediateType = ((BuiltinAggregateFunction)fn_).getIntermediateType(); - } + // This is the function call expr after splitting up to a merge aggregation. + // The function has already been analyzed so just do the minimal sanity + // check here. + // TODO: rethink how we generate the merge aggregation. + AggregateFunction aggFn = (AggregateFunction)fn_; + ColumnType intermediateType = aggFn.getIntermediateType(); type_ = fn_.getReturnType(); if (intermediateType == null) intermediateType = type_; // TODO: this needs to change when the intermediate type != the return type Preconditions.checkArgument(intermediateType.equals(fn_.getReturnType())); - - if (agg_op_ == BuiltinAggregateFunction.Operator.GROUP_CONCAT && - getChildren().size() == 1) { - Expr arg2 = new NullLiteral(); - arg2.analyze(analyzer); - addChild(arg2); - } return; } - ColumnType[] argTypes = new ColumnType[this.children_.size()]; - for (int i = 0; i < this.children_.size(); ++i) { - this.children_.get(i).analyze(analyzer); - argTypes[i] = this.children_.get(i).getType(); + ColumnType[] argTypes = collectChildReturnTypes(); + + // User needs DB access. + Db db = analyzer.getDb(fnName_.getDb(), Privilege.VIEW_METADATA); + if (!db.containsFunction(fnName_.getFunction())) { + throw new AnalysisException(fnName_ + "() unknown"); } - if (agg_op_ != null) { - analyzeBuiltinAggFunction(analyzer); - } else { - setFunction(analyzer, argTypes); + if (fnName_.getFunction().equals("count") && params_.isDistinct()) { + // Treat COUNT(DISTINCT ...) special because of how we do the rewrite. + // There is no version of COUNT() that takes more than 1 argument but after + // the rewrite, we only need count(*). + // TODO: fix how we rewrite count distinct. + argTypes = new ColumnType[0]; + Function searchDesc = new Function(fnName_, argTypes, ColumnType.INVALID, false); + fn_ = db.getFunction(searchDesc, Function.CompareMode.IS_SUBTYPE); + type_ = fn_.getReturnType(); + return; + } + + // TODO: We allow implicit cast from string->timestamp but only + // support avg(timestamp). This means avg(string_col) would work + // from our casting rules. This is not right. + // We need to revisit where implicit casts are allowed for string + // to timestamp + if (fnName_.getFunction().equalsIgnoreCase("avg") && + children_.size() == 1 && children_.get(0).getType().isStringType()) { + throw new AnalysisException( + "AVG requires a numeric or timestamp parameter: " + toSql()); + } + + Function searchDesc = new Function(fnName_, argTypes, ColumnType.INVALID, false); + fn_ = db.getFunction(searchDesc, Function.CompareMode.IS_SUBTYPE); + + if (fn_ == null || !fn_.userVisible()) { + throw new AnalysisException(getFunctionNotFoundError(argTypes)); } - Preconditions.checkState(fn_ != null); if (isAggregateFunction()) { // subexprs must not contain aggregates @@ -381,6 +226,28 @@ public class FunctionCallExpr extends Expr { throw new AnalysisException( "aggregate function cannot contain aggregate parameters: " + this.toSql()); } + + // The catalog contains count() with no arguments to handle count(*) but don't + // accept count(). + // TODO: can this be handled more cleanly. It does seem like a special case since + // no other aggregate functions (currently) can accept '*'. + if (fnName_.getFunction().equalsIgnoreCase("count") && + !params_.isStar() && children_.size() == 0) { + throw new AnalysisException("count() is not allowed."); + } + + // TODO: the distinct rewrite does not handle this but why? + if (params_.isDistinct()) { + if (fnName_.getFunction().equalsIgnoreCase("group_concat")) { + throw new AnalysisException("GROUP_CONCAT() does not support DISTINCT."); + } + if (fn_.getBinaryType() != TFunctionBinaryType.BUILTIN) { + throw new AnalysisException("User defined aggregates do not support DISTINCT."); + } + } + + AggregateFunction aggFn = (AggregateFunction)fn_; + if (aggFn.ignoresDistinct()) params_.setIsDistinct(false); } else { if (params_.isStar()) { throw new AnalysisException("Cannot pass '*' to scalar function."); @@ -390,15 +257,7 @@ public class FunctionCallExpr extends Expr { } } - ColumnType[] args = fn_.getArgs(); - if (args.length > 0) { - // Implicitly cast all the children to match the function if necessary - for (int i = 0; i < argTypes.length; ++i) { - // For varargs, we must compare with the last type in callArgs.argTypes. - int ix = Math.min(args.length - 1, i); - if (!argTypes[i].equals(args[ix])) castChild(args[ix], i); - } - } + castForFunctionCall(); this.type_ = fn_.getReturnType(); } } diff --git a/fe/src/main/java/com/cloudera/impala/analysis/FunctionName.java b/fe/src/main/java/com/cloudera/impala/analysis/FunctionName.java index 3706021a4..abf354f0e 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/FunctionName.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/FunctionName.java @@ -14,8 +14,9 @@ package com.cloudera.impala.analysis; +import com.cloudera.impala.catalog.Catalog; +import com.cloudera.impala.catalog.Db; import com.cloudera.impala.common.AnalysisException; -import com.cloudera.impala.opcode.FunctionOperator; import com.cloudera.impala.thrift.TFunctionName; /** @@ -24,7 +25,8 @@ import com.cloudera.impala.thrift.TFunctionName; */ public class FunctionName { private String db_; - private String fn_; + private final String fn_; + boolean isBuiltin_; public FunctionName(String db, String fn) { db_ = db; @@ -49,14 +51,6 @@ public class FunctionName { return fn_.equalsIgnoreCase(o.fn_); } - // Same as FunctionName but for builtins and we'll leave the case - // as is since we aren't matching by string. - public static FunctionName CreateBuiltinName(String fn) { - FunctionName name = new FunctionName(fn); - name.fn_ = fn; - return name; - } - public FunctionName(TFunctionName thriftName) { db_ = thriftName.db_name.toLowerCase(); fn_ = thriftName.function_name.toLowerCase(); @@ -65,15 +59,14 @@ public class FunctionName { public String getDb() { return db_; } public String getFunction() { return fn_; } public boolean isFullyQualified() { return db_ != null; } + public boolean isBuiltin() { return isBuiltin_; } @Override public String toString() { - if (db_ == null) return fn_; + if (db_ == null || isBuiltin_) return fn_; return db_ + "." + fn_; } - public void setDb(String db) { db_ = db; } - public void analyze(Analyzer analyzer) throws AnalysisException { if (fn_.length() == 0) { throw new AnalysisException("Function name can not be empty."); @@ -89,11 +82,20 @@ public class FunctionName { throw new AnalysisException("Function cannot start with a digit: " + fn_); } - // If the function name is not fully qualified, it must not be the same as a builtin - if (!isFullyQualified() && OpcodeRegistry.instance().getFunctionOperator( - getFunction()) != FunctionOperator.INVALID_OPERATOR) { - throw new AnalysisException( - "Function cannot have the same name as a builtin: " + getFunction()); + // Resolve the database for this function. + if (!isFullyQualified()) { + Db builtinDb = analyzer.getCatalog().getBuiltinsDb(); + if (builtinDb.containsFunction(fn_)) { + // If it isn't fully qualified and is the same name as a builtin, use + // the builtin. + db_ = Catalog.BUILTINS_DB; + isBuiltin_ = true; + } else { + db_ = analyzer.getDefaultDb(); + isBuiltin_ = false; + } + } else { + isBuiltin_ = db_.equals(Catalog.BUILTINS_DB); } } diff --git a/fe/src/main/java/com/cloudera/impala/analysis/InPredicate.java b/fe/src/main/java/com/cloudera/impala/analysis/InPredicate.java index 318fe0d91..702bca5e2 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/InPredicate.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/InPredicate.java @@ -62,7 +62,6 @@ public class InPredicate extends Predicate { protected void toThrift(TExprNode msg) { msg.in_predicate = new TInPredicate(isNotIn_); msg.node_type = TExprNodeType.IN_PRED; - msg.setOpcode(opcode_); } @Override diff --git a/fe/src/main/java/com/cloudera/impala/analysis/LikePredicate.java b/fe/src/main/java/com/cloudera/impala/analysis/LikePredicate.java index 396b8109f..7cc0c8b7e 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/LikePredicate.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/LikePredicate.java @@ -18,35 +18,44 @@ import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Db; +import com.cloudera.impala.catalog.Function.CompareMode; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; -import com.cloudera.impala.thrift.TExprOpcode; import com.cloudera.impala.thrift.TLikePredicate; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; public class LikePredicate extends Predicate { enum Operator { - LIKE("LIKE", TExprOpcode.LIKE), - RLIKE("RLIKE", TExprOpcode.REGEX), - REGEXP("REGEXP", TExprOpcode.REGEX); + LIKE("LIKE"), + RLIKE("RLIKE"), + REGEXP("REGEXP"); private final String description_; - private final TExprOpcode thriftOp_; - private Operator(String description, TExprOpcode thriftOp) { + private Operator(String description) { this.description_ = description; - this.thriftOp_ = thriftOp; } @Override public String toString() { return description_; } + } - public TExprOpcode toThrift() { - return thriftOp_; - } + public static void initBuiltins(Db db) { + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.LIKE.name(), "LikePredicate", "LikeFn", + Lists.newArrayList(ColumnType.STRING, ColumnType.STRING), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.RLIKE.name(), "LikePredicate", "RegexFn", + Lists.newArrayList(ColumnType.STRING, ColumnType.STRING), ColumnType.BOOLEAN)); + db.addBuiltin(ScalarFunction.createBuiltinOperator( + Operator.REGEXP.name(), "LikePredicate", "RegexFn", + Lists.newArrayList(ColumnType.STRING, ColumnType.STRING), ColumnType.BOOLEAN)); } private final Operator op_; @@ -78,7 +87,6 @@ public class LikePredicate extends Predicate { @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.LIKE_PRED; - msg.setOpcode(op_.toThrift()); msg.like_pred = new TLikePredicate("\\"); } @@ -96,6 +104,11 @@ public class LikePredicate extends Predicate { "right operand of " + op_.toString() + " must be of type STRING: " + toSql()); } + fn_ = getBuiltinFunction(analyzer, op_.toString(), collectChildReturnTypes(), + CompareMode.IS_SUBTYPE); + Preconditions.checkState(fn_ != null); + Preconditions.checkState(fn_.getReturnType().isBoolean()); + if (!getChild(1).getType().isNull() && getChild(1).isLiteral() && (op_ == Operator.RLIKE || op_ == Operator.REGEXP)) { // let's make sure the pattern works diff --git a/fe/src/main/java/com/cloudera/impala/analysis/OpcodeRegistry.java b/fe/src/main/java/com/cloudera/impala/analysis/OpcodeRegistry.java deleted file mode 100644 index 548c7bc8a..000000000 --- a/fe/src/main/java/com/cloudera/impala/analysis/OpcodeRegistry.java +++ /dev/null @@ -1,268 +0,0 @@ -// Copyright 2012 Cloudera Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package com.cloudera.impala.analysis; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.cloudera.impala.catalog.Function; -import com.cloudera.impala.common.Pair; -import com.cloudera.impala.opcode.FunctionOperator; -import com.cloudera.impala.opcode.FunctionRegistry; -import com.cloudera.impala.thrift.TExprOpcode; -import com.cloudera.impala.thrift.TFunctionBinaryType; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; - -/** - * The OpcodeRegistry provides a mapping between function signatures and opcodes. The - * supported functions are code-gen'ed and added to the registry with an assigned opcode. - * The opcode is shared with the backend. The frontend can use the registry to look up - * a function's opcode. - * - * The OpcodeRegistry also contains a mapping between function names (as strings) to - * operators. - * - * The OpcodeRegistry is a singleton. - * - * TODO: The opcode registry should be versioned in the FE/BE. - */ -public class OpcodeRegistry { - private final static Logger LOG = LoggerFactory.getLogger(OpcodeRegistry.class); - private final static OpcodeRegistry INSTANCE = new OpcodeRegistry(); - - /** - * Contains all the information about a builtin function. - * TODO: merge with Function and Udf - */ - public static class BuiltinFunction extends Function { - public TExprOpcode opcode; - public FunctionOperator operator; - // If true, this builtin is implemented against the Udf interface. - public final boolean udfInterface; - - // Constructor for searching, specifying the op and arguments - public BuiltinFunction(FunctionOperator operator, ColumnType[] args) { - super(new FunctionName(operator.toString()), - args, ColumnType.INVALID, false); - this.operator = operator; - this.udfInterface = false; - this.setBinaryType(TFunctionBinaryType.BUILTIN); - } - - private BuiltinFunction(boolean udfInterface, TExprOpcode opcode, - FunctionOperator operator, boolean varArgs, ColumnType ret, - ColumnType[] args) { - super(new FunctionName(opcode.toString()), args, ret, varArgs); - this.operator = operator; - this.opcode = opcode; - this.udfInterface = udfInterface; - this.setBinaryType(TFunctionBinaryType.BUILTIN); - } - } - - /** - * This is a mapping of Operator,#args to builtins with a fixed number of arguments. - * The builtin is defined by the operator enum and the arguments - * and is a one to one mapping to opcodes. - * The map is structured this way to more efficiently look for builtin matches. - * Builtins that have the same number of arguments have a potential to be matches - * by allowing types to be implicitly cast. - * Functions with a variable number of arguments are put into the varArgOperations map. - */ - private final Map, List> operations; - - /** - * This is a mapping of Operator,varArgType to builtins of vararg functions only. - * varArgType must be a maximum-resolution type. - * We use a separate map to be able to support multiple vararg builtins for the same - * FunctionOperator. - * Contains a special entry mapping from Operator,NULL_TYPE to builtins for each - * Operator to correctly match varag functions when all args are NULL. - * Limitations: Since we do not consider the number of arguments, each FunctionOperator - * is limited to having one vararg builtin per maximum-resolution PrimitiveType. - * For example, one can have two builtins func(float, int ...) and func(string ...), - * but not func(float, int ...) and func (int ...). - */ - private final Map, List> - varArgOperations; - - /** - * This contains a mapping of function names to a FunctionOperator enum. This is used - * by FunctionCallExpr to go from the parser input to function opcodes. - * This is a many to one mapping (i.e. substr and substring both map to the same - * operation). - * The mappings are filled in in FunctionRegistry.java which is auto-generated. - */ - private final HashMap functionNameMap; - - // Singleton interface - public static OpcodeRegistry instance() { return INSTANCE; } - - /** - * Static utility functions - */ - public static boolean isBitwiseOperation(FunctionOperator operator) { - return operator == FunctionOperator.BITAND || operator == FunctionOperator.BITNOT || - operator == FunctionOperator.BITOR || operator == FunctionOperator.BITXOR; - } - - /** - * Returns the set of function names. - * @return - */ - public Set getFunctionNames() { - return functionNameMap.keySet(); - } - - /** - * Returns the function operator enum. The lookup is case insensitive. - * (i.e. "Substring" --> TExprOperator.STRING_SUBSTR). - * Returns INVALID_OP is that function name is unknown. - */ - public FunctionOperator getFunctionOperator(String fnName) { - fnName = fnName.toLowerCase(); - if (functionNameMap.containsKey(fnName)) return functionNameMap.get(fnName); - return FunctionOperator.INVALID_OPERATOR; - } - - /** - * Query for a function in the registry, specifying the operation, 'op', the arguments. - * If there is no matching signature, null will be returned. - * If there is a match, the matching signature will be returned. - * If 'allowImplicitCasts' is true the matching signature does not have to match the - * input identically, implicit type promotion is allowed. - */ - public BuiltinFunction getFunctionInfo(FunctionOperator op, boolean allowImplicitCasts, - ColumnType ... argTypes) { - Pair lookup = Pair.create(op, argTypes.length); - List> varArgMatchTypes = null; - if (argTypes.length > 0) { - Set maxResolutionTypes = getMaxResolutionTypes(argTypes); - Preconditions.checkNotNull(maxResolutionTypes); - varArgMatchTypes = Lists.newArrayList(); - for (ColumnType maxResolutionType : maxResolutionTypes) { - varArgMatchTypes.add(Pair.create(op, maxResolutionType)); - } - } - List functions = null; - if (operations.containsKey(lookup)) { - functions = operations.get(lookup); - } else if(varArgMatchTypes != null) { - functions = Lists.newArrayList(); - List matchedFunctions = null; - for (Pair varArgsMatchType : varArgMatchTypes) { - matchedFunctions = varArgOperations.get(varArgsMatchType); - if (matchedFunctions != null) functions.addAll(matchedFunctions); - } - } - if (functions == null) return null; - BuiltinFunction compatibleMatch = null; - BuiltinFunction search = new BuiltinFunction(op, argTypes); - for (BuiltinFunction function : functions) { - if (function.compare(search, Function.CompareMode.IS_INDISTINGUISHABLE)) { - return function; - } else if (allowImplicitCasts && compatibleMatch == null - && function.compare(search, Function.CompareMode.IS_SUBTYPE)) { - compatibleMatch = function; - } - } - return compatibleMatch; - } - - /** - * Returns the max resolution type for each argType that is not a NULL_TYPE. If all - * argument types are NULL_TYPE then a set will be returned containing NULL_TYPE. - */ - private Set getMaxResolutionTypes(ColumnType[] argTypes) { - Set maxResolutionTypes = Sets.newHashSet(); - for (int i = 0; i < argTypes.length; ++i) { - if (!argTypes[i].isNull()) { - maxResolutionTypes.add(argTypes[i].getMaxResolutionType()); - } - } - if (maxResolutionTypes.isEmpty()) maxResolutionTypes.add(ColumnType.NULL); - return maxResolutionTypes; - } - - /** - * Add a function with the specified opcode/signature to the registry. - */ - public boolean add(boolean udfInterface, FunctionOperator op, TExprOpcode opcode, - boolean varArgs, ColumnType retType, ColumnType ... args) { - List functions; - Pair lookup = Pair.create(op, args.length); - // Take the last argument's type as the vararg type. - Pair varArgsLookup = null; - // Special signature for vararg functions to handle matching when all args are NULL. - Pair varArgsNullLookup = null; - Preconditions.checkArgument((varArgs) ? args.length > 0 : true); - if (varArgs && args.length > 0) { - varArgsLookup = Pair.create(op, args[args.length - 1].getMaxResolutionType()); - varArgsNullLookup = Pair.create(op, ColumnType.NULL); - } - if (operations.containsKey(lookup)) { - functions = operations.get(lookup); - } else if (varArgsLookup != null && varArgOperations.containsKey(varArgsLookup)) { - functions = varArgOperations.get(varArgsLookup); - } else { - functions = new ArrayList(); - if (varArgs) { - varArgOperations.put(varArgsLookup, functions); - varArgOperations.put(varArgsNullLookup, functions); - } else { - operations.put(lookup, functions); - } - } - - BuiltinFunction function = - new BuiltinFunction(udfInterface, opcode, op, varArgs, retType, args); - if (functions.contains(function)) { - LOG.error("OpcodeRegistry: Function already exists: " + opcode); - return false; - } - functions.add(function); - - return true; - } - - public boolean addFunctionMapping(String functionName, FunctionOperator op) { - if (functionNameMap.containsKey(functionName.toLowerCase())) { - LOG.error("OpcodeRegistry: Function mapping already exists: " + functionName); - return false; - } - functionNameMap.put(functionName.toLowerCase(), op); - return true; - } - - // Singleton interface, don't call the constructor - private OpcodeRegistry() { - operations = Maps.newHashMap(); - varArgOperations = Maps.newHashMap(); - functionNameMap = Maps.newHashMap(); - - // Add all the function signatures to the registry and the function name(string) - // to FunctionOperator mapping - FunctionRegistry.InitFunctions(this); - } -} diff --git a/fe/src/main/java/com/cloudera/impala/analysis/SelectStmt.java b/fe/src/main/java/com/cloudera/impala/analysis/SelectStmt.java index 4f4199cc0..644f18549 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/SelectStmt.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/SelectStmt.java @@ -480,9 +480,7 @@ public class SelectStmt extends QueryStmt { throws AnalysisException, AuthorizationException { Expr.SubstitutionMap result = new Expr.SubstitutionMap(); for (FunctionCallExpr aggExpr : aggExprs) { - if (aggExpr.getAggOp() != BuiltinAggregateFunction.Operator.AVG) { - continue; - } + if (!aggExpr.getFnName().getFunction().equals("avg")) continue; // Transform avg(TIMESTAMP) to cast(avg(cast(TIMESTAMP as DOUBLE)) as TIMESTAMP) CastExpr inCastExpr = null; if (aggExpr.getChild(0).type_.getPrimitiveType() == PrimitiveType.TIMESTAMP) { @@ -497,10 +495,10 @@ public class SelectStmt extends QueryStmt { List countInputExpr = Lists.newArrayList(aggExpr.getChild(0).clone(null)); FunctionCallExpr sumExpr = - new FunctionCallExpr(BuiltinAggregateFunction.Operator.SUM, + new FunctionCallExpr("sum", new FunctionParams(aggExpr.isDistinct(), sumInputExprs)); FunctionCallExpr countExpr = - new FunctionCallExpr(BuiltinAggregateFunction.Operator.COUNT, + new FunctionCallExpr("count", new FunctionParams(aggExpr.isDistinct(), countInputExpr)); ArithmeticExpr divExpr = new ArithmeticExpr(ArithmeticExpr.Operator.DIVIDE, sumExpr, countExpr); diff --git a/fe/src/main/java/com/cloudera/impala/analysis/TimestampArithmeticExpr.java b/fe/src/main/java/com/cloudera/impala/analysis/TimestampArithmeticExpr.java index 57b19f2d5..fdc84d88b 100644 --- a/fe/src/main/java/com/cloudera/impala/analysis/TimestampArithmeticExpr.java +++ b/fe/src/main/java/com/cloudera/impala/analysis/TimestampArithmeticExpr.java @@ -19,9 +19,9 @@ import java.util.Map; import com.cloudera.impala.analysis.ArithmeticExpr.Operator; import com.cloudera.impala.catalog.AuthorizationException; +import com.cloudera.impala.catalog.Function.CompareMode; import com.cloudera.impala.catalog.PrimitiveType; import com.cloudera.impala.common.AnalysisException; -import com.cloudera.impala.opcode.FunctionOperator; import com.cloudera.impala.thrift.TExprNode; import com.cloudera.impala.thrift.TExprNodeType; import com.google.common.base.Preconditions; @@ -143,29 +143,19 @@ public class TimestampArithmeticExpr extends Expr { getChild(1).getType() + "'. Expected an integer type."); } - ColumnType[] argTypes = new ColumnType[this.children_.size()]; - for (int i = 0; i < this.children_.size(); ++i) { - this.children_.get(i).analyze(analyzer); - argTypes[i] = this.children_.get(i).getType(); - } String funcOpName = String.format("%sS_%s", timeUnit_.toString(), (op_ == ArithmeticExpr.Operator.ADD) ? "ADD" : "SUB"); - FunctionOperator funcOp = - OpcodeRegistry.instance().getFunctionOperator(funcOpName); - OpcodeRegistry.BuiltinFunction match = - OpcodeRegistry.instance().getFunctionInfo(funcOp, true, argTypes); - // We have already done type checking to ensure the function will resolve. - Preconditions.checkNotNull(match); + fn_ = getBuiltinFunction(analyzer, funcOpName.toLowerCase(), + collectChildReturnTypes(), CompareMode.IS_SUBTYPE); + Preconditions.checkNotNull(fn_); Preconditions.checkState( - match.getReturnType().getPrimitiveType() == PrimitiveType.TIMESTAMP); - opcode_ = match.opcode; - type_ = match.getReturnType(); + fn_.getReturnType().getPrimitiveType() == PrimitiveType.TIMESTAMP); + type_ = fn_.getReturnType(); } @Override protected void toThrift(TExprNode msg) { msg.node_type = TExprNodeType.COMPUTE_FUNCTION_CALL; - msg.setOpcode(opcode_); } public String getTimeUnitIdent() { return timeUnitIdent_; } diff --git a/fe/src/main/java/com/cloudera/impala/catalog/Uda.java b/fe/src/main/java/com/cloudera/impala/catalog/AggregateFunction.java similarity index 58% rename from fe/src/main/java/com/cloudera/impala/catalog/Uda.java rename to fe/src/main/java/com/cloudera/impala/catalog/AggregateFunction.java index f2729e96e..08f059db6 100644 --- a/fe/src/main/java/com/cloudera/impala/catalog/Uda.java +++ b/fe/src/main/java/com/cloudera/impala/catalog/AggregateFunction.java @@ -22,11 +22,12 @@ import com.cloudera.impala.analysis.FunctionName; import com.cloudera.impala.analysis.HdfsUri; import com.cloudera.impala.thrift.TAggregateFunction; import com.cloudera.impala.thrift.TFunction; +import com.cloudera.impala.thrift.TFunctionBinaryType; /** - * Internal representation of a UDA. + * Internal representation of an aggregate function. */ -public class Uda extends Function { +public class AggregateFunction extends Function { private ColumnType intermediateType_; // The symbol inside the binary at location_ that contains this particular. @@ -37,11 +38,19 @@ public class Uda extends Function { private String mergeFnSymbol_; private String finalizeFnSymbol_; - public Uda(FunctionName fnName, FunctionArgs args, ColumnType retType) { + private static String BE_BUILTINS_CLASS = "AggregateFunctions"; + + // If true, this aggregate function should ignore distinct. + // e.g. min(distinct col) == min(col). + // TODO: currently it is not possible for user functions to specify this. We should + // extend the create aggregate function stmt to allow additional metdata like this. + private boolean ignoresDistinct_; + + public AggregateFunction(FunctionName fnName, FunctionArgs args, ColumnType retType) { super(fnName, args.argTypes, retType, args.hasVarArgs); } - public Uda(FunctionName fnName, List argTypes, + public AggregateFunction(FunctionName fnName, List argTypes, ColumnType retType, ColumnType intermediateType, HdfsUri location, String updateFnSymbol, String initFnSymbol, String serializeFnSymbol, String mergeFnSymbol, String finalizeFnSymbol) { @@ -53,6 +62,20 @@ public class Uda extends Function { serializeFnSymbol_ = serializeFnSymbol; mergeFnSymbol_ = mergeFnSymbol; finalizeFnSymbol_ = finalizeFnSymbol; + ignoresDistinct_ = false; + } + + public static AggregateFunction createBuiltin(Db db, String name, + List argTypes, ColumnType retType, ColumnType intermediateType, + String initFnSymbol, String updateFnSymbol, String mergeFnSymbol, + String serializeFnSymbol, String finalizeFnSymbol, + boolean ignoresDistinct) { + AggregateFunction fn = new AggregateFunction(new FunctionName(db.getName(), name), + argTypes, retType, intermediateType, null, updateFnSymbol, initFnSymbol, + serializeFnSymbol, mergeFnSymbol, finalizeFnSymbol); + fn.setBinaryType(TFunctionBinaryType.BUILTIN); + fn.ignoresDistinct_ = ignoresDistinct; + return fn; } public String getUpdateFnSymbol() { return updateFnSymbol_; } @@ -61,6 +84,7 @@ public class Uda extends Function { public String getMergeFnSymbol() { return mergeFnSymbol_; } public String getFinalizeFnSymbol() { return finalizeFnSymbol_; } public ColumnType getIntermediateType() { return intermediateType_; } + public boolean ignoresDistinct() { return ignoresDistinct_; } public void setUpdateFnSymbol(String fn) { updateFnSymbol_ = fn; } public void setInitFnSymbol(String fn) { initFnSymbol_ = fn; } @@ -72,14 +96,15 @@ public class Uda extends Function { @Override public TFunction toThrift() { TFunction fn = super.toThrift(); - TAggregateFunction uda = new TAggregateFunction(); - uda.setUpdate_fn_symbol(updateFnSymbol_); - uda.setInit_fn_symbol(initFnSymbol_); - if (serializeFnSymbol_ == null) uda.setSerialize_fn_symbol(serializeFnSymbol_); - uda.setMerge_fn_symbol(mergeFnSymbol_); - uda.setFinalize_fn_symbol(finalizeFnSymbol_); - uda.setIntermediate_type(intermediateType_.toThrift()); - fn.setAggregate_fn(uda); + TAggregateFunction agg_fn = new TAggregateFunction(); + agg_fn.setUpdate_fn_symbol(updateFnSymbol_); + agg_fn.setInit_fn_symbol(initFnSymbol_); + if (serializeFnSymbol_ != null) agg_fn.setSerialize_fn_symbol(serializeFnSymbol_); + agg_fn.setMerge_fn_symbol(mergeFnSymbol_); + if (finalizeFnSymbol_ != null) agg_fn.setFinalize_fn_symbol(finalizeFnSymbol_); + agg_fn.setIntermediate_type(intermediateType_.toThrift()); + agg_fn.setIgnores_distinct(ignoresDistinct_); + fn.setAggregate_fn(agg_fn); return fn; } } diff --git a/fe/src/main/java/com/cloudera/impala/catalog/Catalog.java b/fe/src/main/java/com/cloudera/impala/catalog/Catalog.java index fb1eee50e..4097721ab 100644 --- a/fe/src/main/java/com/cloudera/impala/catalog/Catalog.java +++ b/fe/src/main/java/com/cloudera/impala/catalog/Catalog.java @@ -14,17 +14,25 @@ package com.cloudera.impala.catalog; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.log4j.Logger; - +import com.cloudera.impala.analysis.ArithmeticExpr; +import com.cloudera.impala.analysis.BinaryPredicate; +import com.cloudera.impala.analysis.CaseExpr; +import com.cloudera.impala.analysis.CastExpr; +import com.cloudera.impala.analysis.ColumnType; +import com.cloudera.impala.analysis.CompoundPredicate; import com.cloudera.impala.analysis.FunctionName; +import com.cloudera.impala.analysis.LikePredicate; +import com.cloudera.impala.builtins.ScalarBuiltins; import com.cloudera.impala.catalog.MetaStoreClientPool.MetaStoreClient; import com.cloudera.impala.thrift.TCatalogObject; import com.cloudera.impala.thrift.TFunction; @@ -33,6 +41,7 @@ import com.cloudera.impala.thrift.TPartitionKeyValue; import com.cloudera.impala.thrift.TTableName; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; /** @@ -47,6 +56,11 @@ import com.google.common.collect.Lists; * -> Function * Each level has its own synchronization, so the cache of Dbs is synchronized and each * Db has a cache of tables which is synchronized independently. + * + * The catalog is populated with the impala builtins on startup. Builtins and user + * functions are treated identically by the catalog. The builtins go in a specific + * database that the user cannot modify. + * Builtins are populated on startup in initBuiltins(). */ public abstract class Catalog { // Initial catalog version. @@ -54,7 +68,7 @@ public abstract class Catalog { public static final String DEFAULT_DB = "default"; private static final int META_STORE_CLIENT_POOL_SIZE = 5; - private static final Logger LOG = Logger.getLogger(Catalog.class); + public static final String BUILTINS_DB = "_impala_builtins"; protected final MetaStoreClientPool metaStoreClientPool_ = new MetaStoreClientPool(0); @@ -65,6 +79,9 @@ public abstract class Catalog { new AtomicReference>( new ConcurrentHashMap()); + // Cache of the DB containing the builtins. + private static Db builtinsDb_; + /** * Creates a new instance of a Catalog. If initMetastoreClientPool is true, will * also add META_STORE_CLIENT_POOL_SIZE clients to metastoreClientPool_. @@ -73,8 +90,11 @@ public abstract class Catalog { if (initMetastoreClientPool) { metaStoreClientPool_.addClients(META_STORE_CLIENT_POOL_SIZE); } + initBuiltins(); } + public Db getBuiltinsDb() { return builtinsDb_;} + /** * Adds a new database to the catalog, replacing any existing database with the same * name. Returns the previous database with this name, or null if there was no @@ -187,6 +207,10 @@ public abstract class Catalog { return db.getFunction(desc, mode); } + public static Function getBuiltin(Function desc, Function.CompareMode mode) { + return builtinsDb_.getFunction(desc, mode); + } + /** * Removes a function from the catalog. Increments the catalog version and returns * the Function object that was removed if the function existed, otherwise returns @@ -214,10 +238,10 @@ public abstract class Catalog { * Returns true if there is a function with this function name. Parameters * are ignored. */ - public boolean functionExists(FunctionName name) { + public boolean containsFunction(FunctionName name) { Db db = getDb(name.getDb()); if (db == null) return false; - return db.functionExists(name); + return db.containsFunction(name.getFunction()); } /** @@ -365,4 +389,268 @@ public abstract class Catalog { } return result; } + + /** + * Initializes all the builtins. + */ + private void initBuiltins() { + if (builtinsDb_ != null) { + // Only in the FE test setup do we hit this case. + addDb(builtinsDb_); + return; + } + Preconditions.checkState(getDb(BUILTINS_DB) == null); + Preconditions.checkState(builtinsDb_ == null); + builtinsDb_ = new Db(BUILTINS_DB, this); + builtinsDb_.setIsSystemDb(true); + addDb(builtinsDb_); + + // Populate all aggregate builtins. + initAggregateBuiltins(); + + // Populate all scalar builtins. + ArithmeticExpr.initBuiltins(builtinsDb_); + BinaryPredicate.initBuiltins(builtinsDb_); + CastExpr.initBuiltins(builtinsDb_); + CaseExpr.initBuiltins(builtinsDb_); + CompoundPredicate.initBuiltins(builtinsDb_); + LikePredicate.initBuiltins(builtinsDb_); + ScalarBuiltins.initBuiltins(builtinsDb_); + } + + private static final Map HLL_UPDATE_SYMBOL = + ImmutableMap.builder() + .put(ColumnType.BOOLEAN, + "9HllUpdateIN10impala_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.TINYINT, + "9HllUpdateIN10impala_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.SMALLINT, + "9HllUpdateIN10impala_udf11SmallIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.INT, + "9HllUpdateIN10impala_udf6IntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.BIGINT, + "9HllUpdateIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.FLOAT, + "9HllUpdateIN10impala_udf8FloatValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.DOUBLE, + "9HllUpdateIN10impala_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.STRING, + "9HllUpdateIN10impala_udf9StringValEEEvPNS2_15FunctionContextERKT_PS3_") + .put(ColumnType.TIMESTAMP, + "9HllUpdateIN10impala_udf12TimestampValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .build(); + + private static final Map PC_UPDATE_SYMBOL = + ImmutableMap.builder() + .put(ColumnType.BOOLEAN, + "8PcUpdateIN10impala_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.TINYINT, + "8PcUpdateIN10impala_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.SMALLINT, + "8PcUpdateIN10impala_udf11SmallIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.INT, + "8PcUpdateIN10impala_udf6IntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.BIGINT, + "8PcUpdateIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.FLOAT, + "8PcUpdateIN10impala_udf8FloatValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.DOUBLE, + "8PcUpdateIN10impala_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.STRING, + "8PcUpdateIN10impala_udf9StringValEEEvPNS2_15FunctionContextERKT_PS3_") + .put(ColumnType.TIMESTAMP, + "8PcUpdateIN10impala_udf12TimestampValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .build(); + + private static final Map PCSA_UPDATE_SYMBOL = + ImmutableMap.builder() + .put(ColumnType.BOOLEAN, + "10PcsaUpdateIN10impala_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.TINYINT, + "10PcsaUpdateIN10impala_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.SMALLINT, + "10PcsaUpdateIN10impala_udf11SmallIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.INT, + "10PcsaUpdateIN10impala_udf6IntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.BIGINT, + "10PcsaUpdateIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.FLOAT, + "10PcsaUpdateIN10impala_udf8FloatValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.DOUBLE, + "10PcsaUpdateIN10impala_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .put(ColumnType.STRING, + "10PcsaUpdateIN10impala_udf9StringValEEEvPNS2_15FunctionContextERKT_PS3_") + .put(ColumnType.TIMESTAMP, + "10PcsaUpdateIN10impala_udf12TimestampValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE") + .build(); + + private static final Map MIN_UPDATE_SYMBOL = + ImmutableMap.builder() + .put(ColumnType.BOOLEAN, + "3MinIN10impala_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.TINYINT, + "3MinIN10impala_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.SMALLINT, + "3MinIN10impala_udf11SmallIntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.INT, + "3MinIN10impala_udf6IntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.BIGINT, + "3MinIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.FLOAT, + "3MinIN10impala_udf8FloatValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.DOUBLE, + "3MinIN10impala_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.STRING, + "3MinIN10impala_udf9StringValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.TIMESTAMP, + "3MinIN10impala_udf12TimestampValEEEvPNS2_15FunctionContextERKT_PS6_") + .build(); + + private static final Map MAX_UPDATE_SYMBOL = + ImmutableMap.builder() + .put(ColumnType.BOOLEAN, + "3MaxIN10impala_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.TINYINT, + "3MaxIN10impala_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.SMALLINT, + "3MaxIN10impala_udf11SmallIntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.INT, + "3MaxIN10impala_udf6IntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.BIGINT, + "3MaxIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.FLOAT, + "3MaxIN10impala_udf8FloatValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.DOUBLE, + "3MaxIN10impala_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.STRING, + "3MaxIN10impala_udf9StringValEEEvPNS2_15FunctionContextERKT_PS6_") + .put(ColumnType.TIMESTAMP, + "3MaxIN10impala_udf12TimestampValEEEvPNS2_15FunctionContextERKT_PS6_") + .build(); + + // Populate all the aggregate builtins in the catalog. + // null symbols indicate the function does not need that step of the evaluation. + // An empty symbol indicates a TODO for the BE to implement the function. + // TODO: We could also generate this in python but I'm not sure that is easier. + private void initAggregateBuiltins() { + final String prefix = "_ZN6impala18AggregateFunctions"; + final String initNullString = prefix + + "14InitNullStringEPN10impala_udf15FunctionContextEPNS1_9StringValE"; + final String initNull = prefix + + "8InitNullEPN10impala_udf15FunctionContextEPNS1_6AnyValE"; + + Db db = builtinsDb_; + // Count (*) + // TODO: the merge function should be Sum but the way we rewrite distincts + // makes that not work. + db.addBuiltin(AggregateFunction.createBuiltin(db, "count", + new ArrayList(), + ColumnType.BIGINT, ColumnType.BIGINT, + prefix + "8InitZeroIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextEPT_", + prefix + "15CountStarUpdateEPN10impala_udf15FunctionContextEPNS1_9BigIntValE", + prefix + "15CountStarUpdateEPN10impala_udf15FunctionContextEPNS1_9BigIntValE", + null, null, false)); + + for (ColumnType t : ColumnType.getSupportedTypes()) { + if (t.isNull()) continue; // NULL is handled through type promotion. + // Count + db.addBuiltin(AggregateFunction.createBuiltin(db, "count", + Lists.newArrayList(t), ColumnType.BIGINT, ColumnType.BIGINT, + prefix + "8InitZeroIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextEPT_", + prefix + "11CountUpdateEPN10impala_udf15FunctionContextERKNS1_6AnyValEPNS1_9BigIntValE", + prefix + "11CountUpdateEPN10impala_udf15FunctionContextERKNS1_6AnyValEPNS1_9BigIntValE", + null, null, false)); + // Min + String minMaxInit = t.isStringType() ? initNullString : initNull; + db.addBuiltin(AggregateFunction.createBuiltin(db, "min", + Lists.newArrayList(t), t, t, minMaxInit, + prefix + MIN_UPDATE_SYMBOL.get(t), + prefix + MIN_UPDATE_SYMBOL.get(t), + null, null, true)); + // Max + db.addBuiltin(AggregateFunction.createBuiltin(db, "max", + Lists.newArrayList(t), t, t, minMaxInit, + prefix + MAX_UPDATE_SYMBOL.get(t), + prefix + MAX_UPDATE_SYMBOL.get(t), + null, null, true)); + // NDV + // TODO: this needs to switch to CHAR(64) as the intermediate type + db.addBuiltin(AggregateFunction.createBuiltin(db, "ndv", + Lists.newArrayList(t), ColumnType.STRING, ColumnType.STRING, + prefix + "7HllInitEPN10impala_udf15FunctionContextEPNS1_9StringValE", + prefix + HLL_UPDATE_SYMBOL.get(t), + prefix + "8HllMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_", + null, + prefix + "11HllFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE", + true)); + + // distinctpc + // TODO: this needs to switch to CHAR(64) as the intermediate type + db.addBuiltin(AggregateFunction.createBuiltin(db, "distinctpc", + Lists.newArrayList(t), ColumnType.STRING, ColumnType.STRING, + prefix + "6PcInitEPN10impala_udf15FunctionContextEPNS1_9StringValE", + prefix + PC_UPDATE_SYMBOL.get(t), + prefix + "7PcMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_", + null, + prefix + "10PcFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE", + true)); + + // distinctpcsa + // TODO: this needs to switch to CHAR(64) as the intermediate type + db.addBuiltin(AggregateFunction.createBuiltin(db, "distinctpcsa", + Lists.newArrayList(t), ColumnType.STRING, ColumnType.STRING, + prefix + "6PcInitEPN10impala_udf15FunctionContextEPNS1_9StringValE", + prefix + PCSA_UPDATE_SYMBOL.get(t), + prefix + "7PcMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_", + null, + prefix + "12PcsaFinalizeEPN10impala_udf15FunctionContextERKNS1_9StringValE", + true)); + } + + // Sum + db.addBuiltin(AggregateFunction.createBuiltin(db, "sum", + Lists.newArrayList(ColumnType.BIGINT), ColumnType.BIGINT, ColumnType.BIGINT, + initNull, + prefix + "3SumIN10impala_udf9BigIntValES3_EEvPNS2_15FunctionContextERKT_PT0_", + prefix + "3SumIN10impala_udf9BigIntValES3_EEvPNS2_15FunctionContextERKT_PT0_", + null, null, false)); + db.addBuiltin(AggregateFunction.createBuiltin(db, "sum", + Lists.newArrayList(ColumnType.DOUBLE), ColumnType.DOUBLE, ColumnType.DOUBLE, + initNull, + prefix + "3SumIN10impala_udf9DoubleValES3_EEvPNS2_15FunctionContextERKT_PT0_", + prefix + "3SumIN10impala_udf9DoubleValES3_EEvPNS2_15FunctionContextERKT_PT0_", + null, null, false)); + + for (ColumnType t: ColumnType.getNumericTypes()) { + // Avg + // TODO: because of avg rewrite, BE doesn't implement it yet. + db.addBuiltin(AggregateFunction.createBuiltin(db, "avg", + Lists.newArrayList(t), ColumnType.DOUBLE, ColumnType.DOUBLE, + "", "", "", null, "", false)); + } + // Avg(Timestamp) + // TODO: why does this make sense? Avg(timestamp) returns a double. + db.addBuiltin(AggregateFunction.createBuiltin(db, "avg", + Lists.newArrayList(ColumnType.TIMESTAMP), + ColumnType.DOUBLE, ColumnType.DOUBLE, + "", "", "", null, "", false)); + + // Group_concat(string) + db.addBuiltin(AggregateFunction.createBuiltin(db, "group_concat", + Lists.newArrayList(ColumnType.STRING), + ColumnType.STRING, ColumnType.STRING, + initNullString, + prefix + "12StringConcatEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_", + prefix + "12StringConcatEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_", + null, null, false)); + // Group_concat(string, string) + db.addBuiltin(AggregateFunction.createBuiltin(db, "group_concat", + Lists.newArrayList(ColumnType.STRING, ColumnType.STRING), + ColumnType.STRING, ColumnType.STRING, + initNullString, + prefix + + "12StringConcatEPN10impala_udf15FunctionContextERKNS1_9StringValES6_PS4_", + prefix + "12StringConcatEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_", + null, null, false)); + } } diff --git a/fe/src/main/java/com/cloudera/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/com/cloudera/impala/catalog/CatalogServiceCatalog.java index 91477e030..609c8c4b1 100644 --- a/fe/src/main/java/com/cloudera/impala/catalog/CatalogServiceCatalog.java +++ b/fe/src/main/java/com/cloudera/impala/catalog/CatalogServiceCatalog.java @@ -38,6 +38,7 @@ import com.cloudera.impala.common.Pair; import com.cloudera.impala.thrift.TCatalog; import com.cloudera.impala.thrift.TCatalogObject; import com.cloudera.impala.thrift.TCatalogObjectType; +import com.cloudera.impala.thrift.TFunctionBinaryType; import com.cloudera.impala.thrift.TGetAllCatalogObjectsResponse; import com.cloudera.impala.thrift.TTable; import com.cloudera.impala.thrift.TTableName; @@ -400,6 +401,7 @@ public class CatalogServiceCatalog extends Catalog { for (List fns: dbFns.second.values()) { for (Function fn: fns) { + if (fn.getBinaryType() == TFunctionBinaryType.BUILTIN) continue; fn.setCatalogVersion(incrementAndGetCatalogVersion()); db.addFunction(fn); } diff --git a/fe/src/main/java/com/cloudera/impala/catalog/Db.java b/fe/src/main/java/com/cloudera/impala/catalog/Db.java index db064289d..027ae9dff 100644 --- a/fe/src/main/java/com/cloudera/impala/catalog/Db.java +++ b/fe/src/main/java/com/cloudera/impala/catalog/Db.java @@ -19,7 +19,8 @@ import java.util.List; import org.apache.log4j.Logger; -import com.cloudera.impala.analysis.FunctionName; +import com.cloudera.impala.analysis.ColumnType; +import com.cloudera.impala.catalog.Function.CompareMode; import com.cloudera.impala.thrift.TCatalogObjectType; import com.cloudera.impala.thrift.TDatabase; import com.cloudera.impala.thrift.TFunctionType; @@ -54,6 +55,10 @@ public class Db implements CatalogObject { // on this map. private final HashMap> functions_; + // If true, this database is an Impala system database. + // (e.g. can't drop it, can't add tables to it, etc). + private boolean isSystemDb_ = false; + public Db(String name, Catalog catalog) { thriftDb_ = new TDatabase(name); parentCatalog_ = catalog; @@ -68,6 +73,8 @@ public class Db implements CatalogObject { functions_ = new HashMap>(); } + public void setIsSystemDb(boolean b) { isSystemDb_ = b; } + /** * Creates a Db object with no tables based on the given TDatabase thrift struct. */ @@ -75,6 +82,7 @@ public class Db implements CatalogObject { return new Db(db.getDb_name(), parentCatalog); } + public boolean isSystemDb() { return isSystemDb_; } public TDatabase toThrift() { return thriftDb_; } public String getName() { return thriftDb_.getDb_name(); } public TCatalogObjectType getCatalogObjectType() { @@ -132,15 +140,17 @@ public class Db implements CatalogObject { /** * Returns all the function signatures in this DB that match the specified - * fuction type. If the function type is null, all function signatures are returned. + * function type. If the function type is null, all function signatures are returned. */ public List getAllFunctionSignatures(TFunctionType type) { List names = Lists.newArrayList(); synchronized (functions_) { for (List fns: functions_.values()) { for (Function f: fns) { - if (type == null || (type == TFunctionType.SCALAR && f instanceof Udf) || - type == TFunctionType.AGGREGATE && f instanceof Uda) { + if (!f.userVisible()) continue; + if (type == null || + (type == TFunctionType.SCALAR && f instanceof ScalarFunction) || + (type == TFunctionType.AGGREGATE && f instanceof AggregateFunction)) { names.add(f.signatureString()); } } @@ -161,9 +171,9 @@ public class Db implements CatalogObject { /** * See comment in Catalog. */ - public boolean functionExists(FunctionName name) { + public boolean containsFunction(String name) { synchronized (functions_) { - return functions_.get(name.getFunction()) != null; + return functions_.get(name) != null; } } @@ -253,6 +263,27 @@ public class Db implements CatalogObject { return null; } + /** + * Add a builtin with the specified name and signatures to this db. + */ + public void addScalarBuiltin(boolean udfInterface, String fnName, String symbol, + boolean varArgs, ColumnType retType, ColumnType ... args) { + Preconditions.checkState(isSystemDb()); + addBuiltin(ScalarFunction.createBuiltin( + fnName, Lists.newArrayList(args), varArgs, retType, + symbol, udfInterface)); + } + + /** + * Adds a builtin to this database. The function must not already exist. + */ + public void addBuiltin(Function fn) { + Preconditions.checkState(isSystemDb()); + Preconditions.checkState(fn != null); + Preconditions.checkState(getFunction(fn, CompareMode.IS_INDISTINGUISHABLE) == null); + addFunction(fn); + } + /** * Returns a map of functionNames to list of (overloaded) functions with that name. */ diff --git a/fe/src/main/java/com/cloudera/impala/catalog/Function.java b/fe/src/main/java/com/cloudera/impala/catalog/Function.java index 938694943..715142e5c 100644 --- a/fe/src/main/java/com/cloudera/impala/catalog/Function.java +++ b/fe/src/main/java/com/cloudera/impala/catalog/Function.java @@ -64,6 +64,11 @@ public class Function implements CatalogObject { // TODO: we don't currently support varargs with no fixed types. i.e. fn(...) private boolean hasVarArgs_; + // If true (default), this function is called directly by the user. For operators, + // this is false. If false, it also means the function is not visible from + // 'show functions'. + private boolean userVisible_; + // Absolute path in HDFS for the binary that contains this function. // e.g. /udfs/udfs.jar private HdfsUri location_; @@ -80,6 +85,7 @@ public class Function implements CatalogObject { this.argTypes_ = argTypes; } this.retType_ = retType; + this.userVisible_ = true; } public Function(FunctionName name, List args, @@ -102,6 +108,7 @@ public class Function implements CatalogObject { public HdfsUri getLocation() { return location_; } public TFunctionBinaryType getBinaryType() { return binaryType_; } public boolean hasVarArgs() { return hasVarArgs_; } + public boolean userVisible() { return userVisible_; } public ColumnType getVarArgsType() { if (!hasVarArgs_) return ColumnType.INVALID; Preconditions.checkState(argTypes_.length > 0); @@ -112,6 +119,7 @@ public class Function implements CatalogObject { public void setLocation(HdfsUri loc) { location_ = loc; } public void setBinaryType(TFunctionBinaryType type) { binaryType_ = type; } public void setHasVarArgs(boolean v) { hasVarArgs_ = v; } + public void setUserVisible(boolean b) { userVisible_ = b; } // Returns a string with the signature in human readable format: // FnName(argtype1, argtyp2). e.g. Add(int, int) @@ -125,6 +133,12 @@ public class Function implements CatalogObject { return sb.toString(); } + @Override + public boolean equals(Object o) { + if (!(o instanceof Function)) return false; + return compare((Function)o, CompareMode.IS_IDENTICAL); + } + // Compares this to 'other' for mode. public boolean compare(Function other, CompareMode mode) { switch (mode) { @@ -227,8 +241,6 @@ public class Function implements CatalogObject { public TFunction toThrift() { TFunction fn = new TFunction(); - // TODO: this function should have a unique ID - fn.setId(0); fn.setSignature(signatureString()); fn.setName(name_.toThrift()); fn.setBinary_type(binaryType_); @@ -249,12 +261,12 @@ public class Function implements CatalogObject { Function function = null; if (fn.isSetScalar_fn()) { - function = new Udf(FunctionName.fromThrift(fn.getName()), argTypes, + function = new ScalarFunction(FunctionName.fromThrift(fn.getName()), argTypes, ColumnType.fromThrift(fn.getRet_type()), new HdfsUri(fn.getHdfs_location()), fn.getScalar_fn().getSymbol()); } else if (fn.isSetAggregate_fn()) { TAggregateFunction aggFn = fn.getAggregate_fn(); - function = new Uda(FunctionName.fromThrift(fn.getName()), argTypes, + function = new AggregateFunction(FunctionName.fromThrift(fn.getName()), argTypes, ColumnType.fromThrift(fn.getRet_type()), ColumnType.fromThrift(aggFn.getIntermediate_type()), new HdfsUri(fn.getHdfs_location()), aggFn.getUpdate_fn_symbol(), diff --git a/fe/src/main/java/com/cloudera/impala/catalog/ImpaladCatalog.java b/fe/src/main/java/com/cloudera/impala/catalog/ImpaladCatalog.java index 1e925e448..7292a3360 100644 --- a/fe/src/main/java/com/cloudera/impala/catalog/ImpaladCatalog.java +++ b/fe/src/main/java/com/cloudera/impala/catalog/ImpaladCatalog.java @@ -298,6 +298,7 @@ public class ImpaladCatalog extends Catalog { throws AuthorizationException { Preconditions.checkState(dbName != null && !dbName.isEmpty(), "Null or empty database name given as argument to Catalog.getDb"); + if (checkSystemDbAccess(dbName, privilege)) return getDb(dbName); PrivilegeRequestBuilder pb = new PrivilegeRequestBuilder(); if (privilege == Privilege.ANY) { checkAccess(user, pb.any().onAnyTable(dbName).toRequest()); @@ -342,6 +343,10 @@ public class ImpaladCatalog extends Catalog { */ public boolean dbContainsTable(String dbName, String tableName, User user, Privilege privilege) throws AuthorizationException, DatabaseNotFoundException { + if (checkSystemDbAccess(dbName, privilege)) { + return containsTable(dbName, tableName); + } + // Make sure the user has privileges to check if the table exists. checkAccess(user, new PrivilegeRequestBuilder() .allOf(privilege).onTable(dbName, tableName).toRequest()); @@ -361,6 +366,9 @@ public class ImpaladCatalog extends Catalog { public Table getTable(String dbName, String tableName, User user, Privilege privilege) throws AuthorizationException, DatabaseNotFoundException, TableLoadingException { + if (checkSystemDbAccess(dbName, privilege)) { + return getTable(dbName, tableName); + } checkAccess(user, new PrivilegeRequestBuilder() .allOf(privilege).onTable(dbName, tableName).toRequest()); @@ -459,6 +467,26 @@ public class ImpaladCatalog extends Catalog { } } + /** + * Throws an authorization exception if the dbName is a system db + * and p is trying to modify it. + * Returns true if this is a system db and the action is allowed. + */ + public boolean checkSystemDbAccess(String dbName, Privilege privilege) + throws AuthorizationException { + Db db = getDb(dbName); + if (db != null && db.isSystemDb()) { + switch (privilege) { + case VIEW_METADATA: + case ANY: + return true; + default: + throw new AuthorizationException("Cannot modify system database."); + } + } + return false; + } + /** * Adds the given TCatalogObject to the catalog cache. The update may be ignored * (considered out of date) if: @@ -611,4 +639,7 @@ public class ImpaladCatalog extends Catalog { * false otherwise. */ public boolean isReady() { return isReady_.get(); } + + // Only used for testing. + public void setIsReady() { isReady_.set(true); } } diff --git a/fe/src/main/java/com/cloudera/impala/catalog/ScalarFunction.java b/fe/src/main/java/com/cloudera/impala/catalog/ScalarFunction.java new file mode 100644 index 000000000..2bf4cb08d --- /dev/null +++ b/fe/src/main/java/com/cloudera/impala/catalog/ScalarFunction.java @@ -0,0 +1,170 @@ +// Copyright 2012 Cloudera Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.cloudera.impala.catalog; + +import java.util.ArrayList; +import java.util.List; + +import com.cloudera.impala.analysis.ColumnType; +import com.cloudera.impala.analysis.FunctionArgs; +import com.cloudera.impala.analysis.FunctionName; +import com.cloudera.impala.analysis.HdfsUri; +import com.cloudera.impala.thrift.TFunction; +import com.cloudera.impala.thrift.TFunctionBinaryType; +import com.cloudera.impala.thrift.TScalarFunction; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; + +/** + * Internal representation of a scalar function. + */ +public class ScalarFunction extends Function { + // The name inside the binary at location_ that contains this particular + // function. e.g. org.example.MyUdf.class. + private String symbolName_; + + // If true, this function is implemented with the Udf interface. Otherwise, + // it is implemented using the old ComputeFn interface. + // TODO: remove when ComputeFn interface is removed. + private boolean udfInterface_; + + public ScalarFunction(FunctionName fnName, FunctionArgs args, ColumnType retType) { + super(fnName, args.argTypes, retType, args.hasVarArgs); + } + + public ScalarFunction(FunctionName fnName, List argTypes, + ColumnType retType, HdfsUri location, String symbolName) { + super(fnName, argTypes, retType, false); + setLocation(location); + setSymbolName(symbolName); + } + + /** + * Creates a builtin scalar function. This is a helper that wraps a few steps + * into one call. + */ + public static ScalarFunction createBuiltin(String name, ArrayList argTypes, + boolean hasVarArgs, ColumnType retType, String symbol, boolean udfInterface) { + FunctionArgs fnArgs = new FunctionArgs(argTypes, hasVarArgs); + ScalarFunction fn = + new ScalarFunction(new FunctionName(Catalog.BUILTINS_DB, name), fnArgs, retType); + fn.setBinaryType(TFunctionBinaryType.BUILTIN); + fn.setUserVisible(true); + fn.udfInterface_ = udfInterface; + fn.symbolName_ = symbol; + return fn; + } + + /** + * Creates a builtin scalar operator function. This is a helper that wraps a few steps + * into one call. + * TODO: this needs to be kept in sync with what generates the be operator + * implementations. (gen_functions.py). Is there a better way to coordinate this. + */ + public static ScalarFunction createBuiltinOperator(String name, + ArrayList argTypes, ColumnType retType) { + // Operators have a well defined symbol based on the function name and type. + // Convert Add(TINYINT, TINYINT) --> Add_char_char + String beFn = Character.toUpperCase(name.charAt(0)) + name.substring(1); + for (int i = 0; i < argTypes.size(); ++i) { + switch (argTypes.get(i).getPrimitiveType()) { + case BOOLEAN: + beFn += "_bool"; + break; + case TINYINT: + beFn += "_char"; + break; + case SMALLINT: + beFn += "_short"; + break; + case INT: + beFn += "_int"; + break; + case BIGINT: + beFn += "_long"; + break; + case FLOAT: + beFn += "_float"; + break; + case DOUBLE: + beFn += "_double"; + break; + case STRING: + beFn += "_StringValue"; + break; + case TIMESTAMP: + beFn += "_TimestampValue"; + break; + default: + Preconditions.checkState(false); + } + } + + return createBuiltinOperator(name, "ComputeFunctions", beFn, argTypes, retType); + } + + /** + * Create a builtin function with the symbol beClass::beFn + */ + public static ScalarFunction createBuiltinOperator(String name, + String beClass, String beFn, ArrayList argTypes, ColumnType retType) { + FunctionArgs fnArgs = new FunctionArgs(argTypes, false); + ScalarFunction fn = + new ScalarFunction(new FunctionName(Catalog.BUILTINS_DB, name), fnArgs, retType); + fn.setBinaryType(TFunctionBinaryType.BUILTIN); + fn.setUserVisible(false); + fn.udfInterface_ = false; + fn.symbolName_ = GetComputeFnSymbol(beClass, beFn); + return fn; + } + + // Convert ComputeFunctions::Add_char_char + // '_ZN6impala16ComputeFunctions13Add_char_charEPNS_4ExprEPNS_8TupleRowE' + // TODO: this needs to be updated when we retire the ComputeFunction interface. + private static String GetComputeFnSymbol(String beClass, String beFn) { + String result = "_ZN6impala"; + result += beClass.length() + beClass; + result += beFn.length() + beFn; + result += "EPNS_4ExprEPNS_8TupleRowE"; + return result; + } + + /** + * Create a function that is used to search the catalog for a matching builtin. Only + * the fields necessary for matching function prototypes are specified. + */ + public static ScalarFunction createBuiltinSearchDesc(String name, ColumnType[] argTypes, + boolean hasVarArgs) { + FunctionArgs fnArgs = new FunctionArgs( + argTypes == null ? new ArrayList() : Lists.newArrayList(argTypes), + hasVarArgs); + ScalarFunction fn = new ScalarFunction( + new FunctionName(Catalog.BUILTINS_DB, name), fnArgs, ColumnType.INVALID); + fn.setBinaryType(TFunctionBinaryType.BUILTIN); + return fn; + } + + public void setSymbolName(String s) { symbolName_ = s; } + public String getSymbolName() { return symbolName_; } + public boolean isUdfInterface() { return udfInterface_; } + + @Override + public TFunction toThrift() { + TFunction fn = super.toThrift(); + fn.setScalar_fn(new TScalarFunction()); + fn.getScalar_fn().setSymbol(symbolName_); + return fn; + } +} diff --git a/fe/src/main/java/com/cloudera/impala/catalog/Udf.java b/fe/src/main/java/com/cloudera/impala/catalog/Udf.java deleted file mode 100644 index 626bcb057..000000000 --- a/fe/src/main/java/com/cloudera/impala/catalog/Udf.java +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2012 Cloudera Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package com.cloudera.impala.catalog; - -import java.util.List; - -import com.cloudera.impala.analysis.ColumnType; -import com.cloudera.impala.analysis.FunctionArgs; -import com.cloudera.impala.analysis.FunctionName; -import com.cloudera.impala.analysis.HdfsUri; -import com.cloudera.impala.thrift.TFunction; -import com.cloudera.impala.thrift.TScalarFunction; - - -/** - * Internal representation of a UDF. - * TODO: unify this with builtins. - */ - -public class Udf extends Function { - // The name inside the binary at location_ that contains this particular - // UDF. e.g. org.example.MyUdf.class. - private String symbolName_; - - public Udf(FunctionName fnName, FunctionArgs args, ColumnType retType) { - super(fnName, args.argTypes, retType, args.hasVarArgs); - } - - public Udf(FunctionName fnName, List argTypes, - ColumnType retType, HdfsUri location, String symbolName) { - super(fnName, argTypes, retType, false); - setLocation(location); - setSymbolName(symbolName); - } - - public void setSymbolName(String s) { symbolName_ = s; } - public String getSymbolName() { return symbolName_; } - - @Override - public TFunction toThrift() { - TFunction fn = super.toThrift(); - fn.setScalar_fn(new TScalarFunction()); - fn.getScalar_fn().setSymbol(symbolName_); - return fn; - } -} diff --git a/fe/src/main/java/com/cloudera/impala/planner/AggregationNode.java b/fe/src/main/java/com/cloudera/impala/planner/AggregationNode.java index 9b60ec99d..cf0320daf 100644 --- a/fe/src/main/java/com/cloudera/impala/planner/AggregationNode.java +++ b/fe/src/main/java/com/cloudera/impala/planner/AggregationNode.java @@ -26,9 +26,9 @@ import com.cloudera.impala.analysis.Expr; import com.cloudera.impala.analysis.FunctionCallExpr; import com.cloudera.impala.analysis.SlotDescriptor; import com.cloudera.impala.common.Pair; -import com.cloudera.impala.thrift.TAggregateFunctionCall; import com.cloudera.impala.thrift.TAggregationNode; import com.cloudera.impala.thrift.TExplainLevel; +import com.cloudera.impala.thrift.TExpr; import com.cloudera.impala.thrift.TPlanNode; import com.cloudera.impala.thrift.TPlanNodeType; import com.cloudera.impala.thrift.TQueryOptions; @@ -160,10 +160,10 @@ public class AggregationNode extends PlanNode { protected void toThrift(TPlanNode msg) { msg.node_type = TPlanNodeType.AGGREGATION_NODE; - List aggregateFunctions = Lists.newArrayList(); + List aggregateFunctions = Lists.newArrayList(); // only serialize agg exprs that are being materialized for (FunctionCallExpr e: aggInfo_.getMaterializedAggregateExprs()) { - aggregateFunctions.add(e.toTAggregateFunctionCall()); + aggregateFunctions.add(e.treeToThrift()); } msg.agg_node = new TAggregationNode( aggregateFunctions, diff --git a/fe/src/main/java/com/cloudera/impala/service/FeSupport.java b/fe/src/main/java/com/cloudera/impala/service/FeSupport.java index 7b3f5b02e..a84a39c2f 100644 --- a/fe/src/main/java/com/cloudera/impala/service/FeSupport.java +++ b/fe/src/main/java/com/cloudera/impala/service/FeSupport.java @@ -34,6 +34,7 @@ import com.cloudera.impala.thrift.TColumnValue; import com.cloudera.impala.thrift.TExpr; import com.cloudera.impala.thrift.TPrioritizeLoadRequest; import com.cloudera.impala.thrift.TPrioritizeLoadResponse; +import com.cloudera.impala.thrift.TFunction; import com.cloudera.impala.thrift.TQueryContext; import com.cloudera.impala.thrift.TStatus; import com.cloudera.impala.thrift.TSymbolLookupParams; @@ -62,7 +63,7 @@ public class FeSupport { public native static byte[] NativeEvalConstExpr(byte[] thriftExpr, byte[] thriftQueryGlobals); - // Returns a serialize TSymbolLookupResult + // Returns a serialized TSymbolLookupResult public native static byte[] NativeLookupSymbol(byte[] thriftSymbolLookup); // Does an RPCs to the Catalog Server to prioritize the metadata loading of a diff --git a/fe/src/main/java/com/cloudera/impala/service/JniFrontend.java b/fe/src/main/java/com/cloudera/impala/service/JniFrontend.java index 1abbd390d..81dd60dd0 100644 --- a/fe/src/main/java/com/cloudera/impala/service/JniFrontend.java +++ b/fe/src/main/java/com/cloudera/impala/service/JniFrontend.java @@ -325,6 +325,10 @@ public class JniFrontend { } } + public void setCatalogInitialized() { + frontend_.getCatalog().setIsReady(); + } + // Caching this saves ~50ms per call to getHadoopConfigAsHtml private static final Configuration CONF = new Configuration(); diff --git a/fe/src/main/java/com/cloudera/impala/service/MetadataOp.java b/fe/src/main/java/com/cloudera/impala/service/MetadataOp.java index c3fd634a7..4cf749a26 100644 --- a/fe/src/main/java/com/cloudera/impala/service/MetadataOp.java +++ b/fe/src/main/java/com/cloudera/impala/service/MetadataOp.java @@ -22,7 +22,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.cloudera.impala.analysis.ColumnType; -import com.cloudera.impala.analysis.OpcodeRegistry; import com.cloudera.impala.authorization.Privilege; import com.cloudera.impala.authorization.User; import com.cloudera.impala.catalog.Column; @@ -498,13 +497,6 @@ public class MetadataOp { return result; } - Pattern functionPattern = Pattern.compile(convertPattern(functionName)); - - for (String builtinFn: OpcodeRegistry.instance().getFunctionNames()) { - if (!functionPattern.matcher(builtinFn).matches()) continue; - result.rows.add(createFunctionResultRow(builtinFn)); - } - DbsMetadata dbsMetadata = getDbsMetadata(catalog, catalogName, schemaName, null, null, functionName, user); for (List fns: dbsMetadata.functions) { diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex index e8fe984cf..d37719e1b 100644 --- a/fe/src/main/jflex/sql-scanner.flex +++ b/fe/src/main/jflex/sql-scanner.flex @@ -56,7 +56,6 @@ import com.cloudera.impala.analysis.SqlParserSymbols; keywordMap.put("and", new Integer(SqlParserSymbols.KW_AND)); keywordMap.put("as", new Integer(SqlParserSymbols.KW_AS)); keywordMap.put("asc", new Integer(SqlParserSymbols.KW_ASC)); - keywordMap.put("avg", new Integer(SqlParserSymbols.KW_AVG)); keywordMap.put("avro", new Integer(SqlParserSymbols.KW_AVRO)); keywordMap.put("between", new Integer(SqlParserSymbols.KW_BETWEEN)); keywordMap.put("bigint", new Integer(SqlParserSymbols.KW_BIGINT)); @@ -70,7 +69,6 @@ import com.cloudera.impala.analysis.SqlParserSymbols; keywordMap.put("columns", new Integer(SqlParserSymbols.KW_COLUMNS)); keywordMap.put("comment", new Integer(SqlParserSymbols.KW_COMMENT)); keywordMap.put("compute", new Integer(SqlParserSymbols.KW_COMPUTE)); - keywordMap.put("count", new Integer(SqlParserSymbols.KW_COUNT)); keywordMap.put("create", new Integer(SqlParserSymbols.KW_CREATE)); keywordMap.put("cross", new Integer(SqlParserSymbols.KW_CROSS)); keywordMap.put("data", new Integer(SqlParserSymbols.KW_DATA)); @@ -83,8 +81,6 @@ import com.cloudera.impala.analysis.SqlParserSymbols; keywordMap.put("desc", new Integer(SqlParserSymbols.KW_DESC)); keywordMap.put("describe", new Integer(SqlParserSymbols.KW_DESCRIBE)); keywordMap.put("distinct", new Integer(SqlParserSymbols.KW_DISTINCT)); - keywordMap.put("distinctpc", new Integer(SqlParserSymbols.KW_DISTINCTPC)); - keywordMap.put("distinctpcsa", new Integer(SqlParserSymbols.KW_DISTINCTPCSA)); keywordMap.put("div", new Integer(SqlParserSymbols.KW_DIV)); keywordMap.put("double", new Integer(SqlParserSymbols.KW_DOUBLE)); keywordMap.put("drop", new Integer(SqlParserSymbols.KW_DROP)); @@ -107,7 +103,6 @@ import com.cloudera.impala.analysis.SqlParserSymbols; keywordMap.put("function", new Integer(SqlParserSymbols.KW_FUNCTION)); keywordMap.put("functions", new Integer(SqlParserSymbols.KW_FUNCTIONS)); keywordMap.put("group", new Integer(SqlParserSymbols.KW_GROUP)); - keywordMap.put("group_concat", new Integer(SqlParserSymbols.KW_GROUP_CONCAT)); keywordMap.put("having", new Integer(SqlParserSymbols.KW_HAVING)); keywordMap.put("if", new Integer(SqlParserSymbols.KW_IF)); keywordMap.put("in", new Integer(SqlParserSymbols.KW_IN)); @@ -130,11 +125,8 @@ import com.cloudera.impala.analysis.SqlParserSymbols; keywordMap.put("lines", new Integer(SqlParserSymbols.KW_LINES)); keywordMap.put("load", new Integer(SqlParserSymbols.KW_LOAD)); keywordMap.put("location", new Integer(SqlParserSymbols.KW_LOCATION)); - keywordMap.put("max", new Integer(SqlParserSymbols.KW_MAX)); keywordMap.put("merge_fn", new Integer(SqlParserSymbols.KW_MERGE_FN)); keywordMap.put("metadata", new Integer(SqlParserSymbols.KW_METADATA)); - keywordMap.put("min", new Integer(SqlParserSymbols.KW_MIN)); - keywordMap.put("ndv", new Integer(SqlParserSymbols.KW_NDV)); keywordMap.put("not", new Integer(SqlParserSymbols.KW_NOT)); keywordMap.put("null", new Integer(SqlParserSymbols.KW_NULL)); keywordMap.put("nulls", new Integer(SqlParserSymbols.KW_NULLS)); @@ -173,7 +165,6 @@ import com.cloudera.impala.analysis.SqlParserSymbols; keywordMap.put("stored", new Integer(SqlParserSymbols.KW_STORED)); keywordMap.put("straight_join", new Integer(SqlParserSymbols.KW_STRAIGHT_JOIN)); keywordMap.put("string", new Integer(SqlParserSymbols.KW_STRING)); - keywordMap.put("sum", new Integer(SqlParserSymbols.KW_SUM)); keywordMap.put("symbol", new Integer(SqlParserSymbols.KW_SYMBOL)); keywordMap.put("table", new Integer(SqlParserSymbols.KW_TABLE)); keywordMap.put("tables", new Integer(SqlParserSymbols.KW_TABLES)); diff --git a/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeExprsTest.java b/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeExprsTest.java index 4cdad83a7..4f4483204 100644 --- a/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeExprsTest.java +++ b/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeExprsTest.java @@ -28,7 +28,7 @@ import org.junit.Test; import com.cloudera.impala.analysis.TimestampArithmeticExpr.TimeUnit; import com.cloudera.impala.catalog.PrimitiveType; import com.cloudera.impala.catalog.TestSchemaUtils; -import com.cloudera.impala.catalog.Udf; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -947,23 +947,23 @@ public class AnalyzeExprsTest extends AnalyzerTest { // Add a udf default.udf(), default.udf(int), default.udf(string...), // default.udf(int, string...) and functional.udf(double) - catalog_.addFunction(new Udf(new FunctionName("default", "udf"), + catalog_.addFunction(new ScalarFunction(new FunctionName("default", "udf"), new ArrayList(), ColumnType.INT, dummyUri, null)); - catalog_.addFunction(new Udf(new FunctionName("default", "udf"), + catalog_.addFunction(new ScalarFunction(new FunctionName("default", "udf"), Lists.newArrayList(ColumnType.INT), ColumnType.INT, dummyUri, null)); - Udf varArgsUdf1 = new Udf(new FunctionName("default", "udf"), + ScalarFunction varArgsUdf1 = new ScalarFunction(new FunctionName("default", "udf"), Lists.newArrayList(ColumnType.STRING), ColumnType.INT, dummyUri, null); varArgsUdf1.setHasVarArgs(true); catalog_.addFunction(varArgsUdf1); - Udf varArgsUdf2 = new Udf(new FunctionName("default", "udf"), + ScalarFunction varArgsUdf2 = new ScalarFunction(new FunctionName("default", "udf"), Lists.newArrayList( ColumnType.INT, ColumnType.STRING), ColumnType.INT, dummyUri, null); varArgsUdf2.setHasVarArgs(true); catalog_.addFunction(varArgsUdf2); - Udf udf = new Udf(new FunctionName("functional", "udf"), + ScalarFunction udf = new ScalarFunction(new FunctionName("functional", "udf"), Lists.newArrayList(ColumnType.DOUBLE), ColumnType.INT, dummyUri, null); catalog_.addFunction(udf); @@ -1039,7 +1039,7 @@ public class AnalyzeExprsTest extends AnalyzerTest { testFuncExprDepthLimit("lower(", "'abc'", ")"); // UDF. - catalog_.addFunction(new Udf(new FunctionName("default", "udf"), + catalog_.addFunction(new ScalarFunction(new FunctionName("default", "udf"), Lists.newArrayList(ColumnType.INT), ColumnType.INT, new HdfsUri(""), null)); testFuncExprDepthLimit("udf(", "1", ")"); diff --git a/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeStmtsTest.java b/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeStmtsTest.java index 45235a214..37ebb8d8a 100644 --- a/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeStmtsTest.java +++ b/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeStmtsTest.java @@ -20,8 +20,8 @@ import java.lang.reflect.Field; import org.junit.Test; +import com.cloudera.impala.catalog.AggregateFunction; import com.cloudera.impala.catalog.PrimitiveType; -import com.cloudera.impala.catalog.Uda; import com.cloudera.impala.common.AnalysisException; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -462,8 +462,8 @@ public class AnalyzeStmtsTest extends AnalyzerTest { void addTestUda(String name, ColumnType retType, ColumnType... argTypes) { FunctionName fnName = new FunctionName("default", name); - catalog_.addFunction(new Uda(fnName, new FunctionArgs(Lists.newArrayList(argTypes), - false), retType)); + catalog_.addFunction(new AggregateFunction(fnName, + new FunctionArgs(Lists.newArrayList(argTypes), false), retType)); } @Test @@ -485,6 +485,8 @@ public class AnalyzeStmtsTest extends AnalyzerTest { AnalyzesOk("select aggfn(int_col) from functional.alltypesagg"); AnalysisError("select default.AggFn(1)", "aggregation without a FROM clause is not allowed"); + AnalysisError("select aggfn(distinct int_col) from functional.alltypesagg", + "User defined aggregates do not support DISTINCT."); AnalyzesOk("select default.aggfn(int_col) from functional.alltypes"); AnalyzesOk("select count(*) from functional.testtbl"); AnalyzesOk("select min(id), max(id), sum(id) from functional.testtbl"); @@ -505,6 +507,11 @@ public class AnalyzeStmtsTest extends AnalyzerTest { AnalysisError("select 1 from functional.alltypes where aggfn(1)", "aggregate function not allowed in WHERE clause"); + AnalysisError("select count() from functional.alltypes", + "count() is not allowed."); + AnalysisError("select min() from functional.alltypes", + "No matching function with signature: min()."); + // only count() allows '*' AnalysisError("select avg(*) from functional.testtbl", "'*' can only be used in conjunction with COUNT"); @@ -517,32 +524,26 @@ public class AnalyzeStmtsTest extends AnalyzerTest { // multiple args AnalysisError("select count(id, zip) from functional.testtbl", - "COUNT must have DISTINCT for multiple arguments: COUNT(id, zip)"); + "COUNT must have DISTINCT for multiple arguments: count(id, zip)"); AnalysisError("select min(id, zip) from functional.testtbl", - "MIN requires exactly one parameter"); - AnalysisError("select max(id, zip) from functional.testtbl", - "MAX requires exactly one parameter"); - AnalysisError("select sum(id, zip) from functional.testtbl", - "SUM requires exactly one parameter"); - AnalysisError("select avg(id, zip) from functional.testtbl", - "AVG requires exactly one parameter"); + "No matching function with signature: min(BIGINT, INT)."); AnalysisError("select group_concat(name, '-', ',') from functional.testtbl", - "GROUP_CONCAT requires one or two parameters"); + "No matching function with signature: group_concat(STRING, STRING, STRING)"); // nested aggregates AnalysisError("select sum(count(*)) from functional.testtbl", "aggregate function cannot contain aggregate parameters"); AnalysisError("select min(aggfn(int_col)) from functional.alltypes", "aggregate function cannot contain aggregate parameters: " + - "MIN(default.aggfn(int_col))"); + "min(default.aggfn(int_col))"); // wrong type AnalysisError("select sum(timestamp_col) from functional.alltypes", - "SUM requires a numeric parameter: SUM(timestamp_col)"); + "SUM requires a numeric parameter: sum(timestamp_col)"); AnalysisError("select sum(string_col) from functional.alltypes", - "SUM requires a numeric parameter: SUM(string_col)"); + "SUM requires a numeric parameter: sum(string_col)"); AnalysisError("select avg(string_col) from functional.alltypes", - "AVG requires a numeric or timestamp parameter: AVG(string_col)"); + "AVG requires a numeric or timestamp parameter: avg(string_col)"); // aggregate requires table in the FROM clause AnalysisError("select count(*)", "aggregation without a FROM clause is not allowed"); @@ -566,17 +567,14 @@ public class AnalyzeStmtsTest extends AnalyzerTest { AnalyzesOk(query2); } else { AnalysisError(query1, - "GROUP_CONCAT requires first parameter to be of type STRING"); + "No matching function with signature: group_concat("); AnalysisError(query2, - "GROUP_CONCAT requires second parameter to be of type STRING"); + "No matching function with signature: group_concat("); } } // Test distinct estimate for (ColumnType type: typeToLiteralValue_.keySet()) { - AnalyzesOk(String.format( - "select distinctpc(%s) from functional.alltypes", - typeToLiteralValue_.get(type))); AnalyzesOk(String.format( "select ndv(%s) from functional.alltypes", typeToLiteralValue_.get(type))); @@ -586,7 +584,6 @@ public class AnalyzeStmtsTest extends AnalyzerTest { @Test public void TestDistinct() throws AnalysisException { AnalyzesOk("select count(distinct id) as sum_id from functional.testtbl"); - // DISTINCT AnalyzesOk("select count(distinct id) as sum_id from " + "functional.testtbl order by sum_id"); AnalyzesOk("select count(distinct id) as sum_id from " + @@ -608,6 +605,7 @@ public class AnalyzeStmtsTest extends AnalyzerTest { + "from functional.alltypesagg group by 1"); AnalyzesOk("select tinyint_col, count(distinct int_col)," + "sum(distinct int_col) from functional.alltypesagg group by 1"); + AnalyzesOk("select avg(DISTINCT(tinyint_col)) from functional.alltypesagg"); AnalysisError("select tinyint_col, count(distinct int_col)," + "sum(distinct bigint_col) from functional.alltypesagg group by 1", "all DISTINCT aggregate functions need to have the same set of parameters"); @@ -616,7 +614,7 @@ public class AnalyzeStmtsTest extends AnalyzerTest { + "min(distinct smallint_col), max(distinct string_col) " + "from functional.alltypesagg group by 1"); AnalysisError("select group_concat(distinct name) from functional.testtbl", - "GROUP_CONCAT does not support DISTINCT"); + "GROUP_CONCAT() does not support DISTINCT"); } @Test diff --git a/fe/src/test/java/com/cloudera/impala/analysis/AnalyzerTest.java b/fe/src/test/java/com/cloudera/impala/analysis/AnalyzerTest.java index 139db87a1..89e83640b 100644 --- a/fe/src/test/java/com/cloudera/impala/analysis/AnalyzerTest.java +++ b/fe/src/test/java/com/cloudera/impala/analysis/AnalyzerTest.java @@ -35,7 +35,7 @@ import com.cloudera.impala.catalog.Catalog; import com.cloudera.impala.catalog.Function; import com.cloudera.impala.catalog.ImpaladCatalog; import com.cloudera.impala.catalog.PrimitiveType; -import com.cloudera.impala.catalog.Udf; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.common.ImpalaException; import com.cloudera.impala.testutil.TestUtils; @@ -89,7 +89,7 @@ public class AnalyzerTest { protected Function addTestFunction(String db, String fnName, ArrayList args, boolean varArgs) { - Function fn = new Udf( + Function fn = new ScalarFunction( new FunctionName(db, fnName), args, ColumnType.INT, null, null); fn.setHasVarArgs(varArgs); catalog_.addFunction(fn); diff --git a/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java b/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java index d54a0b422..8bca4cd1a 100644 --- a/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java +++ b/fe/src/test/java/com/cloudera/impala/analysis/AuthorizationTest.java @@ -35,7 +35,7 @@ import com.cloudera.impala.authorization.User; import com.cloudera.impala.catalog.AuthorizationException; import com.cloudera.impala.catalog.Catalog; import com.cloudera.impala.catalog.ImpaladCatalog; -import com.cloudera.impala.catalog.Udf; +import com.cloudera.impala.catalog.ScalarFunction; import com.cloudera.impala.common.AnalysisException; import com.cloudera.impala.common.ImpalaException; import com.cloudera.impala.common.InternalException; @@ -282,6 +282,9 @@ public class AuthorizationTest { } catch (AnalysisException e) { Assert.assertEquals(e.getMessage(), "Database does not exist: newdb"); } + + // All users should be able to use the system db. + AuthzOk("use _impala_builtins"); } @Test @@ -376,6 +379,9 @@ public class AuthorizationTest { "'hdfs://localhost:20500/test-warehouse/alltypes'", "User '%s' does not have privileges to access: " + "hdfs://localhost:20500/test-warehouse/alltypes"); + + AuthzError("create table _impala_builtins.tbl(i int)", + "Cannot modify system database."); } @Test @@ -423,6 +429,10 @@ public class AuthorizationTest { AuthzError("create view nodb.alltypes as select * from functional.alltypesagg", "User '%s' does not have privileges to execute 'CREATE' on: " + "nodb.alltypes"); + + AuthzError("create view _impala_builtins.new_view as " + + "select * from functional.alltypesagg", + "Cannot modify system database."); } @Test @@ -446,6 +456,10 @@ public class AuthorizationTest { // No existent db (no permissions). AuthzError("create database nodb", "User '%s' does not have privileges to execute 'CREATE' on: nodb"); + + // No existent db (no permissions). + AuthzError("create database if not exists _impala_builtins", + "Cannot modify system database."); } @Test @@ -473,6 +487,9 @@ public class AuthorizationTest { "User '%s' does not have privileges to execute 'DROP' on: nodb"); AuthzError("drop database if exists nodb", "User '%s' does not have privileges to execute 'DROP' on: nodb"); + + AuthzError("drop database _impala_builtins", + "Cannot modify system database."); } @Test @@ -502,6 +519,10 @@ public class AuthorizationTest { // Using DROP TABLE on a view does not reveal privileged information. AuthzError("drop table functional.view_view", "User '%s' does not have privileges to execute 'DROP' on: functional.view_view"); + + // Using DROP TABLE on a view does not reveal privileged information. + AuthzError("drop table if exists _impala_builtins.tbl", + "Cannot modify system database."); } @Test @@ -531,6 +552,10 @@ public class AuthorizationTest { // Using DROP VIEW on a table does not reveal privileged information. AuthzError("drop view functional.alltypes", "User '%s' does not have privileges to execute 'DROP' on: functional.alltypes"); + + // Using DROP VIEW on a table does not reveal privileged information. + AuthzError("drop view _impala_builtins.my_view", + "Cannot modify system database."); } @Test @@ -776,6 +801,8 @@ public class AuthorizationTest { public void TestShowPermissions() throws AuthorizationException, AnalysisException { AuthzOk("show tables in functional"); AuthzOk("show databases"); + AuthzOk("show tables in _impala_builtins"); + AuthzOk("show functions in _impala_builtins"); // Database exists, user does not have access. AuthzError("show tables in functional_rc", @@ -1011,14 +1038,21 @@ public class AuthorizationTest { "'/test-warehouse/libTestUdfs.so' symbol='NoArgs'"); AuthzOk(adminContext, "drop function if exists f()"); + // Can't add function to system db + AuthzError(adminContext, "create function _impala_builtins.f() returns int location " + + "'/test-warehouse/libTestUdfs.so' symbol='NoArgs'", + "Cannot modify system database.", ADMIN_USER); + AuthzError(adminContext, "drop function if exists pi()", + "Cannot modify system database.", ADMIN_USER); + // Add default.f(), tpch.f() - catalog_.addFunction(new Udf(new FunctionName("default", "f"), + catalog_.addFunction(new ScalarFunction(new FunctionName("default", "f"), new ArrayList(), ColumnType.INT, null, null)); - catalog_.addFunction(new Udf(new FunctionName("tpch", "f"), + catalog_.addFunction(new ScalarFunction(new FunctionName("tpch", "f"), new ArrayList(), ColumnType.INT, null, null)); AuthzError(context, "select default.f()", - "User '%s' does not have privileges to access: default.*", + "User '%s' does not have privileges to access: default", currentUser); // Couldn't create tpch.f() but can run it. AuthzOk(context, "select tpch.f()"); diff --git a/fe/src/test/java/com/cloudera/impala/analysis/ParserTest.java b/fe/src/test/java/com/cloudera/impala/analysis/ParserTest.java index 9685a8a2e..4642a872e 100644 --- a/fe/src/test/java/com/cloudera/impala/analysis/ParserTest.java +++ b/fe/src/test/java/com/cloudera/impala/analysis/ParserTest.java @@ -893,27 +893,19 @@ public class ParserTest { ParsesOk("select count(*), count(a), count(distinct a, b) from t"); ParsesOk("select count(NULL), count(TRUE), count(FALSE), " + "count(distinct TRUE, FALSE, NULL) from t"); - ParserError("select count() from t"); ParsesOk("select count(all *) from t"); ParsesOk("select count(all 1) from t"); ParsesOk("select min(a), min(distinct a) from t"); - ParserError("select min() from t"); ParsesOk("select max(a), max(distinct a) from t"); - ParserError("select max() from t"); ParsesOk("select sum(a), sum(distinct a) from t"); - ParserError("select sum() from t"); ParsesOk("select avg(a), avg(distinct a) from t"); - ParserError("select avg() from t"); ParsesOk("select distinct a, b, c from t"); ParsesOk("select distinctpc(a), distinctpc(distinct a) from t"); - ParserError("select distinctpc() from t"); ParsesOk("select distinctpcsa(a), distinctpcsa(distinct a) from t"); ParsesOk("select ndv(a), ndv(distinct a) from t"); - ParserError("select distinctpcsa() from t"); ParsesOk("select group_concat(a) from t"); ParsesOk("select group_concat(a, ', ') from t"); ParsesOk("select group_concat(a, ', ', c) from t"); - ParserError("select group_concat() from t"); } @Test @@ -1951,9 +1943,9 @@ public class ParserTest { "select from t\n" + " ^\n" + "Encountered: FROM\n" + - "Expected: ALL, AVG, CASE, CAST, COUNT, DISTINCT, DISTINCTPC, " + - "DISTINCTPCSA, FALSE, GROUP_CONCAT, IF, INTERVAL, MAX, MIN, NDV, NOT, NULL, " + - "STRAIGHT_JOIN, SUM, TRUE, IDENTIFIER\n"); + "Expected: ALL, CASE, CAST, DISTINCT, " + + "FALSE, IF, INTERVAL, NOT, NULL, " + + "STRAIGHT_JOIN, TRUE, IDENTIFIER\n"); // missing from ParserError("select c, b, c where a = 5", @@ -1978,8 +1970,8 @@ public class ParserTest { "select c, b, c from t where\n" + " ^\n" + "Encountered: EOF\n" + - "Expected: AVG, CASE, CAST, COUNT, DISTINCTPC, DISTINCTPCSA, FALSE, " + - "GROUP_CONCAT, IF, INTERVAL, MAX, MIN, NDV, NOT, NULL, SUM, TRUE, IDENTIFIER\n"); + "Expected: CASE, CAST, FALSE, " + + "IF, INTERVAL, NOT, NULL, TRUE, IDENTIFIER\n"); // missing predicate in where clause (group by) ParserError("select c, b, c from t where group by a, b", @@ -1987,8 +1979,8 @@ public class ParserTest { "select c, b, c from t where group by a, b\n" + " ^\n" + "Encountered: GROUP\n" + - "Expected: AVG, CASE, CAST, COUNT, DISTINCTPC, DISTINCTPCSA, FALSE, " + - "GROUP_CONCAT, IF, INTERVAL, MAX, MIN, NDV, NOT, NULL, SUM, TRUE, IDENTIFIER\n"); + "Expected: CASE, CAST, FALSE, " + + "IF, INTERVAL, NOT, NULL, TRUE, IDENTIFIER\n"); // unmatched string literal starting with " ParserError("select c, \"b, c from t", @@ -2055,8 +2047,8 @@ public class ParserTest { "...c,c,c,c,c,c,c,c,cd,c,d,d, ,c, from t\n" + " ^\n" + "Encountered: COMMA\n" + - "Expected: AVG, CASE, CAST, COUNT, DISTINCTPC, DISTINCTPCSA, FALSE, " + - "GROUP_CONCAT, IF, INTERVAL, MAX, MIN, NDV, NOT, NULL, SUM, TRUE, IDENTIFIER\n"); + "Expected: CASE, CAST, FALSE, " + + "IF, INTERVAL, NOT, NULL, TRUE, IDENTIFIER\n"); } diff --git a/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java b/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java index 886b7c7af..1f054f7ba 100644 --- a/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java +++ b/fe/src/test/java/com/cloudera/impala/analysis/ToSqlTest.java @@ -136,24 +136,24 @@ public class ToSqlTest extends AnalyzerTest { public void aggregationTest() { testToSql("select COUNT(*), count(id), COUNT(id), SUM(id), AVG(id) " + "from functional.alltypes group by tinyint_col", - "SELECT COUNT(*), COUNT(id), COUNT(id), SUM(id), AVG(id) " + + "SELECT count(*), count(id), count(id), sum(id), avg(id) " + "FROM functional.alltypes GROUP BY tinyint_col"); testToSql("select avg(float_col / id) from functional.alltypes group by tinyint_col", - "SELECT AVG(float_col / id) " + + "SELECT avg(float_col / id) " + "FROM functional.alltypes GROUP BY tinyint_col"); testToSql("select avg(double_col) from functional.alltypes " + "group by int_col, tinyint_col, bigint_col", - "SELECT AVG(double_col) FROM functional.alltypes " + + "SELECT avg(double_col) FROM functional.alltypes " + "GROUP BY int_col, tinyint_col, bigint_col"); // Group by with having clause testToSql("select avg(id) from functional.alltypes " + "group by tinyint_col having count(tinyint_col) > 10", - "SELECT AVG(id) FROM functional.alltypes " + - "GROUP BY tinyint_col HAVING COUNT(tinyint_col) > 10"); + "SELECT avg(id) FROM functional.alltypes " + + "GROUP BY tinyint_col HAVING count(tinyint_col) > 10"); testToSql("select sum(id) from functional.alltypes group by tinyint_col " + "having avg(tinyint_col) > 10 AND count(tinyint_col) > 5", - "SELECT SUM(id) FROM functional.alltypes GROUP BY tinyint_col " + - "HAVING AVG(tinyint_col) > 10 AND COUNT(tinyint_col) > 5"); + "SELECT sum(id) FROM functional.alltypes GROUP BY tinyint_col " + + "HAVING avg(tinyint_col) > 10 AND count(tinyint_col) > 5"); } // Test the toSql() output of the order by clause. @@ -206,11 +206,11 @@ public class ToSqlTest extends AnalyzerTest { "group by bigint_col, int_col " + "having count(int_col) > 10 OR sum(bigint_col) > 20 " + "order by 2 DESC NULLS LAST, 3 ASC", - "SELECT bigint_col, AVG(double_col), SUM(tinyint_col) " + + "SELECT bigint_col, avg(double_col), sum(tinyint_col) " + "FROM functional.alltypes " + "WHERE double_col > 2.5 AND string_col != 'abc' " + "GROUP BY bigint_col, int_col " + - "HAVING COUNT(int_col) > 10 OR SUM(bigint_col) > 20 " + + "HAVING count(int_col) > 10 OR sum(bigint_col) > 20 " + "ORDER BY 2 DESC NULLS LAST, 3 ASC"); } @@ -299,10 +299,10 @@ public class ToSqlTest extends AnalyzerTest { "(select id, string_col from functional.alltypes) t1 inner join " + "(select id, float_col from functional.alltypes) t2 on (t1.id = t2.id) " + "group by t1.id, t2.id having count(t2.float_col) > 2", - "SELECT COUNT(t1.string_col), SUM(t2.float_col) FROM " + + "SELECT count(t1.string_col), sum(t2.float_col) FROM " + "(SELECT id, string_col FROM functional.alltypes) t1 INNER JOIN " + "(SELECT id, float_col FROM functional.alltypes) t2 ON (t1.id = t2.id) " + - "GROUP BY t1.id, t2.id HAVING COUNT(t2.float_col) > 2"); + "GROUP BY t1.id, t2.id HAVING count(t2.float_col) > 2"); // Test undoing expr substitution in order by clause. testToSql("select t1.id, t2.id from " + "(select id, string_col from functional.alltypes) t1 inner join " + @@ -451,9 +451,9 @@ public class ToSqlTest extends AnalyzerTest { testToSql("select count(*), (count(*)), avg(int_col), (avg(int_col)), " + "sum(int_col), (sum(int_col)), min(int_col), (min(int_col)), " + "max(int_col), (max(int_col)) from functional.alltypes", - "SELECT COUNT(*), (COUNT(*)), AVG(int_col), (AVG(int_col)), " + - "SUM(int_col), (SUM(int_col)), MIN(int_col), (MIN(int_col)), " + - "MAX(int_col), (MAX(int_col)) FROM functional.alltypes"); + "SELECT count(*), (count(*)), avg(int_col), (avg(int_col)), " + + "sum(int_col), (sum(int_col)), min(int_col), (min(int_col)), " + + "max(int_col), (max(int_col)) FROM functional.alltypes"); // ArithmeticExpr. testToSql("select 1 * 1, (1 * 1), 2 / 2, (2 / 2), 3 % 3, (3 % 3), " + "4 DIV 4, (4 DIV 4), 5 + 5, (5 + 5), 6 - 6, (6 - 6), 7 & 7, (7 & 7), " + diff --git a/fe/src/test/java/com/cloudera/impala/catalog/CatalogTest.java b/fe/src/test/java/com/cloudera/impala/catalog/CatalogTest.java index c2b805c77..d3889acf7 100644 --- a/fe/src/test/java/com/cloudera/impala/catalog/CatalogTest.java +++ b/fe/src/test/java/com/cloudera/impala/catalog/CatalogTest.java @@ -579,7 +579,7 @@ public class CatalogTest { assertEquals(fnNames.size(), 0); - Udf udf1 = new Udf(new FunctionName("default", "Foo"), + ScalarFunction udf1 = new ScalarFunction(new FunctionName("default", "Foo"), args1, ColumnType.INVALID, new HdfsUri("/Foo"), "Foo.class"); catalog_.addFunction(udf1); fnNames = catalog_.getFunctionSignatures(TFunctionType.SCALAR, "default", null); @@ -587,7 +587,7 @@ public class CatalogTest { assertTrue(fnNames.contains("foo()")); // Same function name, overloaded arguments - Udf udf2 = new Udf(new FunctionName("default", "Foo"), + ScalarFunction udf2 = new ScalarFunction(new FunctionName("default", "Foo"), args2, ColumnType.INVALID, new HdfsUri("/Foo"), "Foo.class"); catalog_.addFunction(udf2); fnNames = catalog_.getFunctionSignatures(TFunctionType.SCALAR, "default", null); @@ -596,7 +596,7 @@ public class CatalogTest { assertTrue(fnNames.contains("foo(INT)")); // Add a function with a new name - Udf udf3 = new Udf(new FunctionName("default", "Bar"), + ScalarFunction udf3 = new ScalarFunction(new FunctionName("default", "Bar"), args2, ColumnType.INVALID, new HdfsUri("/Foo"), "Foo.class"); catalog_.addFunction(udf3); fnNames = catalog_.getFunctionSignatures(TFunctionType.SCALAR, "default", null); diff --git a/fe/src/test/java/com/cloudera/impala/service/FrontendTest.java b/fe/src/test/java/com/cloudera/impala/service/FrontendTest.java index 61974a39c..dc2691617 100644 --- a/fe/src/test/java/com/cloudera/impala/service/FrontendTest.java +++ b/fe/src/test/java/com/cloudera/impala/service/FrontendTest.java @@ -7,6 +7,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.util.List; +import java.util.Set; import org.apache.hive.service.cli.thrift.TGetCatalogsReq; import org.apache.hive.service.cli.thrift.TGetColumnsReq; @@ -28,6 +29,7 @@ import com.cloudera.impala.thrift.TQueryContext; import com.cloudera.impala.thrift.TResultRow; import com.cloudera.impala.thrift.TResultSet; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; /** * Unit test for Frontend.execHiveServer2MetadataOp, which executes a HiveServer2 @@ -171,14 +173,19 @@ public class FrontendTest { assertEquals(6, resp.schema.columns.size()); assertEquals(6, resp.rows.get(0).colVals.size()); - assertEquals(3, resp.rows.size()); + Set fns = Sets.newHashSet(); + for (TResultRow row: resp.rows) { + String fn = row.colVals.get(2).stringVal.toLowerCase(); + fn = fn.substring(0, fn.indexOf('(')); + fns.add(fn); + } + assertEquals(3, fns.size()); List expectedResult = Lists.newArrayList(); expectedResult.add("subdate"); expectedResult.add("substr"); expectedResult.add("substring"); - for (TResultRow row: resp.rows) { - String fn = row.colVals.get(2).stringVal.toLowerCase(); + for (String fn: fns) { assertTrue(fn + " not found", expectedResult.remove(fn)); } } diff --git a/fe/src/test/java/com/cloudera/impala/service/JdbcTest.java b/fe/src/test/java/com/cloudera/impala/service/JdbcTest.java index ed5e43c1d..66399d043 100644 --- a/fe/src/test/java/com/cloudera/impala/service/JdbcTest.java +++ b/fe/src/test/java/com/cloudera/impala/service/JdbcTest.java @@ -212,10 +212,13 @@ public class JdbcTest { @Test public void testMetaDataGetFunctions() throws SQLException { // It should return one function "parse_url". - ResultSet rs = con_.getMetaData().getFunctions(null, null, "parse_url"); + ResultSet rs = con_.getMetaData().getFunctions( + null, null, "sqrt%"); assertTrue(rs.next()); String funcName = rs.getString("FUNCTION_NAME"); - assertEquals("Incorrect function name", "parse_url", funcName.toLowerCase()); + System.err.println(funcName.toLowerCase()); + assertEquals("Incorrect function name", + "sqrt(double)", funcName.toLowerCase()); assertFalse(rs.next()); rs.close(); } diff --git a/fe/src/test/java/com/cloudera/impala/testutil/ImpaladClientExecutor.java b/fe/src/test/java/com/cloudera/impala/testutil/ImpaladClientExecutor.java index 4eae41e15..f219c7363 100644 --- a/fe/src/test/java/com/cloudera/impala/testutil/ImpaladClientExecutor.java +++ b/fe/src/test/java/com/cloudera/impala/testutil/ImpaladClientExecutor.java @@ -200,16 +200,22 @@ public class ImpaladClientExecutor { break; case PARQUET_FILE_SIZE: optionValue = String.valueOf(queryOptions.getParquet_file_size()); + break; case YARN_POOL: optionValue = String.valueOf(queryOptions.yarn_pool); + break; case V_CPU_CORES: optionValue = String.valueOf(queryOptions.v_cpu_cores); + break; case RESERVATION_REQUEST_TIMEOUT: optionValue = String.valueOf(queryOptions.reservation_request_timeout); break; case EXPLAIN_LEVEL: optionValue = String.valueOf(queryOptions.getExplain_level()); break; + case SYNC_DDL: + optionValue = String.valueOf(queryOptions.isSync_ddl()); + break; default: Preconditions.checkState(false, "Unhandled option:" + option.toString()); } diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test b/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test index fdfe4b97a..c8775f624 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test @@ -4,18 +4,18 @@ avg(tinyint_col) from functional.alltypesagg ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*), COUNT(tinyint_col), MIN(tinyint_col), MAX(tinyint_col), SUM(tinyint_col) +| output: count(*), count(tinyint_col), min(tinyint_col), max(tinyint_col), sum(tinyint_col) | 00:SCAN HDFS [functional.alltypesagg] partitions=10/10 size=743.67KB ---- DISTRIBUTEDPLAN 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)), SUM(COUNT(tinyint_col)), MIN(MIN(tinyint_col)), MAX(MAX(tinyint_col)), SUM(SUM(tinyint_col)) +| output: sum(count(*)), sum(count(tinyint_col)), min(min(tinyint_col)), max(max(tinyint_col)), sum(sum(tinyint_col)) | 02:EXCHANGE [PARTITION=UNPARTITIONED] | 01:AGGREGATE -| output: COUNT(*), COUNT(tinyint_col), MIN(tinyint_col), MAX(tinyint_col), SUM(tinyint_col) +| output: count(*), count(tinyint_col), min(tinyint_col), max(tinyint_col), sum(tinyint_col) | 00:SCAN HDFS [functional.alltypesagg] partitions=10/10 size=743.67KB @@ -27,7 +27,7 @@ from functional.alltypesagg group by 2, 1 ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*), MIN(tinyint_col), MAX(tinyint_col), SUM(tinyint_col), COUNT(tinyint_col) +| output: count(*), min(tinyint_col), max(tinyint_col), sum(tinyint_col), count(tinyint_col) | group by: bigint_col, tinyint_col | 00:SCAN HDFS [functional.alltypesagg] @@ -48,13 +48,13 @@ NODE 0: 04:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)), MIN(MIN(tinyint_col)), MAX(MAX(tinyint_col)), SUM(SUM(tinyint_col)), SUM(COUNT(tinyint_col)) +| output: sum(count(*)), min(min(tinyint_col)), max(max(tinyint_col)), sum(sum(tinyint_col)), sum(count(tinyint_col)) | group by: bigint_col, tinyint_col | 02:EXCHANGE [PARTITION=HASH(bigint_col,tinyint_col)] | 01:AGGREGATE -| output: COUNT(*), MIN(tinyint_col), MAX(tinyint_col), SUM(tinyint_col), COUNT(tinyint_col) +| output: count(*), min(tinyint_col), max(tinyint_col), sum(tinyint_col), count(tinyint_col) | group by: bigint_col, tinyint_col | 00:SCAN HDFS [functional.alltypesagg] @@ -67,26 +67,26 @@ having count(id) > 0 order by avg(zip) limit 10 ---- PLAN 02:TOP-N [LIMIT=10] -| order by: SUM(zip) / COUNT(zip) ASC +| order by: sum(zip) / count(zip) ASC | 01:AGGREGATE [FINALIZE] -| output: SUM(id), COUNT(id), SUM(zip), COUNT(zip) -| having: COUNT(id) > 0 +| output: sum(id), count(id), sum(zip), count(zip) +| having: count(id) > 0 | 00:SCAN HDFS [functional.testtbl] partitions=1/1 size=0B ---- DISTRIBUTEDPLAN 02:TOP-N [LIMIT=10] -| order by: SUM(zip) / COUNT(zip) ASC +| order by: sum(zip) / count(zip) ASC | 04:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(id)), SUM(COUNT(id)), SUM(SUM(zip)), SUM(COUNT(zip)) -| having: COUNT(id) > 0 +| output: sum(sum(id)), sum(count(id)), sum(sum(zip)), sum(count(zip)) +| having: count(id) > 0 | 03:EXCHANGE [PARTITION=UNPARTITIONED] | 01:AGGREGATE -| output: SUM(id), COUNT(id), SUM(zip), COUNT(zip) +| output: sum(id), count(id), sum(zip), count(zip) | 00:SCAN HDFS [functional.testtbl] partitions=1/1 size=0B diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test b/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test index 6f3ac91f6..fcf7af1c9 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test @@ -2,18 +2,18 @@ select distinctpc(l_orderkey) from tpch.lineitem ---- PLAN 01:AGGREGATE [FINALIZE] -| output: DISTINCT_PC(l_orderkey) +| output: distinctpc(l_orderkey) | 00:SCAN HDFS [tpch.lineitem] partitions=1/1 size=718.94MB ---- DISTRIBUTEDPLAN 03:AGGREGATE [MERGE FINALIZE] -| output: DISTINCT_PC(DISTINCT_PC(l_orderkey)) +| output: distinctpc(distinctpc(l_orderkey)) | 02:EXCHANGE [PARTITION=UNPARTITIONED] | 01:AGGREGATE -| output: DISTINCT_PC(l_orderkey) +| output: distinctpc(l_orderkey) | 00:SCAN HDFS [tpch.lineitem] partitions=1/1 size=718.94MB @@ -22,31 +22,31 @@ select distinctpc(l_orderkey) from tpch.lineitem select count(distinct l_orderkey), distinctpc(l_orderkey) from tpch.lineitem ---- PLAN 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(l_orderkey), DISTINCT_PC(DISTINCT_PC(l_orderkey)) +| output: count(l_orderkey), distinctpc(distinctpc(l_orderkey)) | 01:AGGREGATE -| output: DISTINCT_PC(l_orderkey) +| output: distinctpc(l_orderkey) | group by: l_orderkey | 00:SCAN HDFS [tpch.lineitem] partitions=1/1 size=718.94MB ---- DISTRIBUTEDPLAN 06:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(l_orderkey)), DISTINCT_PC(DISTINCT_PC(DISTINCT_PC(l_orderkey))) +| output: sum(count(l_orderkey)), distinctpc(distinctpc(distinctpc(l_orderkey))) | 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:AGGREGATE [MERGE] -| output: COUNT(l_orderkey), DISTINCT_PC(DISTINCT_PC(l_orderkey)) +| output: count(l_orderkey), distinctpc(distinctpc(l_orderkey)) | 04:AGGREGATE [MERGE] -| output: DISTINCT_PC(DISTINCT_PC(l_orderkey)) +| output: distinctpc(distinctpc(l_orderkey)) | group by: l_orderkey | 03:EXCHANGE [PARTITION=HASH(l_orderkey)] | 01:AGGREGATE -| output: DISTINCT_PC(l_orderkey) +| output: distinctpc(l_orderkey) | group by: l_orderkey | 00:SCAN HDFS [tpch.lineitem] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test b/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test index 8fc1e4a9c..16690564d 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test @@ -49,7 +49,7 @@ select count(distinct id, zip) from functional.testtbl ---- PLAN 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(if(id IS NULL, NULL, zip)) +| output: count(if(id IS NULL, NULL, zip)) | 01:AGGREGATE | group by: id, zip @@ -58,12 +58,12 @@ from functional.testtbl partitions=1/1 size=0B ---- DISTRIBUTEDPLAN 06:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(if(id IS NULL, NULL, zip))) +| output: sum(count(if(id IS NULL, NULL, zip))) | 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:AGGREGATE [MERGE] -| output: COUNT(if(id IS NULL, NULL, zip)) +| output: count(if(id IS NULL, NULL, zip)) | 04:AGGREGATE [MERGE] | group by: id, zip @@ -82,7 +82,7 @@ from functional.alltypesagg group by 1 ---- PLAN 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(if(int_col IS NULL, NULL, bigint_col)) +| output: count(if(int_col IS NULL, NULL, bigint_col)) | group by: tinyint_col | 01:AGGREGATE @@ -94,7 +94,7 @@ group by 1 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(if(int_col IS NULL, NULL, bigint_col)) +| output: count(if(int_col IS NULL, NULL, bigint_col)) | group by: tinyint_col | 04:AGGREGATE [MERGE] @@ -114,7 +114,7 @@ from functional.alltypesagg group by 1 ---- PLAN 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col), SUM(int_col) +| output: count(int_col), sum(int_col) | group by: tinyint_col | 01:AGGREGATE @@ -126,7 +126,7 @@ group by 1 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col), SUM(int_col) +| output: count(int_col), sum(int_col) | group by: tinyint_col | 04:AGGREGATE [MERGE] @@ -147,11 +147,11 @@ min(distinct smallint_col), max(distinct string_col) from functional.alltypesagg group by 1 ---- PLAN 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col), MIN(MIN(smallint_col)), MAX(MAX(string_col)) +| output: count(int_col), min(min(smallint_col)), max(max(string_col)) | group by: tinyint_col | 01:AGGREGATE -| output: MIN(smallint_col), MAX(string_col) +| output: min(smallint_col), max(string_col) | group by: tinyint_col, int_col | 00:SCAN HDFS [functional.alltypesagg] @@ -160,17 +160,17 @@ from functional.alltypesagg group by 1 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col), MIN(MIN(smallint_col)), MAX(MAX(string_col)) +| output: count(int_col), min(min(smallint_col)), max(max(string_col)) | group by: tinyint_col | 04:AGGREGATE [MERGE] -| output: MIN(MIN(smallint_col)), MAX(MAX(string_col)) +| output: min(min(smallint_col)), max(max(string_col)) | group by: tinyint_col, int_col | 03:EXCHANGE [PARTITION=HASH(tinyint_col)] | 01:AGGREGATE -| output: MIN(smallint_col), MAX(string_col) +| output: min(smallint_col), max(string_col) | group by: tinyint_col, int_col | 00:SCAN HDFS [functional.alltypesagg] @@ -182,11 +182,11 @@ sum(int_col), min(smallint_col), max(bigint_col) from functional.alltypesagg group by 1 ---- PLAN 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col), SUM(int_col), SUM(COUNT(*)), SUM(SUM(int_col)), MIN(MIN(smallint_col)), MAX(MAX(bigint_col)) +| output: count(int_col), sum(int_col), sum(count(*)), sum(sum(int_col)), min(min(smallint_col)), max(max(bigint_col)) | group by: tinyint_col | 01:AGGREGATE -| output: COUNT(*), SUM(int_col), MIN(smallint_col), MAX(bigint_col) +| output: count(*), sum(int_col), min(smallint_col), max(bigint_col) | group by: tinyint_col, int_col | 00:SCAN HDFS [functional.alltypesagg] @@ -195,17 +195,17 @@ from functional.alltypesagg group by 1 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col), SUM(int_col), SUM(COUNT(*)), SUM(SUM(int_col)), MIN(MIN(smallint_col)), MAX(MAX(bigint_col)) +| output: count(int_col), sum(int_col), sum(count(*)), sum(sum(int_col)), min(min(smallint_col)), max(max(bigint_col)) | group by: tinyint_col | 04:AGGREGATE [MERGE] -| output: SUM(COUNT(*)), SUM(SUM(int_col)), MIN(MIN(smallint_col)), MAX(MAX(bigint_col)) +| output: sum(count(*)), sum(sum(int_col)), min(min(smallint_col)), max(max(bigint_col)) | group by: tinyint_col, int_col | 03:EXCHANGE [PARTITION=HASH(tinyint_col)] | 01:AGGREGATE -| output: COUNT(*), SUM(int_col), MIN(smallint_col), MAX(bigint_col) +| output: count(*), sum(int_col), min(smallint_col), max(bigint_col) | group by: tinyint_col, int_col | 00:SCAN HDFS [functional.alltypesagg] @@ -219,10 +219,10 @@ select t1.c, t2.c from (select count(distinct bigint_col) as c from functional.alltypestiny) t2 on (t1.c = t2.c) ---- PLAN 06:HASH JOIN [INNER JOIN] -| hash predicates: COUNT(int_col) = COUNT(bigint_col) +| hash predicates: count(int_col) = count(bigint_col) | |--05:AGGREGATE [MERGE FINALIZE] -| | output: COUNT(bigint_col) +| | output: count(bigint_col) | | | 04:AGGREGATE | | group by: bigint_col @@ -231,7 +231,7 @@ select t1.c, t2.c from | partitions=4/4 size=460B | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col) +| output: count(int_col) | 01:AGGREGATE | group by: int_col @@ -240,17 +240,17 @@ select t1.c, t2.c from partitions=4/4 size=460B ---- DISTRIBUTEDPLAN 06:HASH JOIN [INNER JOIN, BROADCAST] -| hash predicates: COUNT(int_col) = COUNT(bigint_col) +| hash predicates: count(int_col) = count(bigint_col) | |--15:EXCHANGE [PARTITION=UNPARTITIONED] | | | 14:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(bigint_col)) +| | output: sum(count(bigint_col)) | | | 13:EXCHANGE [PARTITION=UNPARTITIONED] | | | 05:AGGREGATE [MERGE] -| | output: COUNT(bigint_col) +| | output: count(bigint_col) | | | 12:AGGREGATE [MERGE] | | group by: bigint_col @@ -264,12 +264,12 @@ select t1.c, t2.c from | partitions=4/4 size=460B | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(int_col)) +| output: sum(count(int_col)) | 09:EXCHANGE [PARTITION=UNPARTITIONED] | 02:AGGREGATE [MERGE] -| output: COUNT(int_col) +| output: count(int_col) | 08:AGGREGATE [MERGE] | group by: int_col diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test b/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test index 10f381c8b..ab0181616 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test @@ -205,7 +205,7 @@ where id < 5 group by 1 ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] @@ -219,13 +219,13 @@ NODE 0: 04:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: int_col | 02:EXCHANGE [PARTITION=HASH(int_col)] | 01:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test b/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test index 75d7c292a..54f30c2b0 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test @@ -17,7 +17,7 @@ where name like 'm%' group by 1 ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: zip | 00:SCAN HDFS [functional.testtbl] @@ -27,13 +27,13 @@ group by 1 04:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: zip | 02:EXCHANGE [PARTITION=HASH(zip)] | 01:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: zip | 00:SCAN HDFS [functional.testtbl] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/insert.test b/testdata/workloads/functional-planner/queries/PlannerTest/insert.test index fb4dc4ed7..75d7e3d50 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/insert.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/insert.test @@ -130,7 +130,7 @@ WRITE TO HDFS [functional.alltypessmall, OVERWRITE=false, PARTITION-KEYS=(year,m | partitions=24 | 01:AGGREGATE [FINALIZE] -| output: MIN(id), MIN(bool_col), MIN(tinyint_col), MIN(smallint_col), MIN(int_col), MIN(bigint_col), MIN(float_col), MIN(double_col), MIN(date_string_col), MIN(string_col), MIN(timestamp_col) +| output: min(id), min(bool_col), min(tinyint_col), min(smallint_col), min(int_col), min(bigint_col), min(float_col), min(double_col), min(date_string_col), min(string_col), min(timestamp_col) | group by: year, month | 00:SCAN HDFS [functional.alltypes] @@ -144,13 +144,13 @@ WRITE TO HDFS [functional.alltypessmall, OVERWRITE=false, PARTITION-KEYS=(year,m | partitions=24 | 03:AGGREGATE [MERGE FINALIZE] -| output: MIN(MIN(id)), MIN(MIN(bool_col)), MIN(MIN(tinyint_col)), MIN(MIN(smallint_col)), MIN(MIN(int_col)), MIN(MIN(bigint_col)), MIN(MIN(float_col)), MIN(MIN(double_col)), MIN(MIN(date_string_col)), MIN(MIN(string_col)), MIN(MIN(timestamp_col)) +| output: min(min(id)), min(min(bool_col)), min(min(tinyint_col)), min(min(smallint_col)), min(min(int_col)), min(min(bigint_col)), min(min(float_col)), min(min(double_col)), min(min(date_string_col)), min(min(string_col)), min(min(timestamp_col)) | group by: year, month | 02:EXCHANGE [PARTITION=HASH(year,month)] | 01:AGGREGATE -| output: MIN(id), MIN(bool_col), MIN(tinyint_col), MIN(smallint_col), MIN(int_col), MIN(bigint_col), MIN(float_col), MIN(double_col), MIN(date_string_col), MIN(string_col), MIN(timestamp_col) +| output: min(id), min(bool_col), min(tinyint_col), min(smallint_col), min(int_col), min(bigint_col), min(float_col), min(double_col), min(date_string_col), min(string_col), min(timestamp_col) | group by: year, month | 00:SCAN HDFS [functional.alltypes] @@ -197,7 +197,7 @@ WRITE TO HDFS [functional.alltypessmall, OVERWRITE=false, PARTITION-KEYS=(2009,m | partitions=12 | 01:AGGREGATE [FINALIZE] -| output: MIN(id), MIN(bool_col), MIN(tinyint_col), MIN(smallint_col), MIN(int_col), MIN(bigint_col), MIN(float_col), MIN(double_col), MIN(date_string_col), MIN(string_col), MIN(timestamp_col) +| output: min(id), min(bool_col), min(tinyint_col), min(smallint_col), min(int_col), min(bigint_col), min(float_col), min(double_col), min(date_string_col), min(string_col), min(timestamp_col) | group by: month | 00:SCAN HDFS [functional.alltypes] @@ -211,13 +211,13 @@ WRITE TO HDFS [functional.alltypessmall, OVERWRITE=false, PARTITION-KEYS=(2009,m | partitions=12 | 03:AGGREGATE [MERGE FINALIZE] -| output: MIN(MIN(id)), MIN(MIN(bool_col)), MIN(MIN(tinyint_col)), MIN(MIN(smallint_col)), MIN(MIN(int_col)), MIN(MIN(bigint_col)), MIN(MIN(float_col)), MIN(MIN(double_col)), MIN(MIN(date_string_col)), MIN(MIN(string_col)), MIN(MIN(timestamp_col)) +| output: min(min(id)), min(min(bool_col)), min(min(tinyint_col)), min(min(smallint_col)), min(min(int_col)), min(min(bigint_col)), min(min(float_col)), min(min(double_col)), min(min(date_string_col)), min(min(string_col)), min(min(timestamp_col)) | group by: month | 02:EXCHANGE [PARTITION=HASH(month)] | 01:AGGREGATE -| output: MIN(id), MIN(bool_col), MIN(tinyint_col), MIN(smallint_col), MIN(int_col), MIN(bigint_col), MIN(float_col), MIN(double_col), MIN(date_string_col), MIN(string_col), MIN(timestamp_col) +| output: min(id), min(bool_col), min(tinyint_col), min(smallint_col), min(int_col), min(bigint_col), min(float_col), min(double_col), min(date_string_col), min(string_col), min(timestamp_col) | group by: month | 00:SCAN HDFS [functional.alltypes] @@ -357,7 +357,7 @@ WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(2010,10)] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: COUNT(int_col) +| output: count(int_col) | group by: string_col | 00:SCAN HDFS [functional.alltypes] @@ -367,13 +367,13 @@ WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(2010,10)] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(int_col)) +| output: sum(count(int_col)) | group by: string_col | 02:EXCHANGE [PARTITION=HASH(string_col)] | 01:AGGREGATE -| output: COUNT(int_col) +| output: count(int_col) | group by: string_col | 00:SCAN HDFS [functional.alltypes] @@ -390,7 +390,7 @@ WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(2010,10)] | partitions=1 | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col) +| output: count(int_col) | group by: string_col | 01:AGGREGATE @@ -403,7 +403,7 @@ WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(2010,10)] | partitions=1 | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(int_col) +| output: count(int_col) | group by: string_col | 04:AGGREGATE [MERGE] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test b/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test index 83a44c49d..b56e95ab8 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test @@ -24,10 +24,10 @@ order by limit 10 ---- PLAN 06:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 05:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_orderkey, o_orderdate, o_shippriority | 04:HASH JOIN [INNER JOIN] @@ -49,21 +49,21 @@ limit 10 predicates: l_shipdate > '1995-03-15' ---- DISTRIBUTEDPLAN 12:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 11:EXCHANGE [PARTITION=UNPARTITIONED] | 06:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: l_orderkey, o_orderdate, o_shippriority | 09:EXCHANGE [PARTITION=HASH(l_orderkey,o_orderdate,o_shippriority)] | 05:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_orderkey, o_orderdate, o_shippriority | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -114,10 +114,10 @@ order by limit 10 ---- PLAN 06:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 05:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_orderkey, o_orderdate, o_shippriority | 04:HASH JOIN [INNER JOIN] @@ -139,21 +139,21 @@ limit 10 predicates: c.c_mktsegment = 'BUILDING' ---- DISTRIBUTEDPLAN 14:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 13:EXCHANGE [PARTITION=UNPARTITIONED] | 06:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 12:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: l_orderkey, o_orderdate, o_shippriority | 11:EXCHANGE [PARTITION=HASH(l_orderkey,o_orderdate,o_shippriority)] | 05:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_orderkey, o_orderdate, o_shippriority | 04:HASH JOIN [INNER JOIN, PARTITIONED] @@ -210,10 +210,10 @@ order by limit 100 ---- PLAN 12:TOP-N [LIMIT=100] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 11:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: n_name | 10:HASH JOIN [INNER JOIN] @@ -252,21 +252,21 @@ limit 100 partitions=1/1 size=718.94MB ---- DISTRIBUTEDPLAN 21:TOP-N [LIMIT=100] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 20:EXCHANGE [PARTITION=UNPARTITIONED] | 12:TOP-N [LIMIT=100] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 19:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: n_name | 18:EXCHANGE [PARTITION=HASH(n_name)] | 11:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: n_name | 10:HASH JOIN [INNER JOIN, BROADCAST] @@ -430,7 +430,7 @@ limit 10 | order by: o_orderpriority ASC | 03:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: o_orderpriority | 02:HASH JOIN [LEFT SEMI JOIN] @@ -453,13 +453,13 @@ limit 10 | order by: o_orderpriority ASC | 08:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: o_orderpriority | 07:EXCHANGE [PARTITION=HASH(o_orderpriority)] | 03:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: o_orderpriority | 02:HASH JOIN [LEFT SEMI JOIN, PARTITIONED] @@ -489,7 +489,7 @@ order by o_orderpriority limit 10 | order by: o_orderpriority ASC | 03:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: o_orderpriority | 02:HASH JOIN [FULL OUTER JOIN] @@ -510,13 +510,13 @@ order by o_orderpriority limit 10 | order by: o_orderpriority ASC | 08:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: o_orderpriority | 07:EXCHANGE [PARTITION=HASH(o_orderpriority)] | 03:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: o_orderpriority | 02:HASH JOIN [FULL OUTER JOIN, PARTITIONED] @@ -544,7 +544,7 @@ order by o_orderpriority limit 10 | order by: o_orderpriority ASC | 03:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: o_orderpriority | 02:HASH JOIN [RIGHT OUTER JOIN] @@ -565,13 +565,13 @@ order by o_orderpriority limit 10 | order by: o_orderpriority ASC | 08:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: o_orderpriority | 07:EXCHANGE [PARTITION=HASH(o_orderpriority)] | 03:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: o_orderpriority | 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED] @@ -596,7 +596,7 @@ from tpch.customer where n_name = 'x' ---- PLAN 05:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | 04:HASH JOIN [LEFT OUTER JOIN] | hash predicates: c_custkey = o_custkey @@ -615,12 +615,12 @@ where n_name = 'x' partitions=1/1 size=23.08MB ---- DISTRIBUTEDPLAN 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | 09:EXCHANGE [PARTITION=UNPARTITIONED] | 05:AGGREGATE -| output: COUNT(*) +| output: count(*) | 04:HASH JOIN [LEFT OUTER JOIN, PARTITIONED] | hash predicates: c_custkey = o_custkey @@ -653,7 +653,7 @@ from tpch.customer where n_name = 'x' ---- PLAN 05:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | 04:CROSS JOIN [BROADCAST] | @@ -671,12 +671,12 @@ where n_name = 'x' partitions=1/1 size=23.08MB ---- DISTRIBUTEDPLAN 09:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | 08:EXCHANGE [PARTITION=UNPARTITIONED] | 05:AGGREGATE -| output: COUNT(*) +| output: count(*) | 04:CROSS JOIN [BROADCAST] | diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test index 1803ec0eb..853302d8f 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test @@ -351,10 +351,10 @@ order by 2 limit 5 ---- PLAN 10:TOP-N [LIMIT=5] -| order by: COUNT(x.day) ASC +| order by: count(x.day) ASC | 09:AGGREGATE [FINALIZE] -| output: COUNT(a.day) +| output: count(a.day) | group by: c.tinyint_col | 08:SELECT @@ -385,10 +385,10 @@ limit 5 partitions=10/10 size=743.67KB ---- DISTRIBUTEDPLAN 10:TOP-N [LIMIT=5] -| order by: COUNT(x.day) ASC +| order by: count(x.day) ASC | 09:AGGREGATE [FINALIZE] -| output: COUNT(a.day) +| output: count(a.day) | group by: c.tinyint_col | 08:SELECT @@ -638,7 +638,7 @@ on (a.int_col = b.int_col and b.bool_col = a.bool_col) | hash predicates: a.int_col = int_col, a.bool_col = bool_col | |--02:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: int_col, bool_col | | | 01:SCAN HDFS [functional.alltypes] @@ -653,13 +653,13 @@ on (a.int_col = b.int_col and b.bool_col = a.bool_col) | hash predicates: a.int_col = int_col, a.bool_col = bool_col | |--05:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: int_col, bool_col | | | 04:EXCHANGE [PARTITION=HASH(int_col,bool_col)] | | | 02:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: int_col, bool_col | | | 01:SCAN HDFS [functional.alltypes] @@ -690,7 +690,7 @@ on (b.int_col = c.int_col and c.bool_col = b.bool_col) | hash predicates: a.int_col = int_col, a.bool_col = bool_col | |--03:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: int_col, bool_col | | | 02:SCAN HDFS [functional.alltypes] @@ -713,13 +713,13 @@ on (b.int_col = c.int_col and c.bool_col = b.bool_col) | hash predicates: a.int_col = int_col, a.bool_col = bool_col | |--07:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: int_col, bool_col | | | 06:EXCHANGE [PARTITION=HASH(int_col,bool_col)] | | | 03:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: int_col, bool_col | | | 02:SCAN HDFS [functional.alltypes] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test index e61aa811d..436a13703 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test @@ -4,7 +4,7 @@ from functional.alltypes a join functional.alltypes b on (a.double_col = b.bigin where b.bigint_col div 2 = 0 ---- PLAN 03:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | 02:HASH JOIN [INNER JOIN] | hash predicates: a.double_col = b.bigint_col @@ -24,16 +24,16 @@ from where b.id < 10 ---- PLAN 03:HASH JOIN [INNER JOIN] -| hash predicates: COUNT(id) = b.id +| hash predicates: count(id) = b.id | |--02:SCAN HDFS [functional.alltypessmall b] | partitions=4/4 size=6.32KB compact | predicates: b.id < 10 | 01:AGGREGATE [FINALIZE] -| output: COUNT(id) +| output: count(id) | group by: int_col, tinyint_col -| having: COUNT(id) < 10 +| having: count(id) < 10 | 00:SCAN HDFS [functional.alltypes] partitions=24/24 size=478.45KB @@ -43,7 +43,7 @@ select count(*) from functional.alltypes where month = id and id = int_col and tinyint_col = int_col and int_col < 2 ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | 00:SCAN HDFS [functional.alltypes] partitions=2/24 size=40.32KB @@ -142,7 +142,7 @@ limit 3 | order by: int_col ASC, tinyint_col ASC | 01:AGGREGATE [FINALIZE] -| output: MAX(id) +| output: max(id) | group by: int_col, tinyint_col | 00:SCAN HDFS [functional.alltypessmall] @@ -170,7 +170,7 @@ limit 3 | order by: int_col ASC, tinyint_col ASC | 01:AGGREGATE [FINALIZE] -| output: MAX(id) +| output: max(id) | group by: int_col, tinyint_col | 00:SCAN HDFS [functional.alltypessmall] @@ -415,9 +415,9 @@ where a.year = 2009 and b.month <= 2 and b.count_col + 1 = 17 and a.tinyint_col | hash predicates: a.id = id, a.tinyint_col = int_col, a.year = year, a.month = month | |--02:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col -| | having: COUNT(*) + 1 = 17 +| | having: count(*) + 1 = 17 | | | 01:SCAN HDFS [functional.alltypessmall] | partitions=2/4 size=3.16KB @@ -442,14 +442,14 @@ NODE 1: |--06:EXCHANGE [BROADCAST] | | | 05:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: year, month, id, int_col -| | having: COUNT(*) + 1 = 17 +| | having: count(*) + 1 = 17 | | | 04:EXCHANGE [PARTITION=HASH(year,month,id,int_col)] | | | 02:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col | | | 01:SCAN HDFS [functional.alltypessmall] @@ -481,9 +481,9 @@ where a.id = b.id and | predicates: a.id = id, a.tinyint_col = int_col, a.year = year, a.month = month | |--02:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col -| | having: COUNT(*) + 1 = 17 +| | having: count(*) + 1 = 17 | | | 01:SCAN HDFS [functional.alltypessmall] | partitions=2/4 size=3.16KB @@ -501,14 +501,14 @@ where a.id = b.id and |--06:EXCHANGE [BROADCAST] | | | 05:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: year, month, id, int_col -| | having: COUNT(*) + 1 = 17 +| | having: count(*) + 1 = 17 | | | 04:EXCHANGE [PARTITION=HASH(year,month,id,int_col)] | | | 02:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col | | | 01:SCAN HDFS [functional.alltypessmall] @@ -538,10 +538,10 @@ where a.year = 2009 and b.month <= 2 and b.count_col + 1 = 17 and a.tinyint_col | hash predicates: a.id = id, a.tinyint_col = int_col, a.year = year, a.month = month | |--03:SELECT -| | predicates: COUNT(*) + 1 = 17 +| | predicates: count(*) + 1 = 17 | | | 02:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col | | limit: 5 | | @@ -570,20 +570,20 @@ NODE 1: |--08:EXCHANGE [BROADCAST] | | | 03:SELECT -| | predicates: COUNT(*) + 1 = 17 +| | predicates: count(*) + 1 = 17 | | | 07:EXCHANGE [PARTITION=UNPARTITIONED] | | limit: 5 | | | 06:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: year, month, id, int_col | | limit: 5 | | | 05:EXCHANGE [PARTITION=HASH(year,month,id,int_col)] | | | 02:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col | | | 01:SCAN HDFS [functional.alltypessmall] @@ -616,10 +616,10 @@ where a.year = 2009 and | predicates: a.id = id, a.tinyint_col = int_col, a.month = month, a.year = year | |--03:SELECT -| | predicates: COUNT(*) + 1 = 17 +| | predicates: count(*) + 1 = 17 | | | 02:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col | | limit: 5 | | @@ -639,20 +639,20 @@ where a.year = 2009 and |--08:EXCHANGE [BROADCAST] | | | 03:SELECT -| | predicates: COUNT(*) + 1 = 17 +| | predicates: count(*) + 1 = 17 | | | 07:EXCHANGE [PARTITION=UNPARTITIONED] | | limit: 5 | | | 06:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: year, month, id, int_col | | limit: 5 | | | 05:EXCHANGE [PARTITION=HASH(year,month,id,int_col)] | | | 02:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: year, month, id, int_col | | | 01:SCAN HDFS [functional.alltypessmall] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/subquery-limit.test b/testdata/workloads/functional-planner/queries/PlannerTest/subquery-limit.test index 4b4832c81..c91bc9537 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/subquery-limit.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/subquery-limit.test @@ -78,10 +78,10 @@ where a.id < 5 limit 5 | limit: 5 | |--03:TOP-N [LIMIT=5] -| | order by: COUNT(*) ASC +| | order by: count(*) ASC | | | 02:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: id | | | 01:SCAN HDFS [functional.alltypes] @@ -102,21 +102,21 @@ where a.id < 5 limit 5 | | limit: 5 | | | 08:TOP-N [LIMIT=5] -| | order by: COUNT(*) ASC +| | order by: count(*) ASC | | | 07:EXCHANGE [PARTITION=UNPARTITIONED] | | | 03:TOP-N [LIMIT=5] -| | order by: COUNT(*) ASC +| | order by: count(*) ASC | | | 06:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: id | | | 05:EXCHANGE [PARTITION=HASH(id)] | | | 02:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: id | | | 01:SCAN HDFS [functional.alltypes] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/subquery.test b/testdata/workloads/functional-planner/queries/PlannerTest/subquery.test index 4929be696..61a64703a 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/subquery.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/subquery.test @@ -16,12 +16,12 @@ where month = 1 | hash predicates: t2.int_col = int_col | |--02:TOP-N [LIMIT=5] -| | order by: COUNT(*) DESC +| | order by: count(*) DESC | | | 01:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: int_col -| | having: COUNT(*) > 1 +| | having: count(*) > 1 | | | 00:SCAN HDFS [functional.alltypessmall] | partitions=1/4 size=1.57KB @@ -38,22 +38,22 @@ where month = 1 | | limit: 5 | | | 08:TOP-N [LIMIT=5] -| | order by: COUNT(*) DESC +| | order by: count(*) DESC | | | 07:EXCHANGE [PARTITION=UNPARTITIONED] | | | 02:TOP-N [LIMIT=5] -| | order by: COUNT(*) DESC +| | order by: count(*) DESC | | | 06:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: int_col -| | having: COUNT(*) > 1 +| | having: count(*) > 1 | | | 05:EXCHANGE [PARTITION=HASH(int_col)] | | | 01:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: int_col | | | 00:SCAN HDFS [functional.alltypessmall] @@ -357,7 +357,7 @@ from (select * from functional.alltypesagg) a group by 1 ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*), MIN(functional.alltypesagg.tinyint_col), MAX(functional.alltypesagg.tinyint_col), SUM(functional.alltypesagg.tinyint_col), COUNT(functional.alltypesagg.tinyint_col) +| output: count(*), min(functional.alltypesagg.tinyint_col), max(functional.alltypesagg.tinyint_col), sum(functional.alltypesagg.tinyint_col), count(functional.alltypesagg.tinyint_col) | group by: functional.alltypesagg.tinyint_col | 00:SCAN HDFS [functional.alltypesagg] @@ -378,13 +378,13 @@ NODE 0: 04:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)), MIN(MIN(tinyint_col)), MAX(MAX(tinyint_col)), SUM(SUM(tinyint_col)), SUM(COUNT(tinyint_col)) +| output: sum(count(*)), min(min(tinyint_col)), max(max(tinyint_col)), sum(sum(tinyint_col)), sum(count(tinyint_col)) | group by: tinyint_col | 02:EXCHANGE [PARTITION=HASH(tinyint_col)] | 01:AGGREGATE -| output: COUNT(*), MIN(functional.alltypesagg.tinyint_col), MAX(functional.alltypesagg.tinyint_col), SUM(functional.alltypesagg.tinyint_col), COUNT(functional.alltypesagg.tinyint_col) +| output: count(*), min(functional.alltypesagg.tinyint_col), max(functional.alltypesagg.tinyint_col), sum(functional.alltypesagg.tinyint_col), count(functional.alltypesagg.tinyint_col) | group by: functional.alltypesagg.tinyint_col | 00:SCAN HDFS [functional.alltypesagg] @@ -399,7 +399,7 @@ group by 1 ) a ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*), MIN(tinyint_col), MAX(tinyint_col), SUM(tinyint_col), COUNT(tinyint_col) +| output: count(*), min(tinyint_col), max(tinyint_col), sum(tinyint_col), count(tinyint_col) | group by: tinyint_col | 00:SCAN HDFS [functional.alltypesagg] @@ -408,13 +408,13 @@ group by 1 04:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)), MIN(MIN(tinyint_col)), MAX(MAX(tinyint_col)), SUM(SUM(tinyint_col)), SUM(COUNT(tinyint_col)) +| output: sum(count(*)), min(min(tinyint_col)), max(max(tinyint_col)), sum(sum(tinyint_col)), sum(count(tinyint_col)) | group by: tinyint_col | 02:EXCHANGE [PARTITION=HASH(tinyint_col)] | 01:AGGREGATE -| output: COUNT(*), MIN(tinyint_col), MAX(tinyint_col), SUM(tinyint_col), COUNT(tinyint_col) +| output: count(*), min(tinyint_col), max(tinyint_col), sum(tinyint_col), count(tinyint_col) | group by: tinyint_col | 00:SCAN HDFS [functional.alltypesagg] @@ -435,7 +435,7 @@ from | order by: c2 ASC, c3 DESC | 01:AGGREGATE [FINALIZE] -| output: SUM(float_col), MIN(float_col) +| output: sum(float_col), min(float_col) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] @@ -449,13 +449,13 @@ from | order by: c2 ASC, c3 DESC | 04:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(float_col)), MIN(MIN(float_col)) +| output: sum(sum(float_col)), min(min(float_col)) | group by: int_col | 03:EXCHANGE [PARTITION=HASH(int_col)] | 01:AGGREGATE -| output: SUM(float_col), MIN(float_col) +| output: sum(float_col), min(float_col) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] @@ -482,14 +482,14 @@ limit 0 | order by: x2 ASC, c1 DESC | 03:AGGREGATE [FINALIZE] -| output: MIN(tinyint_col) +| output: min(tinyint_col) | group by: int_col | 02:TOP-N [LIMIT=1] | order by: int_col ASC, tinyint_col ASC | 01:AGGREGATE [FINALIZE] -| output: MIN(float_col) +| output: min(float_col) | group by: int_col, tinyint_col | 00:SCAN HBASE [functional_hbase.alltypessmall] @@ -498,7 +498,7 @@ limit 0 | order by: x2 ASC, c1 DESC | 03:AGGREGATE [FINALIZE] -| output: MIN(tinyint_col) +| output: min(tinyint_col) | group by: int_col | 08:TOP-N [LIMIT=1] @@ -510,13 +510,13 @@ limit 0 | order by: int_col ASC, tinyint_col ASC | 06:AGGREGATE [MERGE FINALIZE] -| output: MIN(MIN(float_col)) +| output: min(min(float_col)) | group by: int_col, tinyint_col | 05:EXCHANGE [PARTITION=HASH(int_col,tinyint_col)] | 01:AGGREGATE -| output: MIN(float_col) +| output: min(float_col) | group by: int_col, tinyint_col | 00:SCAN HBASE [functional_hbase.alltypessmall] @@ -603,9 +603,9 @@ where c1 is not null and c2 > 10 ---- PLAN 01:AGGREGATE [FINALIZE] -| output: COUNT(*), SUM(functional.alltypesagg.int_col), COUNT(functional.alltypesagg.int_col) +| output: count(*), sum(functional.alltypesagg.int_col), count(functional.alltypesagg.int_col) | group by: functional.alltypesagg.int_col % 7 -| having: int_col % 7 IS NOT NULL, COUNT(*) > 10, SUM(int_col) / COUNT(int_col) > 500.0 OR COUNT(*) = 10 +| having: int_col % 7 IS NOT NULL, count(*) > 10, sum(int_col) / count(int_col) > 500.0 OR count(*) = 10 | 00:SCAN HDFS [functional.alltypesagg] partitions=10/10 size=1.45MB @@ -613,14 +613,14 @@ and c2 > 10 04:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)), SUM(SUM(int_col)), SUM(COUNT(int_col)) +| output: sum(count(*)), sum(sum(int_col)), sum(count(int_col)) | group by: int_col % 7 -| having: int_col % 7 IS NOT NULL, COUNT(*) > 10, SUM(int_col) / COUNT(int_col) > 500.0 OR COUNT(*) = 10 +| having: int_col % 7 IS NOT NULL, count(*) > 10, sum(int_col) / count(int_col) > 500.0 OR count(*) = 10 | 02:EXCHANGE [PARTITION=HASH(int_col % 7)] | 01:AGGREGATE -| output: COUNT(*), SUM(functional.alltypesagg.int_col), COUNT(functional.alltypesagg.int_col) +| output: count(*), sum(functional.alltypesagg.int_col), count(functional.alltypesagg.int_col) | group by: functional.alltypesagg.int_col % 7 | 00:SCAN HDFS [functional.alltypesagg] @@ -686,7 +686,7 @@ from functional.alltypessmall c group by x.smallint_col ---- PLAN 05:AGGREGATE [FINALIZE] -| output: COUNT(b.id) +| output: count(b.id) | group by: a.smallint_col | 04:HASH JOIN [LEFT OUTER JOIN] @@ -707,13 +707,13 @@ group by x.smallint_col 11:EXCHANGE [PARTITION=UNPARTITIONED] | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(x.id)) +| output: sum(count(x.id)) | group by: x.smallint_col | 09:EXCHANGE [PARTITION=HASH(x.smallint_col)] | 05:AGGREGATE -| output: COUNT(b.id) +| output: count(b.id) | group by: a.smallint_col | 04:HASH JOIN [LEFT OUTER JOIN, PARTITIONED] @@ -819,14 +819,14 @@ from functional.alltypessmall c group by x.smallint_col ---- PLAN 06:AGGREGATE [FINALIZE] -| output: SUM(COUNT(a.id)) +| output: sum(count(a.id)) | group by: b.smallint_col | 05:HASH JOIN [INNER JOIN] | hash predicates: c.id = b.smallint_col | |--04:AGGREGATE [FINALIZE] -| | output: COUNT(a.id) +| | output: count(a.id) | | group by: b.smallint_col | | | 03:HASH JOIN [INNER JOIN] @@ -844,13 +844,13 @@ group by x.smallint_col 13:EXCHANGE [PARTITION=UNPARTITIONED] | 12:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(x.cnt)) +| output: sum(sum(x.cnt)) | group by: x.smallint_col | 11:EXCHANGE [PARTITION=HASH(x.smallint_col)] | 06:AGGREGATE -| output: SUM(COUNT(a.id)) +| output: sum(count(a.id)) | group by: b.smallint_col | 05:HASH JOIN [INNER JOIN, BROADCAST] @@ -859,13 +859,13 @@ group by x.smallint_col |--10:EXCHANGE [BROADCAST] | | | 09:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(a.id)) +| | output: sum(count(a.id)) | | group by: b.smallint_col | | | 08:EXCHANGE [PARTITION=HASH(b.smallint_col)] | | | 04:AGGREGATE -| | output: COUNT(a.id) +| | output: count(a.id) | | group by: b.smallint_col | | | 03:HASH JOIN [INNER JOIN, BROADCAST] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/topn.test b/testdata/workloads/functional-planner/queries/PlannerTest/topn.test index 4228daeec..9154763b8 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/topn.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/topn.test @@ -28,10 +28,10 @@ order by 2 desc limit 10 ---- PLAN 02:TOP-N [LIMIT=10] -| order by: COUNT(*) DESC +| order by: count(*) DESC | 01:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: zip | 00:SCAN HDFS [functional.testtbl] @@ -39,21 +39,21 @@ limit 10 predicates: name LIKE 'm%' ---- DISTRIBUTEDPLAN 06:TOP-N [LIMIT=10] -| order by: COUNT(*) DESC +| order by: count(*) DESC | 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:TOP-N [LIMIT=10] -| order by: COUNT(*) DESC +| order by: count(*) DESC | 04:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: zip | 03:EXCHANGE [PARTITION=HASH(zip)] | 01:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: zip | 00:SCAN HDFS [functional.testtbl] @@ -68,31 +68,31 @@ order by 2 limit 4 ---- PLAN 02:TOP-N [LIMIT=4] -| order by: SUM(float_col) ASC +| order by: sum(float_col) ASC | 01:AGGREGATE [FINALIZE] -| output: SUM(float_col) +| output: sum(float_col) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] predicates: id < 5 ---- DISTRIBUTEDPLAN 06:TOP-N [LIMIT=4] -| order by: SUM(float_col) ASC +| order by: sum(float_col) ASC | 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:TOP-N [LIMIT=4] -| order by: SUM(float_col) ASC +| order by: sum(float_col) ASC | 04:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(float_col)) +| output: sum(sum(float_col)) | group by: int_col | 03:EXCHANGE [PARTITION=HASH(int_col)] | 01:AGGREGATE -| output: SUM(float_col) +| output: sum(float_col) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] @@ -105,30 +105,30 @@ order by 2,3 desc limit 0 ---- PLAN 02:TOP-N [LIMIT=0] -| order by: SUM(float_col) ASC, MIN(float_col) DESC +| order by: sum(float_col) ASC, min(float_col) DESC | 01:AGGREGATE [FINALIZE] -| output: SUM(float_col), MIN(float_col) +| output: sum(float_col), min(float_col) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] ---- DISTRIBUTEDPLAN 06:TOP-N [LIMIT=0] -| order by: SUM(float_col) ASC, MIN(float_col) DESC +| order by: sum(float_col) ASC, min(float_col) DESC | 05:EXCHANGE [PARTITION=UNPARTITIONED] | 02:TOP-N [LIMIT=0] -| order by: SUM(float_col) ASC, MIN(float_col) DESC +| order by: sum(float_col) ASC, min(float_col) DESC | 04:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(float_col)), MIN(MIN(float_col)) +| output: sum(sum(float_col)), min(min(float_col)) | group by: int_col | 03:EXCHANGE [PARTITION=HASH(int_col)] | 01:AGGREGATE -| output: SUM(float_col), MIN(float_col) +| output: sum(float_col), min(float_col) | group by: int_col | 00:SCAN HBASE [functional_hbase.alltypessmall] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test index 171d40542..1567c1d07 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test @@ -29,10 +29,10 @@ order by limit 100 ---- PLAN 10:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC +| order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC | 09:AGGREGATE [FINALIZE] -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: i_brand, i_brand_id, i_manufact_id, i_manufact | 08:HASH JOIN [INNER JOIN] @@ -65,21 +65,21 @@ limit 100 partitions=1/1 size=5.25MB ---- DISTRIBUTEDPLAN 19:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC +| order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC | 18:EXCHANGE [PARTITION=UNPARTITIONED] | 10:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC +| order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC | 17:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_ext_sales_price)) +| output: sum(sum(ss_ext_sales_price)) | group by: i_brand, i_brand_id, i_manufact_id, i_manufact | 16:EXCHANGE [PARTITION=HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)] | 09:AGGREGATE -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: i_brand, i_brand_id, i_manufact_id, i_manufact | 08:HASH JOIN [INNER JOIN, BROADCAST] @@ -125,10 +125,10 @@ limit 100 SELECT i_item_id, s_state, - AVG(ss_quantity) agg1, - AVG(ss_list_price) agg2, - AVG(ss_coupon_amt) agg3, - AVG(ss_sales_price) agg4 + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -156,7 +156,7 @@ LIMIT 100 | order by: i_item_id ASC, s_state ASC | 09:AGGREGATE [FINALIZE] -| output: SUM(ss_quantity), COUNT(ss_quantity), SUM(ss_list_price), COUNT(ss_list_price), SUM(ss_coupon_amt), COUNT(ss_coupon_amt), SUM(ss_sales_price), COUNT(ss_sales_price) +| output: sum(ss_quantity), count(ss_quantity), sum(ss_list_price), count(ss_list_price), sum(ss_coupon_amt), count(ss_coupon_amt), sum(ss_sales_price), count(ss_sales_price) | group by: i_item_id, s_state | 08:HASH JOIN [INNER JOIN] @@ -198,13 +198,13 @@ LIMIT 100 | order by: i_item_id ASC, s_state ASC | 16:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_quantity)), SUM(COUNT(ss_quantity)), SUM(SUM(ss_list_price)), SUM(COUNT(ss_list_price)), SUM(SUM(ss_coupon_amt)), SUM(COUNT(ss_coupon_amt)), SUM(SUM(ss_sales_price)), SUM(COUNT(ss_sales_price)) +| output: sum(sum(ss_quantity)), sum(count(ss_quantity)), sum(sum(ss_list_price)), sum(count(ss_list_price)), sum(sum(ss_coupon_amt)), sum(count(ss_coupon_amt)), sum(sum(ss_sales_price)), sum(count(ss_sales_price)) | group by: i_item_id, s_state | 15:EXCHANGE [PARTITION=HASH(i_item_id,s_state)] | 09:AGGREGATE -| output: SUM(ss_quantity), COUNT(ss_quantity), SUM(ss_list_price), COUNT(ss_list_price), SUM(ss_coupon_amt), COUNT(ss_coupon_amt), SUM(ss_sales_price), COUNT(ss_sales_price) +| output: sum(ss_quantity), count(ss_quantity), sum(ss_list_price), count(ss_list_price), sum(ss_coupon_amt), count(ss_coupon_amt), sum(ss_sales_price), count(ss_sales_price) | group by: i_item_id, s_state | 08:HASH JOIN [INNER JOIN, BROADCAST] @@ -271,7 +271,7 @@ from or household_demographics.hd_buy_potential = 'unknown') and household_demographics.hd_vehicle_count > 0 and (case when household_demographics.hd_vehicle_count > 0 then household_demographics.hd_dep_count / household_demographics.hd_vehicle_count else null end) > 1.2 - and store.s_county in ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County', 'Fairfield County', 'Raleigh County', 'Ziebach County', 'Williamson County') + and store.s_county in ('Saginaw County', 'Sumner County', 'Appanoose County', 'Daviess County', 'Fairfield County', 'Raleigh County', 'Ziebach County', 'Williamson County') group by ss_ticket_number, ss_customer_sk @@ -293,9 +293,9 @@ limit 1000 | hash predicates: customer.c_customer_sk = ss_customer_sk | |--07:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: ss_ticket_number, ss_customer_sk -| | having: COUNT(*) >= 15, COUNT(*) <= 20 +| | having: count(*) >= 15, count(*) <= 20 | | | 06:HASH JOIN [INNER JOIN] | | hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk @@ -338,14 +338,14 @@ limit 1000 |--16:EXCHANGE [BROADCAST] | | | 15:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: ss_ticket_number, ss_customer_sk -| | having: COUNT(*) >= 15, COUNT(*) <= 20 +| | having: count(*) >= 15, count(*) <= 20 | | | 14:EXCHANGE [PARTITION=HASH(ss_ticket_number,ss_customer_sk)] | | | 07:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: ss_ticket_number, ss_customer_sk | | | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -408,10 +408,10 @@ order by limit 100 ---- PLAN 04:TOP-N [LIMIT=100] -| order by: year(ss_date) ASC, SUM(ss_ext_sales_price) DESC, item.i_brand_id ASC +| order by: year(ss_date) ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC | 03:AGGREGATE [FINALIZE] -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: year(ss_date), item.i_brand, item.i_brand_id | 02:HASH JOIN [INNER JOIN] @@ -425,21 +425,21 @@ limit 100 partitions=10/120 size=3.88MB ---- DISTRIBUTEDPLAN 09:TOP-N [LIMIT=100] -| order by: year(ss_date) ASC, SUM(ss_ext_sales_price) DESC, item.i_brand_id ASC +| order by: year(ss_date) ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC | 08:EXCHANGE [PARTITION=UNPARTITIONED] | 04:TOP-N [LIMIT=100] -| order by: year(ss_date) ASC, SUM(ss_ext_sales_price) DESC, item.i_brand_id ASC +| order by: year(ss_date) ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC | 07:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_ext_sales_price)) +| output: sum(sum(ss_ext_sales_price)) | group by: year(ss_date), item.i_brand, item.i_brand_id | 06:EXCHANGE [PARTITION=HASH(year(ss_date),item.i_brand,item.i_brand_id)] | 03:AGGREGATE -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: year(ss_date), item.i_brand, item.i_brand_id | 02:HASH JOIN [INNER JOIN, BROADCAST] @@ -459,7 +459,7 @@ SELECT d.d_year, i.i_category_id, i.i_category, - SUM(ss_ext_sales_price) total_sales + sum(ss_ext_sales_price) total_sales FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -481,10 +481,10 @@ ORDER BY LIMIT 100 ---- PLAN 06:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, d.d_year ASC, i.i_category_id ASC, i.i_category ASC +| order by: sum(ss_ext_sales_price) DESC, d.d_year ASC, i.i_category_id ASC, i.i_category ASC | 05:AGGREGATE [FINALIZE] -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: d.d_year, i.i_category_id, i.i_category | 04:HASH JOIN [INNER JOIN] @@ -505,21 +505,21 @@ LIMIT 100 partitions=120/120 size=23.42MB ---- DISTRIBUTEDPLAN 12:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, d.d_year ASC, i.i_category_id ASC, i.i_category ASC +| order by: sum(ss_ext_sales_price) DESC, d.d_year ASC, i.i_category_id ASC, i.i_category ASC | 11:EXCHANGE [PARTITION=UNPARTITIONED] | 06:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, d.d_year ASC, i.i_category_id ASC, i.i_category ASC +| order by: sum(ss_ext_sales_price) DESC, d.d_year ASC, i.i_category_id ASC, i.i_category ASC | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_ext_sales_price)) +| output: sum(sum(ss_ext_sales_price)) | group by: d.d_year, i.i_category_id, i.i_category | 09:EXCHANGE [PARTITION=HASH(d.d_year,i.i_category_id,i.i_category)] | 05:AGGREGATE -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: d.d_year, i.i_category_id, i.i_category | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -547,13 +547,13 @@ LIMIT 100 SELECT s_store_name, s_store_id, - SUM(CASE WHEN (d_day_name='Sunday') THEN ss_sales_price ELSE NULL END) AS sun_sales, - SUM(CASE WHEN (d_day_name='Monday') THEN ss_sales_price ELSE NULL END) AS mon_sales, - SUM(CASE WHEN (d_day_name='Tuesday') THEN ss_sales_price ELSE NULL END) AS tue_sales, - SUM(CASE WHEN (d_day_name='Wednesday') THEN ss_sales_price ELSE NULL END) AS wed_sales, - SUM(CASE WHEN (d_day_name='Thursday') THEN ss_sales_price ELSE NULL END) AS thu_sales, - SUM(CASE WHEN (d_day_name='Friday') THEN ss_sales_price ELSE NULL END) AS fri_sales, - SUM(CASE WHEN (d_day_name='Saturday') THEN ss_sales_price ELSE NULL END) AS sat_sales + sum(CASE WHEN (d_day_name='Sunday') THEN ss_sales_price ELSE NULL END) AS sun_sales, + sum(CASE WHEN (d_day_name='Monday') THEN ss_sales_price ELSE NULL END) AS mon_sales, + sum(CASE WHEN (d_day_name='Tuesday') THEN ss_sales_price ELSE NULL END) AS tue_sales, + sum(CASE WHEN (d_day_name='Wednesday') THEN ss_sales_price ELSE NULL END) AS wed_sales, + sum(CASE WHEN (d_day_name='Thursday') THEN ss_sales_price ELSE NULL END) AS thu_sales, + sum(CASE WHEN (d_day_name='Friday') THEN ss_sales_price ELSE NULL END) AS fri_sales, + sum(CASE WHEN (d_day_name='Saturday') THEN ss_sales_price ELSE NULL END) AS sat_sales FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -578,10 +578,10 @@ ORDER BY LIMIT 100 ---- PLAN 06:TOP-N [LIMIT=100] -| order by: s_store_name ASC, s_store_id ASC, SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC +| order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC | 05:AGGREGATE [FINALIZE] -| output: SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) +| output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | group by: s_store_name, s_store_id | 04:HASH JOIN [INNER JOIN] @@ -602,21 +602,21 @@ LIMIT 100 partitions=120/120 size=23.42MB ---- DISTRIBUTEDPLAN 12:TOP-N [LIMIT=100] -| order by: s_store_name ASC, s_store_id ASC, SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC +| order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC | 11:EXCHANGE [PARTITION=UNPARTITIONED] | 06:TOP-N [LIMIT=100] -| order by: s_store_name ASC, s_store_id ASC, SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC +| order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END)) +| output: sum(sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END)) | group by: s_store_name, s_store_id | 09:EXCHANGE [PARTITION=HASH(s_store_name,s_store_id)] | 05:AGGREGATE -| output: SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) +| output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | group by: s_store_name, s_store_id | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -654,8 +654,8 @@ FROM ss_ticket_number, ss_customer_sk, ca_city bought_city, - SUM(ss_coupon_amt) amt, - SUM(ss_net_profit) profit + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -708,7 +708,7 @@ LIMIT 400 | partitions=1/1 size=12.60MB compact | 09:AGGREGATE [FINALIZE] -| output: SUM(ss_coupon_amt), SUM(ss_net_profit) +| output: sum(ss_coupon_amt), sum(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city | 08:HASH JOIN [INNER JOIN] @@ -769,13 +769,13 @@ LIMIT 400 21:EXCHANGE [PARTITION=HASH(ss_customer_sk)] | 20:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_coupon_amt)), SUM(SUM(ss_net_profit)) +| output: sum(sum(ss_coupon_amt)), sum(sum(ss_net_profit)) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city | 19:EXCHANGE [PARTITION=HASH(ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city)] | 09:AGGREGATE -| output: SUM(ss_coupon_amt), SUM(ss_net_profit) +| output: sum(ss_coupon_amt), sum(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city | 08:HASH JOIN [INNER JOIN, BROADCAST] @@ -821,7 +821,7 @@ SELECT d.d_year, i.i_brand_id AS brand_id, i.i_brand AS brand, - SUM(ss_ext_sales_price) AS ext_price + sum(ss_ext_sales_price) AS ext_price FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -842,10 +842,10 @@ ORDER BY LIMIT 100 ---- PLAN 06:TOP-N [LIMIT=100] -| order by: d.d_year ASC, SUM(ss_ext_sales_price) DESC, i.i_brand_id ASC +| order by: d.d_year ASC, sum(ss_ext_sales_price) DESC, i.i_brand_id ASC | 05:AGGREGATE [FINALIZE] -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: d.d_year, i.i_brand, i.i_brand_id | 04:HASH JOIN [INNER JOIN] @@ -866,21 +866,21 @@ LIMIT 100 partitions=120/120 size=23.42MB ---- DISTRIBUTEDPLAN 12:TOP-N [LIMIT=100] -| order by: d.d_year ASC, SUM(ss_ext_sales_price) DESC, i.i_brand_id ASC +| order by: d.d_year ASC, sum(ss_ext_sales_price) DESC, i.i_brand_id ASC | 11:EXCHANGE [PARTITION=UNPARTITIONED] | 06:TOP-N [LIMIT=100] -| order by: d.d_year ASC, SUM(ss_ext_sales_price) DESC, i.i_brand_id ASC +| order by: d.d_year ASC, sum(ss_ext_sales_price) DESC, i.i_brand_id ASC | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_ext_sales_price)) +| output: sum(sum(ss_ext_sales_price)) | group by: d.d_year, i.i_brand, i.i_brand_id | 09:EXCHANGE [PARTITION=HASH(d.d_year,i.i_brand,i.i_brand_id)] | 05:AGGREGATE -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: d.d_year, i.i_brand, i.i_brand_id | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -923,12 +923,12 @@ from and i_class in('personal', 'portable', 'reference', 'self-help') and i_brand in('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9') ) - or + or (i_category in('Women', 'Music', 'Men') and i_class in('accessories', 'classical', 'fragrances', 'pants') and i_brand in('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1') ) - ) + ) group by i_manufact_id, d_qoy @@ -942,7 +942,7 @@ limit 100 | order by: sum_sales ASC, i_manufact_id ASC | 07:AGGREGATE [FINALIZE] -| output: SUM(ss_sales_price) +| output: sum(ss_sales_price) | group by: i_manufact_id, d_qoy | 06:HASH JOIN [INNER JOIN] @@ -976,13 +976,13 @@ limit 100 | order by: sum_sales ASC, i_manufact_id ASC | 13:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_sales_price)) +| output: sum(sum(ss_sales_price)) | group by: i_manufact_id, d_qoy | 12:EXCHANGE [PARTITION=HASH(i_manufact_id,d_qoy)] | 07:AGGREGATE -| output: SUM(ss_sales_price) +| output: sum(ss_sales_price) | group by: i_manufact_id, d_qoy | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -1017,7 +1017,7 @@ limit 100 SELECT i_brand_id, i_brand, - SUM(ss_ext_sales_price) AS ext_price + sum(ss_ext_sales_price) AS ext_price FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -1036,10 +1036,10 @@ ORDER BY LIMIT 100 ---- PLAN 06:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, i_brand_id ASC +| order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC | 05:AGGREGATE [FINALIZE] -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: i_brand, i_brand_id | 04:HASH JOIN [INNER JOIN] @@ -1060,21 +1060,21 @@ LIMIT 100 partitions=120/120 size=23.42MB ---- DISTRIBUTEDPLAN 12:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, i_brand_id ASC +| order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC | 11:EXCHANGE [PARTITION=UNPARTITIONED] | 06:TOP-N [LIMIT=100] -| order by: SUM(ss_ext_sales_price) DESC, i_brand_id ASC +| order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_ext_sales_price)) +| output: sum(sum(ss_ext_sales_price)) | group by: i_brand, i_brand_id | 09:EXCHANGE [PARTITION=HASH(i_brand,i_brand_id)] | 05:AGGREGATE -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: i_brand, i_brand_id | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -1145,7 +1145,7 @@ from where d_month_seq between 1185 and 1185 + 11 ) y - join + join (select s_store_name s_store_name2, wss.d_week_seq d_week_seq2, @@ -1209,7 +1209,7 @@ limit 100 | | partitions=1/1 size=3.08KB compact | | | 11:AGGREGATE [FINALIZE] -| | output: SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) +| | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | | group by: d_week_seq, ss_store_sk | | | 10:HASH JOIN [INNER JOIN] @@ -1235,7 +1235,7 @@ limit 100 | partitions=1/1 size=3.08KB compact | 03:AGGREGATE [FINALIZE] -| output: SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) +| output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | group by: d_week_seq, ss_store_sk | 02:HASH JOIN [INNER JOIN] @@ -1278,13 +1278,13 @@ limit 100 | | partitions=1/1 size=3.08KB | | | 27:AGGREGATE [MERGE FINALIZE] -| | output: SUM(SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END)) +| | output: sum(sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END)) | | group by: d_week_seq, ss_store_sk | | | 26:EXCHANGE [PARTITION=HASH(d_week_seq,ss_store_sk)] | | | 11:AGGREGATE -| | output: SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) +| | output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | | group by: d_week_seq, ss_store_sk | | | 10:HASH JOIN [INNER JOIN, PARTITIONED] @@ -1320,13 +1320,13 @@ limit 100 | partitions=1/1 size=3.08KB | 21:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END)), SUM(SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END)) +| output: sum(sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END)), sum(sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END)) | group by: d_week_seq, ss_store_sk | 20:EXCHANGE [PARTITION=HASH(d_week_seq,ss_store_sk)] | 03:AGGREGATE -| output: SUM(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), SUM(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) +| output: sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) | group by: d_week_seq, ss_store_sk | 02:HASH JOIN [INNER JOIN, PARTITIONED] @@ -1361,7 +1361,7 @@ from and i_class in('personal', 'portable', 'refernece', 'self-help') and i_brand in('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9') ) - or + or (i_category in('Women', 'Music', 'Men') and i_class in('accessories', 'classical', 'fragrances', 'pants') and i_brand in('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1') @@ -1380,7 +1380,7 @@ limit 100 | order by: i_manager_id ASC, sum_sales ASC | 07:AGGREGATE [FINALIZE] -| output: SUM(ss_sales_price) +| output: sum(ss_sales_price) | group by: i_manager_id, d_moy | 06:HASH JOIN [INNER JOIN] @@ -1414,13 +1414,13 @@ limit 100 | order by: i_manager_id ASC, sum_sales ASC | 13:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_sales_price)) +| output: sum(sum(ss_sales_price)) | group by: i_manager_id, d_moy | 12:EXCHANGE [PARTITION=HASH(i_manager_id,d_moy)] | 07:AGGREGATE -| output: SUM(ss_sales_price) +| output: sum(ss_sales_price) | group by: i_manager_id, d_moy | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -1474,7 +1474,7 @@ from ) sc -- 7,858,212,382 rows join item on (sc.ss_item_sk = item.i_item_sk) join store on (sc.ss_store_sk = store.s_store_sk) - join + join (select ss_store_sk, avg(revenue) as ave @@ -1498,7 +1498,7 @@ where sc.revenue <= 0.1 * sb.ave order by s_store_name, - i_item_desc + i_item_desc limit 100 ---- PLAN 10:TOP-N [LIMIT=100] @@ -1506,14 +1506,14 @@ limit 100 | 09:HASH JOIN [INNER JOIN] | hash predicates: ss_store_sk = ss_store_sk -| other predicates: SUM(ss_sales_price) <= 0.1 * SUM(revenue) / COUNT(revenue) +| other predicates: sum(ss_sales_price) <= 0.1 * sum(revenue) / count(revenue) | |--06:AGGREGATE [FINALIZE] -| | output: SUM(SUM(ss_sales_price)), COUNT(SUM(ss_sales_price)) +| | output: sum(sum(ss_sales_price)), count(sum(ss_sales_price)) | | group by: ss_store_sk | | | 05:AGGREGATE [FINALIZE] -| | output: SUM(ss_sales_price) +| | output: sum(ss_sales_price) | | group by: ss_store_sk, ss_item_sk | | | 04:SCAN HDFS [tpcds.store_sales] @@ -1529,7 +1529,7 @@ limit 100 | hash predicates: item.i_item_sk = ss_item_sk | |--01:AGGREGATE [FINALIZE] -| | output: SUM(ss_sales_price) +| | output: sum(ss_sales_price) | | group by: ss_store_sk, ss_item_sk | | | 00:SCAN HDFS [tpcds.store_sales] @@ -1548,28 +1548,28 @@ limit 100 | 09:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss_store_sk = ss_store_sk -| other predicates: SUM(ss_sales_price) <= 0.1 * SUM(revenue) / COUNT(revenue) +| other predicates: sum(ss_sales_price) <= 0.1 * sum(revenue) / count(revenue) | |--19:EXCHANGE [BROADCAST] | | | 18:AGGREGATE [MERGE FINALIZE] -| | output: SUM(SUM(revenue)), SUM(COUNT(revenue)) +| | output: sum(sum(revenue)), sum(count(revenue)) | | group by: ss_store_sk | | | 17:EXCHANGE [PARTITION=HASH(ss_store_sk)] | | | 06:AGGREGATE -| | output: SUM(SUM(ss_sales_price)), COUNT(SUM(ss_sales_price)) +| | output: sum(sum(ss_sales_price)), count(sum(ss_sales_price)) | | group by: ss_store_sk | | | 16:AGGREGATE [MERGE FINALIZE] -| | output: SUM(SUM(ss_sales_price)) +| | output: sum(sum(ss_sales_price)) | | group by: ss_store_sk, ss_item_sk | | | 15:EXCHANGE [PARTITION=HASH(ss_store_sk,ss_item_sk)] | | | 05:AGGREGATE -| | output: SUM(ss_sales_price) +| | output: sum(ss_sales_price) | | group by: ss_store_sk, ss_item_sk | | | 04:SCAN HDFS [tpcds.store_sales] @@ -1589,13 +1589,13 @@ limit 100 |--13:EXCHANGE [BROADCAST] | | | 12:AGGREGATE [MERGE FINALIZE] -| | output: SUM(SUM(ss_sales_price)) +| | output: sum(sum(ss_sales_price)) | | group by: ss_store_sk, ss_item_sk | | | 11:EXCHANGE [PARTITION=HASH(ss_store_sk,ss_item_sk)] | | | 01:AGGREGATE -| | output: SUM(ss_sales_price) +| | output: sum(ss_sales_price) | | group by: ss_store_sk, ss_item_sk | | | 00:SCAN HDFS [tpcds.store_sales] @@ -1645,7 +1645,7 @@ where current_addr.ca_city <> bought_city order by c_last_name, - ss_ticket_number + ss_ticket_number limit 100 ---- PLAN 12:TOP-N [LIMIT=100] @@ -1656,7 +1656,7 @@ limit 100 | other predicates: current_addr.ca_city != ca_city | |--07:AGGREGATE [FINALIZE] -| | output: SUM(ss_ext_sales_price), SUM(ss_ext_list_price), SUM(ss_ext_tax) +| | output: sum(ss_ext_sales_price), sum(ss_ext_list_price), sum(ss_ext_tax) | | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city | | | 06:HASH JOIN [INNER JOIN] @@ -1706,13 +1706,13 @@ limit 100 |--19:EXCHANGE [BROADCAST] | | | 18:AGGREGATE [MERGE FINALIZE] -| | output: SUM(SUM(ss_ext_sales_price)), SUM(SUM(ss_ext_list_price)), SUM(SUM(ss_ext_tax)) +| | output: sum(sum(ss_ext_sales_price)), sum(sum(ss_ext_list_price)), sum(sum(ss_ext_tax)) | | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city | | | 17:EXCHANGE [PARTITION=HASH(ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city)] | | | 07:AGGREGATE -| | output: SUM(ss_ext_sales_price), SUM(ss_ext_list_price), SUM(ss_ext_tax) +| | output: sum(ss_ext_sales_price), sum(ss_ext_list_price), sum(ss_ext_tax) | | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city | | | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -1767,7 +1767,7 @@ FROM (SELECT ss_ticket_number, ss_customer_sk, - COUNT(*) AS cnt + count(*) AS cnt FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -1802,9 +1802,9 @@ LIMIT 100 | hash predicates: c.c_customer_sk = ss_customer_sk | |--07:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: ss_ticket_number, ss_customer_sk -| | having: COUNT(*) <= 5, COUNT(*) >= 1 +| | having: count(*) <= 5, count(*) >= 1 | | | 06:HASH JOIN [INNER JOIN] | | hash predicates: ss.ss_store_sk = s.s_store_sk @@ -1847,14 +1847,14 @@ LIMIT 100 |--16:EXCHANGE [BROADCAST] | | | 15:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: ss_ticket_number, ss_customer_sk -| | having: COUNT(*) <= 5, COUNT(*) >= 1 +| | having: count(*) <= 5, count(*) >= 1 | | | 14:EXCHANGE [PARTITION=HASH(ss_ticket_number,ss_customer_sk)] | | | 07:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: ss_ticket_number, ss_customer_sk | | | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -1903,8 +1903,8 @@ FROM ss_ticket_number, ss_customer_sk, s_city, - SUM(ss_coupon_amt) AS amt, - SUM(ss_net_profit) AS profit + sum(ss_coupon_amt) AS amt, + sum(ss_net_profit) AS profit FROM store_sales ss JOIN date_dim d ON (ss.ss_sold_date_sk = d.d_date_sk) @@ -1943,7 +1943,7 @@ LIMIT 100 | partitions=1/1 size=12.60MB compact | 07:AGGREGATE [FINALIZE] -| output: SUM(ss_coupon_amt), SUM(ss_net_profit) +| output: sum(ss_coupon_amt), sum(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, s_city | 06:HASH JOIN [INNER JOIN] @@ -1989,13 +1989,13 @@ LIMIT 100 16:EXCHANGE [PARTITION=HASH(ss_customer_sk)] | 15:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_coupon_amt)), SUM(SUM(ss_net_profit)) +| output: sum(sum(ss_coupon_amt)), sum(sum(ss_net_profit)) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, s_city | 14:EXCHANGE [PARTITION=HASH(ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city)] | 07:AGGREGATE -| output: SUM(ss_coupon_amt), SUM(ss_net_profit) +| output: sum(ss_coupon_amt), sum(ss_net_profit) | group by: ss_ticket_number, ss_customer_sk, ss_addr_sk, s_city | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -2050,14 +2050,14 @@ where group by i_item_id order by - i_item_id + i_item_id limit 100 ---- PLAN 08:TOP-N [LIMIT=100] | order by: i_item_id ASC | 07:AGGREGATE [FINALIZE] -| output: SUM(ss_quantity), COUNT(ss_quantity), SUM(ss_list_price), COUNT(ss_list_price), SUM(ss_coupon_amt), COUNT(ss_coupon_amt), SUM(ss_sales_price), COUNT(ss_sales_price) +| output: sum(ss_quantity), count(ss_quantity), sum(ss_list_price), count(ss_list_price), sum(ss_coupon_amt), count(ss_coupon_amt), sum(ss_sales_price), count(ss_sales_price) | group by: i_item_id | 06:HASH JOIN [INNER JOIN] @@ -2092,13 +2092,13 @@ limit 100 | order by: i_item_id ASC | 14:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_quantity)), SUM(COUNT(ss_quantity)), SUM(SUM(ss_list_price)), SUM(COUNT(ss_list_price)), SUM(SUM(ss_coupon_amt)), SUM(COUNT(ss_coupon_amt)), SUM(SUM(ss_sales_price)), SUM(COUNT(ss_sales_price)) +| output: sum(sum(ss_quantity)), sum(count(ss_quantity)), sum(sum(ss_list_price)), sum(count(ss_list_price)), sum(sum(ss_coupon_amt)), sum(count(ss_coupon_amt)), sum(sum(ss_sales_price)), sum(count(ss_sales_price)) | group by: i_item_id | 13:EXCHANGE [PARTITION=HASH(i_item_id)] | 07:AGGREGATE -| output: SUM(ss_quantity), COUNT(ss_quantity), SUM(ss_list_price), COUNT(ss_list_price), SUM(ss_coupon_amt), COUNT(ss_coupon_amt), SUM(ss_sales_price), COUNT(ss_sales_price) +| output: sum(ss_quantity), count(ss_quantity), sum(ss_list_price), count(ss_list_price), sum(ss_coupon_amt), count(ss_coupon_amt), sum(ss_sales_price), count(ss_sales_price) | group by: i_item_id | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -2175,7 +2175,7 @@ limit 100 | order by: sum_sales ASC, s_store_name ASC, i_category ASC | 07:AGGREGATE [FINALIZE] -| output: SUM(ss_sales_price) +| output: sum(ss_sales_price) | group by: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy | 06:HASH JOIN [INNER JOIN] @@ -2210,13 +2210,13 @@ limit 100 | order by: sum_sales ASC, s_store_name ASC, i_category ASC | 13:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_sales_price)) +| output: sum(sum(ss_sales_price)) | group by: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy | 12:EXCHANGE [PARTITION=HASH(i_category,i_class,i_brand,s_store_name,s_company_name,d_moy)] | 07:AGGREGATE -| output: SUM(ss_sales_price) +| output: sum(ss_sales_price) | group by: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -2255,7 +2255,7 @@ select from store_sales join store on (store_sales.ss_store_sk = store.s_store_sk) - join + join (select a.ca_zip from @@ -2271,8 +2271,8 @@ from ca_zip having count( *) > 10 - ) a - left semi join + ) a + left semi join (select substr(ca_zip, 1, 5) ca_zip from @@ -2315,14 +2315,14 @@ where group by s_store_name order by - s_store_name + s_store_name limit 100 ---- PLAN 11:TOP-N [LIMIT=100] | order by: s_store_name ASC | 10:AGGREGATE [FINALIZE] -| output: SUM(ss_net_profit) +| output: sum(ss_net_profit) | group by: s_store_name | 09:HASH JOIN [INNER JOIN] @@ -2336,9 +2336,9 @@ limit 100 | | predicates: substr(ca_zip, 1, 5) IN ('89436', '30868', '65085', '22977', '83927', '77557', '58429', '40697', '80614', '10502', '32779', '91137', '61265', '98294', '17921', '18427', '21203', '59362', '87291', '84093', '21505', '17184', '10866', '67898', '25797', '28055', '18377', '80332', '74535', '21757', '29742', '90885', '29898', '17819', '40811', '25990', '47513', '89531', '91068', '10391', '18846', '99223', '82637', '41368', '83658', '86199', '81625', '26696', '89338', '88425', '32200', '81427', '19053', '77471', '36610', '99823', '43276', '41249', '48584', '83550', '82276', '18842', '78890', '14090', '38123', '40936', '34425', '19850', '43286', '80072', '79188', '54191', '11395', '50497', '84861', '90733', '21068', '57666', '37119', '25004', '57835', '70067', '62878', '95806', '19303', '18840', '19124', '29785', '16737', '16022', '49613', '89977', '68310', '60069', '98360', '48649', '39050', '41793', '25002', '27413', '39736', '47208', '16515', '94808', '57648', '15009', '80015', '42961', '63982', '21744', '71853', '81087', '67468', '34175', '64008', '20261', '11201', '51799', '48043', '45645', '61163', '48375', '36447', '57042', '21218', '41100', '89951', '22745', '35851', '83326', '61125', '78298', '80752', '49858', '52940', '96976', '63792', '11376', '53582', '18717', '90226', '50530', '94203', '99447', '27670', '96577', '57856', '56372', '16165', '23427', '54561', '28806', '44439', '22926', '30123', '61451', '92397', '56979', '92309', '70873', '13355', '21801', '46346', '37562', '56458', '28286', '47306', '99555', '69399', '26234', '47546', '49661', '88601', '35943', '39936', '25632', '24611', '44166', '56648', '30379', '59785', '11110', '14329', '93815', '52226', '71381', '13842', '25612', '63294', '14664', '21077', '82626', '18799', '60915', '81020', '56447', '76619', '11433', '13414', '42548', '92713', '70467', '30884', '47484', '16072', '38936', '13036', '88376', '45539', '35901', '19506', '65690', '73957', '71850', '49231', '14276', '20005', '18384', '76615', '11635', '38177', '55607', '41369', '95447', '58581', '58149', '91946', '33790', '76232', '75692', '95464', '22246', '51061', '56692', '53121', '77209', '15482', '10688', '14868', '45907', '73520', '72666', '25734', '17959', '24677', '66446', '94627', '53535', '15560', '41967', '69297', '11929', '59403', '33283', '52232', '57350', '43933', '40921', '36635', '10827', '71286', '19736', '80619', '25251', '95042', '15526', '36496', '55854', '49124', '81980', '35375', '49157', '63512', '28944', '14946', '36503', '54010', '18767', '23969', '43905', '66979', '33113', '21286', '58471', '59080', '13395', '79144', '70373', '67031', '38360', '26705', '50906', '52406', '26066', '73146', '15884', '31897', '30045', '61068', '45550', '92454', '13376', '14354', '19770', '22928', '97790', '50723', '46081', '30202', '14410', '20223', '88500', '67298', '13261', '14172', '81410', '93578', '83583', '46047', '94167', '82564', '21156', '15799', '86709', '37931', '74703', '83103', '23054', '70470', '72008', '49247', '91911', '69998', '20961', '70070', '63197', '54853', '88191', '91830', '49521', '19454', '81450', '89091', '62378', '25683', '61869', '51744', '36580', '85778', '36871', '48121', '28810', '83712', '45486', '67393', '26935', '42393', '20132', '55349', '86057', '21309', '80218', '10094', '11357', '48819', '39734', '40758', '30432', '21204', '29467', '30214', '61024', '55307', '74621', '11622', '68908', '33032', '52868', '99194', '99900', '84936', '69036', '99149', '45013', '32895', '59004', '32322', '14933', '32936', '33562', '72550', '27385', '58049', '58200', '16808', '21360', '32961', '18586', '79307', '15492') | | | 05:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: substr(ca_zip, 1, 5) -| | having: COUNT(*) > 10 +| | having: count(*) > 10 | | | 04:HASH JOIN [INNER JOIN] | | hash predicates: customer_address.ca_address_sk = customer.c_current_addr_sk @@ -2368,13 +2368,13 @@ limit 100 | order by: s_store_name ASC | 19:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_net_profit)) +| output: sum(sum(ss_net_profit)) | group by: s_store_name | 18:EXCHANGE [PARTITION=HASH(s_store_name)] | 10:AGGREGATE -| output: SUM(ss_net_profit) +| output: sum(ss_net_profit) | group by: s_store_name | 09:HASH JOIN [INNER JOIN, BROADCAST] @@ -2392,14 +2392,14 @@ limit 100 | | predicates: substr(ca_zip, 1, 5) IN ('89436', '30868', '65085', '22977', '83927', '77557', '58429', '40697', '80614', '10502', '32779', '91137', '61265', '98294', '17921', '18427', '21203', '59362', '87291', '84093', '21505', '17184', '10866', '67898', '25797', '28055', '18377', '80332', '74535', '21757', '29742', '90885', '29898', '17819', '40811', '25990', '47513', '89531', '91068', '10391', '18846', '99223', '82637', '41368', '83658', '86199', '81625', '26696', '89338', '88425', '32200', '81427', '19053', '77471', '36610', '99823', '43276', '41249', '48584', '83550', '82276', '18842', '78890', '14090', '38123', '40936', '34425', '19850', '43286', '80072', '79188', '54191', '11395', '50497', '84861', '90733', '21068', '57666', '37119', '25004', '57835', '70067', '62878', '95806', '19303', '18840', '19124', '29785', '16737', '16022', '49613', '89977', '68310', '60069', '98360', '48649', '39050', '41793', '25002', '27413', '39736', '47208', '16515', '94808', '57648', '15009', '80015', '42961', '63982', '21744', '71853', '81087', '67468', '34175', '64008', '20261', '11201', '51799', '48043', '45645', '61163', '48375', '36447', '57042', '21218', '41100', '89951', '22745', '35851', '83326', '61125', '78298', '80752', '49858', '52940', '96976', '63792', '11376', '53582', '18717', '90226', '50530', '94203', '99447', '27670', '96577', '57856', '56372', '16165', '23427', '54561', '28806', '44439', '22926', '30123', '61451', '92397', '56979', '92309', '70873', '13355', '21801', '46346', '37562', '56458', '28286', '47306', '99555', '69399', '26234', '47546', '49661', '88601', '35943', '39936', '25632', '24611', '44166', '56648', '30379', '59785', '11110', '14329', '93815', '52226', '71381', '13842', '25612', '63294', '14664', '21077', '82626', '18799', '60915', '81020', '56447', '76619', '11433', '13414', '42548', '92713', '70467', '30884', '47484', '16072', '38936', '13036', '88376', '45539', '35901', '19506', '65690', '73957', '71850', '49231', '14276', '20005', '18384', '76615', '11635', '38177', '55607', '41369', '95447', '58581', '58149', '91946', '33790', '76232', '75692', '95464', '22246', '51061', '56692', '53121', '77209', '15482', '10688', '14868', '45907', '73520', '72666', '25734', '17959', '24677', '66446', '94627', '53535', '15560', '41967', '69297', '11929', '59403', '33283', '52232', '57350', '43933', '40921', '36635', '10827', '71286', '19736', '80619', '25251', '95042', '15526', '36496', '55854', '49124', '81980', '35375', '49157', '63512', '28944', '14946', '36503', '54010', '18767', '23969', '43905', '66979', '33113', '21286', '58471', '59080', '13395', '79144', '70373', '67031', '38360', '26705', '50906', '52406', '26066', '73146', '15884', '31897', '30045', '61068', '45550', '92454', '13376', '14354', '19770', '22928', '97790', '50723', '46081', '30202', '14410', '20223', '88500', '67298', '13261', '14172', '81410', '93578', '83583', '46047', '94167', '82564', '21156', '15799', '86709', '37931', '74703', '83103', '23054', '70470', '72008', '49247', '91911', '69998', '20961', '70070', '63197', '54853', '88191', '91830', '49521', '19454', '81450', '89091', '62378', '25683', '61869', '51744', '36580', '85778', '36871', '48121', '28810', '83712', '45486', '67393', '26935', '42393', '20132', '55349', '86057', '21309', '80218', '10094', '11357', '48819', '39734', '40758', '30432', '21204', '29467', '30214', '61024', '55307', '74621', '11622', '68908', '33032', '52868', '99194', '99900', '84936', '69036', '99149', '45013', '32895', '59004', '32322', '14933', '32936', '33562', '72550', '27385', '58049', '58200', '16808', '21360', '32961', '18586', '79307', '15492') | | | 15:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: substr(ca_zip, 1, 5) -| | having: COUNT(*) > 10 +| | having: count(*) > 10 | | | 14:EXCHANGE [PARTITION=HASH(substr(ca_zip, 1, 5))] | | | 05:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: substr(ca_zip, 1, 5) | | | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -2427,7 +2427,7 @@ limit 100 ==== # TPCDS-Q96 SELECT - COUNT(*) AS total + count(*) AS total FROM store_sales ss JOIN time_dim td ON (ss.ss_sold_time_sk = td.t_time_sk) @@ -2442,7 +2442,7 @@ WHERE AND s.s_store_name = 'ese' ---- PLAN 07:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | 06:HASH JOIN [INNER JOIN] | hash predicates: ss.ss_store_sk = s.s_store_sk @@ -2469,12 +2469,12 @@ WHERE partitions=120/120 size=23.42MB ---- DISTRIBUTEDPLAN 12:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | 11:EXCHANGE [PARTITION=UNPARTITIONED] | 07:AGGREGATE -| output: COUNT(*) +| output: count(*) | 06:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: ss.ss_store_sk = s.s_store_sk @@ -2534,7 +2534,7 @@ limit 1000 | order by: i_category ASC, i_class ASC, i_item_desc ASC | 03:AGGREGATE [FINALIZE] -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: i_item_desc, i_category, i_class, i_current_price | 02:HASH JOIN [INNER JOIN] @@ -2556,13 +2556,13 @@ limit 1000 | order by: i_category ASC, i_class ASC, i_item_desc ASC | 07:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ss_ext_sales_price)) +| output: sum(sum(ss_ext_sales_price)) | group by: i_item_desc, i_category, i_class, i_current_price | 06:EXCHANGE [PARTITION=HASH(i_item_desc,i_category,i_class,i_current_price)] | 03:AGGREGATE -| output: SUM(ss_ext_sales_price) +| output: sum(ss_ext_sales_price) | group by: i_item_desc, i_category, i_class, i_current_price | 02:HASH JOIN [INNER JOIN, BROADCAST] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test index a46802081..7ef3195af 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test @@ -20,7 +20,7 @@ group by l_linestatus ---- PLAN 01:AGGREGATE [FINALIZE] -| output: SUM(l_quantity), SUM(l_extendedprice), SUM(l_extendedprice * (1.0 - l_discount)), SUM(l_extendedprice * (1.0 - l_discount) * (1.0 + l_tax)), COUNT(l_quantity), COUNT(l_extendedprice), SUM(l_discount), COUNT(l_discount), COUNT(1) +| output: sum(l_quantity), sum(l_extendedprice), sum(l_extendedprice * (1.0 - l_discount)), sum(l_extendedprice * (1.0 - l_discount) * (1.0 + l_tax)), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count(1) | group by: l_returnflag, l_linestatus | 00:SCAN HDFS [tpch.lineitem] @@ -30,13 +30,13 @@ group by 04:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_quantity)), SUM(SUM(l_extendedprice)), SUM(SUM(l_extendedprice * (1.0 - l_discount))), SUM(SUM(l_extendedprice * (1.0 - l_discount) * (1.0 + l_tax))), SUM(COUNT(l_quantity)), SUM(COUNT(l_extendedprice)), SUM(SUM(l_discount)), SUM(COUNT(l_discount)), SUM(COUNT(1)) +| output: sum(sum(l_quantity)), sum(sum(l_extendedprice)), sum(sum(l_extendedprice * (1.0 - l_discount))), sum(sum(l_extendedprice * (1.0 - l_discount) * (1.0 + l_tax))), sum(count(l_quantity)), sum(count(l_extendedprice)), sum(sum(l_discount)), sum(count(l_discount)), sum(count(1)) | group by: l_returnflag, l_linestatus | 02:EXCHANGE [PARTITION=HASH(l_returnflag,l_linestatus)] | 01:AGGREGATE -| output: SUM(l_quantity), SUM(l_extendedprice), SUM(l_extendedprice * (1.0 - l_discount)), SUM(l_extendedprice * (1.0 - l_discount) * (1.0 + l_tax)), COUNT(l_quantity), COUNT(l_extendedprice), SUM(l_discount), COUNT(l_discount), COUNT(1) +| output: sum(l_quantity), sum(l_extendedprice), sum(l_extendedprice * (1.0 - l_discount)), sum(l_extendedprice * (1.0 - l_discount) * (1.0 + l_tax)), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count(1) | group by: l_returnflag, l_linestatus | 00:SCAN HDFS [tpch.lineitem] @@ -148,7 +148,7 @@ WRITE TO HDFS [tpch.q2_minimum_cost_supplier_tmp2, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: MIN(ps_supplycost) +| output: min(ps_supplycost) | group by: p_partkey | 00:SCAN HDFS [tpch.q2_minimum_cost_supplier_tmp1] @@ -158,13 +158,13 @@ WRITE TO HDFS [tpch.q2_minimum_cost_supplier_tmp2, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: MIN(MIN(ps_supplycost)) +| output: min(min(ps_supplycost)) | group by: p_partkey | 02:EXCHANGE [PARTITION=HASH(p_partkey)] | 01:AGGREGATE -| output: MIN(ps_supplycost) +| output: min(ps_supplycost) | group by: p_partkey | 00:SCAN HDFS [tpch.q2_minimum_cost_supplier_tmp1] @@ -246,10 +246,10 @@ order by limit 10 ---- PLAN 06:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 05:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_orderkey, o_orderdate, o_shippriority | 04:HASH JOIN [INNER JOIN] @@ -271,21 +271,21 @@ limit 10 predicates: l_shipdate > '1995-03-15' ---- DISTRIBUTEDPLAN 12:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 11:EXCHANGE [PARTITION=UNPARTITIONED] | 06:TOP-N [LIMIT=10] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC, o_orderdate ASC | 10:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: l_orderkey, o_orderdate, o_shippriority | 09:EXCHANGE [PARTITION=HASH(l_orderkey,o_orderdate,o_shippriority)] | 05:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_orderkey, o_orderdate, o_shippriority | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -334,7 +334,7 @@ limit 10 | order by: o_orderpriority ASC | 04:AGGREGATE [MERGE FINALIZE] -| output: COUNT(l_orderkey) +| output: count(l_orderkey) | group by: o_orderpriority | 03:AGGREGATE @@ -360,7 +360,7 @@ limit 10 | order by: o_orderpriority ASC | 04:AGGREGATE [MERGE FINALIZE] -| output: COUNT(l_orderkey) +| output: count(l_orderkey) | group by: o_orderpriority | 08:AGGREGATE [MERGE] @@ -412,10 +412,10 @@ order by limit 100 ---- PLAN 12:TOP-N [LIMIT=100] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 11:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: n_name | 10:HASH JOIN [INNER JOIN] @@ -454,21 +454,21 @@ limit 100 partitions=1/1 size=718.94MB ---- DISTRIBUTEDPLAN 21:TOP-N [LIMIT=100] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 20:EXCHANGE [PARTITION=UNPARTITIONED] | 12:TOP-N [LIMIT=100] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 19:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: n_name | 18:EXCHANGE [PARTITION=HASH(n_name)] | 11:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: n_name | 10:HASH JOIN [INNER JOIN, BROADCAST] @@ -527,19 +527,19 @@ where l_shipdate >= '1994-01-01' and l_quantity < 24 ---- PLAN 01:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * l_discount) +| output: sum(l_extendedprice * l_discount) | 00:SCAN HDFS [tpch.lineitem] partitions=1/1 size=718.94MB predicates: l_discount >= 0.05, l_discount <= 0.07, l_shipdate >= '1994-01-01', l_shipdate < '1995-01-01', l_quantity < 24.0 ---- DISTRIBUTEDPLAN 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * l_discount)) +| output: sum(sum(l_extendedprice * l_discount)) | 02:EXCHANGE [PARTITION=UNPARTITIONED] | 01:AGGREGATE -| output: SUM(l_extendedprice * l_discount) +| output: sum(l_extendedprice * l_discount) | 00:SCAN HDFS [tpch.lineitem] partitions=1/1 size=718.94MB @@ -582,7 +582,7 @@ limit 100 | order by: year(o_orderdate) ASC | 15:AGGREGATE [FINALIZE] -| output: SUM(CASE WHEN n2.n_name = 'BRAZIL' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(CASE WHEN n2.n_name = 'BRAZIL' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), sum(l_extendedprice * (1.0 - l_discount)) | group by: year(o_orderdate) | 14:HASH JOIN [INNER JOIN] @@ -642,13 +642,13 @@ limit 100 | order by: year(o_orderdate) ASC | 25:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(CASE WHEN n2.n_name = 'BRAZIL' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END)), SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(CASE WHEN n2.n_name = 'BRAZIL' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END)), sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: year(o_orderdate) | 24:EXCHANGE [PARTITION=HASH(year(o_orderdate))] | 15:AGGREGATE -| output: SUM(CASE WHEN n2.n_name = 'BRAZIL' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(CASE WHEN n2.n_name = 'BRAZIL' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), sum(l_extendedprice * (1.0 - l_discount)) | group by: year(o_orderdate) | 14:HASH JOIN [INNER JOIN, BROADCAST] @@ -746,7 +746,7 @@ limit 200 | order by: n.n_name ASC, year(o.o_orderdate) DESC | 11:AGGREGATE [FINALIZE] -| output: SUM(l.l_extendedprice * (1.0 - l.l_discount) - ps.ps_supplycost * l.l_quantity) +| output: sum(l.l_extendedprice * (1.0 - l.l_discount) - ps.ps_supplycost * l.l_quantity) | group by: n.n_name, year(o.o_orderdate) | 10:HASH JOIN [INNER JOIN] @@ -792,13 +792,13 @@ limit 200 | order by: n.n_name ASC, year(o.o_orderdate) DESC | 19:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l.l_extendedprice * (1.0 - l.l_discount) - ps.ps_supplycost * l.l_quantity)) +| output: sum(sum(l.l_extendedprice * (1.0 - l.l_discount) - ps.ps_supplycost * l.l_quantity)) | group by: n.n_name, year(o.o_orderdate) | 18:EXCHANGE [PARTITION=HASH(n.n_name,year(o.o_orderdate))] | 11:AGGREGATE -| output: SUM(l.l_extendedprice * (1.0 - l.l_discount) - ps.ps_supplycost * l.l_quantity) +| output: sum(l.l_extendedprice * (1.0 - l.l_discount) - ps.ps_supplycost * l.l_quantity) | group by: n.n_name, year(o.o_orderdate) | 10:HASH JOIN [INNER JOIN, BROADCAST] @@ -882,10 +882,10 @@ order by limit 20 ---- PLAN 08:TOP-N [LIMIT=20] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 07:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: c_custkey, c_name, CAST(c_acctbal AS BIGINT), c_phone, n_name, c_address, c_comment | 06:HASH JOIN [INNER JOIN] @@ -912,21 +912,21 @@ limit 20 predicates: l.l_returnflag = 'R' ---- DISTRIBUTEDPLAN 15:TOP-N [LIMIT=20] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 14:EXCHANGE [PARTITION=UNPARTITIONED] | 08:TOP-N [LIMIT=20] -| order by: round(SUM(l_extendedprice * (1.0 - l_discount)), 5) DESC +| order by: round(sum(l_extendedprice * (1.0 - l_discount)), 5) DESC | 13:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: c_custkey, c_name, CAST(c_acctbal AS BIGINT), c_phone, n_name, c_address, c_comment | 12:EXCHANGE [PARTITION=HASH(c_custkey,c_name,CAST(c_acctbal AS BIGINT),c_phone,n_name,c_address,c_comment)] | 07:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: c_custkey, c_name, CAST(c_acctbal AS BIGINT), c_phone, n_name, c_address, c_comment | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -973,7 +973,7 @@ WRITE TO HDFS [tpch.q11_part_tmp, OVERWRITE=true] | partitions=1 | 05:AGGREGATE [FINALIZE] -| output: SUM(ps_supplycost * ps_availqty) +| output: sum(ps_supplycost * ps_availqty) | group by: ps_partkey | 04:HASH JOIN [INNER JOIN] @@ -996,13 +996,13 @@ WRITE TO HDFS [tpch.q11_part_tmp, OVERWRITE=true] | partitions=1 | 09:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(ps_supplycost * ps_availqty)) +| output: sum(sum(ps_supplycost * ps_availqty)) | group by: ps_partkey | 08:EXCHANGE [PARTITION=HASH(ps_partkey)] | 05:AGGREGATE -| output: SUM(ps_supplycost * ps_availqty) +| output: sum(ps_supplycost * ps_availqty) | group by: ps_partkey | 04:HASH JOIN [INNER JOIN, BROADCAST] @@ -1033,7 +1033,7 @@ WRITE TO HDFS [tpch.q11_sum_tmp, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: SUM(part_value) +| output: sum(part_value) | 00:SCAN HDFS [tpch.q11_part_tmp] partitions=1/1 size=509.47KB @@ -1042,12 +1042,12 @@ WRITE TO HDFS [tpch.q11_sum_tmp, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(part_value)) +| output: sum(sum(part_value)) | 02:EXCHANGE [PARTITION=UNPARTITIONED] | 01:AGGREGATE -| output: SUM(part_value) +| output: sum(part_value) | 00:SCAN HDFS [tpch.q11_part_tmp] partitions=1/1 size=509.47KB @@ -1088,7 +1088,7 @@ limit 10 | order by: l_shipmode ASC | 03:AGGREGATE [FINALIZE] -| output: SUM(CASE WHEN o_orderpriority = '1-URGENT' OR o_orderpriority = '2-HIGH' THEN 1 ELSE 0 END), SUM(CASE WHEN o_orderpriority != '1-URGENT' AND o_orderpriority != '2-HIGH' THEN 1 ELSE 0 END) +| output: sum(CASE WHEN o_orderpriority = '1-URGENT' OR o_orderpriority = '2-HIGH' THEN 1 ELSE 0 END), sum(CASE WHEN o_orderpriority != '1-URGENT' AND o_orderpriority != '2-HIGH' THEN 1 ELSE 0 END) | group by: l_shipmode | 02:HASH JOIN [INNER JOIN] @@ -1110,13 +1110,13 @@ limit 10 | order by: l_shipmode ASC | 07:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(CASE WHEN o_orderpriority = '1-URGENT' OR o_orderpriority = '2-HIGH' THEN 1 ELSE 0 END)), SUM(SUM(CASE WHEN o_orderpriority != '1-URGENT' AND o_orderpriority != '2-HIGH' THEN 1 ELSE 0 END)) +| output: sum(sum(CASE WHEN o_orderpriority = '1-URGENT' OR o_orderpriority = '2-HIGH' THEN 1 ELSE 0 END)), sum(sum(CASE WHEN o_orderpriority != '1-URGENT' AND o_orderpriority != '2-HIGH' THEN 1 ELSE 0 END)) | group by: l_shipmode | 06:EXCHANGE [PARTITION=HASH(l_shipmode)] | 03:AGGREGATE -| output: SUM(CASE WHEN o_orderpriority = '1-URGENT' OR o_orderpriority = '2-HIGH' THEN 1 ELSE 0 END), SUM(CASE WHEN o_orderpriority != '1-URGENT' AND o_orderpriority != '2-HIGH' THEN 1 ELSE 0 END) +| output: sum(CASE WHEN o_orderpriority = '1-URGENT' OR o_orderpriority = '2-HIGH' THEN 1 ELSE 0 END), sum(CASE WHEN o_orderpriority != '1-URGENT' AND o_orderpriority != '2-HIGH' THEN 1 ELSE 0 END) | group by: l_shipmode | 02:HASH JOIN [INNER JOIN, BROADCAST] @@ -1153,14 +1153,14 @@ order by limit 100 ---- PLAN 05:TOP-N [LIMIT=100] -| order by: COUNT(1) DESC, c_count DESC +| order by: count(1) DESC, c_count DESC | 04:AGGREGATE [FINALIZE] -| output: COUNT(1) -| group by: COUNT(o_orderkey) +| output: count(1) +| group by: count(o_orderkey) | 03:AGGREGATE [FINALIZE] -| output: COUNT(o_orderkey) +| output: count(o_orderkey) | group by: c_custkey | 02:HASH JOIN [RIGHT OUTER JOIN] @@ -1174,31 +1174,31 @@ limit 100 predicates: NOT o.o_comment LIKE '%special%requests%' ---- DISTRIBUTEDPLAN 13:TOP-N [LIMIT=100] -| order by: COUNT(1) DESC, c_count DESC +| order by: count(1) DESC, c_count DESC | 12:EXCHANGE [PARTITION=UNPARTITIONED] | 05:TOP-N [LIMIT=100] -| order by: COUNT(1) DESC, c_count DESC +| order by: count(1) DESC, c_count DESC | 11:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(1)) +| output: sum(count(1)) | group by: c_count | 10:EXCHANGE [PARTITION=HASH(c_count)] | 04:AGGREGATE -| output: COUNT(1) -| group by: COUNT(o_orderkey) +| output: count(1) +| group by: count(o_orderkey) | 09:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(o_orderkey)) +| output: sum(count(o_orderkey)) | group by: c_custkey | 08:EXCHANGE [PARTITION=HASH(c_custkey)] | 03:AGGREGATE -| output: COUNT(o_orderkey) +| output: count(o_orderkey) | group by: c_custkey | 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED] @@ -1228,7 +1228,7 @@ join tpch.part p l.l_shipdate < '1995-10-01' ---- PLAN 03:AGGREGATE [FINALIZE] -| output: SUM(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), sum(l_extendedprice * (1.0 - l_discount)) | 02:HASH JOIN [INNER JOIN] | hash predicates: p.p_partkey = l.l_partkey @@ -1241,12 +1241,12 @@ join tpch.part p partitions=1/1 size=22.83MB ---- DISTRIBUTEDPLAN 06:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END)), SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END)), sum(sum(l_extendedprice * (1.0 - l_discount))) | 05:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE -| output: SUM(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END), sum(l_extendedprice * (1.0 - l_discount)) | 02:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: p.p_partkey = l.l_partkey @@ -1273,7 +1273,7 @@ WRITE TO HDFS [tpch.revenue, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_suppkey | 00:SCAN HDFS [tpch.lineitem] @@ -1284,13 +1284,13 @@ WRITE TO HDFS [tpch.revenue, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | group by: l_suppkey | 02:EXCHANGE [PARTITION=HASH(l_suppkey)] | 01:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | group by: l_suppkey | 00:SCAN HDFS [tpch.lineitem] @@ -1305,7 +1305,7 @@ WRITE TO HDFS [tpch.max_revenue, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: MAX(total_revenue) +| output: max(total_revenue) | 00:SCAN HDFS [tpch.revenue] partitions=1/1 size=137.60KB @@ -1314,12 +1314,12 @@ WRITE TO HDFS [tpch.max_revenue, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: MAX(MAX(total_revenue)) +| output: max(max(total_revenue)) | 02:EXCHANGE [PARTITION=UNPARTITIONED] | 01:AGGREGATE -| output: MAX(total_revenue) +| output: max(total_revenue) | 00:SCAN HDFS [tpch.revenue] partitions=1/1 size=137.60KB @@ -1470,10 +1470,10 @@ order by supplier_cnt desc, p_brand, p_type, p_size limit 1000 ---- PLAN 03:TOP-N [LIMIT=1000] -| order by: COUNT(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC +| order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(ps_suppkey) +| output: count(ps_suppkey) | group by: p_brand, p_type, p_size | 01:AGGREGATE @@ -1484,15 +1484,15 @@ limit 1000 predicates: p_size = 49 OR p_size = 14 OR p_size = 23 OR p_size = 45 OR p_size = 19 OR p_size = 3 OR p_size = 36 OR p_size = 9 ---- DISTRIBUTEDPLAN 07:TOP-N [LIMIT=1000] -| order by: COUNT(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC +| order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC | 06:EXCHANGE [PARTITION=UNPARTITIONED] | 03:TOP-N [LIMIT=1000] -| order by: COUNT(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC +| order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC | 02:AGGREGATE [MERGE FINALIZE] -| output: COUNT(ps_suppkey) +| output: count(ps_suppkey) | group by: p_brand, p_type, p_size | 05:AGGREGATE [MERGE] @@ -1517,7 +1517,7 @@ WRITE TO HDFS [tpch.lineitem_tmp, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: SUM(l_quantity), COUNT(l_quantity) +| output: sum(l_quantity), count(l_quantity) | group by: l_partkey | 00:SCAN HDFS [tpch.lineitem] @@ -1527,13 +1527,13 @@ WRITE TO HDFS [tpch.lineitem_tmp, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_quantity)), SUM(COUNT(l_quantity)) +| output: sum(sum(l_quantity)), sum(count(l_quantity)) | group by: l_partkey | 02:EXCHANGE [PARTITION=HASH(l_partkey)] | 01:AGGREGATE -| output: SUM(l_quantity), COUNT(l_quantity) +| output: sum(l_quantity), count(l_quantity) | group by: l_partkey | 00:SCAN HDFS [tpch.lineitem] @@ -1553,7 +1553,7 @@ where l.l_quantity < lt.t_avg_quantity ---- PLAN 05:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice) +| output: sum(l_extendedprice) | 04:HASH JOIN [INNER JOIN] | hash predicates: p.p_partkey = lt.t_partkey @@ -1573,12 +1573,12 @@ where partitions=1/1 size=718.94MB ---- DISTRIBUTEDPLAN 09:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice)) +| output: sum(sum(l_extendedprice)) | 08:EXCHANGE [PARTITION=UNPARTITIONED] | 05:AGGREGATE -| output: SUM(l_extendedprice) +| output: sum(l_extendedprice) | 04:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: p.p_partkey = lt.t_partkey @@ -1611,7 +1611,7 @@ WRITE TO HDFS [tpch.q18_tmp, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: SUM(l_quantity) +| output: sum(l_quantity) | group by: l_orderkey | 00:SCAN HDFS [tpch.lineitem] @@ -1621,13 +1621,13 @@ WRITE TO HDFS [tpch.q18_tmp, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_quantity)) +| output: sum(sum(l_quantity)) | group by: l_orderkey | 02:EXCHANGE [PARTITION=HASH(l_orderkey)] | 01:AGGREGATE -| output: SUM(l_quantity) +| output: sum(l_quantity) | group by: l_orderkey | 00:SCAN HDFS [tpch.lineitem] @@ -1664,7 +1664,7 @@ limit 100 | order by: CAST(o_totalprice AS BIGINT) ASC, o_orderdate ASC | 07:AGGREGATE [FINALIZE] -| output: SUM(l_quantity) +| output: sum(l_quantity) | group by: c_name, c_custkey, o_orderkey, o_orderdate, CAST(o_totalprice AS BIGINT) | 06:HASH JOIN [INNER JOIN] @@ -1698,13 +1698,13 @@ limit 100 | order by: CAST(o_totalprice AS BIGINT) ASC, o_orderdate ASC | 14:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_quantity)) +| output: sum(sum(l_quantity)) | group by: c_name, c_custkey, o_orderkey, o_orderdate, CAST(o_totalprice AS BIGINT) | 13:EXCHANGE [PARTITION=HASH(c_name,c_custkey,o_orderkey,o_orderdate,CAST(o_totalprice AS BIGINT))] | 07:AGGREGATE -| output: SUM(l_quantity) +| output: sum(l_quantity) | group by: c_name, c_custkey, o_orderkey, o_orderdate, CAST(o_totalprice AS BIGINT) | 06:HASH JOIN [INNER JOIN, BROADCAST] @@ -1772,7 +1772,7 @@ or ) ---- PLAN 03:AGGREGATE [FINALIZE] -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | 02:HASH JOIN [INNER JOIN] | hash predicates: l.l_partkey = p.p_partkey @@ -1785,12 +1785,12 @@ or partitions=1/1 size=718.94MB ---- DISTRIBUTEDPLAN 06:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_extendedprice * (1.0 - l_discount))) +| output: sum(sum(l_extendedprice * (1.0 - l_discount))) | 05:EXCHANGE [PARTITION=UNPARTITIONED] | 03:AGGREGATE -| output: SUM(l_extendedprice * (1.0 - l_discount)) +| output: sum(l_extendedprice * (1.0 - l_discount)) | 02:HASH JOIN [INNER JOIN, BROADCAST] | hash predicates: l.l_partkey = p.p_partkey @@ -1854,7 +1854,7 @@ WRITE TO HDFS [tpch.q20_tmp2, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: SUM(l_quantity) +| output: sum(l_quantity) | group by: l_partkey, l_suppkey | 00:SCAN HDFS [tpch.lineitem] @@ -1865,13 +1865,13 @@ WRITE TO HDFS [tpch.q20_tmp2, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(l_quantity)) +| output: sum(sum(l_quantity)) | group by: l_partkey, l_suppkey | 02:EXCHANGE [PARTITION=HASH(l_partkey,l_suppkey)] | 01:AGGREGATE -| output: SUM(l_quantity) +| output: sum(l_quantity) | group by: l_partkey, l_suppkey | 00:SCAN HDFS [tpch.lineitem] @@ -2059,10 +2059,10 @@ order by limit 100 ---- PLAN 12:TOP-N [LIMIT=100] -| order by: COUNT(*) DESC, s_name ASC +| order by: count(*) DESC, s_name ASC | 11:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: s_name | 10:HASH JOIN [LEFT OUTER JOIN] @@ -2105,21 +2105,21 @@ limit 100 partitions=1/1 size=718.94MB ---- DISTRIBUTEDPLAN 21:TOP-N [LIMIT=100] -| order by: COUNT(*) DESC, s_name ASC +| order by: count(*) DESC, s_name ASC | 20:EXCHANGE [PARTITION=UNPARTITIONED] | 12:TOP-N [LIMIT=100] -| order by: COUNT(*) DESC, s_name ASC +| order by: count(*) DESC, s_name ASC | 19:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: s_name | 18:EXCHANGE [PARTITION=HASH(s_name)] | 11:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: s_name | 10:HASH JOIN [LEFT OUTER JOIN, BROADCAST] @@ -2198,7 +2198,7 @@ WRITE TO HDFS [tpch.q22_customer_tmp1, OVERWRITE=true] | partitions=1 | 01:AGGREGATE [FINALIZE] -| output: SUM(c_acctbal), COUNT(c_acctbal) +| output: sum(c_acctbal), count(c_acctbal) | group by: substr(c_name, 1, 1) | 00:SCAN HDFS [tpch.customer c] @@ -2209,13 +2209,13 @@ WRITE TO HDFS [tpch.q22_customer_tmp1, OVERWRITE=true] | partitions=1 | 03:AGGREGATE [MERGE FINALIZE] -| output: SUM(SUM(c_acctbal)), SUM(COUNT(c_acctbal)) +| output: sum(sum(c_acctbal)), sum(count(c_acctbal)) | group by: substr(c_name, 1, 1) | 02:EXCHANGE [PARTITION=HASH(substr(c_name, 1, 1))] | 01:AGGREGATE -| output: SUM(c_acctbal), COUNT(c_acctbal) +| output: sum(c_acctbal), count(c_acctbal) | group by: substr(c_name, 1, 1) | 00:SCAN HDFS [tpch.customer c] @@ -2256,7 +2256,7 @@ limit 100 | order by: substring(c_phone, 1, 2) ASC | 05:AGGREGATE [FINALIZE] -| output: COUNT(*), SUM(c_acctbal) +| output: count(*), sum(c_acctbal) | group by: substring(c_phone, 1, 2) | 04:HASH JOIN [LEFT OUTER JOIN] @@ -2286,13 +2286,13 @@ limit 100 | order by: substring(c_phone, 1, 2) ASC | 11:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)), SUM(SUM(c_acctbal)) +| output: sum(count(*)), sum(sum(c_acctbal)) | group by: substring(c_phone, 1, 2) | 10:EXCHANGE [PARTITION=HASH(substring(c_phone, 1, 2))] | 05:AGGREGATE -| output: COUNT(*), SUM(c_acctbal) +| output: count(*), sum(c_acctbal) | group by: substring(c_phone, 1, 2) | 04:HASH JOIN [LEFT OUTER JOIN, PARTITIONED] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/union.test b/testdata/workloads/functional-planner/queries/PlannerTest/union.test index 73fcb5b36..e5c754130 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/union.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/union.test @@ -2062,7 +2062,7 @@ where x.int_col < 5 and x.bool_col = false 00:MERGE | |--04:AGGREGATE [FINALIZE] -| | output: COUNT(*) +| | output: count(*) | | group by: int_col, bool_col | | | 03:SCAN HDFS [functional.alltypestiny] @@ -2070,7 +2070,7 @@ where x.int_col < 5 and x.bool_col = false | predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE | 02:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | group by: int_col, bool_col | 01:SCAN HDFS [functional.alltypestiny] @@ -2087,13 +2087,13 @@ NODE 3: |--11:MERGE | | | 08:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(*)) +| | output: sum(count(*)) | | group by: int_col, bool_col | | | 07:EXCHANGE [PARTITION=HASH(int_col,bool_col)] | | | 04:AGGREGATE -| | output: COUNT(*) +| | output: count(*) | | group by: int_col, bool_col | | | 03:SCAN HDFS [functional.alltypestiny] @@ -2103,13 +2103,13 @@ NODE 3: 10:MERGE | 06:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(*)) +| output: sum(count(*)) | group by: int_col, bool_col | 05:EXCHANGE [PARTITION=HASH(int_col,bool_col)] | 02:AGGREGATE -| output: COUNT(*) +| output: count(*) | group by: int_col, bool_col | 01:SCAN HDFS [functional.alltypestiny] @@ -2356,7 +2356,7 @@ select count(*) from ( ) x ---- PLAN 04:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | 03:AGGREGATE [FINALIZE] | group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month @@ -2371,7 +2371,7 @@ select count(*) from ( partitions=24/24 size=956.90KB ---- DISTRIBUTEDPLAN 04:AGGREGATE [FINALIZE] -| output: COUNT(*) +| output: count(*) | 03:AGGREGATE [FINALIZE] | group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/views.test b/testdata/workloads/functional-planner/queries/PlannerTest/views.test index 0a392d474..031964aed 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/views.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/views.test @@ -16,9 +16,9 @@ select * from functional.complex_view | order by: b.string_col ASC | 03:AGGREGATE [FINALIZE] -| output: COUNT(a.bigint_col) +| output: count(a.bigint_col) | group by: b.string_col -| having: COUNT(a.bigint_col) > 1 +| having: count(a.bigint_col) > 1 | 02:HASH JOIN [INNER JOIN] | hash predicates: a.id = b.id @@ -39,14 +39,14 @@ select * from functional.complex_view | order by: b.string_col ASC | 07:AGGREGATE [MERGE FINALIZE] -| output: SUM(COUNT(a.bigint_col)) +| output: sum(count(a.bigint_col)) | group by: b.string_col -| having: COUNT(a.bigint_col) > 1 +| having: count(a.bigint_col) > 1 | 06:EXCHANGE [PARTITION=HASH(b.string_col)] | 03:AGGREGATE -| output: COUNT(a.bigint_col) +| output: count(a.bigint_col) | group by: b.string_col | 02:HASH JOIN [INNER JOIN, BROADCAST] @@ -123,15 +123,15 @@ functional.complex_view t3 where t1.id = t2.x and t2.x = t3.abc | partitions=24/24 size=1.40MB compact | 07:HASH JOIN [INNER JOIN] -| hash predicates: functional.alltypes.id = COUNT(a.bigint_col) +| hash predicates: functional.alltypes.id = count(a.bigint_col) | |--06:TOP-N [LIMIT=100] | | order by: b.string_col ASC | | | 05:AGGREGATE [FINALIZE] -| | output: COUNT(a.bigint_col) +| | output: count(a.bigint_col) | | group by: b.string_col -| | having: COUNT(a.bigint_col) > 1 +| | having: count(a.bigint_col) > 1 | | | 04:HASH JOIN [INNER JOIN] | | hash predicates: a.id = b.id @@ -157,7 +157,7 @@ functional.complex_view t3 where t1.id = t2.x and t2.x = t3.abc | partitions=24/24 size=1.40MB | 07:HASH JOIN [INNER JOIN, BROADCAST] -| hash predicates: functional.alltypes.id = COUNT(a.bigint_col) +| hash predicates: functional.alltypes.id = count(a.bigint_col) | |--14:EXCHANGE [BROADCAST] | | limit: 100 @@ -171,14 +171,14 @@ functional.complex_view t3 where t1.id = t2.x and t2.x = t3.abc | | order by: b.string_col ASC | | | 11:AGGREGATE [MERGE FINALIZE] -| | output: SUM(COUNT(a.bigint_col)) +| | output: sum(count(a.bigint_col)) | | group by: b.string_col -| | having: COUNT(a.bigint_col) > 1 +| | having: count(a.bigint_col) > 1 | | | 10:EXCHANGE [PARTITION=HASH(b.string_col)] | | | 05:AGGREGATE -| | output: COUNT(a.bigint_col) +| | output: count(a.bigint_col) | | group by: b.string_col | | | 04:HASH JOIN [INNER JOIN, BROADCAST] diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test b/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test index 2d3f38198..de846a5ab 100644 --- a/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test +++ b/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test @@ -133,15 +133,15 @@ limit 3 | predicates: c1 > 0 | 03:AGGREGATE [FINALIZE] -| output: MAX(tinyint_col) -| group by: int_col, MAX(id) +| output: max(tinyint_col) +| group by: int_col, max(id) | limit: 10 | 02:TOP-N [LIMIT=5] | order by: int_col ASC, tinyint_col ASC | 01:AGGREGATE [FINALIZE] -| output: MAX(id) +| output: max(id) | group by: int_col, tinyint_col | 00:SCAN HDFS [functional.alltypessmall] @@ -154,8 +154,8 @@ limit 3 | predicates: c1 > 0 | 03:AGGREGATE [FINALIZE] -| output: MAX(tinyint_col) -| group by: int_col, MAX(id) +| output: max(tinyint_col) +| group by: int_col, max(id) | limit: 10 | 09:TOP-N [LIMIT=5] @@ -167,13 +167,13 @@ limit 3 | order by: int_col ASC, tinyint_col ASC | 07:AGGREGATE [MERGE FINALIZE] -| output: MAX(MAX(id)) +| output: max(max(id)) | group by: int_col, tinyint_col | 06:EXCHANGE [PARTITION=HASH(int_col,tinyint_col)] | 01:AGGREGATE -| output: MAX(id) +| output: max(id) | group by: int_col, tinyint_col | 00:SCAN HDFS [functional.alltypessmall] diff --git a/testdata/workloads/functional-query/queries/QueryTest/aggregation.test b/testdata/workloads/functional-query/queries/QueryTest/aggregation.test index 9fee2e7cc..68af71145 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/aggregation.test +++ b/testdata/workloads/functional-query/queries/QueryTest/aggregation.test @@ -726,7 +726,7 @@ timestamp, bigint # Test NULLs in aggregate functions select count(NULL), min(NULL), max(NULL), sum(NULL), avg(NULL) from alltypesagg ---- TYPES -bigint, NULL, NULL, NULL, double +bigint, boolean, boolean, bigint, double ---- RESULTS 0,NULL,NULL,NULL,NULL ==== @@ -734,7 +734,7 @@ bigint, NULL, NULL, NULL, double # Test ignored distinct in MIN and MAX with NULLs select min(distinct NULL), max(distinct NULL) from alltypesagg ---- TYPES -NULL, NULL +boolean, boolean ---- RESULTS NULL,NULL ==== @@ -821,7 +821,7 @@ int, string ---- QUERY # Test group_concat with column delimiter # Will cause all columns save first to be duplicated -select day, group_concat(string_col, string_col) +select day, group_concat(trim(string_col), trim(string_col)) from alltypesagg where id % 200 = day group by day diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test index 5818d3a22..8d3eb0642 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test @@ -34,21 +34,21 @@ limit 100 'Estimated Per-Host Requirements: Memory=144.66MB VCores=5' '' '19:TOP-N [LIMIT=100]' -'| order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' +'| order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' '|' '18:EXCHANGE [PARTITION=UNPARTITIONED]' '|' '10:TOP-N [LIMIT=100]' -'| order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' +'| order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' '|' '17:AGGREGATE [MERGE FINALIZE]' -'| output: SUM(SUM(ss_ext_sales_price))' +'| output: sum(sum(ss_ext_sales_price))' '| group by: i_brand, i_brand_id, i_manufact_id, i_manufact' '|' '16:EXCHANGE [PARTITION=HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]' '|' '09:AGGREGATE' -'| output: SUM(ss_ext_sales_price)' +'| output: sum(ss_ext_sales_price)' '| group by: i_brand, i_brand_id, i_manufact_id, i_manufact' '|' '08:HASH JOIN [INNER JOIN, BROADCAST]' @@ -160,4 +160,4 @@ select * from functional_hbase.alltypessmall '|' '01:SCAN HBASE [functional_hbase.alltypessmall a]' ' predicates: a.year > 2009, a.month = 4' -==== \ No newline at end of file +==== diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test index f89fee81e..5102048a2 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test @@ -34,7 +34,7 @@ limit 100 'Estimated Per-Host Requirements: Memory=144.66MB VCores=5' '' '19:TOP-N [LIMIT=100]' -'| order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' +'| order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' '| hosts=3 per-host-mem=unavailable' '| tuple-ids=5 row-size=48B cardinality=100' '|' @@ -43,12 +43,12 @@ limit 100 '| tuple-ids=5 row-size=48B cardinality=100' '|' '10:TOP-N [LIMIT=100]' -'| order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' +'| order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' '| hosts=3 per-host-mem=4.69KB' '| tuple-ids=5 row-size=48B cardinality=100' '|' '17:AGGREGATE [MERGE FINALIZE]' -'| output: SUM(SUM(ss_ext_sales_price))' +'| output: sum(sum(ss_ext_sales_price))' '| group by: i_brand, i_brand_id, i_manufact_id, i_manufact' '| hosts=3 per-host-mem=10.00MB' '| tuple-ids=5 row-size=48B cardinality=87208' @@ -58,7 +58,7 @@ limit 100 '| tuple-ids=5 row-size=48B cardinality=87208' '|' '09:AGGREGATE' -'| output: SUM(ss_ext_sales_price)' +'| output: sum(ss_ext_sales_price)' '| group by: i_brand, i_brand_id, i_manufact_id, i_manufact' '| hosts=3 per-host-mem=10.00MB' '| tuple-ids=5 row-size=48B cardinality=87208' @@ -260,4 +260,4 @@ select * from functional_hbase.alltypessmall ' column stats: all' ' hosts=100 per-host-mem=1.00GB' ' tuple-ids=0 row-size=97B cardinality=4' -==== \ No newline at end of file +==== diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test index ae5097e4e..c8cc71c29 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test +++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test @@ -35,7 +35,7 @@ limit 100 '' 'F07:PLAN FRAGMENT [PARTITION=UNPARTITIONED]' ' 19:TOP-N [LIMIT=100]' -' | order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' +' | order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' ' | hosts=3 per-host-mem=unavailable' ' | tuple-ids=5 row-size=48B cardinality=100' ' |' @@ -46,12 +46,12 @@ limit 100 'F06:PLAN FRAGMENT [PARTITION=HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]' ' DATASTREAM SINK [FRAGMENT=F07, EXCHANGE=18, PARTITION=UNPARTITIONED]' ' 10:TOP-N [LIMIT=100]' -' | order by: SUM(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' +' | order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC' ' | hosts=3 per-host-mem=4.69KB' ' | tuple-ids=5 row-size=48B cardinality=100' ' |' ' 17:AGGREGATE [MERGE FINALIZE]' -' | output: SUM(SUM(ss_ext_sales_price))' +' | output: sum(sum(ss_ext_sales_price))' ' | group by: i_brand, i_brand_id, i_manufact_id, i_manufact' ' | hosts=3 per-host-mem=10.00MB' ' | tuple-ids=5 row-size=48B cardinality=87208' @@ -63,7 +63,7 @@ limit 100 'F02:PLAN FRAGMENT [PARTITION=HASH(customer_address.ca_address_sk)]' ' DATASTREAM SINK [FRAGMENT=F06, EXCHANGE=16, PARTITION=HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]' ' 09:AGGREGATE' -' | output: SUM(ss_ext_sales_price)' +' | output: sum(ss_ext_sales_price)' ' | group by: i_brand, i_brand_id, i_manufact_id, i_manufact' ' | hosts=3 per-host-mem=10.00MB' ' | tuple-ids=5 row-size=48B cardinality=87208' @@ -291,4 +291,4 @@ select * from functional_hbase.alltypessmall ' column stats: all' ' hosts=3 per-host-mem=32.00MB' ' tuple-ids=1 row-size=0B cardinality=100' -==== \ No newline at end of file +==== diff --git a/testdata/workloads/functional-query/queries/QueryTest/hive-udf.test b/testdata/workloads/functional-query/queries/QueryTest/hive-udf.test index e9efa4856..0cf94f5e5 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/hive-udf.test +++ b/testdata/workloads/functional-query/queries/QueryTest/hive-udf.test @@ -1,12 +1,5 @@ ==== ---- QUERY -select udf_test.trim('HelloWorld') ----- RESULTS -'HelloWorld' ----- TYPES -STRING -==== ----- QUERY select udf_test.hive_pi() ---- RESULTS 3.141592653589793 diff --git a/testdata/workloads/functional-query/queries/QueryTest/load-hive-udfs.test b/testdata/workloads/functional-query/queries/QueryTest/load-hive-udfs.test index d79c19586..99be9b70e 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/load-hive-udfs.test +++ b/testdata/workloads/functional-query/queries/QueryTest/load-hive-udfs.test @@ -6,6 +6,7 @@ drop function if exists udf_test.hive_floor(double); drop function if exists udf_test.hive_mod(int, int); drop function if exists udf_test.hive_bin(bigint); drop function if exists udf_test.hive_lower(string); +drop function if exists udf_test.hive_trim(string); drop function if exists udf_test.identity(boolean); drop function if exists udf_test.identity(tinyint);