move hivesterix codebase into hyracks fullstack
git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_staging@2420 123451ca-8445-de46-9d55-352943316053
diff --git a/src/main/assembly/binary-assembly.xml b/src/main/assembly/binary-assembly.xml
new file mode 100755
index 0000000..0500499
--- /dev/null
+++ b/src/main/assembly/binary-assembly.xml
@@ -0,0 +1,19 @@
+<assembly>
+ <id>binary-assembly</id>
+ <formats>
+ <format>zip</format>
+ <format>dir</format>
+ </formats>
+ <includeBaseDirectory>false</includeBaseDirectory>
+ <fileSets>
+ <fileSet>
+ <directory>target/appassembler/bin</directory>
+ <outputDirectory>bin</outputDirectory>
+ <fileMode>0755</fileMode>
+ </fileSet>
+ <fileSet>
+ <directory>target/appassembler/lib</directory>
+ <outputDirectory>lib</outputDirectory>
+ </fileSet>
+ </fileSets>
+</assembly>
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionConstant.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionConstant.java
new file mode 100644
index 0000000..bf4806f
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionConstant.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+/**
+ * some constants for expression
+ *
+ * @author yingyib
+ *
+ */
+public class ExpressionConstant {
+
+ /**
+ * name space for function identifier
+ */
+ public static String NAMESPACE = "hive";
+
+ /**
+ * field expression: modeled as function in Algebricks
+ */
+ public static String FIELDACCESS = "fieldaccess";
+
+ /**
+ * null string: modeled as null in Algebricks
+ */
+ public static String NULL = "null";
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveAlgebricksBuiltInFunctionMap.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveAlgebricksBuiltInFunctionMap.java
new file mode 100644
index 0000000..450e8ad
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveAlgebricksBuiltInFunctionMap.java
@@ -0,0 +1,81 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import java.util.HashMap;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class HiveAlgebricksBuiltInFunctionMap {
+
+ /**
+ * hive auqa builtin function map instance
+ */
+ public static HiveAlgebricksBuiltInFunctionMap INSTANCE = new HiveAlgebricksBuiltInFunctionMap();
+
+ /**
+ * hive to Algebricks function name mapping
+ */
+ private HashMap<String, FunctionIdentifier> hiveToAlgebricksMap = new HashMap<String, FunctionIdentifier>();
+
+ /**
+ * Algebricks to hive function name mapping
+ */
+ private HashMap<FunctionIdentifier, String> AlgebricksToHiveMap = new HashMap<FunctionIdentifier, String>();
+
+ /**
+ * the bi-directional mapping between hive functions and Algebricks functions
+ */
+ private HiveAlgebricksBuiltInFunctionMap() {
+ hiveToAlgebricksMap.put("and", AlgebricksBuiltinFunctions.AND);
+ hiveToAlgebricksMap.put("or", AlgebricksBuiltinFunctions.OR);
+ hiveToAlgebricksMap.put("!", AlgebricksBuiltinFunctions.NOT);
+ hiveToAlgebricksMap.put("not", AlgebricksBuiltinFunctions.NOT);
+ hiveToAlgebricksMap.put("=", AlgebricksBuiltinFunctions.EQ);
+ hiveToAlgebricksMap.put("<>", AlgebricksBuiltinFunctions.NEQ);
+ hiveToAlgebricksMap.put(">", AlgebricksBuiltinFunctions.GT);
+ hiveToAlgebricksMap.put("<", AlgebricksBuiltinFunctions.LT);
+ hiveToAlgebricksMap.put(">=", AlgebricksBuiltinFunctions.GE);
+ hiveToAlgebricksMap.put("<=", AlgebricksBuiltinFunctions.LE);
+
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.AND, "and");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.OR, "or");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NOT, "!");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NOT, "not");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.EQ, "=");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NEQ, "<>");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.GT, ">");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.LT, "<");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.GE, ">=");
+ AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.LE, "<=");
+ }
+
+ /**
+ * get hive function name from Algebricks function identifier
+ *
+ * @param AlgebricksId
+ * @return hive
+ */
+ public String getHiveFunctionName(FunctionIdentifier AlgebricksId) {
+ return AlgebricksToHiveMap.get(AlgebricksId);
+ }
+
+ /**
+ * get hive UDF or Generic class's corresponding built-in functions
+ *
+ * @param funcClass
+ * @return function identifier
+ */
+ public FunctionIdentifier getAlgebricksFunctionId(Class<?> funcClass) {
+ Description annotation = (Description) funcClass.getAnnotation(Description.class);
+ String hiveUDFName = "";
+ if (annotation == null) {
+ hiveUDFName = null;
+ return null;
+ } else {
+ hiveUDFName = annotation.name();
+ return hiveToAlgebricksMap.get(hiveUDFName);
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveExpressionTypeComputer.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveExpressionTypeComputer.java
new file mode 100644
index 0000000..e10e8c1
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveExpressionTypeComputer.java
@@ -0,0 +1,179 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class HiveExpressionTypeComputer implements IExpressionTypeComputer {
+
+ public static IExpressionTypeComputer INSTANCE = new HiveExpressionTypeComputer();
+
+ @Override
+ public Object getType(ILogicalExpression expr, IMetadataProvider<?, ?> metadataProvider,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ /**
+ * function expression
+ */
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ IFunctionInfo funcInfo = funcExpr.getFunctionInfo();
+
+ /**
+ * argument expressions, types, object inspectors
+ */
+ List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();
+ List<TypeInfo> argumentTypes = new ArrayList<TypeInfo>();
+
+ /**
+ * get types of argument
+ */
+ for (Mutable<ILogicalExpression> argument : arguments) {
+ TypeInfo type = (TypeInfo) getType(argument.getValue(), metadataProvider, env);
+ argumentTypes.add(type);
+ }
+
+ ObjectInspector[] childrenOIs = new ObjectInspector[argumentTypes.size()];
+
+ /**
+ * get object inspector
+ */
+ for (int i = 0; i < argumentTypes.size(); i++) {
+ childrenOIs[i] = TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(argumentTypes.get(i));
+ }
+
+ /**
+ * type inference for scalar function
+ */
+ if (funcExpr instanceof ScalarFunctionCallExpression) {
+
+ FunctionIdentifier AlgebricksId = funcInfo.getFunctionIdentifier();
+ Object functionInfo = ((HiveFunctionInfo) funcInfo).getInfo();
+ String udfName = HiveAlgebricksBuiltInFunctionMap.INSTANCE.getHiveFunctionName(AlgebricksId);
+ GenericUDF udf;
+ if (udfName != null) {
+ /**
+ * get corresponding function info for built-in functions
+ */
+ FunctionInfo fInfo = FunctionRegistry.getFunctionInfo(udfName);
+ udf = fInfo.getGenericUDF();
+ } else if (functionInfo != null) {
+ /**
+ * for GenericUDFBridge: we should not call get type of this
+ * hive expression, because parameters may have been
+ * changed!
+ */
+ ExprNodeGenericFuncDesc hiveExpr = (ExprNodeGenericFuncDesc) functionInfo;
+ udf = hiveExpr.getGenericUDF();
+ } else {
+ /**
+ * for other generic UDF
+ */
+ Class<?> udfClass;
+ try {
+ udfClass = Class.forName(AlgebricksId.getName());
+ udf = (GenericUDF) udfClass.newInstance();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ }
+ }
+ /**
+ * doing the actual type inference
+ */
+ ObjectInspector oi = null;
+ try {
+ oi = udf.initialize(childrenOIs);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+
+ TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(oi);
+ return exprType;
+
+ } else if (funcExpr instanceof AggregateFunctionCallExpression) {
+ /**
+ * hive aggregation info
+ */
+ AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo())
+ .getInfo();
+ /**
+ * type inference for aggregation function
+ */
+ GenericUDAFEvaluator result = aggregateDesc.getGenericUDAFEvaluator();
+
+ ObjectInspector returnOI = null;
+ try {
+ returnOI = result.init(aggregateDesc.getMode(), childrenOIs);
+ } catch (HiveException e) {
+ e.printStackTrace();
+ }
+ TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(returnOI);
+ return exprType;
+ } else if (funcExpr instanceof UnnestingFunctionCallExpression) {
+ /**
+ * type inference for UDTF function
+ */
+ UDTFDesc hiveDesc = (UDTFDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo()).getInfo();
+ GenericUDTF udtf = hiveDesc.getGenericUDTF();
+ ObjectInspector returnOI = null;
+ try {
+ returnOI = udtf.initialize(childrenOIs);
+ } catch (HiveException e) {
+ e.printStackTrace();
+ }
+ TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(returnOI);
+ return exprType;
+ } else {
+ throw new IllegalStateException("unrecognized function expression " + expr.getClass().getName());
+ }
+ } else if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ /**
+ * get type for variable in the environment
+ */
+ VariableReferenceExpression varExpr = (VariableReferenceExpression) expr;
+ LogicalVariable var = varExpr.getVariableReference();
+ TypeInfo type = (TypeInfo) env.getVarType(var);
+ return type;
+ } else if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ /**
+ * get type for constant, from its java class
+ */
+ ConstantExpression constExpr = (ConstantExpression) expr;
+ HivesterixConstantValue value = (HivesterixConstantValue) constExpr.getValue();
+ TypeInfo type = TypeInfoFactory.getPrimitiveTypeInfoFromJavaPrimitive(value.getObject().getClass());
+ return type;
+ } else {
+ throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveFunctionInfo.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveFunctionInfo.java
new file mode 100644
index 0000000..ced8d02
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveFunctionInfo.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+
+public class HiveFunctionInfo implements IFunctionInfo, Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * primary function identifier
+ */
+ private transient FunctionIdentifier fid;
+
+ /**
+ * secondary function identifier: function name
+ */
+ private transient Object secondaryFid;
+
+ public HiveFunctionInfo(FunctionIdentifier fid, Object secondFid) {
+ this.fid = fid;
+ this.secondaryFid = secondFid;
+ }
+
+ @Override
+ public FunctionIdentifier getFunctionIdentifier() {
+ return fid;
+ }
+
+ public Object getInfo() {
+ return secondaryFid;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
new file mode 100644
index 0000000..d43d7bd
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+/**
+ * generate merge aggregation expression from an aggregation expression
+ *
+ * @author yingyib
+ *
+ */
+public class HiveMergeAggregationExpressionFactory implements IMergeAggregationExpressionFactory {
+
+ public static IMergeAggregationExpressionFactory INSTANCE = new HiveMergeAggregationExpressionFactory();
+
+ @Override
+ public ILogicalExpression createMergeAggregation(ILogicalExpression expr, IOptimizationContext context)
+ throws AlgebricksException {
+ /**
+ * type inference for scalar function
+ */
+ if (expr instanceof AggregateFunctionCallExpression) {
+ AggregateFunctionCallExpression funcExpr = (AggregateFunctionCallExpression) expr;
+ /**
+ * hive aggregation info
+ */
+ AggregationDesc aggregator = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo()).getInfo();
+ LogicalVariable inputVar = context.newVar();
+ ExprNodeDesc col = new ExprNodeColumnDesc(TypeInfoFactory.voidTypeInfo, inputVar.toString(), null, false);
+ ArrayList<ExprNodeDesc> parameters = new ArrayList<ExprNodeDesc>();
+ parameters.add(col);
+
+ GenericUDAFEvaluator.Mode mergeMode;
+ if (aggregator.getMode() == GenericUDAFEvaluator.Mode.PARTIAL1)
+ mergeMode = GenericUDAFEvaluator.Mode.PARTIAL2;
+ else if (aggregator.getMode() == GenericUDAFEvaluator.Mode.COMPLETE)
+ mergeMode = GenericUDAFEvaluator.Mode.FINAL;
+ else
+ mergeMode = aggregator.getMode();
+ AggregationDesc mergeDesc = new AggregationDesc(aggregator.getGenericUDAFName(),
+ aggregator.getGenericUDAFEvaluator(), parameters, aggregator.getDistinct(), mergeMode);
+
+ String UDAFName = mergeDesc.getGenericUDAFName();
+ List<Mutable<ILogicalExpression>> arguments = new ArrayList<Mutable<ILogicalExpression>>();
+ arguments.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputVar)));
+
+ FunctionIdentifier funcId = new FunctionIdentifier(ExpressionConstant.NAMESPACE, UDAFName + "("
+ + mergeDesc.getMode() + ")");
+ HiveFunctionInfo funcInfo = new HiveFunctionInfo(funcId, mergeDesc);
+ AggregateFunctionCallExpression aggregationExpression = new AggregateFunctionCallExpression(funcInfo,
+ false, arguments);
+ return aggregationExpression;
+ } else {
+ throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
+ }
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveNullableTypeComputer.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveNullableTypeComputer.java
new file mode 100644
index 0000000..906e3ce
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveNullableTypeComputer.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+
+public class HiveNullableTypeComputer implements INullableTypeComputer {
+
+ public static INullableTypeComputer INSTANCE = new HiveNullableTypeComputer();
+
+ @Override
+ public Object makeNullableType(Object type) throws AlgebricksException {
+ return type;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivePartialAggregationTypeComputer.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivePartialAggregationTypeComputer.java
new file mode 100644
index 0000000..c74966c
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivePartialAggregationTypeComputer.java
@@ -0,0 +1,102 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class HivePartialAggregationTypeComputer implements IPartialAggregationTypeComputer {
+
+ public static IPartialAggregationTypeComputer INSTANCE = new HivePartialAggregationTypeComputer();
+
+ @Override
+ public Object getType(ILogicalExpression expr, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ IExpressionTypeComputer tc = HiveExpressionTypeComputer.INSTANCE;
+ /**
+ * function expression
+ */
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+
+ /**
+ * argument expressions, types, object inspectors
+ */
+ List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();
+ List<TypeInfo> argumentTypes = new ArrayList<TypeInfo>();
+
+ /**
+ * get types of argument
+ */
+ for (Mutable<ILogicalExpression> argument : arguments) {
+ TypeInfo type = (TypeInfo) tc.getType(argument.getValue(), metadataProvider, env);
+ argumentTypes.add(type);
+ }
+
+ ObjectInspector[] childrenOIs = new ObjectInspector[argumentTypes.size()];
+
+ /**
+ * get object inspector
+ */
+ for (int i = 0; i < argumentTypes.size(); i++) {
+ childrenOIs[i] = TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(argumentTypes.get(i));
+ }
+
+ /**
+ * type inference for scalar function
+ */
+ if (funcExpr instanceof AggregateFunctionCallExpression) {
+ /**
+ * hive aggregation info
+ */
+ AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo())
+ .getInfo();
+ /**
+ * type inference for aggregation function
+ */
+ GenericUDAFEvaluator result = aggregateDesc.getGenericUDAFEvaluator();
+
+ ObjectInspector returnOI = null;
+ try {
+ returnOI = result.init(getPartialMode(aggregateDesc.getMode()), childrenOIs);
+ } catch (HiveException e) {
+ e.printStackTrace();
+ }
+ TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(returnOI);
+ return exprType;
+ } else {
+ throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
+ }
+ } else {
+ throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
+ }
+ }
+
+ private Mode getPartialMode(Mode mode) {
+ Mode partialMode;
+ if (mode == Mode.FINAL)
+ partialMode = Mode.PARTIAL2;
+ else if (mode == Mode.COMPLETE)
+ partialMode = Mode.PARTIAL1;
+ else
+ partialMode = mode;
+ return partialMode;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivesterixConstantValue.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivesterixConstantValue.java
new file mode 100644
index 0000000..3d35e1f
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivesterixConstantValue.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+
+public class HivesterixConstantValue implements IAlgebricksConstantValue {
+
+ private Object object;
+
+ public HivesterixConstantValue(Object object) {
+ this.setObject(object);
+ }
+
+ @Override
+ public boolean isFalse() {
+ return object == Boolean.FALSE;
+ }
+
+ @Override
+ public boolean isNull() {
+ return object == null;
+ }
+
+ @Override
+ public boolean isTrue() {
+ return object == Boolean.TRUE;
+ }
+
+ public void setObject(Object object) {
+ this.object = object;
+ }
+
+ public Object getObject() {
+ return object;
+ }
+
+ @Override
+ public String toString() {
+ return object.toString();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof HivesterixConstantValue)) {
+ return false;
+ }
+ HivesterixConstantValue v2 = (HivesterixConstantValue) o;
+ return object.equals(v2.getObject());
+ }
+
+ @Override
+ public int hashCode() {
+ return object.hashCode();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/expression/Schema.java b/src/main/java/edu/uci/ics/hivesterix/logical/expression/Schema.java
new file mode 100644
index 0000000..3982d33
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/expression/Schema.java
@@ -0,0 +1,39 @@
+package edu.uci.ics.hivesterix.logical.expression;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+
+public class Schema implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private List<String> fieldNames;
+
+ private List<TypeInfo> fieldTypes;
+
+ public Schema(List<String> fieldNames, List<TypeInfo> fieldTypes) {
+ this.fieldNames = fieldNames;
+ this.fieldTypes = fieldTypes;
+ }
+
+ public ObjectInspector toObjectInspector() {
+ return LazyUtils.getLazyObjectInspector(fieldNames, fieldTypes);
+ }
+
+ public List<String> getNames() {
+ return fieldNames;
+ }
+
+ public List<TypeInfo> getTypes() {
+ return fieldTypes;
+ }
+
+ public Object[] getSchema() {
+ return fieldTypes.toArray();
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
new file mode 100644
index 0000000..7586d83
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
@@ -0,0 +1,805 @@
+package edu.uci.ics.hivesterix.logical.plan;
+
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.ExtractOperator;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.GroupByOperator;
+import org.apache.hadoop.hive.ql.exec.JoinOperator;
+import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator;
+import org.apache.hadoop.hive.ql.exec.LimitOperator;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.exec.UDTFOperator;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeNullDesc;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+import org.apache.hadoop.hive.ql.plan.api.OperatorType;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+import edu.uci.ics.hivesterix.logical.expression.ExpressionConstant;
+import edu.uci.ics.hivesterix.logical.expression.HiveAlgebricksBuiltInFunctionMap;
+import edu.uci.ics.hivesterix.logical.expression.HiveFunctionInfo;
+import edu.uci.ics.hivesterix.logical.expression.HivesterixConstantValue;
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.logical.plan.visitor.ExtractVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.FilterVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.GroupByVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.JoinVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.LateralViewJoinVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.LimitVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.MapJoinVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.ProjectVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.SortVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.TableScanWriteVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.UnionVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Visitor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+
+public class HiveAlgebricksTranslator implements Translator {
+
+ private int currentVariable = 0;
+
+ private List<Mutable<ILogicalOperator>> logicalOp = new ArrayList<Mutable<ILogicalOperator>>();
+
+ private boolean continueTraverse = true;
+
+ private IMetadataProvider<PartitionDesc, Object> metaData;
+
+ /**
+ * map variable name to the logical variable
+ */
+ private HashMap<String, LogicalVariable> nameToLogicalVariableMap = new HashMap<String, LogicalVariable>();
+
+ /**
+ * map field name to LogicalVariable
+ */
+ private HashMap<String, LogicalVariable> fieldToLogicalVariableMap = new HashMap<String, LogicalVariable>();
+
+ /**
+ * map logical variable to name
+ */
+ private HashMap<LogicalVariable, String> logicalVariableToFieldMap = new HashMap<LogicalVariable, String>();
+
+ /**
+ * asterix root operators
+ */
+ private List<Mutable<ILogicalOperator>> rootOperators = new ArrayList<Mutable<ILogicalOperator>>();
+
+ /**
+ * a list of visitors
+ */
+ private List<Visitor> visitors = new ArrayList<Visitor>();
+
+ /**
+ * output writer to print things out
+ */
+ private static PrintWriter outputWriter = new PrintWriter(new OutputStreamWriter(System.out));
+
+ /**
+ * map a logical variable to type info
+ */
+ private HashMap<LogicalVariable, TypeInfo> variableToType = new HashMap<LogicalVariable, TypeInfo>();
+
+ @Override
+ public LogicalVariable getVariable(String fieldName, TypeInfo type) {
+ LogicalVariable var = fieldToLogicalVariableMap.get(fieldName);
+ if (var == null) {
+ currentVariable++;
+ var = new LogicalVariable(currentVariable);
+ fieldToLogicalVariableMap.put(fieldName, var);
+ nameToLogicalVariableMap.put(var.toString(), var);
+ variableToType.put(var, type);
+ logicalVariableToFieldMap.put(var, fieldName);
+ }
+ return var;
+ }
+
+ @Override
+ public LogicalVariable getNewVariable(String fieldName, TypeInfo type) {
+ currentVariable++;
+ LogicalVariable var = new LogicalVariable(currentVariable);
+ fieldToLogicalVariableMap.put(fieldName, var);
+ nameToLogicalVariableMap.put(var.toString(), var);
+ variableToType.put(var, type);
+ logicalVariableToFieldMap.put(var, fieldName);
+ return var;
+ }
+
+ @Override
+ public void replaceVariable(LogicalVariable oldVar, LogicalVariable newVar) {
+ String name = this.logicalVariableToFieldMap.get(oldVar);
+ if (name != null) {
+ fieldToLogicalVariableMap.put(name, newVar);
+ nameToLogicalVariableMap.put(newVar.toString(), newVar);
+ nameToLogicalVariableMap.put(oldVar.toString(), newVar);
+ logicalVariableToFieldMap.put(newVar, name);
+ }
+ }
+
+ @Override
+ public IMetadataProvider<PartitionDesc, Object> getMetadataProvider() {
+ return metaData;
+ }
+
+ /**
+ * only get an variable, without rewriting it
+ *
+ * @param fieldName
+ * @return
+ */
+ private LogicalVariable getVariableOnly(String fieldName) {
+ return fieldToLogicalVariableMap.get(fieldName);
+ }
+
+ private void updateVariable(String fieldName, LogicalVariable variable) {
+ LogicalVariable var = fieldToLogicalVariableMap.get(fieldName);
+ if (var == null) {
+ fieldToLogicalVariableMap.put(fieldName, variable);
+ nameToLogicalVariableMap.put(fieldName, variable);
+ } else if (!var.equals(variable)) {
+ // System.out.println("!!!replace variables!!!");
+ fieldToLogicalVariableMap.put(fieldName, variable);
+ nameToLogicalVariableMap.put(fieldName, variable);
+ }
+ }
+
+ /**
+ * get a list of logical variables from the schema
+ *
+ * @param schema
+ * @return
+ */
+ @Override
+ public List<LogicalVariable> getVariablesFromSchema(Schema schema) {
+ List<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+ List<String> names = schema.getNames();
+
+ for (String name : names)
+ variables.add(nameToLogicalVariableMap.get(name));
+ return variables;
+ }
+
+ /**
+ * get variable to typeinfo map
+ *
+ * @return
+ */
+ public HashMap<LogicalVariable, TypeInfo> getVariableContext() {
+ return this.variableToType;
+ }
+
+ /**
+ * get the number of variables
+ *
+ * @return
+ */
+ public int getVariableCounter() {
+ return currentVariable + 1;
+ }
+
+ /**
+ * translate from hive operator tree to asterix operator tree
+ *
+ * @param hive
+ * roots
+ * @return Algebricks roots
+ */
+ public void translate(List<Operator> hiveRoot, ILogicalOperator parentOperator,
+ HashMap<String, PartitionDesc> aliasToPathMap) throws AlgebricksException {
+ /**
+ * register visitors
+ */
+ visitors.add(new FilterVisitor());
+ visitors.add(new GroupByVisitor());
+ visitors.add(new JoinVisitor());
+ visitors.add(new LateralViewJoinVisitor());
+ visitors.add(new UnionVisitor());
+ visitors.add(new LimitVisitor());
+ visitors.add(new MapJoinVisitor());
+ visitors.add(new ProjectVisitor());
+ visitors.add(new SortVisitor());
+ visitors.add(new ExtractVisitor());
+ visitors.add(new TableScanWriteVisitor(aliasToPathMap));
+
+ List<Mutable<ILogicalOperator>> refList = translate(hiveRoot, new MutableObject<ILogicalOperator>(
+ parentOperator));
+ insertReplicateOperator(refList);
+ if (refList != null)
+ rootOperators.addAll(refList);
+ }
+
+ /**
+ * translate operator DAG
+ *
+ * @param hiveRoot
+ * @param AlgebricksParentOperator
+ * @return
+ */
+ private List<Mutable<ILogicalOperator>> translate(List<Operator> hiveRoot,
+ Mutable<ILogicalOperator> AlgebricksParentOperator) throws AlgebricksException {
+
+ for (Operator hiveOperator : hiveRoot) {
+ continueTraverse = true;
+ Mutable<ILogicalOperator> currentOperatorRef = null;
+ if (hiveOperator.getType() == OperatorType.FILTER) {
+ FilterOperator fop = (FilterOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.REDUCESINK) {
+ ReduceSinkOperator fop = (ReduceSinkOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.JOIN) {
+ JoinOperator fop = (JoinOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null) {
+ continueTraverse = true;
+ break;
+ } else
+ continueTraverse = false;
+ }
+ if (currentOperatorRef == null)
+ return null;
+ } else if (hiveOperator.getType() == OperatorType.LATERALVIEWJOIN) {
+ LateralViewJoinOperator fop = (LateralViewJoinOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ if (currentOperatorRef == null)
+ return null;
+ } else if (hiveOperator.getType() == OperatorType.MAPJOIN) {
+ MapJoinOperator fop = (MapJoinOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null) {
+ continueTraverse = true;
+ break;
+ } else
+ continueTraverse = false;
+ }
+ if (currentOperatorRef == null)
+ return null;
+ } else if (hiveOperator.getType() == OperatorType.SELECT) {
+ SelectOperator fop = (SelectOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.EXTRACT) {
+ ExtractOperator fop = (ExtractOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.GROUPBY) {
+ GroupByOperator fop = (GroupByOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.TABLESCAN) {
+ TableScanOperator fop = (TableScanOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.FILESINK) {
+ FileSinkOperator fop = (FileSinkOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(fop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.LIMIT) {
+ LimitOperator lop = (LimitOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(lop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.UDTF) {
+ UDTFOperator lop = (UDTFOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(lop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null)
+ break;
+ }
+ } else if (hiveOperator.getType() == OperatorType.UNION) {
+ UnionOperator lop = (UnionOperator) hiveOperator;
+ for (Visitor visitor : visitors) {
+ currentOperatorRef = visitor.visit(lop, AlgebricksParentOperator, this);
+ if (currentOperatorRef != null) {
+ continueTraverse = true;
+ break;
+ } else
+ continueTraverse = false;
+ }
+ } else
+ ;
+ if (hiveOperator.getChildOperators() != null && hiveOperator.getChildOperators().size() > 0
+ && continueTraverse) {
+ List<Operator> children = hiveOperator.getChildOperators();
+ if (currentOperatorRef == null)
+ currentOperatorRef = AlgebricksParentOperator;
+ translate(children, currentOperatorRef);
+ }
+ if (hiveOperator.getChildOperators() == null || hiveOperator.getChildOperators().size() == 0)
+ logicalOp.add(currentOperatorRef);
+ }
+ return logicalOp;
+ }
+
+ /**
+ * used in select, group by to get no-column-expression columns
+ *
+ * @param cols
+ * @return
+ */
+ public ILogicalOperator getAssignOperator(Mutable<ILogicalOperator> parent, List<ExprNodeDesc> cols,
+ ArrayList<LogicalVariable> variables) {
+
+ ArrayList<Mutable<ILogicalExpression>> expressions = new ArrayList<Mutable<ILogicalExpression>>();
+
+ /**
+ * variables to be appended in the assign operator
+ */
+ ArrayList<LogicalVariable> appendedVariables = new ArrayList<LogicalVariable>();
+
+ // one variable can only be assigned once
+ for (ExprNodeDesc hiveExpr : cols) {
+ rewriteExpression(hiveExpr);
+
+ if (hiveExpr instanceof ExprNodeColumnDesc) {
+ ExprNodeColumnDesc desc2 = (ExprNodeColumnDesc) hiveExpr;
+ String fieldName = desc2.getTabAlias() + "." + desc2.getColumn();
+
+ // System.out.println("project expr: " + fieldName);
+
+ if (fieldName.indexOf("$$") < 0) {
+ LogicalVariable var = getVariable(fieldName, hiveExpr.getTypeInfo());
+ desc2.setColumn(var.toString());
+ desc2.setTabAlias("");
+ variables.add(var);
+ } else {
+ LogicalVariable var = nameToLogicalVariableMap.get(desc2.getColumn());
+ String name = this.logicalVariableToFieldMap.get(var);
+ var = this.getVariableOnly(name);
+ variables.add(var);
+ }
+ } else {
+ Mutable<ILogicalExpression> asterixExpr = translateScalarFucntion(hiveExpr);
+ expressions.add(asterixExpr);
+ LogicalVariable var = getVariable(hiveExpr.getExprString() + asterixExpr.hashCode(),
+ hiveExpr.getTypeInfo());
+ variables.add(var);
+ appendedVariables.add(var);
+ }
+ }
+
+ /**
+ * create an assign operator to deal with appending
+ */
+ ILogicalOperator assignOp = null;
+ if (appendedVariables.size() > 0) {
+ assignOp = new AssignOperator(appendedVariables, expressions);
+ assignOp.getInputs().add(parent);
+ }
+ return assignOp;
+ }
+
+ private ILogicalPlan plan;
+
+ public ILogicalPlan genLogicalPlan() {
+ plan = new ALogicalPlanImpl(rootOperators);
+ return plan;
+ }
+
+ public void printOperators() throws AlgebricksException {
+ LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
+ StringBuilder buffer = new StringBuilder();
+ PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
+ outputWriter.println(buffer);
+ outputWriter.println("rewritten variables: ");
+ outputWriter.flush();
+ printVariables();
+
+ }
+
+ public static void setOutputPrinter(PrintWriter writer) {
+ outputWriter = writer;
+ }
+
+ private void printVariables() {
+ Set<Entry<String, LogicalVariable>> entries = fieldToLogicalVariableMap.entrySet();
+
+ for (Entry<String, LogicalVariable> entry : entries) {
+ outputWriter.println(entry.getKey() + " -> " + entry.getValue());
+ }
+ outputWriter.flush();
+ }
+
+ /**
+ * generate the object inspector for the output of an operator
+ *
+ * @param operator
+ * The Hive operator
+ * @return an ObjectInspector object
+ */
+ public Schema generateInputSchema(Operator operator) {
+ List<String> variableNames = new ArrayList<String>();
+ List<TypeInfo> typeList = new ArrayList<TypeInfo>();
+ List<ColumnInfo> columns = operator.getSchema().getSignature();
+
+ for (ColumnInfo col : columns) {
+ // typeList.add();
+ TypeInfo type = col.getType();
+ typeList.add(type);
+
+ String fieldName = col.getInternalName();
+ variableNames.add(fieldName);
+ }
+
+ return new Schema(variableNames, typeList);
+ }
+
+ /**
+ * rewrite the names of output columns for feature expression evaluators to
+ * use
+ *
+ * @param operator
+ */
+ public void rewriteOperatorOutputSchema(Operator operator) {
+ List<ColumnInfo> columns = operator.getSchema().getSignature();
+
+ for (ColumnInfo column : columns) {
+ String columnName = column.getTabAlias() + "." + column.getInternalName();
+ if (columnName.indexOf("$$") < 0) {
+ LogicalVariable var = getVariable(columnName, column.getType());
+ column.setInternalName(var.toString());
+ }
+ }
+ }
+
+ @Override
+ public void rewriteOperatorOutputSchema(List<LogicalVariable> variables, Operator operator) {
+
+ printOperatorSchema(operator);
+ List<ColumnInfo> columns = operator.getSchema().getSignature();
+ if (variables.size() != columns.size()) {
+ System.out.println("output cardinality error " + operator.getName() + " variable size: " + variables.size()
+ + " expected " + columns.size());
+ }
+
+ for (int i = 0; i < variables.size(); i++) {
+ LogicalVariable var = variables.get(i);
+ ColumnInfo column = columns.get(i);
+ String fieldName = column.getTabAlias() + "." + column.getInternalName();
+ if (fieldName.indexOf("$$") < 0) {
+ updateVariable(fieldName, var);
+ column.setInternalName(var.toString());
+ }
+ }
+ printOperatorSchema(operator);
+ }
+
+ /**
+ * rewrite an expression and substitute variables
+ *
+ * @param expr
+ * hive expression
+ */
+ public void rewriteExpression(ExprNodeDesc expr) {
+ if (expr instanceof ExprNodeColumnDesc) {
+ ExprNodeColumnDesc desc = (ExprNodeColumnDesc) expr;
+ String fieldName = desc.getTabAlias() + "." + desc.getColumn();
+ if (fieldName.indexOf("$$") < 0) {
+ LogicalVariable var = getVariableOnly(fieldName);
+ if (var == null) {
+ fieldName = "." + desc.getColumn();
+ var = getVariableOnly(fieldName);
+ if (var == null) {
+ fieldName = "null." + desc.getColumn();
+ var = getVariableOnly(fieldName);
+ if (var == null) {
+ System.out.println(fieldName + " is wrong!!! ");
+ }
+ }
+ }
+ String name = this.logicalVariableToFieldMap.get(var);
+ var = getVariableOnly(name);
+ desc.setColumn(var.toString());
+ }
+ } else {
+ if (expr.getChildren() != null && expr.getChildren().size() > 0) {
+ List<ExprNodeDesc> children = expr.getChildren();
+ for (ExprNodeDesc desc : children)
+ rewriteExpression(desc);
+ }
+ }
+ }
+
+ /**
+ * rewrite an expression and substitute variables
+ *
+ * @param expr
+ * hive expression
+ */
+ public void rewriteExpressionPartial(ExprNodeDesc expr) {
+ if (expr instanceof ExprNodeColumnDesc) {
+ ExprNodeColumnDesc desc = (ExprNodeColumnDesc) expr;
+ String fieldName = desc.getTabAlias() + "." + desc.getColumn();
+ if (fieldName.indexOf("$$") < 0) {
+ LogicalVariable var = getVariableOnly(fieldName);
+ desc.setColumn(var.toString());
+ }
+ } else {
+ if (expr.getChildren() != null && expr.getChildren().size() > 0) {
+ List<ExprNodeDesc> children = expr.getChildren();
+ for (ExprNodeDesc desc : children)
+ rewriteExpressionPartial(desc);
+ }
+ }
+ }
+
+ private void printOperatorSchema(Operator operator) {
+ System.out.println(operator.getName());
+ List<ColumnInfo> columns = operator.getSchema().getSignature();
+ for (ColumnInfo column : columns) {
+ System.out.print(column.getTabAlias() + "." + column.getInternalName() + " ");
+ }
+ System.out.println();
+ }
+
+ /**
+ * translate scalar function expression
+ *
+ * @param hiveExpr
+ * @return
+ */
+ public Mutable<ILogicalExpression> translateScalarFucntion(ExprNodeDesc hiveExpr) {
+ ILogicalExpression AlgebricksExpr;
+
+ if (hiveExpr instanceof ExprNodeGenericFuncDesc) {
+ List<Mutable<ILogicalExpression>> arguments = new ArrayList<Mutable<ILogicalExpression>>();
+ List<ExprNodeDesc> children = hiveExpr.getChildren();
+
+ for (ExprNodeDesc child : children)
+ arguments.add(translateScalarFucntion(child));
+
+ ExprNodeGenericFuncDesc funcExpr = (ExprNodeGenericFuncDesc) hiveExpr;
+ GenericUDF genericUdf = funcExpr.getGenericUDF();
+ UDF udf = null;
+ if (genericUdf instanceof GenericUDFBridge) {
+ GenericUDFBridge bridge = (GenericUDFBridge) genericUdf;
+ try {
+ udf = bridge.getUdfClass().newInstance();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * set up the hive function
+ */
+ Object hiveFunction = genericUdf;
+ if (udf != null)
+ hiveFunction = udf;
+
+ FunctionIdentifier funcId = HiveAlgebricksBuiltInFunctionMap.INSTANCE.getAlgebricksFunctionId(hiveFunction
+ .getClass());
+ if (funcId == null) {
+ funcId = new FunctionIdentifier(ExpressionConstant.NAMESPACE, hiveFunction.getClass().getName());
+ }
+
+ Object functionInfo = null;
+ if (genericUdf instanceof GenericUDFBridge) {
+ functionInfo = funcExpr;
+ }
+
+ /**
+ * generate the function call expression
+ */
+ ScalarFunctionCallExpression AlgebricksFuncExpr = new ScalarFunctionCallExpression(new HiveFunctionInfo(
+ funcId, functionInfo), arguments);
+ AlgebricksExpr = AlgebricksFuncExpr;
+
+ } else if (hiveExpr instanceof ExprNodeColumnDesc) {
+ ExprNodeColumnDesc column = (ExprNodeColumnDesc) hiveExpr;
+ LogicalVariable var = this.getVariable(column.getColumn());
+ AlgebricksExpr = new VariableReferenceExpression(var);
+
+ } else if (hiveExpr instanceof ExprNodeFieldDesc) {
+ FunctionIdentifier funcId;
+ funcId = new FunctionIdentifier(ExpressionConstant.NAMESPACE, ExpressionConstant.FIELDACCESS);
+
+ ScalarFunctionCallExpression AlgebricksFuncExpr = new ScalarFunctionCallExpression(new HiveFunctionInfo(
+ funcId, hiveExpr));
+ AlgebricksExpr = AlgebricksFuncExpr;
+ } else if (hiveExpr instanceof ExprNodeConstantDesc) {
+ ExprNodeConstantDesc hiveConst = (ExprNodeConstantDesc) hiveExpr;
+ Object value = hiveConst.getValue();
+ AlgebricksExpr = new ConstantExpression(new HivesterixConstantValue(value));
+ } else if (hiveExpr instanceof ExprNodeNullDesc) {
+ FunctionIdentifier funcId;
+ funcId = new FunctionIdentifier(ExpressionConstant.NAMESPACE, ExpressionConstant.NULL);
+
+ ScalarFunctionCallExpression AlgebricksFuncExpr = new ScalarFunctionCallExpression(new HiveFunctionInfo(
+ funcId, hiveExpr));
+
+ AlgebricksExpr = AlgebricksFuncExpr;
+ } else {
+ throw new IllegalStateException("unknown hive expression");
+ }
+ return new MutableObject<ILogicalExpression>(AlgebricksExpr);
+ }
+
+ /**
+ * translate aggregation function expression
+ *
+ * @param aggregateDesc
+ * @return
+ */
+ public Mutable<ILogicalExpression> translateAggregation(AggregationDesc aggregateDesc) {
+
+ String UDAFName = aggregateDesc.getGenericUDAFName();
+
+ List<Mutable<ILogicalExpression>> arguments = new ArrayList<Mutable<ILogicalExpression>>();
+ List<ExprNodeDesc> children = aggregateDesc.getParameters();
+
+ for (ExprNodeDesc child : children)
+ arguments.add(translateScalarFucntion(child));
+
+ FunctionIdentifier funcId = new FunctionIdentifier(ExpressionConstant.NAMESPACE, UDAFName + "("
+ + aggregateDesc.getMode() + ")");
+ HiveFunctionInfo funcInfo = new HiveFunctionInfo(funcId, aggregateDesc);
+ AggregateFunctionCallExpression aggregationExpression = new AggregateFunctionCallExpression(funcInfo, false,
+ arguments);
+ return new MutableObject<ILogicalExpression>(aggregationExpression);
+ }
+
+ /**
+ * translate aggregation function expression
+ *
+ * @param aggregator
+ * @return
+ */
+ public Mutable<ILogicalExpression> translateUnnestFunction(UDTFDesc udtfDesc, Mutable<ILogicalExpression> argument) {
+
+ String UDTFName = udtfDesc.getUDTFName();
+
+ FunctionIdentifier funcId = new FunctionIdentifier(ExpressionConstant.NAMESPACE, UDTFName);
+ UnnestingFunctionCallExpression unnestingExpression = new UnnestingFunctionCallExpression(new HiveFunctionInfo(
+ funcId, udtfDesc));
+ unnestingExpression.getArguments().add(argument);
+ return new MutableObject<ILogicalExpression>(unnestingExpression);
+ }
+
+ /**
+ * get typeinfo
+ */
+ @Override
+ public TypeInfo getType(LogicalVariable var) {
+ return variableToType.get(var);
+ }
+
+ /**
+ * get variable from variable name
+ */
+ @Override
+ public LogicalVariable getVariable(String name) {
+ return nameToLogicalVariableMap.get(name);
+ }
+
+ @Override
+ public LogicalVariable getVariableFromFieldName(String fieldName) {
+ return this.getVariableOnly(fieldName);
+ }
+
+ /**
+ * set the metadata provider
+ */
+ @Override
+ public void setMetadataProvider(IMetadataProvider<PartitionDesc, Object> metadata) {
+ this.metaData = metadata;
+ }
+
+ /**
+ * insert ReplicateOperator when necessary
+ */
+ private void insertReplicateOperator(List<Mutable<ILogicalOperator>> roots) {
+ Map<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>> childToParentsMap = new HashMap<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>>();
+ buildChildToParentsMapping(roots, childToParentsMap);
+ for (Entry<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>> entry : childToParentsMap.entrySet()) {
+ List<Mutable<ILogicalOperator>> pList = entry.getValue();
+ if (pList.size() > 1) {
+ ILogicalOperator rop = new ReplicateOperator(pList.size());
+ Mutable<ILogicalOperator> ropRef = new MutableObject<ILogicalOperator>(rop);
+ Mutable<ILogicalOperator> childRef = entry.getKey();
+ rop.getInputs().add(childRef);
+ for (Mutable<ILogicalOperator> parentRef : pList) {
+ ILogicalOperator parentOp = parentRef.getValue();
+ int index = parentOp.getInputs().indexOf(childRef);
+ parentOp.getInputs().set(index, ropRef);
+ }
+ }
+ }
+ }
+
+ /**
+ * build the mapping from child to Parents
+ *
+ * @param roots
+ * @param childToParentsMap
+ */
+ private void buildChildToParentsMapping(List<Mutable<ILogicalOperator>> roots,
+ Map<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>> map) {
+ for (Mutable<ILogicalOperator> opRef : roots) {
+ List<Mutable<ILogicalOperator>> childRefs = opRef.getValue().getInputs();
+ for (Mutable<ILogicalOperator> childRef : childRefs) {
+ List<Mutable<ILogicalOperator>> parentList = map.get(childRef);
+ if (parentList == null) {
+ parentList = new ArrayList<Mutable<ILogicalOperator>>();
+ map.put(childRef, parentList);
+ }
+ if (!parentList.contains(opRef))
+ parentList.add(opRef);
+ }
+ buildChildToParentsMapping(childRefs, map);
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveLogicalPlanAndMetaData.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveLogicalPlanAndMetaData.java
new file mode 100644
index 0000000..a34371a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveLogicalPlanAndMetaData.java
@@ -0,0 +1,34 @@
+package edu.uci.ics.hivesterix.logical.plan;
+
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class HiveLogicalPlanAndMetaData implements ILogicalPlanAndMetadata {
+
+ IMetadataProvider metadata;
+ ILogicalPlan plan;
+
+ public HiveLogicalPlanAndMetaData(ILogicalPlan plan, IMetadataProvider metadata) {
+ this.plan = plan;
+ this.metadata = metadata;
+ }
+
+ @Override
+ public IMetadataProvider getMetadataProvider() {
+ return metadata;
+ }
+
+ @Override
+ public ILogicalPlan getPlan() {
+ return plan;
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getClusterLocations() {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveOperatorAnnotations.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveOperatorAnnotations.java
new file mode 100644
index 0000000..bf59900
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveOperatorAnnotations.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.hivesterix.logical.plan;
+
+public class HiveOperatorAnnotations {
+
+ // hints
+ public static final String LOCAL_GROUP_BY = "LOCAL_GROUP_BY";
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ExtractVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ExtractVisitor.java
new file mode 100644
index 0000000..6b0aacd
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ExtractVisitor.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hadoop.hive.ql.exec.ExtractOperator;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public class ExtractVisitor extends DefaultVisitor {
+
+ @Override
+ public Mutable<ILogicalOperator> visit(ExtractOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) {
+ Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));
+ operator.setSchema(operator.getParentOperators().get(0).getSchema());
+ List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);
+ t.rewriteOperatorOutputSchema(latestOutputSchema, operator);
+ return null;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
new file mode 100644
index 0000000..529fc09
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.FilterDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+
+public class FilterVisitor extends DefaultVisitor {
+
+ @Override
+ public Mutable<ILogicalOperator> visit(FilterOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) {
+ Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));
+
+ FilterDesc desc = (FilterDesc) operator.getConf();
+ ExprNodeDesc predicate = desc.getPredicate();
+ t.rewriteExpression(predicate);
+
+ Mutable<ILogicalExpression> exprs = t.translateScalarFucntion(desc.getPredicate());
+ ILogicalOperator currentOperator = new SelectOperator(exprs);
+ currentOperator.getInputs().add(AlgebricksParentOperatorRef);
+
+ // populate the schema from upstream operator
+ operator.setSchema(operator.getParentOperators().get(0).getSchema());
+ List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);
+ t.rewriteOperatorOutputSchema(latestOutputSchema, operator);
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/GroupByVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/GroupByVisitor.java
new file mode 100644
index 0000000..4f6a31f
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/GroupByVisitor.java
@@ -0,0 +1,263 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.GroupByOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.GroupByDesc;
+import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
+import org.apache.hadoop.hive.ql.plan.api.OperatorType;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.plan.HiveOperatorAnnotations;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+
+public class GroupByVisitor extends DefaultVisitor {
+
+ private List<Mutable<ILogicalExpression>> AlgebricksAggs = new ArrayList<Mutable<ILogicalExpression>>();
+ private List<IFunctionInfo> localAggs = new ArrayList<IFunctionInfo>();
+ private boolean isDistinct = false;
+ private boolean gbyKeyNotRedKey = false;
+
+ @Override
+ public Mutable<ILogicalOperator> visit(GroupByOperator operator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException {
+
+ // get descriptors
+ GroupByDesc desc = (GroupByDesc) operator.getConf();
+ GroupByDesc.Mode mode = desc.getMode();
+
+ List<ExprNodeDesc> keys = desc.getKeys();
+ List<AggregationDesc> aggregators = desc.getAggregators();
+
+ Operator child = operator.getChildOperators().get(0);
+
+ if (child.getType() == OperatorType.REDUCESINK) {
+ List<ExprNodeDesc> partKeys = ((ReduceSinkDesc) child.getConf()).getPartitionCols();
+ if (keys.size() != partKeys.size())
+ gbyKeyNotRedKey = true;
+ }
+
+ if (mode == GroupByDesc.Mode.PARTIAL1 || mode == GroupByDesc.Mode.HASH || mode == GroupByDesc.Mode.COMPLETE
+ || (aggregators.size() == 0 && isDistinct == false) || gbyKeyNotRedKey) {
+ AlgebricksAggs.clear();
+ // add an assign operator if the key is not a column expression
+ ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();
+ ILogicalOperator currentOperator = null;
+ ILogicalOperator assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, keys, keyVariables);
+ if (assignOperator != null) {
+ currentOperator = assignOperator;
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ // get key variable expression list
+ List<Mutable<ILogicalExpression>> keyExprs = new ArrayList<Mutable<ILogicalExpression>>();
+ for (LogicalVariable var : keyVariables) {
+ keyExprs.add(t.translateScalarFucntion(new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, var
+ .toString(), "", false)));
+ }
+
+ if (aggregators.size() == 0) {
+ List<Mutable<ILogicalExpression>> distinctExprs = new ArrayList<Mutable<ILogicalExpression>>();
+ for (LogicalVariable var : keyVariables) {
+ Mutable<ILogicalExpression> varExpr = new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(var));
+ distinctExprs.add(varExpr);
+ }
+ t.rewriteOperatorOutputSchema(keyVariables, operator);
+ isDistinct = true;
+ ILogicalOperator lop = new DistinctOperator(distinctExprs);
+ lop.getInputs().add(AlgebricksParentOperatorRef);
+ return new MutableObject<ILogicalOperator>(lop);
+ }
+
+ // get the pair<LogicalVariable, ILogicalExpression> list
+ List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> keyParameters = new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>();
+ keyVariables.clear();
+ for (Mutable<ILogicalExpression> expr : keyExprs) {
+ LogicalVariable keyVar = t.getVariable(expr.getValue().toString(), TypeInfoFactory.unknownTypeInfo);
+ keyParameters.add(new Pair(keyVar, expr));
+ keyVariables.add(keyVar);
+ }
+
+ // get the parameters for the aggregator operator
+ ArrayList<LogicalVariable> aggVariables = new ArrayList<LogicalVariable>();
+ ArrayList<Mutable<ILogicalExpression>> aggExprs = new ArrayList<Mutable<ILogicalExpression>>();
+
+ // get the type of each aggregation function
+ HashMap<AggregationDesc, TypeInfo> aggToType = new HashMap<AggregationDesc, TypeInfo>();
+ List<ColumnInfo> columns = operator.getSchema().getSignature();
+ int offset = keys.size();
+ for (int i = offset; i < columns.size(); i++) {
+ aggToType.put(aggregators.get(i - offset), columns.get(i).getType());
+ }
+
+ localAggs.clear();
+ // rewrite parameter expressions for all aggregators
+ for (AggregationDesc aggregator : aggregators) {
+ for (ExprNodeDesc parameter : aggregator.getParameters()) {
+ t.rewriteExpression(parameter);
+ }
+ Mutable<ILogicalExpression> aggExpr = t.translateAggregation(aggregator);
+ AbstractFunctionCallExpression localAggExpr = (AbstractFunctionCallExpression) aggExpr.getValue();
+ localAggs.add(localAggExpr.getFunctionInfo());
+
+ AggregationDesc logicalAgg = new AggregationDesc(aggregator.getGenericUDAFName(),
+ aggregator.getGenericUDAFEvaluator(), aggregator.getParameters(), aggregator.getDistinct(),
+ Mode.COMPLETE);
+ Mutable<ILogicalExpression> logicalAggExpr = t.translateAggregation(logicalAgg);
+
+ AlgebricksAggs.add(logicalAggExpr);
+ if (!gbyKeyNotRedKey)
+ aggExprs.add(logicalAggExpr);
+ else
+ aggExprs.add(aggExpr);
+
+ aggVariables.add(t.getVariable(aggregator.getExprString() + aggregator.getMode(),
+ aggToType.get(aggregator)));
+ }
+
+ if (child.getType() != OperatorType.REDUCESINK)
+ gbyKeyNotRedKey = false;
+
+ // get the sub plan list
+ AggregateOperator aggOperator = new AggregateOperator(aggVariables, aggExprs);
+ NestedTupleSourceOperator nestedTupleSource = new NestedTupleSourceOperator(
+ new MutableObject<ILogicalOperator>());
+ aggOperator.getInputs().add(new MutableObject<ILogicalOperator>(nestedTupleSource));
+
+ List<Mutable<ILogicalOperator>> subRoots = new ArrayList<Mutable<ILogicalOperator>>();
+ subRoots.add(new MutableObject<ILogicalOperator>(aggOperator));
+ ILogicalPlan subPlan = new ALogicalPlanImpl(subRoots);
+ List<ILogicalPlan> subPlans = new ArrayList<ILogicalPlan>();
+ subPlans.add(subPlan);
+
+ // create the group by operator
+ currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator(
+ keyParameters, new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>(), subPlans);
+ currentOperator.getInputs().add(AlgebricksParentOperatorRef);
+ nestedTupleSource.getDataSourceReference().setValue(currentOperator);
+
+ List<LogicalVariable> outputVariables = new ArrayList<LogicalVariable>();
+ outputVariables.addAll(keyVariables);
+ outputVariables.addAll(aggVariables);
+ t.rewriteOperatorOutputSchema(outputVariables, operator);
+
+ if (gbyKeyNotRedKey) {
+ currentOperator.getAnnotations().put(HiveOperatorAnnotations.LOCAL_GROUP_BY, Boolean.TRUE);
+ }
+
+ HiveConf conf = ConfUtil.getHiveConf();
+ Boolean extGby = conf.getBoolean("hive.algebricks.groupby.external", false);
+
+ if (extGby && isSerializable(aggregators)) {
+ currentOperator.getAnnotations().put(OperatorAnnotations.USE_EXTERNAL_GROUP_BY, Boolean.TRUE);
+ }
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ } else {
+ isDistinct = false;
+ // rewrite parameter expressions for all aggregators
+ int i = 0;
+ for (AggregationDesc aggregator : aggregators) {
+ for (ExprNodeDesc parameter : aggregator.getParameters()) {
+ t.rewriteExpression(parameter);
+ }
+ Mutable<ILogicalExpression> agg = t.translateAggregation(aggregator);
+ AggregateFunctionCallExpression originalAgg = (AggregateFunctionCallExpression) AlgebricksAggs.get(i)
+ .getValue();
+ originalAgg.setStepOneAggregate(localAggs.get(i));
+ AggregateFunctionCallExpression currentAgg = (AggregateFunctionCallExpression) agg.getValue();
+ if (currentAgg.getFunctionInfo() != null) {
+ originalAgg.setTwoStep(true);
+ originalAgg.setStepTwoAggregate(currentAgg.getFunctionInfo());
+ }
+ i++;
+ }
+ return null;
+ }
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {
+ Operator downStream = (Operator) operator.getChildOperators().get(0);
+ if (!(downStream instanceof GroupByOperator)) {
+ return null;
+ }
+
+ ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();
+ List<ExprNodeDesc> keys = desc.getKeyCols();
+ List<ExprNodeDesc> values = desc.getValueCols();
+
+ // insert assign for keys
+ ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();
+ t.getAssignOperator(AlgebricksParentOperatorRef, keys, keyVariables);
+
+ // insert assign for values
+ ArrayList<LogicalVariable> valueVariables = new ArrayList<LogicalVariable>();
+ t.getAssignOperator(AlgebricksParentOperatorRef, values, valueVariables);
+
+ ArrayList<LogicalVariable> columns = new ArrayList<LogicalVariable>();
+ columns.addAll(keyVariables);
+ columns.addAll(valueVariables);
+
+ t.rewriteOperatorOutputSchema(columns, operator);
+ return null;
+ }
+
+ private boolean isSerializable(List<AggregationDesc> descs) throws AlgebricksException {
+ try {
+ for (AggregationDesc desc : descs) {
+ GenericUDAFEvaluator udaf = desc.getGenericUDAFEvaluator();
+ AggregationBuffer buf = udaf.getNewAggregationBuffer();
+ Class<?> bufferClass = buf.getClass();
+ Field[] fields = bufferClass.getDeclaredFields();
+ for (Field field : fields) {
+ field.setAccessible(true);
+ String type = field.getType().toString();
+ if (!(type.equals("int") || type.equals("long") || type.equals("float") || type.equals("double") || type
+ .equals("boolean"))) {
+ return false;
+ }
+ }
+
+ }
+ return true;
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/JoinVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/JoinVisitor.java
new file mode 100644
index 0000000..41a492d
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/JoinVisitor.java
@@ -0,0 +1,414 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.JoinOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
+import org.apache.hadoop.hive.ql.plan.JoinDesc;
+import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+
+public class JoinVisitor extends DefaultVisitor {
+
+ /**
+ * reduce sink operator to variables
+ */
+ private HashMap<Operator, List<LogicalVariable>> reduceSinkToKeyVariables = new HashMap<Operator, List<LogicalVariable>>();
+
+ /**
+ * reduce sink operator to variables
+ */
+ private HashMap<Operator, List<String>> reduceSinkToFieldNames = new HashMap<Operator, List<String>>();
+
+ /**
+ * reduce sink operator to variables
+ */
+ private HashMap<Operator, List<TypeInfo>> reduceSinkToTypes = new HashMap<Operator, List<TypeInfo>>();
+
+ /**
+ * map a join operator (in hive) to its parent operators (in hive)
+ */
+ private HashMap<Operator, List<Operator>> operatorToHiveParents = new HashMap<Operator, List<Operator>>();
+
+ /**
+ * map a join operator (in hive) to its parent operators (in asterix)
+ */
+ private HashMap<Operator, List<ILogicalOperator>> operatorToAsterixParents = new HashMap<Operator, List<ILogicalOperator>>();
+
+ /**
+ * the latest traversed reduce sink operator
+ */
+ private Operator latestReduceSink = null;
+
+ /**
+ * the latest generated parent for join
+ */
+ private ILogicalOperator latestAlgebricksOperator = null;
+
+ /**
+ * process a join operator
+ */
+ @Override
+ public Mutable<ILogicalOperator> visit(JoinOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) {
+ latestAlgebricksOperator = AlgebricksParentOperator.getValue();
+ translateJoinOperatorPreprocess(operator, t);
+ List<Operator> parents = operatorToHiveParents.get(operator);
+ if (parents.size() < operator.getParentOperators().size()) {
+ return null;
+ } else {
+ ILogicalOperator joinOp = translateJoinOperator(operator, AlgebricksParentOperator, t);
+ // clearStatus();
+ return new MutableObject<ILogicalOperator>(joinOp);
+ }
+ }
+
+ private void reorder(Byte[] order, List<ILogicalOperator> parents, List<Operator> hiveParents) {
+ ILogicalOperator[] lops = new ILogicalOperator[parents.size()];
+ Operator[] ops = new Operator[hiveParents.size()];
+
+ for (Operator op : hiveParents) {
+ ReduceSinkOperator rop = (ReduceSinkOperator) op;
+ ReduceSinkDesc rdesc = rop.getConf();
+ int tag = rdesc.getTag();
+
+ int index = -1;
+ for (int i = 0; i < order.length; i++)
+ if (order[i] == tag) {
+ index = i;
+ break;
+ }
+ lops[index] = parents.get(hiveParents.indexOf(op));
+ ops[index] = op;
+ }
+
+ parents.clear();
+ hiveParents.clear();
+
+ for (int i = 0; i < lops.length; i++) {
+ parents.add(lops[i]);
+ hiveParents.add(ops[i]);
+ }
+ }
+
+ /**
+ * translate a hive join operator to asterix join operator->assign
+ * operator->project operator
+ *
+ * @param parentOperator
+ * @param operator
+ * @return
+ */
+ private ILogicalOperator translateJoinOperator(Operator operator, Mutable<ILogicalOperator> parentOperator,
+ Translator t) {
+
+ JoinDesc joinDesc = (JoinDesc) operator.getConf();
+
+ // get the projection expression (already re-written) from each source
+ // table
+ Map<Byte, List<ExprNodeDesc>> exprMap = joinDesc.getExprs();
+ reorder(joinDesc.getTagOrder(), operatorToAsterixParents.get(operator), operatorToHiveParents.get(operator));
+
+ // make an reduce join operator
+ ILogicalOperator currentOperator = generateJoinTree(joinDesc.getCondsList(),
+ operatorToAsterixParents.get(operator), operatorToHiveParents.get(operator), 0, t);
+ parentOperator = new MutableObject<ILogicalOperator>(currentOperator);
+
+ // add assign and project operator on top of a join
+ // output variables
+ ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+ Set<Entry<Byte, List<ExprNodeDesc>>> entries = exprMap.entrySet();
+ Iterator<Entry<Byte, List<ExprNodeDesc>>> iterator = entries.iterator();
+ while (iterator.hasNext()) {
+ List<ExprNodeDesc> outputExprs = iterator.next().getValue();
+ ILogicalOperator assignOperator = t.getAssignOperator(parentOperator, outputExprs, variables);
+
+ if (assignOperator != null) {
+ currentOperator = assignOperator;
+ parentOperator = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+ }
+
+ ILogicalOperator po = new ProjectOperator(variables);
+ po.getInputs().add(parentOperator);
+ t.rewriteOperatorOutputSchema(variables, operator);
+ return po;
+ }
+
+ /**
+ * deal with reduce sink operator for the case of join
+ */
+ @Override
+ public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator, Mutable<ILogicalOperator> parentOperator,
+ Translator t) {
+
+ Operator downStream = (Operator) operator.getChildOperators().get(0);
+ if (!(downStream instanceof JoinOperator))
+ return null;
+
+ ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();
+ List<ExprNodeDesc> keys = desc.getKeyCols();
+ List<ExprNodeDesc> values = desc.getValueCols();
+ List<ExprNodeDesc> partitionCols = desc.getPartitionCols();
+
+ /**
+ * rewrite key, value, paritioncol expressions
+ */
+ for (ExprNodeDesc key : keys)
+ t.rewriteExpression(key);
+ for (ExprNodeDesc value : values)
+ t.rewriteExpression(value);
+ for (ExprNodeDesc col : partitionCols)
+ t.rewriteExpression(col);
+
+ ILogicalOperator currentOperator = null;
+
+ // add assign operator for keys if necessary
+ ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();
+ ILogicalOperator assignOperator = t.getAssignOperator(parentOperator, keys, keyVariables);
+ if (assignOperator != null) {
+ currentOperator = assignOperator;
+ parentOperator = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ // add assign operator for values if necessary
+ ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+ assignOperator = t.getAssignOperator(parentOperator, values, variables);
+ if (assignOperator != null) {
+ currentOperator = assignOperator;
+ parentOperator = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ // unified schema: key, value
+ ArrayList<LogicalVariable> unifiedKeyValues = new ArrayList<LogicalVariable>();
+ unifiedKeyValues.addAll(keyVariables);
+ for (LogicalVariable value : variables)
+ if (keyVariables.indexOf(value) < 0)
+ unifiedKeyValues.add(value);
+
+ // insert projection operator, it is a *must*,
+ // in hive, reduce sink sometimes also do the projection operator's
+ // task
+ currentOperator = new ProjectOperator(unifiedKeyValues);
+ currentOperator.getInputs().add(parentOperator);
+ parentOperator = new MutableObject<ILogicalOperator>(currentOperator);
+
+ reduceSinkToKeyVariables.put(operator, keyVariables);
+ List<String> fieldNames = new ArrayList<String>();
+ List<TypeInfo> types = new ArrayList<TypeInfo>();
+ for (LogicalVariable var : unifiedKeyValues) {
+ fieldNames.add(var.toString());
+ types.add(t.getType(var));
+ }
+ reduceSinkToFieldNames.put(operator, fieldNames);
+ reduceSinkToTypes.put(operator, types);
+ t.rewriteOperatorOutputSchema(variables, operator);
+
+ latestAlgebricksOperator = currentOperator;
+ latestReduceSink = operator;
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ /**
+ * partial rewrite a join operator
+ *
+ * @param operator
+ * @param t
+ */
+ private void translateJoinOperatorPreprocess(Operator operator, Translator t) {
+ JoinDesc desc = (JoinDesc) operator.getConf();
+ ReduceSinkDesc reduceSinkDesc = (ReduceSinkDesc) latestReduceSink.getConf();
+ int tag = reduceSinkDesc.getTag();
+
+ Map<Byte, List<ExprNodeDesc>> exprMap = desc.getExprs();
+ List<ExprNodeDesc> exprs = exprMap.get(Byte.valueOf((byte) tag));
+
+ for (ExprNodeDesc expr : exprs)
+ t.rewriteExpression(expr);
+
+ List<Operator> parents = operatorToHiveParents.get(operator);
+ if (parents == null) {
+ parents = new ArrayList<Operator>();
+ operatorToHiveParents.put(operator, parents);
+ }
+ parents.add(latestReduceSink);
+
+ List<ILogicalOperator> asterixParents = operatorToAsterixParents.get(operator);
+ if (asterixParents == null) {
+ asterixParents = new ArrayList<ILogicalOperator>();
+ operatorToAsterixParents.put(operator, asterixParents);
+ }
+ asterixParents.add(latestAlgebricksOperator);
+ }
+
+ // generate a join tree from a list of exchange/reducesink operator
+ // both exchanges and reduce sinks have the same order
+ private ILogicalOperator generateJoinTree(List<JoinCondDesc> conds, List<ILogicalOperator> exchanges,
+ List<Operator> reduceSinks, int offset, Translator t) {
+ // get a list of reduce sink descs (input descs)
+ int inputSize = reduceSinks.size() - offset;
+
+ if (inputSize == 2) {
+ ILogicalOperator currentRoot;
+
+ List<ReduceSinkDesc> reduceSinkDescs = new ArrayList<ReduceSinkDesc>();
+ for (int i = reduceSinks.size() - 1; i >= offset; i--)
+ reduceSinkDescs.add((ReduceSinkDesc) reduceSinks.get(i).getConf());
+
+ // get the object inspector for the join
+ List<String> fieldNames = new ArrayList<String>();
+ List<TypeInfo> types = new ArrayList<TypeInfo>();
+ for (int i = reduceSinks.size() - 1; i >= offset; i--) {
+ fieldNames.addAll(reduceSinkToFieldNames.get(reduceSinks.get(i)));
+ types.addAll(reduceSinkToTypes.get(reduceSinks.get(i)));
+ }
+
+ // get number of equality conjunctions in the final join condition
+ int size = reduceSinkDescs.get(0).getKeyCols().size();
+
+ // make up the join conditon expression
+ List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();
+ for (int i = 0; i < size; i++) {
+ // create a join key pair
+ List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();
+ for (ReduceSinkDesc sink : reduceSinkDescs) {
+ keyPair.add(sink.getKeyCols().get(i));
+ }
+ // create a hive equal condition
+ ExprNodeDesc equality = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,
+ new GenericUDFOPEqual(), keyPair);
+ // add the equal condition to the conjunction list
+ joinConditionChildren.add(equality);
+ }
+ // get final conjunction expression
+ ExprNodeDesc conjunct = null;
+
+ if (joinConditionChildren.size() > 1)
+ conjunct = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),
+ joinConditionChildren);
+ else if (joinConditionChildren.size() == 1)
+ conjunct = joinConditionChildren.get(0);
+ else {
+ // there is no join equality condition, equal-join
+ conjunct = new ExprNodeConstantDesc(TypeInfoFactory.booleanTypeInfo, new Boolean(true));
+ }
+ // get an ILogicalExpression from hive's expression
+ Mutable<ILogicalExpression> expression = t.translateScalarFucntion(conjunct);
+
+ Mutable<ILogicalOperator> leftBranch = new MutableObject<ILogicalOperator>(exchanges.get(exchanges.size() - 1));
+ Mutable<ILogicalOperator> rightBranch = new MutableObject<ILogicalOperator>(exchanges.get(exchanges.size() - 2));
+ // get the join operator
+ if (conds.get(offset).getType() == JoinDesc.LEFT_OUTER_JOIN) {
+ currentRoot = new LeftOuterJoinOperator(expression);
+ Mutable<ILogicalOperator> temp = leftBranch;
+ leftBranch = rightBranch;
+ rightBranch = temp;
+ } else if (conds.get(offset).getType() == JoinDesc.RIGHT_OUTER_JOIN) {
+ currentRoot = new LeftOuterJoinOperator(expression);
+ } else
+ currentRoot = new InnerJoinOperator(expression);
+
+ currentRoot.getInputs().add(leftBranch);
+ currentRoot.getInputs().add(rightBranch);
+
+ // rewriteOperatorOutputSchema(variables, operator);
+ return currentRoot;
+ } else {
+ // get the child join operator and insert and one-to-one exchange
+ ILogicalOperator joinSrcOne = generateJoinTree(conds, exchanges, reduceSinks, offset + 1, t);
+ // joinSrcOne.addInput(childJoin);
+
+ ILogicalOperator currentRoot;
+
+ List<ReduceSinkDesc> reduceSinkDescs = new ArrayList<ReduceSinkDesc>();
+ for (int i = offset; i < offset + 2; i++)
+ reduceSinkDescs.add((ReduceSinkDesc) reduceSinks.get(i).getConf());
+
+ // get the object inspector for the join
+ List<String> fieldNames = new ArrayList<String>();
+ List<TypeInfo> types = new ArrayList<TypeInfo>();
+ for (int i = offset; i < reduceSinks.size(); i++) {
+ fieldNames.addAll(reduceSinkToFieldNames.get(reduceSinks.get(i)));
+ types.addAll(reduceSinkToTypes.get(reduceSinks.get(i)));
+ }
+
+ // get number of equality conjunctions in the final join condition
+ int size = reduceSinkDescs.get(0).getKeyCols().size();
+
+ // make up the join condition expression
+ List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();
+ for (int i = 0; i < size; i++) {
+ // create a join key pair
+ List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();
+ for (ReduceSinkDesc sink : reduceSinkDescs) {
+ keyPair.add(sink.getKeyCols().get(i));
+ }
+ // create a hive equal condition
+ ExprNodeDesc equality = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,
+ new GenericUDFOPEqual(), keyPair);
+ // add the equal condition to the conjunction list
+ joinConditionChildren.add(equality);
+ }
+ // get final conjunction expression
+ ExprNodeDesc conjunct = null;
+
+ if (joinConditionChildren.size() > 1)
+ conjunct = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),
+ joinConditionChildren);
+ else if (joinConditionChildren.size() == 1)
+ conjunct = joinConditionChildren.get(0);
+ else {
+ // there is no join equality condition, full outer join
+ conjunct = new ExprNodeConstantDesc(TypeInfoFactory.booleanTypeInfo, new Boolean(true));
+ }
+ // get an ILogicalExpression from hive's expression
+ Mutable<ILogicalExpression> expression = t.translateScalarFucntion(conjunct);
+
+ Mutable<ILogicalOperator> leftBranch = new MutableObject<ILogicalOperator>(joinSrcOne);
+ Mutable<ILogicalOperator> rightBranch = new MutableObject<ILogicalOperator>(exchanges.get(offset));
+
+ // get the join operator
+ if (conds.get(offset).getType() == JoinDesc.LEFT_OUTER_JOIN) {
+ currentRoot = new LeftOuterJoinOperator(expression);
+ Mutable<ILogicalOperator> temp = leftBranch;
+ leftBranch = rightBranch;
+ rightBranch = temp;
+ } else if (conds.get(offset).getType() == JoinDesc.RIGHT_OUTER_JOIN) {
+ currentRoot = new LeftOuterJoinOperator(expression);
+ } else
+ currentRoot = new InnerJoinOperator(expression);
+
+ // set the inputs from Algebricks join operator
+ // add the current table
+ currentRoot.getInputs().add(leftBranch);
+ currentRoot.getInputs().add(rightBranch);
+
+ return currentRoot;
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
new file mode 100644
index 0000000..56b0ef4
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
@@ -0,0 +1,121 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.UDTFOperator;
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+
+/**
+ * The lateral view join operator is used for FROM src LATERAL VIEW udtf()...
+ * This operator was implemented with the following operator DAG in mind.
+ *
+ * For a query such as
+ *
+ * SELECT pageid, adid.* FROM example_table LATERAL VIEW explode(adid_list) AS
+ * adid
+ *
+ * The top of the operator DAG will look similar to
+ *
+ * [Table Scan] | [Lateral View Forward] / \ [Select](*) [Select](adid_list) | |
+ * | [UDTF] (explode) \ / [Lateral View Join] | | [Select] (pageid, adid.*) |
+ * ....
+ *
+ * Rows from the table scan operator are first to a lateral view forward
+ * operator that just forwards the row and marks the start of a LV. The select
+ * operator on the left picks all the columns while the select operator on the
+ * right picks only the columns needed by the UDTF.
+ *
+ * The output of select in the left branch and output of the UDTF in the right
+ * branch are then sent to the lateral view join (LVJ). In most cases, the UDTF
+ * will generate > 1 row for every row received from the TS, while the left
+ * select operator will generate only one. For each row output from the TS, the
+ * LVJ outputs all possible rows that can be created by joining the row from the
+ * left select and one of the rows output from the UDTF.
+ *
+ * Additional lateral views can be supported by adding a similar DAG after the
+ * previous LVJ operator.
+ */
+
+public class LateralViewJoinVisitor extends DefaultVisitor {
+
+ private UDTFDesc udtf;
+
+ private List<Mutable<ILogicalOperator>> parents = new ArrayList<Mutable<ILogicalOperator>>();
+
+ @Override
+ public Mutable<ILogicalOperator> visit(LateralViewJoinOperator operator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException {
+
+ parents.add(AlgebricksParentOperatorRef);
+ if (operator.getParentOperators().size() > parents.size()) {
+ return null;
+ }
+
+ Operator parent0 = operator.getParentOperators().get(0);
+ Operator parent1 = operator.getParentOperators().get(1);
+ List<LogicalVariable> variables;
+
+ ILogicalOperator parentOperator;
+ ILogicalExpression unnestArg;
+ if (parent0 instanceof UDTFOperator) {
+ variables = t.getVariablesFromSchema(t.generateInputSchema(parent1));
+ List<LogicalVariable> unnestVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(parents.get(1).getValue(), unnestVars);
+ unnestArg = new VariableReferenceExpression(unnestVars.get(0));
+ parentOperator = parents.get(1).getValue();
+ } else {
+ variables = t.getVariablesFromSchema(t.generateInputSchema(parent0));
+ List<LogicalVariable> unnestVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(parents.get(0).getValue(), unnestVars);
+ unnestArg = new VariableReferenceExpression(unnestVars.get(0));
+ parentOperator = parents.get(0).getValue();
+ }
+
+ LogicalVariable var = t.getVariable(udtf.toString(), TypeInfoFactory.unknownTypeInfo);
+
+ Mutable<ILogicalExpression> unnestExpr = t.translateUnnestFunction(udtf, new MutableObject<ILogicalExpression>(
+ unnestArg));
+ ILogicalOperator currentOperator = new UnnestOperator(var, unnestExpr);
+
+ List<LogicalVariable> outputVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(parentOperator, outputVars);
+ outputVars.add(var);
+ currentOperator.getInputs().add(new MutableObject<ILogicalOperator>(parentOperator));
+
+ parents.clear();
+ udtf = null;
+ t.rewriteOperatorOutputSchema(outputVars, operator);
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(UDTFOperator operator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {
+ Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));
+ udtf = (UDTFDesc) operator.getConf();
+
+ // populate the schema from upstream operator
+ operator.setSchema(operator.getParentOperators().get(0).getSchema());
+ List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);
+ t.rewriteOperatorOutputSchema(latestOutputSchema, operator);
+ return null;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LimitVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LimitVisitor.java
new file mode 100644
index 0000000..d7d6d67
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LimitVisitor.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.LimitOperator;
+import org.apache.hadoop.hive.ql.plan.LimitDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.HivesterixConstantValue;
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+
+public class LimitVisitor extends DefaultVisitor {
+
+ @Override
+ public Mutable<ILogicalOperator> visit(LimitOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) {
+ Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));
+
+ LimitDesc desc = (LimitDesc) operator.getConf();
+ int limit = desc.getLimit();
+ Integer limitValue = new Integer(limit);
+
+ ILogicalExpression expr = new ConstantExpression(new HivesterixConstantValue(limitValue));
+ ILogicalOperator currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator(
+ expr, true);
+ currentOperator.getInputs().add(AlgebricksParentOperatorRef);
+
+ operator.setSchema(operator.getParentOperators().get(0).getSchema());
+ List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);
+ t.rewriteOperatorOutputSchema(latestOutputSchema, operator);
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
new file mode 100644
index 0000000..f8af063
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
@@ -0,0 +1,171 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+
+public class MapJoinVisitor extends DefaultVisitor {
+
+ /**
+ * map a join operator (in hive) to its parent operators (in asterix)
+ */
+ private HashMap<Operator, List<Mutable<ILogicalOperator>>> opMap = new HashMap<Operator, List<Mutable<ILogicalOperator>>>();
+
+ @Override
+ public Mutable<ILogicalOperator> visit(MapJoinOperator operator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {
+ List<Operator<? extends Serializable>> joinSrc = operator.getParentOperators();
+ List<Mutable<ILogicalOperator>> parents = opMap.get(operator);
+ if (parents == null) {
+ parents = new ArrayList<Mutable<ILogicalOperator>>();
+ opMap.put(operator, parents);
+ }
+ parents.add(AlgebricksParentOperatorRef);
+ if (joinSrc.size() != parents.size())
+ return null;
+
+ ILogicalOperator currentOperator;
+ // make an map join operator
+ // TODO: will have trouble for n-way joins
+ MapJoinDesc joinDesc = (MapJoinDesc) operator.getConf();
+
+ Map<Byte, List<ExprNodeDesc>> keyMap = joinDesc.getKeys();
+ // get the projection expression (already re-written) from each source
+ // table
+ Map<Byte, List<ExprNodeDesc>> exprMap = joinDesc.getExprs();
+
+ int inputSize = operator.getParentOperators().size();
+ // get a list of reduce sink descs (input descs)
+
+ // get the parent operator
+ List<Mutable<ILogicalOperator>> parentOps = parents;
+
+ List<String> fieldNames = new ArrayList<String>();
+ List<TypeInfo> types = new ArrayList<TypeInfo>();
+ for (Operator ts : joinSrc) {
+ List<ColumnInfo> columns = ts.getSchema().getSignature();
+ for (ColumnInfo col : columns) {
+ fieldNames.add(col.getInternalName());
+ types.add(col.getType());
+ }
+ }
+
+ // get number of equality conjunctions in the final join condition
+ Set<Entry<Byte, List<ExprNodeDesc>>> keyEntries = keyMap.entrySet();
+ List<List<ExprNodeDesc>> keyLists = new ArrayList<List<ExprNodeDesc>>();
+ Iterator<Entry<Byte, List<ExprNodeDesc>>> entry = keyEntries.iterator();
+
+ int size = 0;
+ if (entry.hasNext())
+ size = entry.next().getValue().size();
+
+ // make up the join conditon expression
+ List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();
+ for (int i = 0; i < size; i++) {
+ // create a join key pair
+ List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();
+ for (int j = 0; j < inputSize; j++) {
+ keyPair.add(keyMap.get(Byte.valueOf((byte) j)).get(i));
+ }
+ // create a hive equal condition
+ ExprNodeDesc equality = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,
+ new GenericUDFOPEqual(), keyPair);
+ // add the equal condition to the conjunction list
+ joinConditionChildren.add(equality);
+ }
+ // get final conjunction expression
+ ExprNodeDesc conjunct = null;
+
+ if (joinConditionChildren.size() > 1)
+ conjunct = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),
+ joinConditionChildren);
+ else if (joinConditionChildren.size() == 1)
+ conjunct = joinConditionChildren.get(0);
+ else {
+ // there is no join equality condition, full outer join
+ conjunct = new ExprNodeConstantDesc(TypeInfoFactory.booleanTypeInfo, new Boolean(true));
+ }
+ // get an ILogicalExpression from hive's expression
+ Mutable<ILogicalExpression> expression = t.translateScalarFucntion(conjunct);
+
+ ArrayList<LogicalVariable> left = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> right = new ArrayList<LogicalVariable>();
+
+ Set<Entry<Byte, List<ExprNodeDesc>>> kentries = keyMap.entrySet();
+ Iterator<Entry<Byte, List<ExprNodeDesc>>> kiterator = kentries.iterator();
+ int iteration = 0;
+ ILogicalOperator assignOperator = null;
+ while (kiterator.hasNext()) {
+ List<ExprNodeDesc> outputExprs = kiterator.next().getValue();
+
+ if (iteration == 0)
+ assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, outputExprs, left);
+ else
+ assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, outputExprs, right);
+
+ if (assignOperator != null) {
+ currentOperator = assignOperator;
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+ iteration++;
+ }
+
+ List<Mutable<ILogicalOperator>> inputs = parentOps;
+
+ // get the join operator
+ currentOperator = new InnerJoinOperator(expression);
+
+ // set the inputs from asterix join operator
+ for (Mutable<ILogicalOperator> input : inputs)
+ currentOperator.getInputs().add(input);
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+
+ // add assign and project operator
+ // output variables
+ ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+ Set<Entry<Byte, List<ExprNodeDesc>>> entries = exprMap.entrySet();
+ Iterator<Entry<Byte, List<ExprNodeDesc>>> iterator = entries.iterator();
+ while (iterator.hasNext()) {
+ List<ExprNodeDesc> outputExprs = iterator.next().getValue();
+ assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, outputExprs, variables);
+
+ if (assignOperator != null) {
+ currentOperator = assignOperator;
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+ }
+
+ currentOperator = new ProjectOperator(variables);
+ currentOperator.getInputs().add(AlgebricksParentOperatorRef);
+ t.rewriteOperatorOutputSchema(variables, operator);
+ // opMap.clear();
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ProjectVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ProjectVisitor.java
new file mode 100644
index 0000000..eb0922f
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ProjectVisitor.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.SelectDesc;
+
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+
+public class ProjectVisitor extends DefaultVisitor {
+
+ /**
+ * translate project operator
+ */
+ @Override
+ public Mutable<ILogicalOperator> visit(SelectOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) {
+
+ SelectDesc desc = (SelectDesc) operator.getConf();
+
+ if (desc == null)
+ return null;
+
+ List<ExprNodeDesc> cols = desc.getColList();
+
+ if (cols == null)
+ return null;
+
+ // insert assign operator if necessary
+ ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+
+ for (ExprNodeDesc expr : cols)
+ t.rewriteExpression(expr);
+
+ ILogicalOperator assignOp = t.getAssignOperator(AlgebricksParentOperator, cols, variables);
+ ILogicalOperator currentOperator = null;
+ if (assignOp != null) {
+ currentOperator = assignOp;
+ AlgebricksParentOperator = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ currentOperator = new ProjectOperator(variables);
+ currentOperator.getInputs().add(AlgebricksParentOperator);
+ t.rewriteOperatorOutputSchema(variables, operator);
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/SortVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/SortVisitor.java
new file mode 100644
index 0000000..a4fc765
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/SortVisitor.java
@@ -0,0 +1,112 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.ExtractOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
+
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+
+public class SortVisitor extends DefaultVisitor {
+
+ @Override
+ public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException {
+ ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();
+ Operator downStream = (Operator) operator.getChildOperators().get(0);
+ List<ExprNodeDesc> keys = desc.getKeyCols();
+ if (!(downStream instanceof ExtractOperator && desc.getNumReducers() == 1 && keys.size() > 0)) {
+ return null;
+ }
+
+ List<ExprNodeDesc> schema = new ArrayList<ExprNodeDesc>();
+ List<ExprNodeDesc> values = desc.getValueCols();
+ List<ExprNodeDesc> partitionCols = desc.getPartitionCols();
+ for (ExprNodeDesc key : keys) {
+ t.rewriteExpression(key);
+ }
+ for (ExprNodeDesc value : values) {
+ t.rewriteExpression(value);
+ }
+ for (ExprNodeDesc col : partitionCols) {
+ t.rewriteExpression(col);
+ }
+
+ // add a order-by operator and limit if any
+ List<Pair<IOrder, Mutable<ILogicalExpression>>> pairs = new ArrayList<Pair<IOrder, Mutable<ILogicalExpression>>>();
+ char[] orders = desc.getOrder().toCharArray();
+ int i = 0;
+ for (ExprNodeDesc key : keys) {
+ Mutable<ILogicalExpression> expr = t.translateScalarFucntion(key);
+ IOrder order = orders[i] == '+' ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
+
+ Pair<IOrder, Mutable<ILogicalExpression>> pair = new Pair<IOrder, Mutable<ILogicalExpression>>(order, expr);
+ pairs.add(pair);
+ i++;
+ }
+
+ // get input variables
+ ArrayList<LogicalVariable> inputVariables = new ArrayList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(AlgebricksParentOperatorRef.getValue(), inputVariables);
+
+ ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();
+ ILogicalOperator currentOperator;
+ ILogicalOperator assignOp = t.getAssignOperator(AlgebricksParentOperatorRef, keys, keyVariables);
+ if (assignOp != null) {
+ currentOperator = assignOp;
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ OrderColumn[] keyColumns = new OrderColumn[keyVariables.size()];
+
+ for (int j = 0; j < keyColumns.length; j++)
+ keyColumns[j] = new OrderColumn(keyVariables.get(j), pairs.get(j).first.getKind());
+
+ // handle order operator
+ currentOperator = new OrderOperator(pairs);
+ currentOperator.getInputs().add(AlgebricksParentOperatorRef);
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+
+ // project back, remove generated sort-key columns if any
+ if (assignOp != null) {
+ currentOperator = new ProjectOperator(inputVariables);
+ currentOperator.getInputs().add(AlgebricksParentOperatorRef);
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ /**
+ * a special rule for hive's order by output schema of reduce sink
+ * operator only contains the columns
+ */
+ for (ExprNodeDesc value : values) {
+ schema.add(value);
+ }
+
+ ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+ ILogicalOperator assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, schema, variables);
+ t.rewriteOperatorOutputSchema(variables, operator);
+
+ if (assignOperator != null) {
+ currentOperator = assignOperator;
+ AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);
+ }
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
new file mode 100644
index 0000000..6071716
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
@@ -0,0 +1,136 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveDataSink;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveDataSource;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveMetaDataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+
+public class TableScanWriteVisitor extends DefaultVisitor {
+
+ /**
+ * map from alias to partition desc
+ */
+ private HashMap<String, PartitionDesc> aliasToPathMap;
+
+ /**
+ * map from partition desc to data source
+ */
+ private HashMap<PartitionDesc, IDataSource<PartitionDesc>> dataSourceMap = new HashMap<PartitionDesc, IDataSource<PartitionDesc>>();
+
+ /**
+ * constructor
+ *
+ * @param aliasToPathMap
+ */
+ public TableScanWriteVisitor(HashMap<String, PartitionDesc> aliasToPathMap) {
+ this.aliasToPathMap = aliasToPathMap;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(TableScanOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ TableScanDesc desc = (TableScanDesc) operator.getConf();
+ if (desc == null) {
+ List<LogicalVariable> schema = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(AlgebricksParentOperator.getValue(), schema);
+ t.rewriteOperatorOutputSchema(schema, operator);
+ return null;
+ }
+
+ List<ColumnInfo> columns = operator.getSchema().getSignature();
+ for (int i = columns.size() - 1; i >= 0; i--)
+ if (columns.get(i).getIsVirtualCol() == true)
+ columns.remove(i);
+
+ // start with empty tuple operator
+ List<TypeInfo> types = new ArrayList<TypeInfo>();
+ ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+ List<String> names = new ArrayList<String>();
+ for (ColumnInfo column : columns) {
+ types.add(column.getType());
+
+ LogicalVariable var = t.getVariableFromFieldName(column.getTabAlias() + "." + column.getInternalName());
+ LogicalVariable varNew;
+
+ if (var != null) {
+ varNew = t.getVariable(
+ column.getTabAlias() + "." + column.getInternalName() + operator.toString(),
+ column.getType());
+ t.replaceVariable(var, varNew);
+ var = varNew;
+ } else
+ var = t.getNewVariable(column.getTabAlias() + "." + column.getInternalName(), column.getType());
+
+ variables.add(var);
+ names.add(column.getInternalName());
+ }
+ Schema currentSchema = new Schema(names, types);
+
+ String alias = desc.getAlias();
+ PartitionDesc partDesc = aliasToPathMap.get(alias);
+ IDataSource<PartitionDesc> dataSource = new HiveDataSource<PartitionDesc>(partDesc, currentSchema.getSchema());
+ ILogicalOperator currentOperator = new DataSourceScanOperator(variables, dataSource);
+
+ // set empty tuple source operator
+ ILogicalOperator ets = new EmptyTupleSourceOperator();
+ currentOperator.getInputs().add(new MutableObject<ILogicalOperator>(ets));
+
+ // setup data source
+ dataSourceMap.put(partDesc, dataSource);
+ t.rewriteOperatorOutputSchema(variables, operator);
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) {
+
+ if (hiveOperator.getChildOperators() != null && hiveOperator.getChildOperators().size() > 0)
+ return null;
+
+ Schema currentSchema = t.generateInputSchema(hiveOperator.getParentOperators().get(0));
+
+ IDataSink sink = new HiveDataSink(hiveOperator, currentSchema.getSchema());
+ List<Mutable<ILogicalExpression> > exprList = new ArrayList<Mutable<ILogicalExpression>>();
+ for (String column : currentSchema.getNames()) {
+ exprList.add(new MutableObject<ILogicalExpression> (new VariableReferenceExpression(t.getVariable(column))));
+ }
+
+ ILogicalOperator currentOperator = new WriteOperator(exprList, sink);
+ if (AlgebricksParentOperator != null) {
+ currentOperator.getInputs().add(AlgebricksParentOperator);
+ }
+
+ IMetadataProvider<PartitionDesc, Object> metaData = new HiveMetaDataProvider<PartitionDesc, Object>(
+ hiveOperator, currentSchema, dataSourceMap);
+ t.setMetadataProvider(metaData);
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
new file mode 100644
index 0000000..96b9463
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
@@ -0,0 +1,62 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.DefaultVisitor;
+import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+
+public class UnionVisitor extends DefaultVisitor {
+
+ List<Mutable<ILogicalOperator>> parents = new ArrayList<Mutable<ILogicalOperator>>();
+
+ @Override
+ public Mutable<ILogicalOperator> visit(UnionOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+
+ parents.add(AlgebricksParentOperator);
+ if (operator.getParentOperators().size() > parents.size()) {
+ return null;
+ }
+
+ List<LogicalVariable> leftVars = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> rightVars = new ArrayList<LogicalVariable>();
+
+ VariableUtilities.getUsedVariables(parents.get(0).getValue(), leftVars);
+ VariableUtilities.getUsedVariables(parents.get(1).getValue(), rightVars);
+
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> triples = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
+ List<LogicalVariable> unionVars = new ArrayList<LogicalVariable>();
+
+ for (int i = 0; i < leftVars.size(); i++) {
+ LogicalVariable unionVar = t.getVariable(
+ leftVars.get(i).getId() + "union" + AlgebricksParentOperator.hashCode(),
+ TypeInfoFactory.unknownTypeInfo);
+ unionVars.add(unionVar);
+ Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+ leftVars.get(i), rightVars.get(i), unionVar);
+ t.replaceVariable(leftVars.get(i), unionVar);
+ t.replaceVariable(rightVars.get(i), unionVar);
+ triples.add(triple);
+ }
+ ILogicalOperator currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator(
+ triples);
+ for (Mutable<ILogicalOperator> parent : parents)
+ currentOperator.getInputs().add(parent);
+
+ t.rewriteOperatorOutputSchema(unionVars, operator);
+ parents.clear();
+ return new MutableObject<ILogicalOperator>(currentOperator);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/DefaultVisitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/DefaultVisitor.java
new file mode 100644
index 0000000..972deed
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/DefaultVisitor.java
@@ -0,0 +1,145 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor.base;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hadoop.hive.ql.exec.CollectOperator;
+import org.apache.hadoop.hive.ql.exec.ExtractOperator;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.ForwardOperator;
+import org.apache.hadoop.hive.ql.exec.GroupByOperator;
+import org.apache.hadoop.hive.ql.exec.JoinOperator;
+import org.apache.hadoop.hive.ql.exec.LateralViewForwardOperator;
+import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator;
+import org.apache.hadoop.hive.ql.exec.LimitOperator;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.MapOperator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.ScriptOperator;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.UDTFOperator;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+
+/**
+ * a default empty implementation of visitor
+ *
+ * @author yingyib
+ */
+public class DefaultVisitor implements Visitor {
+
+ @Override
+ public Mutable<ILogicalOperator> visit(CollectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(JoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(ExtractOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(MapJoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(SMBMapJoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ public Mutable<ILogicalOperator> visit(ReduceSinkOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(FilterOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(ForwardOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(GroupByOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(LateralViewForwardOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(LateralViewJoinOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(LimitOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(MapOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(ScriptOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(SelectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(TableScanOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(UDTFOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Mutable<ILogicalOperator> visit(UnionOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,
+ Translator t) throws AlgebricksException {
+ return null;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
new file mode 100644
index 0000000..da87f53
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
@@ -0,0 +1,169 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor.base;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public interface Translator {
+
+ /**
+ * generate input schema
+ *
+ * @param operator
+ * @return
+ */
+ public Schema generateInputSchema(Operator operator);
+
+ /**
+ * rewrite the names of output columns for feture expression evaluators to
+ * use
+ *
+ * @param operator
+ */
+ public void rewriteOperatorOutputSchema(List<LogicalVariable> vars, Operator operator);
+
+ /**
+ * rewrite the names of output columns for feture expression evaluators to
+ * use
+ *
+ * @param operator
+ */
+ public void rewriteOperatorOutputSchema(Operator operator);
+
+ /**
+ * rewrite an expression and substitute variables
+ *
+ * @param expr
+ * hive expression
+ */
+ public void rewriteExpression(ExprNodeDesc expr);
+
+ /**
+ * rewrite an expression and substitute variables
+ *
+ * @param expr
+ * hive expression
+ */
+ public void rewriteExpressionPartial(ExprNodeDesc expr);
+
+ /**
+ * get an assign operator as a child of parent
+ *
+ * @param parent
+ * @param cols
+ * @param variables
+ * @return
+ */
+ public ILogicalOperator getAssignOperator(Mutable<ILogicalOperator> parent, List<ExprNodeDesc> cols,
+ ArrayList<LogicalVariable> variables);
+
+ /**
+ * get type for a logical variable
+ *
+ * @param var
+ * @return type info
+ */
+ public TypeInfo getType(LogicalVariable var);
+
+ /**
+ * translate an expression from hive to Algebricks
+ *
+ * @param desc
+ * @return
+ */
+ public Mutable<ILogicalExpression> translateScalarFucntion(ExprNodeDesc desc);
+
+ /**
+ * translate an aggregation from hive to Algebricks
+ *
+ * @param aggregateDesc
+ * @return
+ */
+ public Mutable<ILogicalExpression> translateAggregation(AggregationDesc aggregateDesc);
+
+ /**
+ * translate unnesting (UDTF) function expression
+ *
+ * @param aggregator
+ * @return
+ */
+ public Mutable<ILogicalExpression> translateUnnestFunction(UDTFDesc udtfDesc, Mutable<ILogicalExpression> argument);
+
+ /**
+ * get variable from a schema
+ *
+ * @param schema
+ * @return
+ */
+ public List<LogicalVariable> getVariablesFromSchema(Schema schema);
+
+ /**
+ * get variable from name
+ *
+ * @param name
+ * @return
+ */
+ public LogicalVariable getVariable(String name);
+
+ /**
+ * get variable from field name
+ *
+ * @param name
+ * @return
+ */
+ public LogicalVariable getVariableFromFieldName(String name);
+
+ /**
+ * get variable from name, type
+ *
+ * @param fieldName
+ * @param type
+ * @return
+ */
+ public LogicalVariable getVariable(String fieldName, TypeInfo type);
+
+ /**
+ * get new variable from name, type
+ *
+ * @param fieldName
+ * @param type
+ * @return
+ */
+ public LogicalVariable getNewVariable(String fieldName, TypeInfo type);
+
+ /**
+ * set the metadata provider
+ *
+ * @param metadata
+ */
+ public void setMetadataProvider(IMetadataProvider<PartitionDesc, Object> metadata);
+
+ /**
+ * get the metadata provider
+ *
+ * @param metadata
+ */
+ public IMetadataProvider<PartitionDesc, Object> getMetadataProvider();
+
+ /**
+ * replace the variable
+ *
+ * @param oldVar
+ * @param newVar
+ */
+ public void replaceVariable(LogicalVariable oldVar, LogicalVariable newVar);
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Visitor.java b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Visitor.java
new file mode 100644
index 0000000..495f3ee
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Visitor.java
@@ -0,0 +1,85 @@
+package edu.uci.ics.hivesterix.logical.plan.visitor.base;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hadoop.hive.ql.exec.CollectOperator;
+import org.apache.hadoop.hive.ql.exec.ExtractOperator;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.ForwardOperator;
+import org.apache.hadoop.hive.ql.exec.GroupByOperator;
+import org.apache.hadoop.hive.ql.exec.JoinOperator;
+import org.apache.hadoop.hive.ql.exec.LateralViewForwardOperator;
+import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator;
+import org.apache.hadoop.hive.ql.exec.LimitOperator;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.MapOperator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.ScriptOperator;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.UDTFOperator;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+
+public interface Visitor {
+
+ public Mutable<ILogicalOperator> visit(CollectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(JoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(ExtractOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(MapJoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(SMBMapJoinOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(FilterOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(ForwardOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(GroupByOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(LateralViewForwardOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(LateralViewJoinOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(LimitOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(MapOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(ScriptOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(SelectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(TableScanOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(ReduceSinkOperator hiveOperator,
+ Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(UDTFOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+
+ public Mutable<ILogicalOperator> visit(UnionOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,
+ Translator t) throws AlgebricksException;
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java b/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
new file mode 100644
index 0000000..7e4e271
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
@@ -0,0 +1,113 @@
+package edu.uci.ics.hivesterix.optimizer.rulecollections;
+
+import java.util.LinkedList;
+
+import edu.uci.ics.hivesterix.optimizer.rules.InsertProjectBeforeWriteRule;
+import edu.uci.ics.hivesterix.optimizer.rules.IntroduceEarlyProjectRule;
+import edu.uci.ics.hivesterix.optimizer.rules.LocalGroupByRule;
+import edu.uci.ics.hivesterix.optimizer.rules.RemoveRedundantSelectRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.BreakSelectIntoConjunctsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexJoinInferenceRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractGbyExpressionsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.FactorRedundantGroupAndDecorVarsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InferTypesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineVariablesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InsertProjectBeforeUnionRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceAggregateCombinerRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByCombinerRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushLimitDownRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectDownRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectIntoDataSourceScanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectDownRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectIntoJoinRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ReinferAllTypesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantProjectionRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveUnusedAssignAndAggregateRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetAlgebricksPhysicalOperatorsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetExecutionModeRule;
+
+public final class HiveRuleCollections {
+
+ public final static LinkedList<IAlgebraicRewriteRule> NORMALIZATION = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ NORMALIZATION.add(new EliminateSubplanRule());
+ NORMALIZATION.add(new IntroduceAggregateCombinerRule());
+ NORMALIZATION.add(new BreakSelectIntoConjunctsRule());
+ NORMALIZATION.add(new IntroduceAggregateCombinerRule());
+ NORMALIZATION.add(new PushSelectIntoJoinRule());
+ NORMALIZATION.add(new ExtractGbyExpressionsRule());
+ NORMALIZATION.add(new RemoveRedundantSelectRule());
+ }
+
+ public final static LinkedList<IAlgebraicRewriteRule> COND_PUSHDOWN_AND_JOIN_INFERENCE = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new PushSelectDownRule());
+ COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new InlineVariablesRule());
+ COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new FactorRedundantGroupAndDecorVarsRule());
+ COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new EliminateSubplanRule());
+ }
+
+ public final static LinkedList<IAlgebraicRewriteRule> LOAD_FIELDS = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ // should LoadRecordFieldsRule be applied in only one pass over the
+ // plan?
+ LOAD_FIELDS.add(new InlineVariablesRule());
+ // LOAD_FIELDS.add(new RemoveUnusedAssignAndAggregateRule());
+ LOAD_FIELDS.add(new ComplexJoinInferenceRule());
+ LOAD_FIELDS.add(new InferTypesRule());
+ }
+
+ public final static LinkedList<IAlgebraicRewriteRule> OP_PUSHDOWN = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ OP_PUSHDOWN.add(new PushProjectDownRule());
+ OP_PUSHDOWN.add(new PushSelectDownRule());
+ }
+
+ public final static LinkedList<IAlgebraicRewriteRule> DATA_EXCHANGE = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ DATA_EXCHANGE.add(new SetExecutionModeRule());
+ }
+
+ public final static LinkedList<IAlgebraicRewriteRule> CONSOLIDATION = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ CONSOLIDATION.add(new RemoveRedundantProjectionRule());
+ CONSOLIDATION.add(new ConsolidateSelectsRule());
+ CONSOLIDATION.add(new IntroduceEarlyProjectRule());
+ CONSOLIDATION.add(new ConsolidateAssignsRule());
+ CONSOLIDATION.add(new IntroduceGroupByCombinerRule());
+ CONSOLIDATION.add(new RemoveUnusedAssignAndAggregateRule());
+ }
+
+ public final static LinkedList<IAlgebraicRewriteRule> PHYSICAL_PLAN_REWRITES = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ PHYSICAL_PLAN_REWRITES.add(new PullSelectOutOfEqJoin());
+ PHYSICAL_PLAN_REWRITES.add(new SetAlgebricksPhysicalOperatorsRule());
+ PHYSICAL_PLAN_REWRITES.add(new EnforceStructuralPropertiesRule());
+ PHYSICAL_PLAN_REWRITES.add(new PushProjectDownRule());
+ PHYSICAL_PLAN_REWRITES.add(new SetAlgebricksPhysicalOperatorsRule());
+ PHYSICAL_PLAN_REWRITES.add(new PushLimitDownRule());
+ PHYSICAL_PLAN_REWRITES.add(new InsertProjectBeforeWriteRule());
+ PHYSICAL_PLAN_REWRITES.add(new InsertProjectBeforeUnionRule());
+ }
+
+ public final static LinkedList<IAlgebraicRewriteRule> prepareJobGenRules = new LinkedList<IAlgebraicRewriteRule>();
+ static {
+ prepareJobGenRules.add(new ReinferAllTypesRule());
+ prepareJobGenRules.add(new IsolateHyracksOperatorsRule(
+ HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
+ prepareJobGenRules.add(new ExtractCommonOperatorsRule());
+ prepareJobGenRules.add(new LocalGroupByRule());
+ prepareJobGenRules.add(new PushProjectIntoDataSourceScanRule());
+ prepareJobGenRules.add(new ReinferAllTypesRule());
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/InsertProjectBeforeWriteRule.java b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/InsertProjectBeforeWriteRule.java
new file mode 100644
index 0000000..90777ee
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/InsertProjectBeforeWriteRule.java
@@ -0,0 +1,79 @@
+package edu.uci.ics.hivesterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class InsertProjectBeforeWriteRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ /**
+ * When the input schema to WriteOperator is different from the output
+ * schema in terms of variable order, add a project operator to get the
+ * write order
+ */
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.WRITE) {
+ return false;
+ }
+ WriteOperator opWrite = (WriteOperator) op;
+ ArrayList<LogicalVariable> finalSchema = new ArrayList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(opWrite, finalSchema);
+ ArrayList<LogicalVariable> inputSchema = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(opWrite, inputSchema);
+ if (!isIdentical(finalSchema, inputSchema)) {
+ ProjectOperator projectOp = new ProjectOperator(finalSchema);
+ Mutable<ILogicalOperator> parentOpRef = opWrite.getInputs().get(0);
+ projectOp.getInputs().add(parentOpRef);
+ opWrite.getInputs().clear();
+ opWrite.getInputs().add(new MutableObject<ILogicalOperator>(projectOp));
+ projectOp.setPhysicalOperator(new StreamProjectPOperator());
+ projectOp.setExecutionMode(ExecutionMode.PARTITIONED);
+
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) parentOpRef.getValue();
+ if (op2.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ ProjectOperator pi2 = (ProjectOperator) op2;
+ parentOpRef.setValue(pi2.getInputs().get(0).getValue());
+ }
+ context.computeAndSetTypeEnvironmentForOperator(projectOp);
+ return true;
+ } else
+ return false;
+
+ }
+
+ private boolean isIdentical(List<LogicalVariable> finalSchema, List<LogicalVariable> inputSchema) {
+ int finalSchemaSize = finalSchema.size();
+ int inputSchemaSize = inputSchema.size();
+ if (finalSchemaSize != inputSchemaSize)
+ throw new IllegalStateException("final output schema variables missing!");
+ for (int i = 0; i < finalSchemaSize; i++) {
+ LogicalVariable var1 = finalSchema.get(i);
+ LogicalVariable var2 = inputSchema.get(i);
+ if (!var1.equals(var2))
+ return false;
+ }
+ return true;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/IntroduceEarlyProjectRule.java b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/IntroduceEarlyProjectRule.java
new file mode 100644
index 0000000..96815ff
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/IntroduceEarlyProjectRule.java
@@ -0,0 +1,72 @@
+package edu.uci.ics.hivesterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceEarlyProjectRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ AbstractLogicalOperator middleOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ List<LogicalVariable> deliveredVars = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
+
+ VariableUtilities.getUsedVariables(op, deliveredVars);
+ VariableUtilities.getUsedVariables(middleOp, usedVars);
+ VariableUtilities.getProducedVariables(middleOp, producedVars);
+
+ Set<LogicalVariable> requiredVariables = new HashSet<LogicalVariable>();
+ requiredVariables.addAll(deliveredVars);
+ requiredVariables.addAll(usedVars);
+ requiredVariables.removeAll(producedVars);
+
+ if (middleOp.getInputs().size() <= 0 || middleOp.getInputs().size() > 1)
+ return false;
+
+ AbstractLogicalOperator targetOp = (AbstractLogicalOperator) middleOp.getInputs().get(0).getValue();
+ if (targetOp.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN)
+ return false;
+
+ Set<LogicalVariable> deliveredEarlyVars = new HashSet<LogicalVariable>();
+ VariableUtilities.getLiveVariables(targetOp, deliveredEarlyVars);
+
+ deliveredEarlyVars.removeAll(requiredVariables);
+ if (deliveredEarlyVars.size() > 0) {
+ ArrayList<LogicalVariable> requiredVars = new ArrayList<LogicalVariable>();
+ requiredVars.addAll(requiredVariables);
+ ILogicalOperator earlyProjectOp = new ProjectOperator(requiredVars);
+ Mutable<ILogicalOperator> earlyProjectOpRef = new MutableObject<ILogicalOperator>(earlyProjectOp);
+ Mutable<ILogicalOperator> targetRef = middleOp.getInputs().get(0);
+ middleOp.getInputs().set(0, earlyProjectOpRef);
+ earlyProjectOp.getInputs().add(targetRef);
+ context.computeAndSetTypeEnvironmentForOperator(earlyProjectOp);
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/LocalGroupByRule.java b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/LocalGroupByRule.java
new file mode 100644
index 0000000..90ca008
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/LocalGroupByRule.java
@@ -0,0 +1,66 @@
+package edu.uci.ics.hivesterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hivesterix.logical.plan.HiveOperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class LocalGroupByRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ Boolean localGby = (Boolean) op.getAnnotations().get(HiveOperatorAnnotations.LOCAL_GROUP_BY);
+ if (localGby != null && localGby.equals(Boolean.TRUE)) {
+ Boolean hashGby = (Boolean) op.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY);
+ Boolean externalGby = (Boolean) op.getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY);
+ if ((hashGby != null && (hashGby.equals(Boolean.TRUE)) || (externalGby != null && externalGby
+ .equals(Boolean.TRUE)))) {
+ reviseExchange(op);
+ } else {
+ ILogicalOperator child = op.getInputs().get(0).getValue();
+ AbstractLogicalOperator childOp = (AbstractLogicalOperator) child;
+ while (child.getInputs().size() > 0) {
+ if (childOp.getOperatorTag() == LogicalOperatorTag.ORDER)
+ break;
+ else {
+ child = child.getInputs().get(0).getValue();
+ childOp = (AbstractLogicalOperator) child;
+ }
+ }
+ if (childOp.getOperatorTag() == LogicalOperatorTag.ORDER)
+ reviseExchange(childOp);
+ }
+ return true;
+ }
+ return false;
+ }
+
+ private void reviseExchange(AbstractLogicalOperator op) {
+ ExchangeOperator exchange = (ExchangeOperator) op.getInputs().get(0).getValue();
+ IPhysicalOperator physicalOp = exchange.getPhysicalOperator();
+ if (physicalOp.getOperatorTag() == PhysicalOperatorTag.HASH_PARTITION_EXCHANGE) {
+ exchange.setPhysicalOperator(new OneToOneExchangePOperator());
+ }
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/RemoveRedundantSelectRule.java b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/RemoveRedundantSelectRule.java
new file mode 100644
index 0000000..44ff12d
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/RemoveRedundantSelectRule.java
@@ -0,0 +1,44 @@
+package edu.uci.ics.hivesterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class RemoveRedundantSelectRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ AbstractLogicalOperator inputOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (inputOp.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator selectOp = (SelectOperator) op;
+ SelectOperator inputSelectOp = (SelectOperator) inputOp;
+ ILogicalExpression expr1 = selectOp.getCondition().getValue();
+ ILogicalExpression expr2 = inputSelectOp.getCondition().getValue();
+
+ if (expr1.equals(expr2)) {
+ selectOp.getInputs().set(0, inputSelectOp.getInputs().get(0));
+ return true;
+ }
+ return false;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/config/ConfUtil.java b/src/main/java/edu/uci/ics/hivesterix/runtime/config/ConfUtil.java
new file mode 100644
index 0000000..10596b6
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/config/ConfUtil.java
@@ -0,0 +1,138 @@
+package edu.uci.ics.hivesterix.runtime.config;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+
+public class ConfUtil {
+
+ private static JobConf job;
+ private static HiveConf hconf;
+ private static String[] NCs;
+ private static Map<String, List<String>> ncMapping;
+ private static IHyracksClientConnection hcc = null;
+ private static ClusterTopology topology = null;
+
+ public static JobConf getJobConf(Class<? extends InputFormat> format, Path path) {
+ JobConf conf = new JobConf();
+ if (job != null)
+ conf = job;
+
+ String hadoopPath = System.getProperty("HADOOP_HOME", "/hadoop");
+ Path pathCore = new Path(hadoopPath + "/conf/core-site.xml");
+ conf.addResource(pathCore);
+ Path pathMapRed = new Path(hadoopPath + "/conf/mapred-site.xml");
+ conf.addResource(pathMapRed);
+ Path pathHDFS = new Path(hadoopPath + "/conf/hdfs-site.xml");
+ conf.addResource(pathHDFS);
+
+ conf.setInputFormat(format);
+ FileInputFormat.setInputPaths(conf, path);
+ return conf;
+ }
+
+ public static JobConf getJobConf() {
+ JobConf conf = new JobConf();
+ if (job != null)
+ conf = job;
+
+ String hadoopPath = System.getProperty("HADOOP_HOME", "/hadoop");
+ Path pathCore = new Path(hadoopPath + "/conf/core-site.xml");
+ conf.addResource(pathCore);
+ Path pathMapRed = new Path(hadoopPath + "/conf/mapred-site.xml");
+ conf.addResource(pathMapRed);
+ Path pathHDFS = new Path(hadoopPath + "/conf/hdfs-site.xml");
+ conf.addResource(pathHDFS);
+
+ return conf;
+ }
+
+ public static void setJobConf(JobConf conf) {
+ job = conf;
+ }
+
+ public static void setHiveConf(HiveConf hiveConf) {
+ hconf = hiveConf;
+ }
+
+ public static HiveConf getHiveConf() {
+ if (hconf == null) {
+ hconf = new HiveConf(SessionState.class);
+ hconf.addResource(new Path("conf/hive-default.xml"));
+ }
+ return hconf;
+ }
+
+ public static String[] getNCs() throws AlgebricksException {
+ if (NCs == null) {
+ try {
+ loadClusterConfig();
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
+ return NCs;
+ }
+
+ public static Map<String, List<String>> getNCMapping() throws AlgebricksException {
+ if (ncMapping == null) {
+ try {
+ loadClusterConfig();
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
+ return ncMapping;
+ }
+
+ private static void loadClusterConfig() {
+ try {
+ getHiveConf();
+ String ipAddress = hconf.get("hive.hyracks.host");
+ int port = Integer.parseInt(hconf.get("hive.hyracks.port"));
+ int mpl = Integer.parseInt(hconf.get("hive.hyracks.parrallelism"));
+ hcc = new HyracksConnection(ipAddress, port);
+ topology = hcc.getClusterTopology();
+ Map<String, NodeControllerInfo> ncNameToNcInfos = hcc.getNodeControllerInfos();
+ NCs = new String[ncNameToNcInfos.size() * mpl];
+ ncMapping = new HashMap<String, List<String>>();
+ int i = 0;
+ for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {
+ String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().getIpAddress())
+ .getHostAddress();
+ List<String> matchedNCs = ncMapping.get(ipAddr);
+ if (matchedNCs == null) {
+ matchedNCs = new ArrayList<String>();
+ ncMapping.put(ipAddr, matchedNCs);
+ }
+ matchedNCs.add(entry.getKey());
+ for (int j = i * mpl; j < i * mpl + mpl; j++)
+ NCs[j] = entry.getKey();
+ i++;
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ public static ClusterTopology getClusterTopology() {
+ if (topology == null)
+ loadClusterConfig();
+ return topology;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
new file mode 100644
index 0000000..ad02239
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
@@ -0,0 +1,169 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.io.BytesWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyFactory;
+import edu.uci.ics.hivesterix.serde.lazy.LazyObject;
+import edu.uci.ics.hivesterix.serde.lazy.LazySerDe;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractExpressionEvaluator implements ICopyEvaluator {
+
+ private List<ICopyEvaluator> children;
+
+ private ExprNodeEvaluator evaluator;
+
+ private IDataOutputProvider out;
+
+ private ObjectInspector inspector;
+
+ /**
+ * output object inspector
+ */
+ private ObjectInspector outputInspector;
+
+ /**
+ * cached row object
+ */
+ private LazyObject<? extends ObjectInspector> cachedRowObject;
+
+ /**
+ * serializer/derialzer for lazy object
+ */
+ private SerDe lazySer;
+
+ /**
+ * data output
+ */
+ DataOutput dataOutput;
+
+ public AbstractExpressionEvaluator(ExprNodeEvaluator hiveEvaluator, ObjectInspector oi, IDataOutputProvider output)
+ throws AlgebricksException {
+ evaluator = hiveEvaluator;
+ out = output;
+ inspector = oi;
+ dataOutput = out.getDataOutput();
+ }
+
+ protected ObjectInspector getRowInspector() {
+ return null;
+ }
+
+ protected IDataOutputProvider getIDataOutputProvider() {
+ return out;
+ }
+
+ protected ExprNodeEvaluator getHiveEvaluator() {
+ return evaluator;
+ }
+
+ public ObjectInspector getObjectInspector() {
+ return inspector;
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference r) throws AlgebricksException {
+ // initialize hive evaluator
+ try {
+ if (outputInspector == null)
+ outputInspector = evaluator.initialize(inspector);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ }
+
+ readIntoCache(r);
+ try {
+ Object result = evaluator.evaluate(cachedRowObject);
+
+ // if (result == null) {
+ // result = evaluator.evaluate(cachedRowObject);
+ //
+ // // check if result is null
+ //
+ // String errorMsg = "serialize null object in \n output " +
+ // outputInspector.toString() + " \n input "
+ // + inspector.toString() + "\n ";
+ // errorMsg += "";
+ // List<Object> columns = ((StructObjectInspector)
+ // inspector).getStructFieldsDataAsList(cachedRowObject);
+ // for (Object column : columns) {
+ // errorMsg += column.toString() + " ";
+ // }
+ // errorMsg += "\n";
+ // Log.info(errorMsg);
+ // System.out.println(errorMsg);
+ // // result = new BooleanWritable(true);
+ // throw new IllegalStateException(errorMsg);
+ // }
+
+ serializeResult(result);
+ } catch (HiveException e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ } catch (IOException e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ }
+ }
+
+ /**
+ * serialize the result
+ *
+ * @param result
+ * the evaluation result
+ * @throws IOException
+ * @throws AlgebricksException
+ */
+ private void serializeResult(Object result) throws IOException, AlgebricksException {
+ if (lazySer == null)
+ lazySer = new LazySerDe();
+
+ try {
+ BytesWritable outputWritable = (BytesWritable) lazySer.serialize(result, outputInspector);
+ dataOutput.write(outputWritable.getBytes(), 0, outputWritable.getLength());
+ } catch (SerDeException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ /**
+ * bind the tuple reference to the cached row object
+ *
+ * @param r
+ */
+ private void readIntoCache(IFrameTupleReference r) {
+ if (cachedRowObject == null)
+ cachedRowObject = (LazyObject<? extends ObjectInspector>) LazyFactory.createLazyObject(inspector);
+ cachedRowObject.init(r);
+ }
+
+ /**
+ * set a list of children of this evaluator
+ *
+ * @param children
+ */
+ public void setChildren(List<ICopyEvaluator> children) {
+ this.children = children;
+ }
+
+ public void addChild(ICopyEvaluator child) {
+ if (children == null)
+ children = new ArrayList<ICopyEvaluator>();
+ children.add(child);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
new file mode 100644
index 0000000..e500376
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
@@ -0,0 +1,224 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.BytesWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyObject;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyAggregateFunction;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class AggregationFunctionEvaluator implements ICopyAggregateFunction {
+
+ /**
+ * the mode of aggregation function
+ */
+ private GenericUDAFEvaluator.Mode mode;
+
+ /**
+ * an array of evaluators
+ */
+ private ExprNodeEvaluator[] evaluators;
+
+ /**
+ * udaf evaluator partial
+ */
+ private GenericUDAFEvaluator udafPartial;
+
+ /**
+ * udaf evaluator complete
+ */
+ private GenericUDAFEvaluator udafComplete;
+
+ /**
+ * cached parameter objects
+ */
+ private Object[] cachedParameters;
+
+ /**
+ * cached row objects
+ */
+ private LazyObject<? extends ObjectInspector> cachedRowObject;
+
+ /**
+ * the output channel
+ */
+ private DataOutput out;
+
+ /**
+ * aggregation buffer
+ */
+ private AggregationBuffer aggBuffer;
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ private SerDe lazySer;
+
+ /**
+ * the output object inspector for this aggregation function
+ */
+ private ObjectInspector outputInspector;
+
+ /**
+ * the output object inspector for this aggregation function
+ */
+ private ObjectInspector outputInspectorPartial;
+
+ /**
+ * parameter inspectors
+ */
+ private ObjectInspector[] parameterInspectors;
+
+ /**
+ * output make sure the aggregation functio has least object creation
+ *
+ * @param desc
+ * @param oi
+ * @param output
+ */
+ public AggregationFunctionEvaluator(List<ExprNodeDesc> inputs, List<TypeInfo> inputTypes, String genericUDAFName,
+ GenericUDAFEvaluator.Mode aggMode, boolean distinct, ObjectInspector oi, DataOutput output,
+ ExprNodeEvaluator[] evals, ObjectInspector[] pInspectors, Object[] parameterCache, SerDe serde,
+ LazyObject<? extends ObjectInspector> row, GenericUDAFEvaluator udafunctionPartial,
+ GenericUDAFEvaluator udafunctionComplete, ObjectInspector outputOi, ObjectInspector outputOiPartial) {
+ // shared object across threads
+ this.out = output;
+ this.mode = aggMode;
+ this.parameterInspectors = pInspectors;
+
+ // thread local objects
+ this.evaluators = evals;
+ this.cachedParameters = parameterCache;
+ this.cachedRowObject = row;
+ this.lazySer = serde;
+ this.udafPartial = udafunctionPartial;
+ this.udafComplete = udafunctionComplete;
+ this.outputInspector = outputOi;
+ this.outputInspectorPartial = outputOiPartial;
+ }
+
+ @Override
+ public void init() throws AlgebricksException {
+ try {
+ aggBuffer = udafPartial.getNewAggregationBuffer();
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ readIntoCache(tuple);
+ processRow();
+ }
+
+ private void processRow() throws AlgebricksException {
+ try {
+ // get values by evaluating them
+ for (int i = 0; i < cachedParameters.length; i++) {
+ cachedParameters[i] = evaluators[i].evaluate(cachedRowObject);
+ }
+ processAggregate();
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ private void processAggregate() throws HiveException {
+ /**
+ * accumulate the aggregation function
+ */
+ switch (mode) {
+ case PARTIAL1:
+ case COMPLETE:
+ udafPartial.iterate(aggBuffer, cachedParameters);
+ break;
+ case PARTIAL2:
+ case FINAL:
+ if (udafPartial instanceof GenericUDAFCount.GenericUDAFCountEvaluator) {
+ Object parameter = ((PrimitiveObjectInspector) parameterInspectors[0])
+ .getPrimitiveWritableObject(cachedParameters[0]);
+ udafPartial.merge(aggBuffer, parameter);
+ } else
+ udafPartial.merge(aggBuffer, cachedParameters[0]);
+ break;
+ default:
+ break;
+ }
+ }
+
+ /**
+ * serialize the result
+ *
+ * @param result
+ * the evaluation result
+ * @throws IOException
+ * @throws AlgebricksException
+ */
+ private void serializeResult(Object result, ObjectInspector oi) throws IOException, AlgebricksException {
+ try {
+ BytesWritable outputWritable = (BytesWritable) lazySer.serialize(result, oi);
+ out.write(outputWritable.getBytes(), 0, outputWritable.getLength());
+ } catch (SerDeException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ /**
+ * bind the tuple reference to the cached row object
+ *
+ * @param r
+ */
+ private void readIntoCache(IFrameTupleReference r) {
+ cachedRowObject.init(r);
+ }
+
+ @Override
+ public void finish() throws AlgebricksException {
+ // aggregator
+ try {
+ Object result = null;
+ result = udafPartial.terminatePartial(aggBuffer);
+ if (mode == GenericUDAFEvaluator.Mode.COMPLETE || mode == GenericUDAFEvaluator.Mode.FINAL) {
+ result = udafComplete.terminate(aggBuffer);
+ serializeResult(result, outputInspector);
+ } else {
+ serializeResult(result, outputInspectorPartial);
+ }
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void finishPartial() throws AlgebricksException {
+ // aggregator.
+ try {
+ Object result = null;
+ // get aggregations
+ result = udafPartial.terminatePartial(aggBuffer);
+ serializeResult(result, outputInspectorPartial);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
new file mode 100644
index 0000000..1933253
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
@@ -0,0 +1,248 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.BytesWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyObject;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunction;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class AggregatuibFunctionSerializableEvaluator implements ICopySerializableAggregateFunction {
+
+ /**
+ * the mode of aggregation function
+ */
+ private GenericUDAFEvaluator.Mode mode;
+
+ /**
+ * an array of evaluators
+ */
+ private ExprNodeEvaluator[] evaluators;
+
+ /**
+ * udaf evaluator partial
+ */
+ private GenericUDAFEvaluator udafPartial;
+
+ /**
+ * udaf evaluator complete
+ */
+ private GenericUDAFEvaluator udafComplete;
+
+ /**
+ * cached parameter objects
+ */
+ private Object[] cachedParameters;
+
+ /**
+ * cached row objects
+ */
+ private LazyObject<? extends ObjectInspector> cachedRowObject;
+
+ /**
+ * aggregation buffer
+ */
+ private SerializableBuffer aggBuffer;
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ private SerDe lazySer;
+
+ /**
+ * the output object inspector for this aggregation function
+ */
+ private ObjectInspector outputInspector;
+
+ /**
+ * the output object inspector for this aggregation function
+ */
+ private ObjectInspector outputInspectorPartial;
+
+ /**
+ * parameter inspectors
+ */
+ private ObjectInspector[] parameterInspectors;
+
+ /**
+ * output make sure the aggregation functio has least object creation
+ *
+ * @param desc
+ * @param oi
+ * @param output
+ */
+ public AggregatuibFunctionSerializableEvaluator(List<ExprNodeDesc> inputs, List<TypeInfo> inputTypes,
+ String genericUDAFName, GenericUDAFEvaluator.Mode aggMode, boolean distinct, ObjectInspector oi,
+ ExprNodeEvaluator[] evals, ObjectInspector[] pInspectors, Object[] parameterCache, SerDe serde,
+ LazyObject<? extends ObjectInspector> row, GenericUDAFEvaluator udafunctionPartial,
+ GenericUDAFEvaluator udafunctionComplete, ObjectInspector outputOi, ObjectInspector outputOiPartial)
+ throws AlgebricksException {
+ // shared object across threads
+ this.mode = aggMode;
+ this.parameterInspectors = pInspectors;
+
+ // thread local objects
+ this.evaluators = evals;
+ this.cachedParameters = parameterCache;
+ this.cachedRowObject = row;
+ this.lazySer = serde;
+ this.udafPartial = udafunctionPartial;
+ this.udafComplete = udafunctionComplete;
+ this.outputInspector = outputOi;
+ this.outputInspectorPartial = outputOiPartial;
+
+ try {
+ aggBuffer = (SerializableBuffer) udafPartial.getNewAggregationBuffer();
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void init(DataOutput output) throws AlgebricksException {
+ try {
+ udafPartial.reset(aggBuffer);
+ outputAggBuffer(aggBuffer, output);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple, byte[] data, int start, int len) throws AlgebricksException {
+ deSerializeAggBuffer(aggBuffer, data, start, len);
+ readIntoCache(tuple);
+ processRow();
+ serializeAggBuffer(aggBuffer, data, start, len);
+ }
+
+ private void processRow() throws AlgebricksException {
+ try {
+ // get values by evaluating them
+ for (int i = 0; i < cachedParameters.length; i++) {
+ cachedParameters[i] = evaluators[i].evaluate(cachedRowObject);
+ }
+ processAggregate();
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ private void processAggregate() throws HiveException {
+ /**
+ * accumulate the aggregation function
+ */
+ switch (mode) {
+ case PARTIAL1:
+ case COMPLETE:
+ udafPartial.iterate(aggBuffer, cachedParameters);
+ break;
+ case PARTIAL2:
+ case FINAL:
+ if (udafPartial instanceof GenericUDAFCount.GenericUDAFCountEvaluator) {
+ Object parameter = ((PrimitiveObjectInspector) parameterInspectors[0])
+ .getPrimitiveWritableObject(cachedParameters[0]);
+ udafPartial.merge(aggBuffer, parameter);
+ } else
+ udafPartial.merge(aggBuffer, cachedParameters[0]);
+ break;
+ default:
+ break;
+ }
+ }
+
+ /**
+ * serialize the result
+ *
+ * @param result
+ * the evaluation result
+ * @throws IOException
+ * @throws AlgebricksException
+ */
+ private void serializeResult(Object result, ObjectInspector oi, DataOutput out) throws IOException,
+ AlgebricksException {
+ try {
+ BytesWritable outputWritable = (BytesWritable) lazySer.serialize(result, oi);
+ out.write(outputWritable.getBytes(), 0, outputWritable.getLength());
+ } catch (SerDeException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ /**
+ * bind the tuple reference to the cached row object
+ *
+ * @param r
+ */
+ private void readIntoCache(IFrameTupleReference r) {
+ cachedRowObject.init(r);
+ }
+
+ @Override
+ public void finish(byte[] data, int start, int len, DataOutput output) throws AlgebricksException {
+ deSerializeAggBuffer(aggBuffer, data, start, len);
+ // aggregator
+ try {
+ Object result = null;
+ result = udafPartial.terminatePartial(aggBuffer);
+ if (mode == GenericUDAFEvaluator.Mode.COMPLETE || mode == GenericUDAFEvaluator.Mode.FINAL) {
+ result = udafComplete.terminate(aggBuffer);
+ serializeResult(result, outputInspector, output);
+ } else {
+ serializeResult(result, outputInspectorPartial, output);
+ }
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void finishPartial(byte[] data, int start, int len, DataOutput output) throws AlgebricksException {
+ deSerializeAggBuffer(aggBuffer, data, start, len);
+ // aggregator.
+ try {
+ Object result = null;
+ // get aggregations
+ result = udafPartial.terminatePartial(aggBuffer);
+ serializeResult(result, outputInspectorPartial, output);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ private void serializeAggBuffer(SerializableBuffer buffer, byte[] data, int start, int len)
+ throws AlgebricksException {
+ buffer.serializeAggBuffer(data, start, len);
+ }
+
+ private void deSerializeAggBuffer(SerializableBuffer buffer, byte[] data, int start, int len)
+ throws AlgebricksException {
+ buffer.deSerializeAggBuffer(data, start, len);
+ }
+
+ private void outputAggBuffer(SerializableBuffer buffer, DataOutput out) throws AlgebricksException {
+ try {
+ buffer.serializeAggBuffer(out);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/BufferSerDeUtil.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/BufferSerDeUtil.java
new file mode 100644
index 0000000..96065e5
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/BufferSerDeUtil.java
@@ -0,0 +1,67 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+public class BufferSerDeUtil {
+
+ public static double getDouble(byte[] bytes, int offset) {
+ return Double.longBitsToDouble(getLong(bytes, offset));
+ }
+
+ public static float getFloat(byte[] bytes, int offset) {
+ return Float.intBitsToFloat(getInt(bytes, offset));
+ }
+
+ public static boolean getBoolean(byte[] bytes, int offset) {
+ if (bytes[offset] == 0)
+ return false;
+ else
+ return true;
+ }
+
+ public static int getInt(byte[] bytes, int offset) {
+ return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
+ + ((bytes[offset + 3] & 0xff) << 0);
+ }
+
+ public static long getLong(byte[] bytes, int offset) {
+ return (((long) (bytes[offset] & 0xff)) << 56) + (((long) (bytes[offset + 1] & 0xff)) << 48)
+ + (((long) (bytes[offset + 2] & 0xff)) << 40) + (((long) (bytes[offset + 3] & 0xff)) << 32)
+ + (((long) (bytes[offset + 4] & 0xff)) << 24) + (((long) (bytes[offset + 5] & 0xff)) << 16)
+ + (((long) (bytes[offset + 6] & 0xff)) << 8) + (((long) (bytes[offset + 7] & 0xff)) << 0);
+ }
+
+ public static void writeBoolean(boolean value, byte[] bytes, int offset) {
+ if (value)
+ bytes[offset] = (byte) 1;
+ else
+ bytes[offset] = (byte) 0;
+ }
+
+ public static void writeInt(int value, byte[] bytes, int offset) {
+ bytes[offset++] = (byte) (value >> 24);
+ bytes[offset++] = (byte) (value >> 16);
+ bytes[offset++] = (byte) (value >> 8);
+ bytes[offset++] = (byte) (value);
+ }
+
+ public static void writeLong(long value, byte[] bytes, int offset) {
+ bytes[offset++] = (byte) (value >> 56);
+ bytes[offset++] = (byte) (value >> 48);
+ bytes[offset++] = (byte) (value >> 40);
+ bytes[offset++] = (byte) (value >> 32);
+ bytes[offset++] = (byte) (value >> 24);
+ bytes[offset++] = (byte) (value >> 16);
+ bytes[offset++] = (byte) (value >> 8);
+ bytes[offset++] = (byte) (value);
+ }
+
+ public static void writeDouble(double value, byte[] bytes, int offset) {
+ long lValue = Double.doubleToLongBits(value);
+ writeLong(lValue, bytes, offset);
+ }
+
+ public static void writeFloat(float value, byte[] bytes, int offset) {
+ int iValue = Float.floatToIntBits(value);
+ writeInt(iValue, bytes, offset);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ColumnExpressionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ColumnExpressionEvaluator.java
new file mode 100644
index 0000000..5647f6a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ColumnExpressionEvaluator.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeColumnEvaluator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ColumnExpressionEvaluator extends AbstractExpressionEvaluator {
+
+ public ColumnExpressionEvaluator(ExprNodeColumnDesc expr, ObjectInspector oi, IDataOutputProvider output)
+ throws AlgebricksException {
+ super(new ExprNodeColumnEvaluator(expr), oi, output);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ConstantExpressionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ConstantExpressionEvaluator.java
new file mode 100644
index 0000000..d8796ea
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ConstantExpressionEvaluator.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeConstantEvaluator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ConstantExpressionEvaluator extends AbstractExpressionEvaluator {
+
+ public ConstantExpressionEvaluator(ExprNodeConstantDesc expr, ObjectInspector oi, IDataOutputProvider output)
+ throws AlgebricksException {
+ super(new ExprNodeConstantEvaluator(expr), oi, output);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ExpressionTranslator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ExpressionTranslator.java
new file mode 100644
index 0000000..c40ef73
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ExpressionTranslator.java
@@ -0,0 +1,213 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeNullDesc;
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.expression.ExpressionConstant;
+import edu.uci.ics.hivesterix.logical.expression.HiveAlgebricksBuiltInFunctionMap;
+import edu.uci.ics.hivesterix.logical.expression.HiveFunctionInfo;
+import edu.uci.ics.hivesterix.logical.expression.HivesterixConstantValue;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+
+public class ExpressionTranslator {
+
+ public static Object getHiveExpression(ILogicalExpression expr, IVariableTypeEnvironment env) throws Exception {
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ /**
+ * function expression
+ */
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ IFunctionInfo funcInfo = funcExpr.getFunctionInfo();
+ FunctionIdentifier fid = funcInfo.getFunctionIdentifier();
+
+ if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {
+ Object info = ((HiveFunctionInfo) funcInfo).getInfo();
+ ExprNodeFieldDesc desc = (ExprNodeFieldDesc) info;
+ return new ExprNodeFieldDesc(desc.getTypeInfo(), desc.getDesc(), desc.getFieldName(), desc.getIsList());
+ }
+
+ if (fid.getName().equals(ExpressionConstant.NULL)) {
+ return new ExprNodeNullDesc();
+ }
+
+ /**
+ * argument expressions: translate argument expressions recursively
+ * first, this logic is shared in scalar, aggregation and unnesting
+ * function
+ */
+ List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();
+ List<ExprNodeDesc> parameters = new ArrayList<ExprNodeDesc>();
+ for (Mutable<ILogicalExpression> argument : arguments) {
+ /**
+ * parameters could not be aggregate function desc
+ */
+ ExprNodeDesc parameter = (ExprNodeDesc) getHiveExpression(argument.getValue(), env);
+ parameters.add(parameter);
+ }
+
+ /**
+ * get expression
+ */
+ if (funcExpr instanceof ScalarFunctionCallExpression) {
+ String udfName = HiveAlgebricksBuiltInFunctionMap.INSTANCE.getHiveFunctionName(fid);
+ GenericUDF udf;
+ if (udfName != null) {
+ /**
+ * get corresponding function info for built-in functions
+ */
+ FunctionInfo fInfo = FunctionRegistry.getFunctionInfo(udfName);
+ udf = fInfo.getGenericUDF();
+
+ int inputSize = parameters.size();
+ List<ExprNodeDesc> currentDescs = new ArrayList<ExprNodeDesc>();
+
+ // generate expression tree if necessary
+ while (inputSize > 2) {
+ int pairs = inputSize / 2;
+ for (int i = 0; i < pairs; i++) {
+ List<ExprNodeDesc> descs = new ArrayList<ExprNodeDesc>();
+ descs.add(parameters.get(2 * i));
+ descs.add(parameters.get(2 * i + 1));
+ ExprNodeDesc desc = ExprNodeGenericFuncDesc.newInstance(udf, descs);
+ currentDescs.add(desc);
+ }
+
+ if (inputSize % 2 != 0) {
+ // List<ExprNodeDesc> descs = new
+ // ArrayList<ExprNodeDesc>();
+ // ExprNodeDesc lastExpr =
+ // currentDescs.remove(currentDescs.size() - 1);
+ // descs.add(lastExpr);
+ currentDescs.add(parameters.get(inputSize - 1));
+ // ExprNodeDesc desc =
+ // ExprNodeGenericFuncDesc.newInstance(udf, descs);
+ // currentDescs.add(desc);
+ }
+ inputSize = currentDescs.size();
+ parameters.clear();
+ parameters.addAll(currentDescs);
+ currentDescs.clear();
+ }
+
+ } else {
+ Object secondInfo = ((HiveFunctionInfo) funcInfo).getInfo();
+ if (secondInfo != null) {
+
+ /**
+ * for GenericUDFBridge: we should not call get type of
+ * this hive expression, because parameters may have
+ * been changed!
+ */
+ ExprNodeGenericFuncDesc hiveExpr = (ExprNodeGenericFuncDesc) ((HiveFunctionInfo) funcInfo)
+ .getInfo();
+ udf = hiveExpr.getGenericUDF();
+ } else {
+ /**
+ * for other generic UDF
+ */
+ Class<?> udfClass;
+ try {
+ udfClass = Class.forName(fid.getName());
+ udf = (GenericUDF) udfClass.newInstance();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ }
+ }
+ }
+ /**
+ * get hive generic function expression
+ */
+ ExprNodeDesc desc = ExprNodeGenericFuncDesc.newInstance(udf, parameters);
+ return desc;
+ } else if (funcExpr instanceof AggregateFunctionCallExpression) {
+ /**
+ * hive aggregation info
+ */
+ AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo())
+ .getInfo();
+ /**
+ * set parameters
+ */
+ aggregateDesc.setParameters((ArrayList<ExprNodeDesc>) parameters);
+
+ List<TypeInfo> originalParameterTypeInfos = new ArrayList<TypeInfo>();
+ for (ExprNodeDesc parameter : parameters) {
+ if (parameter.getTypeInfo() instanceof StructTypeInfo) {
+ originalParameterTypeInfos.add(TypeInfoFactory.doubleTypeInfo);
+ } else
+ originalParameterTypeInfos.add(parameter.getTypeInfo());
+ }
+
+ GenericUDAFEvaluator eval = FunctionRegistry.getGenericUDAFEvaluator(
+ aggregateDesc.getGenericUDAFName(), originalParameterTypeInfos, aggregateDesc.getDistinct(),
+ false);
+
+ AggregationDesc newAggregateDesc = new AggregationDesc(aggregateDesc.getGenericUDAFName(), eval,
+ aggregateDesc.getParameters(), aggregateDesc.getDistinct(), aggregateDesc.getMode());
+ return newAggregateDesc;
+ } else if (funcExpr instanceof UnnestingFunctionCallExpression) {
+ /**
+ * type inference for UDTF function
+ */
+ UDTFDesc hiveDesc = (UDTFDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo()).getInfo();
+ String funcName = hiveDesc.getUDTFName();
+ FunctionInfo fi = FunctionRegistry.getFunctionInfo(funcName);
+ GenericUDTF udtf = fi.getGenericUDTF();
+ UDTFDesc desc = new UDTFDesc(udtf);
+ return desc;
+ } else {
+ throw new IllegalStateException("unrecognized function expression " + expr.getClass().getName());
+ }
+ } else if ((expr.getExpressionTag() == LogicalExpressionTag.VARIABLE)) {
+ /**
+ * get type for variable in the environment
+ */
+ VariableReferenceExpression varExpr = (VariableReferenceExpression) expr;
+ LogicalVariable var = varExpr.getVariableReference();
+ TypeInfo typeInfo = (TypeInfo) env.getVarType(var);
+ ExprNodeDesc desc = new ExprNodeColumnDesc(typeInfo, var.toString(), "", false);
+ return desc;
+ } else if ((expr.getExpressionTag() == LogicalExpressionTag.CONSTANT)) {
+ /**
+ * get expression for constant in the environment
+ */
+ ConstantExpression varExpr = (ConstantExpression) expr;
+ Object value = ((HivesterixConstantValue) varExpr.getValue()).getObject();
+ ExprNodeDesc desc = new ExprNodeConstantDesc(value);
+ return desc;
+ } else {
+ throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FieldExpressionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FieldExpressionEvaluator.java
new file mode 100644
index 0000000..35560b6
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FieldExpressionEvaluator.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeFieldEvaluator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class FieldExpressionEvaluator extends AbstractExpressionEvaluator {
+
+ public FieldExpressionEvaluator(ExprNodeFieldDesc expr, ObjectInspector oi, IDataOutputProvider output)
+ throws AlgebricksException {
+ super(new ExprNodeFieldEvaluator(expr), oi, output);
+ }
+
+}
\ No newline at end of file
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FunctionExpressionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FunctionExpressionEvaluator.java
new file mode 100644
index 0000000..7ffec7a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FunctionExpressionEvaluator.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeGenericFuncEvaluator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class FunctionExpressionEvaluator extends AbstractExpressionEvaluator {
+
+ public FunctionExpressionEvaluator(ExprNodeGenericFuncDesc expr, ObjectInspector oi, IDataOutputProvider output)
+ throws AlgebricksException {
+ super(new ExprNodeGenericFuncEvaluator(expr), oi, output);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/NullExpressionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/NullExpressionEvaluator.java
new file mode 100644
index 0000000..ca60385
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/NullExpressionEvaluator.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import org.apache.hadoop.hive.ql.exec.ExprNodeNullEvaluator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeNullDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class NullExpressionEvaluator extends AbstractExpressionEvaluator {
+
+ public NullExpressionEvaluator(ExprNodeNullDesc expr, ObjectInspector oi, IDataOutputProvider output)
+ throws AlgebricksException {
+ super(new ExprNodeNullEvaluator(expr), oi, output);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/SerializableBuffer.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/SerializableBuffer.java
new file mode 100644
index 0000000..676989e
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/SerializableBuffer.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer;
+
+public interface SerializableBuffer extends AggregationBuffer {
+
+ public void deSerializeAggBuffer(byte[] data, int start, int len);
+
+ public void serializeAggBuffer(byte[] data, int start, int len);
+
+ public void serializeAggBuffer(DataOutput output) throws IOException;
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
new file mode 100644
index 0000000..284d3d2
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
@@ -0,0 +1,143 @@
+package edu.uci.ics.hivesterix.runtime.evaluator;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+import org.apache.hadoop.hive.ql.udf.generic.Collector;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.io.BytesWritable;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.serde.lazy.LazyColumnar;
+import edu.uci.ics.hivesterix.serde.lazy.LazyFactory;
+import edu.uci.ics.hivesterix.serde.lazy.LazyObject;
+import edu.uci.ics.hivesterix.serde.lazy.LazySerDe;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyUnnestingFunction;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class UDTFFunctionEvaluator implements ICopyUnnestingFunction, Collector {
+
+ /**
+ * udtf function
+ */
+ private UDTFDesc func;
+
+ /**
+ * input object inspector
+ */
+ private ObjectInspector inputInspector;
+
+ /**
+ * output object inspector
+ */
+ private ObjectInspector outputInspector;
+
+ /**
+ * object inspector for udtf
+ */
+ private ObjectInspector[] udtfInputOIs;
+
+ /**
+ * generic udtf
+ */
+ private GenericUDTF udtf;
+
+ /**
+ * data output
+ */
+ private DataOutput out;
+
+ /**
+ * the input row object
+ */
+ private LazyColumnar cachedRowObject;
+
+ /**
+ * cached row object (input)
+ */
+ private Object[] cachedInputObjects;
+
+ /**
+ * serialization/deserialization
+ */
+ private SerDe lazySerDe;
+
+ /**
+ * columns feed into UDTF
+ */
+ private int[] columns;
+
+ public UDTFFunctionEvaluator(UDTFDesc desc, Schema schema, int[] cols, DataOutput output) {
+ this.func = desc;
+ this.inputInspector = schema.toObjectInspector();
+ udtf = func.getGenericUDTF();
+ out = output;
+ columns = cols;
+ }
+
+ @Override
+ public void init(IFrameTupleReference tuple) throws AlgebricksException {
+ cachedInputObjects = new LazyObject[columns.length];
+ try {
+ cachedRowObject = (LazyColumnar) LazyFactory.createLazyObject(inputInspector);
+ outputInspector = udtf.initialize(udtfInputOIs);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ udtf.setCollector(this);
+ lazySerDe = new LazySerDe();
+ readIntoCache(tuple);
+ }
+
+ @Override
+ public boolean step() throws AlgebricksException {
+ try {
+ udtf.process(cachedInputObjects);
+ return true;
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ /**
+ * bind the tuple reference to the cached row object
+ *
+ * @param r
+ */
+ private void readIntoCache(IFrameTupleReference r) {
+ cachedRowObject.init(r);
+ for (int i = 0; i < cachedInputObjects.length; i++) {
+ cachedInputObjects[i] = cachedRowObject.getField(columns[i]);
+ }
+ }
+
+ /**
+ * serialize the result
+ *
+ * @param result
+ * the evaluation result
+ * @throws IOException
+ * @throws AlgebricksException
+ */
+ private void serializeResult(Object result) throws SerDeException, IOException {
+ BytesWritable outputWritable = (BytesWritable) lazySerDe.serialize(result, outputInspector);
+ out.write(outputWritable.getBytes(), 0, outputWritable.getLength());
+ }
+
+ @Override
+ public void collect(Object input) throws HiveException {
+ try {
+ serializeResult(input);
+ } catch (IOException e) {
+ throw new HiveException(e);
+ } catch (SerDeException e) {
+ throw new HiveException(e);
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java b/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
new file mode 100644
index 0000000..2f16573
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
@@ -0,0 +1,557 @@
+package edu.uci.ics.hivesterix.runtime.exec;
+
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ConditionalTask;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.MapRedTask;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
+import org.apache.hadoop.hive.ql.plan.FetchWork;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.hive.ql.plan.MapredLocalWork;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.HiveExpressionTypeComputer;
+import edu.uci.ics.hivesterix.logical.expression.HiveMergeAggregationExpressionFactory;
+import edu.uci.ics.hivesterix.logical.expression.HiveNullableTypeComputer;
+import edu.uci.ics.hivesterix.logical.expression.HivePartialAggregationTypeComputer;
+import edu.uci.ics.hivesterix.logical.plan.HiveAlgebricksTranslator;
+import edu.uci.ics.hivesterix.logical.plan.HiveLogicalPlanAndMetaData;
+import edu.uci.ics.hivesterix.optimizer.rulecollections.HiveRuleCollections;
+import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
+import edu.uci.ics.hivesterix.runtime.factory.evaluator.HiveExpressionRuntimeProvider;
+import edu.uci.ics.hivesterix.runtime.factory.nullwriter.HiveNullWriterFactory;
+import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryBooleanInspectorFactory;
+import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryIntegerInspectorFactory;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy.Policy;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryComparatorFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveNormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HivePrinterFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveSerializerDeserializerProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveTypeTraitProvider;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder.DefaultOptimizationContextFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class HyracksExecutionEngine implements IExecutionEngine {
+
+ private static final Log LOG = LogFactory.getLog(HyracksExecutionEngine.class.getName());
+
+ // private static final String[] locConstraints = {}
+
+ private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_LOGICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+ private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_PHYSICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+ static {
+ SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+ SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
+ SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+ HiveRuleCollections.NORMALIZATION));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ HiveRuleCollections.COND_PUSHDOWN_AND_JOIN_INFERENCE));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+ HiveRuleCollections.LOAD_FIELDS));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ HiveRuleCollections.OP_PUSHDOWN));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ HiveRuleCollections.DATA_EXCHANGE));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ HiveRuleCollections.CONSOLIDATION));
+
+ DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ HiveRuleCollections.PHYSICAL_PLAN_REWRITES));
+ DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ HiveRuleCollections.prepareJobGenRules));
+ }
+
+ /**
+ * static configurations for compiler
+ */
+ private HeuristicCompilerFactoryBuilder builder;
+
+ /**
+ * compiler
+ */
+ private ICompiler compiler;
+
+ /**
+ * physical optimization config
+ */
+ private PhysicalOptimizationConfig physicalOptimizationConfig;
+
+ /**
+ * final ending operators
+ */
+ private List<Operator> leaveOps = new ArrayList<Operator>();
+
+ /**
+ * tasks that are already visited
+ */
+ private Map<Task<? extends Serializable>, Boolean> tasksVisited = new HashMap<Task<? extends Serializable>, Boolean>();
+
+ /**
+ * hyracks job spec
+ */
+ private JobSpecification jobSpec;
+
+ /**
+ * hive configuration
+ */
+ private HiveConf conf;
+
+ /**
+ * plan printer
+ */
+ private PrintWriter planPrinter;
+
+ public HyracksExecutionEngine(HiveConf conf) {
+ this.conf = conf;
+ init(conf);
+ }
+
+ public HyracksExecutionEngine(HiveConf conf, PrintWriter planPrinter) {
+ this.conf = conf;
+ this.planPrinter = planPrinter;
+ init(conf);
+ }
+
+ private void init(HiveConf conf) {
+ builder = new HeuristicCompilerFactoryBuilder(DefaultOptimizationContextFactory.INSTANCE);
+ builder.setLogicalRewrites(DEFAULT_LOGICAL_REWRITES);
+ builder.setPhysicalRewrites(DEFAULT_PHYSICAL_REWRITES);
+ builder.setIMergeAggregationExpressionFactory(HiveMergeAggregationExpressionFactory.INSTANCE);
+ builder.setExpressionTypeComputer(HiveExpressionTypeComputer.INSTANCE);
+ builder.setNullableTypeComputer(HiveNullableTypeComputer.INSTANCE);
+
+ long memSizeExternalGby = conf.getLong("hive.algebricks.groupby.external.memory", 268435456);
+ long memSizeExternalSort = conf.getLong("hive.algebricks.sort.memory", 536870912);
+ int frameSize = conf.getInt("hive.algebricks.framesize", 32768);
+
+ physicalOptimizationConfig = new PhysicalOptimizationConfig();
+ int frameLimitExtGby = (int) (memSizeExternalGby / frameSize);
+ physicalOptimizationConfig.setMaxFramesExternalGroupBy(frameLimitExtGby);
+ int frameLimitExtSort = (int) (memSizeExternalSort / frameSize);
+ physicalOptimizationConfig.setMaxFramesExternalSort(frameLimitExtSort);
+ builder.setPhysicalOptimizationConfig(physicalOptimizationConfig);
+ }
+
+ @Override
+ public int compileJob(List<Task<? extends Serializable>> rootTasks) {
+ // clean up
+ leaveOps.clear();
+ tasksVisited.clear();
+ jobSpec = null;
+
+ HashMap<String, PartitionDesc> aliasToPath = new HashMap<String, PartitionDesc>();
+ List<Operator> rootOps = generateRootOperatorDAG(rootTasks, aliasToPath);
+
+ // get all leave Ops
+ getLeaves(rootOps, leaveOps);
+
+ HiveAlgebricksTranslator translator = new HiveAlgebricksTranslator();
+ try {
+ translator.translate(rootOps, null, aliasToPath);
+
+ ILogicalPlan plan = translator.genLogicalPlan();
+
+ if (plan.getRoots() != null && plan.getRoots().size() > 0 && plan.getRoots().get(0).getValue() != null) {
+ translator.printOperators();
+ System.out.println("translate complete");
+ ILogicalPlanAndMetadata planAndMetadata = new HiveLogicalPlanAndMetaData(plan,
+ translator.getMetadataProvider());
+
+ ICompilerFactory compilerFactory = builder.create();
+ compiler = compilerFactory.createCompiler(planAndMetadata.getPlan(),
+ planAndMetadata.getMetadataProvider(), translator.getVariableCounter());
+
+ // run optimization and re-writing rules for Hive plan
+ compiler.optimize();
+
+ // print optimized plan
+ LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
+ StringBuilder buffer = new StringBuilder();
+ PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
+ String planStr = buffer.toString();
+ System.out.println(planStr);
+
+ if (planPrinter != null)
+ planPrinter.print(planStr);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ return 1;
+ }
+
+ return 0;
+ }
+
+ private void codeGen() throws AlgebricksException {
+ // number of cpu cores in the cluster
+ builder.setClusterLocations(new AlgebricksAbsolutePartitionConstraint(ConfUtil.getNCs()));
+ //builder.setClusterTopology(ConfUtil.getClusterTopology());
+ builder.setBinaryBooleanInspectorFactory(HiveBinaryBooleanInspectorFactory.INSTANCE);
+ builder.setBinaryIntegerInspectorFactory(HiveBinaryIntegerInspectorFactory.INSTANCE);
+ builder.setComparatorFactoryProvider(HiveBinaryComparatorFactoryProvider.INSTANCE);
+ builder.setExpressionRuntimeProvider(HiveExpressionRuntimeProvider.INSTANCE);
+ builder.setHashFunctionFactoryProvider(HiveBinaryHashFunctionFactoryProvider.INSTANCE);
+ builder.setPrinterProvider(HivePrinterFactoryProvider.INSTANCE);
+ builder.setSerializerDeserializerProvider(HiveSerializerDeserializerProvider.INSTANCE);
+ builder.setNullWriterFactory(HiveNullWriterFactory.INSTANCE);
+ builder.setNormalizedKeyComputerFactoryProvider(HiveNormalizedKeyComputerFactoryProvider.INSTANCE);
+ builder.setPartialAggregationTypeComputer(HivePartialAggregationTypeComputer.INSTANCE);
+ builder.setTypeTraitProvider(HiveTypeTraitProvider.INSTANCE);
+
+ jobSpec = compiler.createJob(null);
+
+ // set the policy
+ String policyStr = conf.get("hive.hyracks.connectorpolicy");
+ if (policyStr == null)
+ policyStr = "PIPELINING";
+ Policy policyValue = Policy.valueOf(policyStr);
+ jobSpec.setConnectorPolicyAssignmentPolicy(new HiveConnectorPolicyAssignmentPolicy(policyValue));
+
+ // execute the job
+ System.out.println(jobSpec.toString());
+ }
+
+ @Override
+ public int executeJob() {
+ try {
+ codeGen();
+ executeHyraxJob(jobSpec);
+ } catch (Exception e) {
+ e.printStackTrace();
+ return 1;
+ }
+ return 0;
+ }
+
+ private List<Operator> generateRootOperatorDAG(List<Task<? extends Serializable>> rootTasks,
+ HashMap<String, PartitionDesc> aliasToPath) {
+
+ List<Operator> rootOps = new ArrayList<Operator>();
+ List<Task<? extends Serializable>> toDelete = new ArrayList<Task<? extends Serializable>>();
+ tasksVisited.clear();
+
+ for (int i = rootTasks.size() - 1; i >= 0; i--) {
+ /**
+ * list of map-reduce tasks
+ */
+ Task<? extends Serializable> task = rootTasks.get(i);
+ // System.out.println("!" + task.getName());
+
+ if (task instanceof MapRedTask) {
+ List<Operator> mapRootOps = articulateMapReduceOperators(task, rootOps, aliasToPath, rootTasks);
+ if (i == 0)
+ rootOps.addAll(mapRootOps);
+ else {
+ List<Operator> leaves = new ArrayList<Operator>();
+ getLeaves(rootOps, leaves);
+
+ List<Operator> mapChildren = new ArrayList<Operator>();
+ for (Operator childMap : mapRootOps) {
+ if (childMap instanceof TableScanOperator) {
+ TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
+ if (topDesc == null)
+ mapChildren.add(childMap);
+ else {
+ rootOps.add(childMap);
+ }
+ } else
+ mapChildren.add(childMap);
+ }
+
+ if (mapChildren.size() > 0) {
+ for (Operator leaf : leaves)
+ leaf.setChildOperators(mapChildren);
+ for (Operator child : mapChildren)
+ child.setParentOperators(leaves);
+ }
+ }
+
+ MapredWork mr = (MapredWork) task.getWork();
+ HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
+
+ addAliasToPartition(aliasToPath, map);
+ toDelete.add(task);
+ }
+ }
+
+ for (Task<? extends Serializable> task : toDelete)
+ rootTasks.remove(task);
+
+ return rootOps;
+ }
+
+ private void addAliasToPartition(HashMap<String, PartitionDesc> aliasToPath, HashMap<String, PartitionDesc> map) {
+ Iterator<String> keys = map.keySet().iterator();
+ while (keys.hasNext()) {
+ String key = keys.next();
+ PartitionDesc part = map.get(key);
+ String[] names = key.split(":");
+ for (String name : names) {
+ aliasToPath.put(name, part);
+ }
+ }
+ }
+
+ private List<Operator> articulateMapReduceOperators(Task task, List<Operator> rootOps,
+ HashMap<String, PartitionDesc> aliasToPath, List<Task<? extends Serializable>> rootTasks) {
+ // System.out.println("!"+task.getName());
+ if (!(task instanceof MapRedTask)) {
+ if (!(task instanceof ConditionalTask)) {
+ rootTasks.add(task);
+ return null;
+ } else {
+ // remove map-reduce branches in condition task
+ ConditionalTask condition = (ConditionalTask) task;
+ List<Task<? extends Serializable>> branches = condition.getListTasks();
+ boolean existMR = false;
+ for (int i = branches.size() - 1; i >= 0; i--) {
+ Task branch = branches.get(i);
+ if (branch instanceof MapRedTask) {
+ return articulateMapReduceOperators(branch, rootOps, aliasToPath, rootTasks);
+ }
+ }
+ rootTasks.add(task);
+ return null;
+ }
+ }
+
+ MapredWork mr = (MapredWork) task.getWork();
+ HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
+
+ // put all aliasToParitionDesc mapping into the map
+ addAliasToPartition(aliasToPath, map);
+
+ MapRedTask mrtask = (MapRedTask) task;
+ MapredWork work = (MapredWork) mrtask.getWork();
+ HashMap<String, Operator<? extends Serializable>> operators = work.getAliasToWork();
+
+ Set entries = operators.entrySet();
+ Iterator<Entry<String, Operator>> iterator = entries.iterator();
+ List<Operator> mapRootOps = new ArrayList<Operator>();
+
+ // get map root operators
+ while (iterator.hasNext()) {
+ Operator next = iterator.next().getValue();
+ if (!mapRootOps.contains(next)) {
+ // clear that only for the case of union
+ mapRootOps.add(next);
+ }
+ }
+
+ // get map local work
+ MapredLocalWork localWork = work.getMapLocalWork();
+ if (localWork != null) {
+ HashMap<String, Operator<? extends Serializable>> localOperators = localWork.getAliasToWork();
+
+ Set localEntries = localOperators.entrySet();
+ Iterator<Entry<String, Operator>> localIterator = localEntries.iterator();
+ while (localIterator.hasNext()) {
+ mapRootOps.add(localIterator.next().getValue());
+ }
+
+ HashMap<String, FetchWork> localFetch = localWork.getAliasToFetchWork();
+ Set localFetchEntries = localFetch.entrySet();
+ Iterator<Entry<String, FetchWork>> localFetchIterator = localFetchEntries.iterator();
+ while (localFetchIterator.hasNext()) {
+ Entry<String, FetchWork> fetchMap = localFetchIterator.next();
+ FetchWork fetch = fetchMap.getValue();
+ String alias = fetchMap.getKey();
+ List<PartitionDesc> dirPart = fetch.getPartDesc();
+
+ // temporary hack: put the first partitionDesc into the map
+ aliasToPath.put(alias, dirPart.get(0));
+ }
+ }
+
+ Boolean visited = tasksVisited.get(task);
+ if (visited != null && visited.booleanValue() == true) {
+ return mapRootOps;
+ }
+
+ // do that only for union operator
+ for (Operator op : mapRootOps)
+ if (op.getParentOperators() != null)
+ op.getParentOperators().clear();
+
+ List<Operator> mapLeaves = new ArrayList<Operator>();
+ downToLeaves(mapRootOps, mapLeaves);
+ List<Operator> reduceOps = new ArrayList<Operator>();
+
+ if (work.getReducer() != null)
+ reduceOps.add(work.getReducer());
+
+ for (Operator mapLeaf : mapLeaves) {
+ mapLeaf.setChildOperators(reduceOps);
+ }
+
+ for (Operator reduceOp : reduceOps) {
+ if (reduceOp != null)
+ reduceOp.setParentOperators(mapLeaves);
+ }
+
+ List<Operator> leafs = new ArrayList<Operator>();
+ if (reduceOps.size() > 0) {
+ downToLeaves(reduceOps, leafs);
+ } else {
+ leafs = mapLeaves;
+ }
+
+ List<Operator> mapChildren = new ArrayList<Operator>();
+ if (task.getChildTasks() != null && task.getChildTasks().size() > 0) {
+ System.out.println("have child tasks!!");
+ for (Object child : task.getChildTasks()) {
+ System.out.println(child.getClass().getName());
+ List<Operator> childMapOps = articulateMapReduceOperators((Task) child, rootOps, aliasToPath, rootTasks);
+ if (childMapOps == null)
+ continue;
+
+ for (Operator childMap : childMapOps) {
+ if (childMap instanceof TableScanOperator) {
+ TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
+ if (topDesc == null)
+ mapChildren.add(childMap);
+ else {
+ rootOps.add(childMap);
+ }
+ } else {
+ // if not table scan, add the child
+ mapChildren.add(childMap);
+ }
+ }
+ }
+
+ if (mapChildren.size() > 0) {
+ int i = 0;
+ for (Operator leaf : leafs) {
+ if (leaf.getChildOperators() == null || leaf.getChildOperators().size() == 0)
+ leaf.setChildOperators(new ArrayList<Operator>());
+ leaf.getChildOperators().add(mapChildren.get(i));
+ i++;
+ }
+ i = 0;
+ for (Operator child : mapChildren) {
+ if (child.getParentOperators() == null || child.getParentOperators().size() == 0)
+ child.setParentOperators(new ArrayList<Operator>());
+ child.getParentOperators().add(leafs.get(i));
+ i++;
+ }
+ }
+ }
+
+ // mark this task as visited
+ this.tasksVisited.put(task, true);
+ return mapRootOps;
+ }
+
+ /**
+ * down to leaf nodes
+ *
+ * @param ops
+ * @param leaves
+ */
+ private void downToLeaves(List<Operator> ops, List<Operator> leaves) {
+
+ // Operator currentOp;
+ for (Operator op : ops) {
+ if (op != null && op.getChildOperators() != null && op.getChildOperators().size() > 0) {
+ downToLeaves(op.getChildOperators(), leaves);
+ } else {
+ if (op != null && leaves.indexOf(op) < 0)
+ leaves.add(op);
+ }
+ }
+ }
+
+ private void getLeaves(List<Operator> roots, List<Operator> currentLeaves) {
+ for (Operator op : roots) {
+ List<Operator> children = op.getChildOperators();
+ if (children == null || children.size() <= 0) {
+ currentLeaves.add(op);
+ } else {
+ getLeaves(children, currentLeaves);
+ }
+ }
+ }
+
+ private void executeHyraxJob(JobSpecification job) throws Exception {
+ String ipAddress = conf.get("hive.hyracks.host");
+ int port = Integer.parseInt(conf.get("hive.hyracks.port"));
+ String applicationName = conf.get("hive.hyracks.app");
+ System.out.println("connect to " + ipAddress + " " + port);
+
+ IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+
+ System.out.println("get connected");
+ long start = System.currentTimeMillis();
+ JobId jobId = hcc.startJob(applicationName, job);
+ hcc.waitForCompletion(jobId);
+
+ System.out.println("job finished: " + jobId.toString());
+ // call all leave nodes to end
+ for (Operator leaf : leaveOps) {
+ jobClose(leaf);
+ }
+
+ long end = System.currentTimeMillis();
+ System.err.println(start + " " + end + " " + (end - start));
+ }
+
+ /**
+ * mv to final directory on hdfs (not real final)
+ *
+ * @param leaf
+ * @throws Exception
+ */
+ private void jobClose(Operator leaf) throws Exception {
+ FileSinkOperator fsOp = (FileSinkOperator) leaf;
+ FileSinkDesc desc = fsOp.getConf();
+ boolean isNativeTable = !desc.getTableInfo().isNonNative();
+ if ((conf != null) && isNativeTable) {
+ String specPath = desc.getDirName();
+ DynamicPartitionCtx dpCtx = desc.getDynPartCtx();
+ // for 0.7.0
+ fsOp.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx);
+ // for 0.8.0
+ // Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx,
+ // desc);
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/exec/IExecutionEngine.java b/src/main/java/edu/uci/ics/hivesterix/runtime/exec/IExecutionEngine.java
new file mode 100644
index 0000000..c64a39b
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/exec/IExecutionEngine.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.hivesterix.runtime.exec;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.Task;
+
+public interface IExecutionEngine {
+
+ /**
+ * compile the job
+ *
+ * @param rootTasks
+ * : Hive MapReduce plan
+ * @return 0 pass, 1 fail
+ */
+ public int compileJob(List<Task<? extends Serializable>> rootTasks);
+
+ /**
+ * execute the job with latest compiled plan
+ *
+ * @return
+ */
+ public int executeJob();
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryAscComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryAscComparatorFactory.java
new file mode 100644
index 0000000..f3b76e4
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryAscComparatorFactory.java
@@ -0,0 +1,34 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveByteBinaryAscComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveByteBinaryAscComparatorFactory INSTANCE = new HiveByteBinaryAscComparatorFactory();
+
+ private HiveByteBinaryAscComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private byte left;
+ private byte right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = b1[s1];
+ right = b2[s2];
+ if (left > right)
+ return 1;
+ else if (left == right)
+ return 0;
+ else
+ return -1;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryDescComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryDescComparatorFactory.java
new file mode 100644
index 0000000..8d452dc
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryDescComparatorFactory.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveByteBinaryDescComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveByteBinaryDescComparatorFactory INSTANCE = new HiveByteBinaryDescComparatorFactory();
+
+ private HiveByteBinaryDescComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private byte left;
+ private byte right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = b1[s1];
+ right = b2[s2];
+ if (left > right)
+ return -1;
+ else if (left == right)
+ return 0;
+ else
+ return 1;
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryAscComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryAscComparatorFactory.java
new file mode 100644
index 0000000..0b5350a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryAscComparatorFactory.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveDoubleBinaryAscComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveDoubleBinaryAscComparatorFactory INSTANCE = new HiveDoubleBinaryAscComparatorFactory();
+
+ private HiveDoubleBinaryAscComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private double left;
+ private double right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b1, s1));
+ right = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b2, s2));
+ if (left > right)
+ return 1;
+ else if (left == right)
+ return 0;
+ else
+ return -1;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryDescComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryDescComparatorFactory.java
new file mode 100644
index 0000000..2405956
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryDescComparatorFactory.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveDoubleBinaryDescComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveDoubleBinaryDescComparatorFactory INSTANCE = new HiveDoubleBinaryDescComparatorFactory();
+
+ private HiveDoubleBinaryDescComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private double left;
+ private double right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b1, s1));
+ right = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b2, s2));
+ if (left > right)
+ return -1;
+ else if (left == right)
+ return 0;
+ else
+ return 1;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryAscComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryAscComparatorFactory.java
new file mode 100644
index 0000000..05a43e6
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryAscComparatorFactory.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveFloatBinaryAscComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveFloatBinaryAscComparatorFactory INSTANCE = new HiveFloatBinaryAscComparatorFactory();
+
+ private HiveFloatBinaryAscComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private float left;
+ private float right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b1, s1));
+ right = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b2, s2));
+ if (left > right)
+ return 1;
+ else if (left == right)
+ return 0;
+ else
+ return -1;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryDescComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryDescComparatorFactory.java
new file mode 100644
index 0000000..2c44f97
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryDescComparatorFactory.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveFloatBinaryDescComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveFloatBinaryDescComparatorFactory INSTANCE = new HiveFloatBinaryDescComparatorFactory();
+
+ private HiveFloatBinaryDescComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private float left;
+ private float right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b1, s1));
+ right = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b2, s2));
+ if (left > right)
+ return -1;
+ else if (left == right)
+ return 0;
+ else
+ return 1;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryAscComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryAscComparatorFactory.java
new file mode 100644
index 0000000..0127791
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryAscComparatorFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveIntegerBinaryAscComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static final HiveIntegerBinaryAscComparatorFactory INSTANCE = new HiveIntegerBinaryAscComparatorFactory();
+
+ private HiveIntegerBinaryAscComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private VInt left = new VInt();
+ private VInt right = new VInt();
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ LazyUtils.readVInt(b1, s1, left);
+ LazyUtils.readVInt(b2, s2, right);
+
+ if (left.length != l1 || right.length != l2)
+ throw new IllegalArgumentException("length mismatch in int comparator function actual: "
+ + left.length + "," + right.length + " expected " + l1 + "," + l2);
+
+ if (left.value > right.value)
+ return 1;
+ else if (left.value == right.value)
+ return 0;
+ else
+ return -1;
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryDescComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryDescComparatorFactory.java
new file mode 100644
index 0000000..5116337
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryDescComparatorFactory.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveIntegerBinaryDescComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static final HiveIntegerBinaryDescComparatorFactory INSTANCE = new HiveIntegerBinaryDescComparatorFactory();
+
+ private HiveIntegerBinaryDescComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private VInt left = new VInt();
+ private VInt right = new VInt();
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ LazyUtils.readVInt(b1, s1, left);
+ LazyUtils.readVInt(b2, s2, right);
+ if (left.length != l1 || right.length != l2)
+ throw new IllegalArgumentException("length mismatch in int comparator function actual: "
+ + left.length + " expected " + l1);
+ if (left.value > right.value)
+ return -1;
+ else if (left.value == right.value)
+ return 0;
+ else
+ return 1;
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryAscComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryAscComparatorFactory.java
new file mode 100644
index 0000000..fa416a9
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryAscComparatorFactory.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VLong;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveLongBinaryAscComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static final HiveLongBinaryAscComparatorFactory INSTANCE = new HiveLongBinaryAscComparatorFactory();
+
+ private HiveLongBinaryAscComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private VLong left = new VLong();
+ private VLong right = new VLong();
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ LazyUtils.readVLong(b1, s1, left);
+ LazyUtils.readVLong(b2, s2, right);
+ if (left.length != l1 || right.length != l2)
+ throw new IllegalArgumentException("length mismatch in int comparator function actual: "
+ + left.length + " expected " + l1);
+ if (left.value > right.value)
+ return 1;
+ else if (left.value == right.value)
+ return 0;
+ else
+ return -1;
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryDescComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryDescComparatorFactory.java
new file mode 100644
index 0000000..e72dc62
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryDescComparatorFactory.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VLong;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveLongBinaryDescComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static final HiveLongBinaryDescComparatorFactory INSTANCE = new HiveLongBinaryDescComparatorFactory();
+
+ private HiveLongBinaryDescComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private VLong left = new VLong();
+ private VLong right = new VLong();
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ LazyUtils.readVLong(b1, s1, left);
+ LazyUtils.readVLong(b2, s2, right);
+ if (left.length != l1 || right.length != l2)
+ throw new IllegalArgumentException("length mismatch in int comparator function actual: "
+ + left.length + " expected " + l1);
+ if (left.value > right.value)
+ return -1;
+ else if (left.value == right.value)
+ return 0;
+ else
+ return 1;
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryAscComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryAscComparatorFactory.java
new file mode 100644
index 0000000..a3745fa
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryAscComparatorFactory.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveShortBinaryAscComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveShortBinaryAscComparatorFactory INSTANCE = new HiveShortBinaryAscComparatorFactory();
+
+ private HiveShortBinaryAscComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private short left;
+ private short right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = LazyUtils.byteArrayToShort(b1, s1);
+ right = LazyUtils.byteArrayToShort(b2, s2);
+ if (left > right)
+ return 1;
+ else if (left == right)
+ return 0;
+ else
+ return -1;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryDescComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryDescComparatorFactory.java
new file mode 100644
index 0000000..44d3f43
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryDescComparatorFactory.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveShortBinaryDescComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveShortBinaryDescComparatorFactory INSTANCE = new HiveShortBinaryDescComparatorFactory();
+
+ private HiveShortBinaryDescComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private short left;
+ private short right;
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ left = LazyUtils.byteArrayToShort(b1, s1);
+ right = LazyUtils.byteArrayToShort(b2, s2);
+ if (left > right)
+ return -1;
+ else if (left == right)
+ return 0;
+ else
+ return 1;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryAscComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryAscComparatorFactory.java
new file mode 100644
index 0000000..6da9716
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryAscComparatorFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveStringBinaryAscComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveStringBinaryAscComparatorFactory INSTANCE = new HiveStringBinaryAscComparatorFactory();
+
+ private HiveStringBinaryAscComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private VInt leftLen = new VInt();
+ private VInt rightLen = new VInt();
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ LazyUtils.readVInt(b1, s1, leftLen);
+ LazyUtils.readVInt(b2, s2, rightLen);
+
+ if (leftLen.value + leftLen.length != l1 || rightLen.value + rightLen.length != l2)
+ throw new IllegalStateException("parse string: length mismatch, expected "
+ + (leftLen.value + leftLen.length) + ", " + (rightLen.value + rightLen.length)
+ + " but get " + l1 + ", " + l2);
+
+ return Text.Comparator.compareBytes(b1, s1 + leftLen.length, l1 - leftLen.length, b2, s2
+ + rightLen.length, l2 - rightLen.length);
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryDescComparatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryDescComparatorFactory.java
new file mode 100644
index 0000000..c579711
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryDescComparatorFactory.java
@@ -0,0 +1,39 @@
+package edu.uci.ics.hivesterix.runtime.factory.comparator;
+
+import org.apache.hadoop.io.WritableComparator;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveStringBinaryDescComparatorFactory implements IBinaryComparatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveStringBinaryDescComparatorFactory INSTANCE = new HiveStringBinaryDescComparatorFactory();
+
+ private HiveStringBinaryDescComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+ private VInt leftLen = new VInt();
+ private VInt rightLen = new VInt();
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ LazyUtils.readVInt(b1, s1, leftLen);
+ LazyUtils.readVInt(b2, s2, rightLen);
+
+ if (leftLen.value + leftLen.length != l1 || rightLen.value + rightLen.length != l2)
+ throw new IllegalStateException("parse string: length mismatch, expected "
+ + (leftLen.value + leftLen.length) + ", " + (rightLen.value + rightLen.length)
+ + " but get " + l1 + ", " + l2);
+
+ return -WritableComparator.compareBytes(b1, s1 + leftLen.length, l1 - leftLen.length, b2, s2
+ + rightLen.length, l2 - rightLen.length);
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
new file mode 100644
index 0000000..365379a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
@@ -0,0 +1,368 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.AggregationFunctionEvaluator;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hivesterix.serde.lazy.LazyFactory;
+import edu.uci.ics.hivesterix.serde.lazy.LazyObject;
+import edu.uci.ics.hivesterix.serde.lazy.LazySerDe;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyAggregateFunctionFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class AggregationFunctionFactory implements ICopyAggregateFunctionFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * list of parameters' serialization
+ */
+ private List<String> parametersSerialization = new ArrayList<String>();
+
+ /**
+ * the name of the udf
+ */
+ private String genericUDAFName;
+
+ /**
+ * aggregation mode
+ */
+ private GenericUDAFEvaluator.Mode mode;
+
+ /**
+ * list of type info
+ */
+ private List<TypeInfo> types = new ArrayList<TypeInfo>();
+
+ /**
+ * distinct or not
+ */
+ private boolean distinct;
+
+ /**
+ * the schema of incoming rows
+ */
+ private Schema rowSchema;
+
+ /**
+ * list of parameters
+ */
+ private transient List<ExprNodeDesc> parametersOrigin;
+
+ /**
+ * row inspector
+ */
+ private transient ObjectInspector rowInspector = null;
+
+ /**
+ * output object inspector
+ */
+ private transient ObjectInspector outputInspector = null;
+
+ /**
+ * output object inspector
+ */
+ private transient ObjectInspector outputInspectorPartial = null;
+
+ /**
+ * parameter inspectors
+ */
+ private transient ObjectInspector[] parameterInspectors = null;
+
+ /**
+ * expression desc
+ */
+ private transient HashMap<Long, List<ExprNodeDesc>> parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
+
+ /**
+ * evaluators
+ */
+ private transient HashMap<Long, ExprNodeEvaluator[]> evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
+
+ /**
+ * cached parameter objects
+ */
+ private transient HashMap<Long, Object[]> cachedParameters = new HashMap<Long, Object[]>();
+
+ /**
+ * cached row object: one per thread
+ */
+ private transient HashMap<Long, LazyObject<? extends ObjectInspector>> cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ private transient HashMap<Long, SerDe> serDe = new HashMap<Long, SerDe>();
+
+ /**
+ * udaf evaluators
+ */
+ private transient HashMap<Long, GenericUDAFEvaluator> udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
+
+ /**
+ * udaf evaluators
+ */
+ private transient HashMap<Long, GenericUDAFEvaluator> udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
+
+ /**
+ * aggregation function desc
+ */
+ private transient AggregationDesc aggregator;
+
+ /**
+ *
+ * @param aggregator
+ * Algebricks function call expression
+ * @param oi
+ * schema
+ */
+ public AggregationFunctionFactory(AggregateFunctionCallExpression expression, Schema oi,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+
+ try {
+ aggregator = (AggregationDesc) ExpressionTranslator.getHiveExpression(expression, env);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ }
+ init(aggregator.getParameters(), aggregator.getGenericUDAFName(), aggregator.getMode(),
+ aggregator.getDistinct(), oi);
+ }
+
+ /**
+ * constructor of aggregation function factory
+ *
+ * @param inputs
+ * @param name
+ * @param udafMode
+ * @param distinct
+ * @param oi
+ */
+ private void init(List<ExprNodeDesc> inputs, String name, GenericUDAFEvaluator.Mode udafMode, boolean distinct,
+ Schema oi) {
+ parametersOrigin = inputs;
+ genericUDAFName = name;
+ mode = udafMode;
+ this.distinct = distinct;
+ rowSchema = oi;
+
+ for (ExprNodeDesc input : inputs) {
+ TypeInfo type = input.getTypeInfo();
+ if (type instanceof StructTypeInfo) {
+ types.add(TypeInfoFactory.doubleTypeInfo);
+ } else
+ types.add(type);
+
+ String s = Utilities.serializeExpression(input);
+ parametersSerialization.add(s);
+ }
+ }
+
+ @Override
+ public synchronized ICopyAggregateFunction createAggregateFunction(IDataOutputProvider provider)
+ throws AlgebricksException {
+ if (parametersOrigin == null) {
+ Configuration config = new Configuration();
+ config.setClassLoader(this.getClass().getClassLoader());
+ /**
+ * in case of class.forname(...) call in hive code
+ */
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+
+ parametersOrigin = new ArrayList<ExprNodeDesc>();
+ for (String serialization : parametersSerialization) {
+ parametersOrigin.add(Utilities.deserializeExpression(serialization, config));
+ }
+ }
+
+ /**
+ * exprs
+ */
+ if (parameterExprs == null)
+ parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
+
+ /**
+ * evaluators
+ */
+ if (evaluators == null)
+ evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
+
+ /**
+ * cached parameter objects
+ */
+ if (cachedParameters == null)
+ cachedParameters = new HashMap<Long, Object[]>();
+
+ /**
+ * cached row object: one per thread
+ */
+ if (cachedRowObjects == null)
+ cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ if (serDe == null)
+ serDe = new HashMap<Long, SerDe>();
+
+ /**
+ * UDAF functions
+ */
+ if (udafsComplete == null)
+ udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
+
+ /**
+ * UDAF functions
+ */
+ if (udafsPartial == null)
+ udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
+
+ if (parameterInspectors == null)
+ parameterInspectors = new ObjectInspector[parametersOrigin.size()];
+
+ if (rowInspector == null)
+ rowInspector = rowSchema.toObjectInspector();
+
+ // get current thread id
+ long threadId = Thread.currentThread().getId();
+
+ /**
+ * expressions, expressions are thread local
+ */
+ List<ExprNodeDesc> parameters = parameterExprs.get(threadId);
+ if (parameters == null) {
+ parameters = new ArrayList<ExprNodeDesc>();
+ for (ExprNodeDesc parameter : parametersOrigin)
+ parameters.add(parameter.clone());
+ parameterExprs.put(threadId, parameters);
+ }
+
+ /**
+ * cached parameter objects
+ */
+ Object[] cachedParas = cachedParameters.get(threadId);
+ if (cachedParas == null) {
+ cachedParas = new Object[parameters.size()];
+ cachedParameters.put(threadId, cachedParas);
+ }
+
+ /**
+ * cached row object: one per thread
+ */
+ LazyObject<? extends ObjectInspector> cachedRowObject = cachedRowObjects.get(threadId);
+ if (cachedRowObject == null) {
+ cachedRowObject = LazyFactory.createLazyObject(rowInspector);
+ cachedRowObjects.put(threadId, cachedRowObject);
+ }
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ SerDe lazySer = serDe.get(threadId);
+ if (lazySer == null) {
+ lazySer = new LazySerDe();
+ serDe.put(threadId, lazySer);
+ }
+
+ /**
+ * evaluators
+ */
+ ExprNodeEvaluator[] evals = evaluators.get(threadId);
+ if (evals == null) {
+ evals = new ExprNodeEvaluator[parameters.size()];
+ evaluators.put(threadId, evals);
+ }
+
+ GenericUDAFEvaluator udafPartial;
+ GenericUDAFEvaluator udafComplete;
+
+ // initialize object inspectors
+ try {
+ /**
+ * evaluators, udf, object inpsectors are shared in one thread
+ */
+ for (int i = 0; i < evals.length; i++) {
+ if (evals[i] == null) {
+ evals[i] = ExprNodeEvaluatorFactory.get(parameters.get(i));
+ if (parameterInspectors[i] == null) {
+ parameterInspectors[i] = evals[i].initialize(rowInspector);
+ } else {
+ evals[i].initialize(rowInspector);
+ }
+ }
+ }
+
+ udafComplete = udafsComplete.get(threadId);
+ if (udafComplete == null) {
+ try {
+ udafComplete = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ udafsComplete.put(threadId, udafComplete);
+ udafComplete.init(mode, parameterInspectors);
+ }
+
+ // multiple stage group by, determined by the mode parameter
+ if (outputInspector == null)
+ outputInspector = udafComplete.init(mode, parameterInspectors);
+
+ // initial partial gby udaf
+ GenericUDAFEvaluator.Mode partialMode;
+ // adjust mode for external groupby
+ if (mode == GenericUDAFEvaluator.Mode.COMPLETE)
+ partialMode = GenericUDAFEvaluator.Mode.PARTIAL1;
+ else if (mode == GenericUDAFEvaluator.Mode.FINAL)
+ partialMode = GenericUDAFEvaluator.Mode.PARTIAL2;
+ else
+ partialMode = mode;
+ udafPartial = udafsPartial.get(threadId);
+ if (udafPartial == null) {
+ try {
+ udafPartial = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ udafPartial.init(partialMode, parameterInspectors);
+ udafsPartial.put(threadId, udafPartial);
+ }
+
+ // multiple stage group by, determined by the mode parameter
+ if (outputInspectorPartial == null)
+ outputInspectorPartial = udafPartial.init(partialMode, parameterInspectors);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e);
+ }
+
+ return new AggregationFunctionEvaluator(parameters, types, genericUDAFName, mode, distinct, rowInspector,
+ provider.getDataOutput(), evals, parameterInspectors, cachedParas, lazySer, cachedRowObject,
+ udafPartial, udafComplete, outputInspector, outputInspectorPartial);
+ }
+
+ public String toString() {
+ return "aggregation function expression evaluator factory: " + this.genericUDAFName;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
new file mode 100644
index 0000000..7a48319
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
@@ -0,0 +1,367 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.AggregatuibFunctionSerializableEvaluator;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hivesterix.serde.lazy.LazyFactory;
+import edu.uci.ics.hivesterix.serde.lazy.LazyObject;
+import edu.uci.ics.hivesterix.serde.lazy.LazySerDe;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunctionFactory;
+
+public class AggregationFunctionSerializableFactory implements ICopySerializableAggregateFunctionFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * list of parameters' serialization
+ */
+ private List<String> parametersSerialization = new ArrayList<String>();
+
+ /**
+ * the name of the udf
+ */
+ private String genericUDAFName;
+
+ /**
+ * aggregation mode
+ */
+ private GenericUDAFEvaluator.Mode mode;
+
+ /**
+ * list of type info
+ */
+ private List<TypeInfo> types = new ArrayList<TypeInfo>();
+
+ /**
+ * distinct or not
+ */
+ private boolean distinct;
+
+ /**
+ * the schema of incoming rows
+ */
+ private Schema rowSchema;
+
+ /**
+ * list of parameters
+ */
+ private transient List<ExprNodeDesc> parametersOrigin;
+
+ /**
+ * row inspector
+ */
+ private transient ObjectInspector rowInspector = null;
+
+ /**
+ * output object inspector
+ */
+ private transient ObjectInspector outputInspector = null;
+
+ /**
+ * output object inspector
+ */
+ private transient ObjectInspector outputInspectorPartial = null;
+
+ /**
+ * parameter inspectors
+ */
+ private transient ObjectInspector[] parameterInspectors = null;
+
+ /**
+ * expression desc
+ */
+ private transient HashMap<Long, List<ExprNodeDesc>> parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
+
+ /**
+ * evaluators
+ */
+ private transient HashMap<Long, ExprNodeEvaluator[]> evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
+
+ /**
+ * cached parameter objects
+ */
+ private transient HashMap<Long, Object[]> cachedParameters = new HashMap<Long, Object[]>();
+
+ /**
+ * cached row object: one per thread
+ */
+ private transient HashMap<Long, LazyObject<? extends ObjectInspector>> cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ private transient HashMap<Long, SerDe> serDe = new HashMap<Long, SerDe>();
+
+ /**
+ * udaf evaluators
+ */
+ private transient HashMap<Long, GenericUDAFEvaluator> udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
+
+ /**
+ * udaf evaluators
+ */
+ private transient HashMap<Long, GenericUDAFEvaluator> udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
+
+ /**
+ * aggregation function desc
+ */
+ private transient AggregationDesc aggregator;
+
+ /**
+ *
+ * @param aggregator
+ * Algebricks function call expression
+ * @param oi
+ * schema
+ */
+ public AggregationFunctionSerializableFactory(AggregateFunctionCallExpression expression, Schema oi,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+
+ try {
+ aggregator = (AggregationDesc) ExpressionTranslator.getHiveExpression(expression, env);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ }
+ init(aggregator.getParameters(), aggregator.getGenericUDAFName(), aggregator.getMode(),
+ aggregator.getDistinct(), oi);
+ }
+
+ /**
+ * constructor of aggregation function factory
+ *
+ * @param inputs
+ * @param name
+ * @param udafMode
+ * @param distinct
+ * @param oi
+ */
+ private void init(List<ExprNodeDesc> inputs, String name, GenericUDAFEvaluator.Mode udafMode, boolean distinct,
+ Schema oi) {
+ parametersOrigin = inputs;
+ genericUDAFName = name;
+ mode = udafMode;
+ this.distinct = distinct;
+ rowSchema = oi;
+
+ for (ExprNodeDesc input : inputs) {
+ TypeInfo type = input.getTypeInfo();
+ if (type instanceof StructTypeInfo) {
+ types.add(TypeInfoFactory.doubleTypeInfo);
+ } else
+ types.add(type);
+
+ String s = Utilities.serializeExpression(input);
+ parametersSerialization.add(s);
+ }
+ }
+
+ @Override
+ public synchronized ICopySerializableAggregateFunction createAggregateFunction() throws AlgebricksException {
+ if (parametersOrigin == null) {
+ Configuration config = new Configuration();
+ config.setClassLoader(this.getClass().getClassLoader());
+ /**
+ * in case of class.forname(...) call in hive code
+ */
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+
+ parametersOrigin = new ArrayList<ExprNodeDesc>();
+ for (String serialization : parametersSerialization) {
+ parametersOrigin.add(Utilities.deserializeExpression(serialization, config));
+ }
+ }
+
+ /**
+ * exprs
+ */
+ if (parameterExprs == null)
+ parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
+
+ /**
+ * evaluators
+ */
+ if (evaluators == null)
+ evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
+
+ /**
+ * cached parameter objects
+ */
+ if (cachedParameters == null)
+ cachedParameters = new HashMap<Long, Object[]>();
+
+ /**
+ * cached row object: one per thread
+ */
+ if (cachedRowObjects == null)
+ cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ if (serDe == null)
+ serDe = new HashMap<Long, SerDe>();
+
+ /**
+ * UDAF functions
+ */
+ if (udafsComplete == null)
+ udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
+
+ /**
+ * UDAF functions
+ */
+ if (udafsPartial == null)
+ udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
+
+ if (parameterInspectors == null)
+ parameterInspectors = new ObjectInspector[parametersOrigin.size()];
+
+ if (rowInspector == null)
+ rowInspector = rowSchema.toObjectInspector();
+
+ // get current thread id
+ long threadId = Thread.currentThread().getId();
+
+ /**
+ * expressions, expressions are thread local
+ */
+ List<ExprNodeDesc> parameters = parameterExprs.get(threadId);
+ if (parameters == null) {
+ parameters = new ArrayList<ExprNodeDesc>();
+ for (ExprNodeDesc parameter : parametersOrigin)
+ parameters.add(parameter.clone());
+ parameterExprs.put(threadId, parameters);
+ }
+
+ /**
+ * cached parameter objects
+ */
+ Object[] cachedParas = cachedParameters.get(threadId);
+ if (cachedParas == null) {
+ cachedParas = new Object[parameters.size()];
+ cachedParameters.put(threadId, cachedParas);
+ }
+
+ /**
+ * cached row object: one per thread
+ */
+ LazyObject<? extends ObjectInspector> cachedRowObject = cachedRowObjects.get(threadId);
+ if (cachedRowObject == null) {
+ cachedRowObject = LazyFactory.createLazyObject(rowInspector);
+ cachedRowObjects.put(threadId, cachedRowObject);
+ }
+
+ /**
+ * we only use lazy serde to do serialization
+ */
+ SerDe lazySer = serDe.get(threadId);
+ if (lazySer == null) {
+ lazySer = new LazySerDe();
+ serDe.put(threadId, lazySer);
+ }
+
+ /**
+ * evaluators
+ */
+ ExprNodeEvaluator[] evals = evaluators.get(threadId);
+ if (evals == null) {
+ evals = new ExprNodeEvaluator[parameters.size()];
+ evaluators.put(threadId, evals);
+ }
+
+ GenericUDAFEvaluator udafPartial;
+ GenericUDAFEvaluator udafComplete;
+
+ // initialize object inspectors
+ try {
+ /**
+ * evaluators, udf, object inpsectors are shared in one thread
+ */
+ for (int i = 0; i < evals.length; i++) {
+ if (evals[i] == null) {
+ evals[i] = ExprNodeEvaluatorFactory.get(parameters.get(i));
+ if (parameterInspectors[i] == null) {
+ parameterInspectors[i] = evals[i].initialize(rowInspector);
+ } else {
+ evals[i].initialize(rowInspector);
+ }
+ }
+ }
+
+ udafComplete = udafsComplete.get(threadId);
+ if (udafComplete == null) {
+ try {
+ udafComplete = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ udafsComplete.put(threadId, udafComplete);
+ udafComplete.init(mode, parameterInspectors);
+ }
+
+ // multiple stage group by, determined by the mode parameter
+ if (outputInspector == null)
+ outputInspector = udafComplete.init(mode, parameterInspectors);
+
+ // initial partial gby udaf
+ GenericUDAFEvaluator.Mode partialMode;
+ // adjust mode for external groupby
+ if (mode == GenericUDAFEvaluator.Mode.COMPLETE)
+ partialMode = GenericUDAFEvaluator.Mode.PARTIAL1;
+ else if (mode == GenericUDAFEvaluator.Mode.FINAL)
+ partialMode = GenericUDAFEvaluator.Mode.PARTIAL2;
+ else
+ partialMode = mode;
+ udafPartial = udafsPartial.get(threadId);
+ if (udafPartial == null) {
+ try {
+ udafPartial = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
+ } catch (HiveException e) {
+ throw new AlgebricksException(e);
+ }
+ udafPartial.init(partialMode, parameterInspectors);
+ udafsPartial.put(threadId, udafPartial);
+ }
+
+ // multiple stage group by, determined by the mode parameter
+ if (outputInspectorPartial == null)
+ outputInspectorPartial = udafPartial.init(partialMode, parameterInspectors);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e);
+ }
+
+ return new AggregatuibFunctionSerializableEvaluator(parameters, types, genericUDAFName, mode, distinct,
+ rowInspector, evals, parameterInspectors, cachedParas, lazySer, cachedRowObject, udafPartial,
+ udafComplete, outputInspector, outputInspectorPartial);
+ }
+
+ public String toString() {
+ return "aggregation function expression evaluator factory: " + this.genericUDAFName;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ColumnExpressionEvaluatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ColumnExpressionEvaluatorFactory.java
new file mode 100644
index 0000000..dc21be7
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ColumnExpressionEvaluatorFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.ColumnExpressionEvaluator;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ColumnExpressionEvaluatorFactory implements ICopyEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private ExprNodeColumnDesc expr;
+
+ private Schema inputSchema;
+
+ public ColumnExpressionEvaluatorFactory(ILogicalExpression expression, Schema schema, IVariableTypeEnvironment env)
+ throws AlgebricksException {
+ try {
+ expr = (ExprNodeColumnDesc) ExpressionTranslator.getHiveExpression(expression, env);
+ } catch (Exception e) {
+ throw new AlgebricksException(e.getMessage());
+ }
+ inputSchema = schema;
+ }
+
+ public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+ return new ColumnExpressionEvaluator(expr, inputSchema.toObjectInspector(), output);
+ }
+
+ public String toString() {
+ return "column expression evaluator factory: " + expr.toString();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ConstantExpressionEvaluatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ConstantExpressionEvaluatorFactory.java
new file mode 100644
index 0000000..69e2171
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ConstantExpressionEvaluatorFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.ConstantExpressionEvaluator;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ConstantExpressionEvaluatorFactory implements ICopyEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private ExprNodeConstantDesc expr;
+
+ private Schema schema;
+
+ public ConstantExpressionEvaluatorFactory(ILogicalExpression expression, Schema inputSchema,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+ try {
+ expr = (ExprNodeConstantDesc) ExpressionTranslator.getHiveExpression(expression, env);
+ } catch (Exception e) {
+ throw new AlgebricksException(e.getMessage());
+ }
+ schema = inputSchema;
+ }
+
+ public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+ return new ConstantExpressionEvaluator(expr, schema.toObjectInspector(), output);
+ }
+
+ public String toString() {
+ return "constant expression evaluator factory: " + expr.toString();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/FieldExpressionEvaluatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/FieldExpressionEvaluatorFactory.java
new file mode 100644
index 0000000..eddfb9b
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/FieldExpressionEvaluatorFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hivesterix.runtime.evaluator.FieldExpressionEvaluator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class FieldExpressionEvaluatorFactory implements ICopyEvaluatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ private ExprNodeFieldDesc expr;
+
+ private Schema inputSchema;
+
+ public FieldExpressionEvaluatorFactory(ILogicalExpression expression, Schema schema, IVariableTypeEnvironment env)
+ throws AlgebricksException {
+ try {
+ expr = (ExprNodeFieldDesc) ExpressionTranslator.getHiveExpression(expression, env);
+ } catch (Exception e) {
+ throw new AlgebricksException(e.getMessage());
+ }
+ inputSchema = schema;
+ }
+
+ public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+ return new FieldExpressionEvaluator(expr, inputSchema.toObjectInspector(), output);
+ }
+
+ public String toString() {
+ return "field access expression evaluator factory: " + expr.toString();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/HiveExpressionRuntimeProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/HiveExpressionRuntimeProvider.java
new file mode 100644
index 0000000..842d96e
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/HiveExpressionRuntimeProvider.java
@@ -0,0 +1,167 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+import edu.uci.ics.hivesterix.logical.expression.ExpressionConstant;
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.AggregateFunctionFactoryAdapter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.UnnestingFunctionFactoryAdapter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+
+public class HiveExpressionRuntimeProvider implements IExpressionRuntimeProvider {
+
+ public static final IExpressionRuntimeProvider INSTANCE = new HiveExpressionRuntimeProvider();
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ Schema schema = this.getSchema(inputSchemas[0], env);
+ return new AggregateFunctionFactoryAdapter(new AggregationFunctionFactory(expr, schema, env));
+ }
+
+ @Override
+ public ICopySerializableAggregateFunctionFactory createSerializableAggregateFunctionFactory(
+ AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
+ JobGenContext context) throws AlgebricksException {
+ Schema schema = this.getSchema(inputSchemas[0], env);
+ return new AggregationFunctionSerializableFactory(expr, schema, env);
+ }
+
+ @Override
+ public IRunningAggregateEvaluatorFactory createRunningAggregateFunctionFactory(StatefulFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public IUnnestingEvaluatorFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ Schema schema = this.getSchema(inputSchemas[0], env);
+ return new UnnestingFunctionFactoryAdapter(new UnnestingFunctionFactory(expr, schema, env));
+ }
+
+ public IScalarEvaluatorFactory createEvaluatorFactory(ILogicalExpression expr, IVariableTypeEnvironment env,
+ IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
+ switch (expr.getExpressionTag()) {
+ case VARIABLE: {
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ return new ScalarEvaluatorFactoryAdapter(createVariableEvaluatorFactory(v, env, inputSchemas, context));
+ }
+ case CONSTANT: {
+ ConstantExpression c = (ConstantExpression) expr;
+ return new ScalarEvaluatorFactoryAdapter(createConstantEvaluatorFactory(c, env, inputSchemas, context));
+ }
+ case FUNCTION_CALL: {
+ AbstractFunctionCallExpression fun = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier fid = fun.getFunctionIdentifier();
+
+ if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {
+ return new ScalarEvaluatorFactoryAdapter(createFieldExpressionEvaluatorFactory(fun, env,
+ inputSchemas, context));
+ }
+
+ if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {
+ return new ScalarEvaluatorFactoryAdapter(createNullExpressionEvaluatorFactory(fun, env,
+ inputSchemas, context));
+ }
+
+ if (fun.getKind() == FunctionKind.SCALAR) {
+ ScalarFunctionCallExpression scalar = (ScalarFunctionCallExpression) fun;
+ return new ScalarEvaluatorFactoryAdapter(createScalarFunctionEvaluatorFactory(scalar, env,
+ inputSchemas, context));
+ } else {
+ throw new AlgebricksException("Cannot create evaluator for function " + fun + " of kind "
+ + fun.getKind());
+ }
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ private ICopyEvaluatorFactory createVariableEvaluatorFactory(VariableReferenceExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ Schema schema = this.getSchema(inputSchemas[0], env);
+ return new ColumnExpressionEvaluatorFactory(expr, schema, env);
+ }
+
+ private ICopyEvaluatorFactory createScalarFunctionEvaluatorFactory(AbstractFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ List<String> names = new ArrayList<String>();
+ List<TypeInfo> types = new ArrayList<TypeInfo>();
+ for (IOperatorSchema inputSchema : inputSchemas) {
+ Schema schema = this.getSchema(inputSchema, env);
+ names.addAll(schema.getNames());
+ types.addAll(schema.getTypes());
+ }
+ Schema inputSchema = new Schema(names, types);
+ return new ScalarFunctionExpressionEvaluatorFactory(expr, inputSchema, env);
+ }
+
+ private ICopyEvaluatorFactory createFieldExpressionEvaluatorFactory(AbstractFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ Schema schema = this.getSchema(inputSchemas[0], env);
+ return new FieldExpressionEvaluatorFactory(expr, schema, env);
+ }
+
+ private ICopyEvaluatorFactory createNullExpressionEvaluatorFactory(AbstractFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ Schema schema = this.getSchema(inputSchemas[0], env);
+ return new NullExpressionEvaluatorFactory(expr, schema, env);
+ }
+
+ private ICopyEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr, IVariableTypeEnvironment env,
+ IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
+ Schema schema = this.getSchema(inputSchemas[0], env);
+ return new ConstantExpressionEvaluatorFactory(expr, schema, env);
+ }
+
+ private Schema getSchema(IOperatorSchema inputSchema, IVariableTypeEnvironment env) throws AlgebricksException {
+ List<String> names = new ArrayList<String>();
+ List<TypeInfo> types = new ArrayList<TypeInfo>();
+ Iterator<LogicalVariable> variables = inputSchema.iterator();
+ while (variables.hasNext()) {
+ LogicalVariable var = variables.next();
+ names.add(var.toString());
+ types.add((TypeInfo) env.getVarType(var));
+ }
+
+ Schema schema = new Schema(names, types);
+ return schema;
+ }
+
+}
\ No newline at end of file
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/NullExpressionEvaluatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/NullExpressionEvaluatorFactory.java
new file mode 100644
index 0000000..075ed91
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/NullExpressionEvaluatorFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import org.apache.hadoop.hive.ql.plan.ExprNodeNullDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hivesterix.runtime.evaluator.NullExpressionEvaluator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class NullExpressionEvaluatorFactory implements ICopyEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private ExprNodeNullDesc expr;
+
+ private Schema schema;
+
+ public NullExpressionEvaluatorFactory(ILogicalExpression expression, Schema intputSchema,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+ try {
+ expr = (ExprNodeNullDesc) ExpressionTranslator.getHiveExpression(expression, env);
+ } catch (Exception e) {
+ throw new AlgebricksException(e.getMessage());
+ }
+ schema = intputSchema;
+ }
+
+ public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+ return new NullExpressionEvaluator(expr, schema.toObjectInspector(), output);
+ }
+
+ public String toString() {
+ return "null expression evaluator factory: " + expr.toString();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ScalarFunctionExpressionEvaluatorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ScalarFunctionExpressionEvaluatorFactory.java
new file mode 100644
index 0000000..fe7ddfb
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ScalarFunctionExpressionEvaluatorFactory.java
@@ -0,0 +1,69 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hivesterix.runtime.evaluator.FunctionExpressionEvaluator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ScalarFunctionExpressionEvaluatorFactory implements ICopyEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private transient ExprNodeGenericFuncDesc expr;
+
+ private String exprSerialization;
+
+ private Schema inputSchema;
+
+ private transient Configuration config;
+
+ public ScalarFunctionExpressionEvaluatorFactory(ILogicalExpression expression, Schema schema,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+ try {
+ expr = (ExprNodeGenericFuncDesc) ExpressionTranslator.getHiveExpression(expression, env);
+
+ exprSerialization = Utilities.serializeExpression(expr);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException(e.getMessage());
+ }
+ inputSchema = schema;
+ }
+
+ public synchronized ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+ if (expr == null) {
+ configClassLoader();
+ expr = (ExprNodeGenericFuncDesc) Utilities.deserializeExpression(exprSerialization, config);
+ }
+
+ ExprNodeGenericFuncDesc funcDesc = (ExprNodeGenericFuncDesc) expr.clone();
+ return new FunctionExpressionEvaluator(funcDesc, inputSchema.toObjectInspector(), output);
+ }
+
+ private void configClassLoader() {
+ config = new Configuration();
+ ClassLoader loader = this.getClass().getClassLoader();
+ config.setClassLoader(loader);
+ Thread.currentThread().setContextClassLoader(loader);
+ }
+
+ public String toString() {
+ if (expr == null){
+ configClassLoader();
+ expr = (ExprNodeGenericFuncDesc) Utilities.deserializeExpression(exprSerialization, new Configuration());
+ }
+
+ return "function expression evaluator factory: " + expr.getExprString();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/UnnestingFunctionFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/UnnestingFunctionFactory.java
new file mode 100644
index 0000000..4657ae1
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/UnnestingFunctionFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hivesterix.runtime.factory.evaluator;
+
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.evaluator.ExpressionTranslator;
+import edu.uci.ics.hivesterix.runtime.evaluator.UDTFFunctionEvaluator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyUnnestingFunction;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyUnnestingFunctionFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class UnnestingFunctionFactory implements ICopyUnnestingFunctionFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private UDTFDesc expr;
+
+ private Schema inputSchema;
+
+ private int[] columns;
+
+ public UnnestingFunctionFactory(ILogicalExpression expression, Schema schema, IVariableTypeEnvironment env)
+ throws AlgebricksException {
+ try {
+ expr = (UDTFDesc) ExpressionTranslator.getHiveExpression(expression, env);
+ } catch (Exception e) {
+ throw new AlgebricksException(e.getMessage());
+ }
+ inputSchema = schema;
+ }
+
+ @Override
+ public ICopyUnnestingFunction createUnnestingFunction(IDataOutputProvider provider) throws AlgebricksException {
+ return new UDTFFunctionEvaluator(expr, inputSchema, columns, provider.getDataOutput());
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveDoubleBinaryHashFunctionFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveDoubleBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..b636009
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveDoubleBinaryHashFunctionFactory.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class HiveDoubleBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveDoubleBinaryHashFunctionFactory INSTANCE = new HiveDoubleBinaryHashFunctionFactory();
+
+ private HiveDoubleBinaryHashFunctionFactory() {
+ }
+
+ @Override
+ public IBinaryHashFunction createBinaryHashFunction() {
+ // TODO Auto-generated method stub
+ return new IBinaryHashFunction() {
+ private Double value;
+
+ @Override
+ public int hash(byte[] bytes, int offset, int length) {
+ value = Double.longBitsToDouble(LazyUtils.byteArrayToLong(bytes, offset));
+ return value.hashCode();
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveIntegerBinaryHashFunctionFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveIntegerBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..90e6ce4
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveIntegerBinaryHashFunctionFactory.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class HiveIntegerBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static IBinaryHashFunctionFactory INSTANCE = new HiveIntegerBinaryHashFunctionFactory();
+
+ private HiveIntegerBinaryHashFunctionFactory() {
+ }
+
+ @Override
+ public IBinaryHashFunction createBinaryHashFunction() {
+
+ return new IBinaryHashFunction() {
+ private VInt value = new VInt();
+
+ @Override
+ public int hash(byte[] bytes, int offset, int length) {
+ LazyUtils.readVInt(bytes, offset, value);
+ if (value.length != length)
+ throw new IllegalArgumentException("length mismatch in int hash function actual: " + length
+ + " expected " + value.length);
+ return value.value;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveLongBinaryHashFunctionFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveLongBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..1b61f67
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveLongBinaryHashFunctionFactory.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VLong;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class HiveLongBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static IBinaryHashFunctionFactory INSTANCE = new HiveLongBinaryHashFunctionFactory();
+
+ private HiveLongBinaryHashFunctionFactory() {
+ }
+
+ @Override
+ public IBinaryHashFunction createBinaryHashFunction() {
+
+ return new IBinaryHashFunction() {
+ private VLong value = new VLong();
+
+ @Override
+ public int hash(byte[] bytes, int offset, int length) {
+ LazyUtils.readVLong(bytes, offset, value);
+ return (int) value.value;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveRawBinaryHashFunctionFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveRawBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..f2b7b44
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveRawBinaryHashFunctionFactory.java
@@ -0,0 +1,31 @@
+package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class HiveRawBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static IBinaryHashFunctionFactory INSTANCE = new HiveRawBinaryHashFunctionFactory();
+
+ private HiveRawBinaryHashFunctionFactory() {
+
+ }
+
+ @Override
+ public IBinaryHashFunction createBinaryHashFunction() {
+
+ return new IBinaryHashFunction() {
+
+ @Override
+ public int hash(byte[] bytes, int offset, int length) {
+ int value = 1;
+ int end = offset + length;
+ for (int i = offset; i < end; i++)
+ value = value * 31 + (int) bytes[i];
+ return value;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveStingBinaryHashFunctionFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveStingBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..a9cf6fd
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveStingBinaryHashFunctionFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class HiveStingBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static HiveStingBinaryHashFunctionFactory INSTANCE = new HiveStingBinaryHashFunctionFactory();
+
+ private HiveStingBinaryHashFunctionFactory() {
+ }
+
+ @Override
+ public IBinaryHashFunction createBinaryHashFunction() {
+ // TODO Auto-generated method stub
+ return new IBinaryHashFunction() {
+ private VInt len = new VInt();
+
+ @Override
+ public int hash(byte[] bytes, int offset, int length) {
+ LazyUtils.readVInt(bytes, offset, len);
+ if (len.value + len.length != length)
+ throw new IllegalStateException("parse string: length mismatch, expected "
+ + (len.value + len.length) + " but get " + length);
+ return hashBytes(bytes, offset + len.length, length - len.length);
+ }
+
+ public int hashBytes(byte[] bytes, int offset, int length) {
+ int value = 1;
+ int end = offset + length;
+ for (int i = offset; i < end; i++)
+ value = value * 31 + (int) bytes[i];
+ return value;
+ }
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleAscNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleAscNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..6ac012f
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleAscNormalizedKeyComputerFactory.java
@@ -0,0 +1,24 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class HiveDoubleAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+
+ return new INormalizedKeyComputer() {
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ int header = LazyUtils.byteArrayToInt(bytes, start);
+ long unsignedValue = (long) header;
+ return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleDescNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleDescNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..3044109
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleDescNormalizedKeyComputerFactory.java
@@ -0,0 +1,24 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class HiveDoubleDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final INormalizedKeyComputerFactory ascNormalizedKeyComputerFactory = new HiveDoubleAscNormalizedKeyComputerFactory();
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+ return new INormalizedKeyComputer() {
+ private INormalizedKeyComputer nmkComputer = ascNormalizedKeyComputerFactory.createNormalizedKeyComputer();
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ int nk = nmkComputer.normalize(bytes, start, length);
+ return (int) ((long) Integer.MAX_VALUE - (long) (nk - Integer.MIN_VALUE));
+ }
+
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerAscNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerAscNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..a1d4d48
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerAscNormalizedKeyComputerFactory.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class HiveIntegerAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+
+ return new INormalizedKeyComputer() {
+ private VInt vint = new VInt();
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ LazyUtils.readVInt(bytes, start, vint);
+ if (vint.length != length)
+ throw new IllegalArgumentException("length mismatch in int comparator function actual: "
+ + vint.length + " expected " + length);
+ long unsignedValue = (long) vint.value;
+ return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerDescNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerDescNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..39d4a4b
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerDescNormalizedKeyComputerFactory.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class HiveIntegerDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+
+ return new INormalizedKeyComputer() {
+ private VInt vint = new VInt();
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ LazyUtils.readVInt(bytes, start, vint);
+ if (vint.length != length)
+ throw new IllegalArgumentException("length mismatch in int comparator function actual: "
+ + vint.length + " expected " + length);
+ long unsignedValue = (long) vint.value;
+ return (int) ((long)0xffffffff - unsignedValue);
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongAscNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongAscNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..079e069
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongAscNormalizedKeyComputerFactory.java
@@ -0,0 +1,63 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VLong;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class HiveLongAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+
+ return new INormalizedKeyComputer() {
+ private static final int POSTIVE_LONG_MASK = (3 << 30);
+ private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
+ private static final int NEGATIVE_LONG_MASK = (0 << 30);
+ private VLong vlong = new VLong();
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ LazyUtils.readVLong(bytes, start, vlong);
+ if (vlong.length != length)
+ throw new IllegalArgumentException("length mismatch in int comparator function actual: "
+ + vlong.length + " expected " + length);
+ long value = (long) vlong.value;
+ int highValue = (int) (value >> 32);
+ if (highValue > 0) {
+ /**
+ * larger than Integer.MAX
+ */
+ int highNmk = getKey(highValue);
+ highNmk >>= 2;
+ highNmk |= POSTIVE_LONG_MASK;
+ return highNmk;
+ } else if (highValue == 0) {
+ /**
+ * smaller than Integer.MAX but >=0
+ */
+ int lowNmk = (int) value;
+ lowNmk >>= 2;
+ lowNmk |= NON_NEGATIVE_INT_MASK;
+ return lowNmk;
+ } else {
+ /**
+ * less than 0; TODO: have not optimized for that
+ */
+ int highNmk = getKey(highValue);
+ highNmk >>= 2;
+ highNmk |= NEGATIVE_LONG_MASK;
+ return highNmk;
+ }
+ }
+
+ private int getKey(int value) {
+ long unsignedFirstValue = (long) value;
+ int nmk = (int) ((unsignedFirstValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+ return nmk;
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongDescNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongDescNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..cc5661b
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongDescNormalizedKeyComputerFactory.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class HiveLongDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final INormalizedKeyComputerFactory ascNormalizedKeyComputerFactory = new HiveIntegerAscNormalizedKeyComputerFactory();
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+ return new INormalizedKeyComputer() {
+ private INormalizedKeyComputer nmkComputer = ascNormalizedKeyComputerFactory.createNormalizedKeyComputer();
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ int nk = nmkComputer.normalize(bytes, start, length);
+ return (int) ((long) Integer.MAX_VALUE - (long) (nk - Integer.MIN_VALUE));
+ }
+
+ };
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringAscNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringAscNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..d0429d6
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringAscNormalizedKeyComputerFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class HiveStringAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+
+ return new INormalizedKeyComputer() {
+ private VInt len = new VInt();
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ LazyUtils.readVInt(bytes, start, len);
+
+ if (len.value + len.length != length)
+ throw new IllegalStateException("parse string: length mismatch, expected "
+ + (len.value + len.length) + " but get " + length);
+ int nk = 0;
+ int offset = start + len.length;
+ for (int i = 0; i < 2; ++i) {
+ nk <<= 16;
+ if (i < len.value) {
+ char character = UTF8StringPointable.charAt(bytes, offset);
+ nk += ((int) character) & 0xffff;
+ offset += UTF8StringPointable.charSize(bytes, offset);
+ }
+ }
+ return nk;
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringDescNormalizedKeyComputerFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringDescNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..15b2d27
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringDescNormalizedKeyComputerFactory.java
@@ -0,0 +1,37 @@
+package edu.uci.ics.hivesterix.runtime.factory.normalize;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class HiveStringDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+ return new INormalizedKeyComputer() {
+ private VInt len = new VInt();
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ LazyUtils.readVInt(bytes, start, len);
+ if (len.value + len.length != length)
+ throw new IllegalStateException("parse string: length mismatch, expected "
+ + (len.value + len.length) + " but get " + length);
+ int nk = 0;
+ int offset = start + len.length;
+ for (int i = 0; i < 2; ++i) {
+ nk <<= 16;
+ if (i < len.value) {
+ nk += ((int) UTF8StringPointable.charAt(bytes, offset)) & 0xffff;
+ offset += UTF8StringPointable.charSize(bytes, offset);
+ }
+ }
+ return (int) ((long) 0xffffffff - (long) nk);
+ }
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/factory/nullwriter/HiveNullWriterFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/nullwriter/HiveNullWriterFactory.java
new file mode 100644
index 0000000..590bd61
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/factory/nullwriter/HiveNullWriterFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hivesterix.runtime.factory.nullwriter;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class HiveNullWriterFactory implements INullWriterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static HiveNullWriterFactory INSTANCE = new HiveNullWriterFactory();
+
+ @Override
+ public INullWriter createNullWriter() {
+ return new HiveNullWriter();
+ }
+}
+
+class HiveNullWriter implements INullWriter {
+
+ @Override
+ public void writeNull(DataOutput out) throws HyracksDataException {
+ // do nothing
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspector.java b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspector.java
new file mode 100644
index 0000000..677e20e
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspector.java
@@ -0,0 +1,19 @@
+package edu.uci.ics.hivesterix.runtime.inspector;
+
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
+
+public class HiveBinaryBooleanInspector implements IBinaryBooleanInspector {
+
+ HiveBinaryBooleanInspector() {
+ }
+
+ @Override
+ public boolean getBooleanValue(byte[] bytes, int offset, int length) {
+ if (length == 0)
+ return false;
+ if (length != 1)
+ throw new IllegalStateException("boolean field error: with length " + length);
+ return bytes[0] == 1;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspectorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspectorFactory.java
new file mode 100644
index 0000000..22a6065
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspectorFactory.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hivesterix.runtime.inspector;
+
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public class HiveBinaryBooleanInspectorFactory implements IBinaryBooleanInspectorFactory {
+ private static final long serialVersionUID = 1L;
+ public static HiveBinaryBooleanInspectorFactory INSTANCE = new HiveBinaryBooleanInspectorFactory();
+
+ private HiveBinaryBooleanInspectorFactory() {
+
+ }
+
+ @Override
+ public IBinaryBooleanInspector createBinaryBooleanInspector(IHyracksTaskContext arg0) {
+ return new HiveBinaryBooleanInspector();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspector.java b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspector.java
new file mode 100644
index 0000000..555afee
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspector.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.hivesterix.runtime.inspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspector;
+
+public class HiveBinaryIntegerInspector implements IBinaryIntegerInspector {
+ private VInt value = new VInt();
+
+ HiveBinaryIntegerInspector() {
+ }
+
+ @Override
+ public int getIntegerValue(byte[] bytes, int offset, int length) {
+ LazyUtils.readVInt(bytes, offset, value);
+ if (value.length != length)
+ throw new IllegalArgumentException("length mismatch in int hash function actual: " + length + " expected "
+ + value.length);
+ return value.value;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspectorFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspectorFactory.java
new file mode 100644
index 0000000..bb93a60
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspectorFactory.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hivesterix.runtime.inspector;
+
+import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public class HiveBinaryIntegerInspectorFactory implements IBinaryIntegerInspectorFactory {
+ private static final long serialVersionUID = 1L;
+ public static HiveBinaryIntegerInspectorFactory INSTANCE = new HiveBinaryIntegerInspectorFactory();
+
+ private HiveBinaryIntegerInspectorFactory() {
+
+ }
+
+ @Override
+ public IBinaryIntegerInspector createBinaryIntegerInspector(IHyracksTaskContext arg0) {
+ return new HiveBinaryIntegerInspector();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveConnectorPolicyAssignmentPolicy.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveConnectorPolicyAssignmentPolicy.java
new file mode 100644
index 0000000..bdced12
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveConnectorPolicyAssignmentPolicy.java
@@ -0,0 +1,65 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.SendSideMaterializedBlockingConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+
+public class HiveConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy {
+ public enum Policy {
+ PIPELINING, SEND_SIDE_MAT_PIPELINING, SEND_SIDE_MAT_BLOCKING, SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING;
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ private final IConnectorPolicy pipeliningPolicy = new PipeliningConnectorPolicy();
+ private final IConnectorPolicy sendSideMatPipeliningPolicy = new SendSideMaterializedPipeliningConnectorPolicy();
+ private final IConnectorPolicy sendSideMatBlockingPolicy = new SendSideMaterializedBlockingConnectorPolicy();
+ private final IConnectorPolicy sendSideMatReceiveSideMatBlockingPolicy = new SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy();
+ private final Policy policy;
+
+ public HiveConnectorPolicyAssignmentPolicy(Policy policy) {
+ this.policy = policy;
+ }
+
+ @Override
+ public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
+ int[] fanouts) {
+ if (c instanceof MToNPartitioningMergingConnectorDescriptor) {
+ // avoid deadlocks
+ switch (policy) {
+ case PIPELINING:
+ case SEND_SIDE_MAT_PIPELINING:
+ return sendSideMatPipeliningPolicy;
+ case SEND_SIDE_MAT_BLOCKING:
+ return sendSideMatBlockingPolicy;
+ case SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING:
+ return sendSideMatReceiveSideMatBlockingPolicy;
+ default:
+ return sendSideMatPipeliningPolicy;
+ }
+ } else if (c instanceof MToNPartitioningConnectorDescriptor) {
+ // support different repartitioning policies
+ switch (policy) {
+ case PIPELINING:
+ return pipeliningPolicy;
+ case SEND_SIDE_MAT_PIPELINING:
+ return sendSideMatPipeliningPolicy;
+ case SEND_SIDE_MAT_BLOCKING:
+ return sendSideMatBlockingPolicy;
+ case SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING:
+ return sendSideMatReceiveSideMatBlockingPolicy;
+ default:
+ return pipeliningPolicy;
+ }
+ } else {
+ // pipelining for other connectors
+ return pipeliningPolicy;
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSink.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSink.java
new file mode 100644
index 0000000..ccc2e6c
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSink.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+
+public class HiveDataSink implements IDataSink {
+
+ private Object[] schema;
+
+ private Object fsOperator;
+
+ public HiveDataSink(Object sink, Object[] sourceSchema) {
+ schema = sourceSchema;
+ fsOperator = sink;
+ }
+
+ @Override
+ public Object getId() {
+ return fsOperator;
+ }
+
+ @Override
+ public Object[] getSchemaTypes() {
+ return schema;
+ }
+
+ public IPartitioningProperty getPartitioningProperty() {
+ return new RandomPartitioningProperty(new HiveDomain());
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSource.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSource.java
new file mode 100644
index 0000000..67b743b
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSource.java
@@ -0,0 +1,47 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+
+public class HiveDataSource<P> implements IDataSource<P> {
+
+ private P source;
+
+ private Object[] schema;
+
+ public HiveDataSource(P dataSource, Object[] sourceSchema) {
+ source = dataSource;
+ schema = sourceSchema;
+ }
+
+ @Override
+ public P getId() {
+ return source;
+ }
+
+ @Override
+ public Object[] getSchemaTypes() {
+ return schema;
+ }
+
+ @Override
+ public void computeFDs(List<LogicalVariable> scanVariables, List<FunctionalDependency> fdList) {
+ }
+
+ @Override
+ public IDataSourcePropertiesProvider getPropertiesProvider() {
+ return new HiveDataSourcePartitioningProvider();
+ }
+
+ @Override
+ public String toString() {
+ PartitionDesc desc = (PartitionDesc) source;
+ return desc.getTableName();
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSourcePartitioningProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSourcePartitioningProvider.java
new file mode 100644
index 0000000..bb9c4ce
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSourcePartitioningProvider.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+
+public class HiveDataSourcePartitioningProvider implements IDataSourcePropertiesProvider {
+
+ @Override
+ public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables) {
+ IPartitioningProperty property = new RandomPartitioningProperty(new HiveDomain());
+ IPhysicalPropertiesVector vector = new StructuralPropertiesVector(property,
+ new LinkedList<ILocalStructuralProperty>());
+ return vector;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDomain.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDomain.java
new file mode 100644
index 0000000..8b1d3b5
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDomain.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class HiveDomain implements INodeDomain {
+
+ @Override
+ public boolean sameAs(INodeDomain domain) {
+ return true;
+ }
+
+ @Override
+ public Integer cardinality() {
+ return 0;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
new file mode 100644
index 0000000..a6cdbe0
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
@@ -0,0 +1,129 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+
+import edu.uci.ics.hivesterix.logical.expression.HiveFunctionInfo;
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+@SuppressWarnings("rawtypes")
+public class HiveMetaDataProvider<S, T> implements IMetadataProvider<S, T> {
+
+ private Operator fileSink;
+ private Schema outputSchema;
+ private HashMap<S, IDataSource<S>> dataSourceMap;
+
+ public HiveMetaDataProvider(Operator fsOp, Schema oi, HashMap<S, IDataSource<S>> map) {
+ fileSink = fsOp;
+ outputSchema = oi;
+ dataSourceMap = map;
+ }
+
+ @Override
+ public IDataSourceIndex<T, S> findDataSourceIndex(T indexId, S dataSourceId) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public IDataSource<S> findDataSource(S id) throws AlgebricksException {
+ return dataSourceMap.get(id);
+ }
+
+ @Override
+ public boolean scannerOperatorIsLeaf(IDataSource<S> dataSource) {
+ return true;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec)
+ throws AlgebricksException {
+
+ S desc = dataSource.getId();
+ HiveScanRuntimeGenerator generator = new HiveScanRuntimeGenerator((PartitionDesc) desc);
+ return generator.getRuntimeOperatorAndConstraint(dataSource, scanVariables, projectVariables, projectPushed,
+ context, jobSpec);
+ }
+
+ @Override
+ public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
+ int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc) {
+
+ HiveWriteRuntimeGenerator generator = new HiveWriteRuntimeGenerator((FileSinkOperator) fileSink, outputSchema);
+ return generator.getWriterRuntime(inputDesc);
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> arg0,
+ IOperatorSchema arg1, List<LogicalVariable> arg2, LogicalVariable arg3, RecordDescriptor arg4,
+ JobGenContext arg5, JobSpecification arg6) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> arg0,
+ IOperatorSchema arg1, List<LogicalVariable> arg2, LogicalVariable arg3, RecordDescriptor arg4,
+ JobGenContext arg5, JobSpecification arg6) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> arg0,
+ IOperatorSchema arg1, List<LogicalVariable> arg2, LogicalVariable arg3, JobGenContext arg4,
+ JobSpecification arg5) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public IFunctionInfo lookupFunction(FunctionIdentifier arg0) {
+ return new HiveFunctionInfo(arg0, null);
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(
+ IDataSourceIndex<T, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
+ IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+ throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(
+ IDataSourceIndex<T, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
+ IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+ throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveOperatorSchema.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveOperatorSchema.java
new file mode 100644
index 0000000..cdb0e95
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveOperatorSchema.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+
+public class HiveOperatorSchema implements IOperatorSchema {
+
+ private final Map<LogicalVariable, Integer> varMap;
+
+ private final List<LogicalVariable> varList;
+
+ public HiveOperatorSchema() {
+ varMap = new HashMap<LogicalVariable, Integer>();
+ varList = new ArrayList<LogicalVariable>();
+ }
+
+ @Override
+ public void addAllVariables(IOperatorSchema source) {
+ for (LogicalVariable v : source) {
+ varMap.put(v, varList.size());
+ varList.add(v);
+ }
+ }
+
+ @Override
+ public void addAllNewVariables(IOperatorSchema source) {
+ for (LogicalVariable v : source) {
+ if (varMap.get(v) == null) {
+ varMap.put(v, varList.size());
+ varList.add(v);
+ }
+ }
+ }
+
+ @Override
+ public int addVariable(LogicalVariable var) {
+ int idx = varList.size();
+ varMap.put(var, idx);
+ varList.add(var);
+ return idx;
+ }
+
+ @Override
+ public void clear() {
+ varMap.clear();
+ varList.clear();
+ }
+
+ @Override
+ public int findVariable(LogicalVariable var) {
+ Integer i = varMap.get(var);
+ if (i == null) {
+ return -1;
+ }
+ return i;
+ }
+
+ @Override
+ public int getSize() {
+ return varList.size();
+ }
+
+ @Override
+ public LogicalVariable getVariable(int index) {
+ return varList.get(index);
+ }
+
+ @Override
+ public Iterator<LogicalVariable> iterator() {
+ return varList.iterator();
+ }
+
+ @Override
+ public String toString() {
+ return varMap.toString();
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
new file mode 100644
index 0000000..69fb66a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
@@ -0,0 +1,106 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
+import edu.uci.ics.hivesterix.runtime.operator.filescan.HiveFileScanOperatorDescriptor;
+import edu.uci.ics.hivesterix.runtime.operator.filescan.HiveFileSplitProvider;
+import edu.uci.ics.hivesterix.runtime.operator.filescan.HiveTupleParserFactory;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class HiveScanRuntimeGenerator {
+
+ private PartitionDesc fileDesc;
+
+ private transient Path filePath;
+
+ private String filePathName;
+
+ private Properties properties;
+
+ public HiveScanRuntimeGenerator(PartitionDesc path) {
+ fileDesc = path;
+ properties = fileDesc.getProperties();
+
+ String inputPath = (String) properties.getProperty("location");
+
+ if (inputPath.startsWith("file:")) {
+ // Windows
+ String[] strs = inputPath.split(":");
+ filePathName = strs[strs.length - 1];
+ } else {
+ // Linux
+ filePathName = inputPath;
+ }
+
+ filePath = new Path(filePathName);
+ }
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRuntimeOperatorAndConstraint(
+ IDataSource dataSource, List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables,
+ boolean projectPushed, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
+ // get the correct delimiter from Hive metastore or other data
+ // structures
+ IOperatorSchema propagatedSchema = new HiveOperatorSchema();
+
+ List<LogicalVariable> outputVariables = projectPushed ? projectVariables : scanVariables;
+ for (LogicalVariable var : outputVariables)
+ propagatedSchema.addVariable(var);
+
+ int[] outputColumnsOffset = new int[scanVariables.size()];
+ int i = 0;
+ for (LogicalVariable var : scanVariables)
+ if (outputVariables.contains(var)) {
+ int offset = outputVariables.indexOf(var);
+ outputColumnsOffset[i++] = offset;
+ } else
+ outputColumnsOffset[i++] = -1;
+
+ Object[] schemaTypes = dataSource.getSchemaTypes();
+ // get record descriptor
+ RecordDescriptor recDescriptor = mkRecordDescriptor(propagatedSchema, schemaTypes, context);
+
+ // setup the run time operator
+ JobConf conf = ConfUtil.getJobConf(fileDesc.getInputFileFormatClass(), filePath);
+ int clusterSize = ConfUtil.getNCs().length;
+ IFileSplitProvider fsprovider = new HiveFileSplitProvider(conf, filePathName, clusterSize);
+ ITupleParserFactory tupleParserFactory = new HiveTupleParserFactory(fileDesc, conf, outputColumnsOffset);
+ HiveFileScanOperatorDescriptor opDesc = new HiveFileScanOperatorDescriptor(jobSpec, fsprovider,
+ tupleParserFactory, recDescriptor);
+
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(opDesc, opDesc.getPartitionConstraint());
+ }
+
+ @SuppressWarnings("unchecked")
+ private static RecordDescriptor mkRecordDescriptor(IOperatorSchema opSchema, Object[] types, JobGenContext context)
+ throws AlgebricksException {
+ ISerializerDeserializer[] fields = new ISerializerDeserializer[opSchema.getSize()];
+ ISerializerDeserializerProvider sdp = context.getSerializerDeserializerProvider();
+ int i = 0;
+ for (LogicalVariable var : opSchema) {
+ Object t = types[i];
+ fields[i] = sdp.getSerializerDeserializer(t);
+ i++;
+ }
+ return new RecordDescriptor(fields);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveWriteRuntimeGenerator.java b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveWriteRuntimeGenerator.java
new file mode 100644
index 0000000..bd5bc7f
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveWriteRuntimeGenerator.java
@@ -0,0 +1,37 @@
+package edu.uci.ics.hivesterix.runtime.jobgen;
+
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
+import edu.uci.ics.hivesterix.runtime.operator.filewrite.HivePushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class HiveWriteRuntimeGenerator {
+ private FileSinkOperator fileSink;
+
+ private Schema inputSchema;
+
+ public HiveWriteRuntimeGenerator(FileSinkOperator fsOp, Schema oi) {
+ fileSink = fsOp;
+ inputSchema = oi;
+ }
+
+ /**
+ * get the write runtime
+ *
+ * @param inputDesc
+ * @return
+ */
+ public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriterRuntime(RecordDescriptor inputDesc) {
+ JobConf conf = ConfUtil.getJobConf();
+ IPushRuntimeFactory factory = new HivePushRuntimeFactory(inputDesc, conf, fileSink, inputSchema);
+ Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> pair = new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(
+ factory, null);
+ return pair;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveFileSplitProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveFileSplitProvider.java
new file mode 100644
index 0000000..642280d
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveFileSplitProvider.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hivesterix.runtime.operator.filescan;
+
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+
+public abstract class AbstractHiveFileSplitProvider implements IFileSplitProvider {
+
+ @Override
+ public FileSplit[] getFileSplits() {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ public abstract org.apache.hadoop.mapred.FileSplit[] getFileSplitArray();
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveTupleParser.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveTupleParser.java
new file mode 100644
index 0000000..a76a383
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveTupleParser.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.hivesterix.runtime.operator.filescan;
+
+import java.io.InputStream;
+
+import org.apache.hadoop.mapred.FileSplit;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+
+public abstract class AbstractHiveTupleParser implements ITupleParser {
+
+ @Override
+ public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+ // empty implementation
+ }
+
+ /**
+ * method for parsing HDFS file split
+ *
+ * @param split
+ * @param writer
+ */
+ abstract public void parse(FileSplit split, IFrameWriter writer) throws HyracksDataException;
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileScanOperatorDescriptor.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..00a9d0a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileScanOperatorDescriptor.java
@@ -0,0 +1,164 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * 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 from
+ *
+ * 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 edu.uci.ics.hivesterix.runtime.operator.filescan;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.mapred.FileSplit;
+
+import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class HiveFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * tuple parser factory
+ */
+ private final ITupleParserFactory tupleParserFactory;
+
+ /**
+ * Hive file split
+ */
+ private Partition[] parts;
+
+ /**
+ * IFileSplitProvider
+ */
+ private IFileSplitProvider fileSplitProvider;
+
+ /**
+ * constrains in the form of host DNS names
+ */
+ private String[] constraintsByHostNames;
+
+ /**
+ * ip-to-node controller mapping
+ */
+ private Map<String, List<String>> ncMapping;
+
+ /**
+ * an array of NCs
+ */
+ private String[] NCs;
+
+ /**
+ *
+ * @param spec
+ * @param fsProvider
+ */
+ public HiveFileScanOperatorDescriptor(JobSpecification spec, IFileSplitProvider fsProvider,
+ ITupleParserFactory tupleParserFactory, RecordDescriptor rDesc) {
+ super(spec, 0, 1);
+ this.tupleParserFactory = tupleParserFactory;
+ recordDescriptors[0] = rDesc;
+ fileSplitProvider = fsProvider;
+ }
+
+ /**
+ * set partition constraint at the first time it is called the number of
+ * partitions is obtained from HDFS name node
+ */
+ public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException {
+ FileSplit[] returnedSplits = ((AbstractHiveFileSplitProvider) fileSplitProvider).getFileSplitArray();
+ Random random = new Random(System.currentTimeMillis());
+ ncMapping = ConfUtil.getNCMapping();
+ NCs = ConfUtil.getNCs();
+
+ int size = 0;
+ for (FileSplit split : returnedSplits)
+ if (split != null)
+ size++;
+
+ FileSplit[] splits = new FileSplit[size];
+ for (int i = 0; i < returnedSplits.length; i++)
+ if (returnedSplits[i] != null)
+ splits[i] = returnedSplits[i];
+
+ System.out.println("!!! number of splits: " + splits.length);
+ constraintsByHostNames = new String[splits.length];
+ for (int i = 0; i < splits.length; i++) {
+ try {
+ String[] loc = splits[i].getLocations();
+ Collections.shuffle(Arrays.asList(loc), random);
+ if (loc.length > 0) {
+ InetAddress[] allIps = InetAddress.getAllByName(loc[0]);
+ for (InetAddress ip : allIps) {
+ if (ncMapping.get(ip.getHostAddress()) != null) {
+ List<String> ncs = ncMapping.get(ip.getHostAddress());
+ int pos = random.nextInt(ncs.size());
+ constraintsByHostNames[i] = ncs.get(pos);
+ } else {
+ int pos = random.nextInt(NCs.length);
+ constraintsByHostNames[i] = NCs[pos];
+ }
+ }
+ } else {
+ int pos = random.nextInt(NCs.length);
+ constraintsByHostNames[i] = NCs[pos];
+ if (splits[i].getLength() > 0)
+ throw new IllegalStateException("non local scanner non locations!!");
+ }
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ parts = new Partition[splits.length];
+ for (int i = 0; i < splits.length; i++) {
+ parts[i] = new Partition(splits[i]);
+ }
+ return new AlgebricksAbsolutePartitionConstraint(constraintsByHostNames);
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+
+ final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
+ final int partitionId = partition;
+
+ return new AbstractUnaryOutputSourceOperatorNodePushable() {
+
+ @Override
+ public void initialize() throws HyracksDataException {
+ writer.open();
+ FileSplit split = parts[partitionId].toFileSplit();
+ if (split == null)
+ throw new HyracksDataException("partition " + partitionId + " is null!");
+ ((AbstractHiveTupleParser) tp).parse(split, writer);
+ writer.close();
+ }
+ };
+ }
+}
\ No newline at end of file
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileSplitProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileSplitProvider.java
new file mode 100644
index 0000000..9e85ac4
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileSplitProvider.java
@@ -0,0 +1,108 @@
+package edu.uci.ics.hivesterix.runtime.operator.filescan;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.util.UUID;
+
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.eclipse.jetty.util.log.Log;
+
+public class HiveFileSplitProvider extends AbstractHiveFileSplitProvider {
+
+ private transient InputFormat format;
+ private transient JobConf conf;
+ private String confContent;
+ final private int nPartition;
+ private transient FileSplit[] splits;
+ private String inputFormatStr;
+
+ public HiveFileSplitProvider(JobConf conf, String filePath, int nPartition) {
+ format = conf.getInputFormat();
+ inputFormatStr = format.getClass().getName();
+ this.conf = conf;
+ this.nPartition = nPartition;
+ writeConfContent();
+ }
+
+ private void writeConfContent() {
+ File dir = new File("hadoop-conf-tmp");
+ if (!dir.exists()) {
+ dir.mkdir();
+ }
+
+ String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";
+ try {
+ DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(fileName)));
+ conf.writeXml(out);
+ out.close();
+
+ DataInputStream in = new DataInputStream(new FileInputStream(fileName));
+ StringBuffer buffer = new StringBuffer();
+ String line;
+ while ((line = in.readLine()) != null) {
+ buffer.append(line + "\n");
+ }
+ in.close();
+ confContent = buffer.toString();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ private void readConfContent() {
+ File dir = new File("hadoop-conf-tmp");
+ if (!dir.exists()) {
+ dir.mkdir();
+ }
+
+ String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";
+ try {
+ PrintWriter out = new PrintWriter((new OutputStreamWriter(new FileOutputStream(new File(fileName)))));
+ out.write(confContent);
+ out.close();
+ conf = new JobConf(fileName);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ /**
+ * get the HDFS file split
+ */
+ public FileSplit[] getFileSplitArray() {
+ readConfContent();
+ conf.setClassLoader(this.getClass().getClassLoader());
+ format = conf.getInputFormat();
+ //int splitSize = conf.getInt("mapred.min.split.size", 0);
+
+ if (splits == null) {
+ try {
+ splits = (org.apache.hadoop.mapred.FileSplit[]) format.getSplits(conf, nPartition);
+ System.out.println("hdfs split number: " + splits.length);
+ } catch (IOException e) {
+ String inputPath = conf.get("mapred.input.dir");
+ String hdfsURL = conf.get("fs.default.name");
+ String alternatePath = inputPath.replaceAll(hdfsURL, "file:");
+ conf.set("mapred.input.dir", alternatePath);
+ try {
+ splits = (org.apache.hadoop.mapred.FileSplit[]) format.getSplits(conf, nPartition);
+ System.out.println("hdfs split number: " + splits.length);
+ } catch (IOException e1) {
+ e1.printStackTrace();
+ Log.debug(e1.getMessage());
+ return null;
+ }
+ }
+ }
+ return splits;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParser.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParser.java
new file mode 100644
index 0000000..ffcaa1d
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParser.java
@@ -0,0 +1,219 @@
+package edu.uci.ics.hivesterix.runtime.operator.filescan;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hivesterix.serde.parser.IHiveParser;
+import edu.uci.ics.hivesterix.serde.parser.TextToBinaryTupleParser;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class HiveTupleParser extends AbstractHiveTupleParser {
+
+ private int[] outputColumnsOffset;
+ /**
+ * class of input format
+ */
+ private InputFormat inputFormat;
+
+ /**
+ * serialization/deserialization object
+ */
+ private SerDe serDe;
+
+ /**
+ * the input row object inspector
+ */
+ private ObjectInspector objectInspector;
+
+ /**
+ * the hadoop job conf
+ */
+ private JobConf job;
+
+ /**
+ * Hyrax context to control resource allocation
+ */
+ private final IHyracksTaskContext ctx;
+
+ /**
+ * lazy serde: format flow in between operators
+ */
+ private final SerDe outputSerDe;
+
+ /**
+ * the parser from hive data to binary data
+ */
+ private IHiveParser parser = null;
+
+ /**
+ * parser for any hive input format
+ *
+ * @param inputFormatClass
+ * @param serDeClass
+ * @param tbl
+ * @param conf
+ * @throws AlgebricksException
+ */
+ public HiveTupleParser(String inputFormatClass, String serDeClass, String outputSerDeClass, Properties tbl,
+ JobConf conf, final IHyracksTaskContext ctx, int[] outputColumnsOffset) throws AlgebricksException {
+ try {
+ conf.setClassLoader(this.getClass().getClassLoader());
+
+ inputFormat = (InputFormat) ReflectionUtils.newInstance(Class.forName(inputFormatClass), conf);
+ job = conf;
+
+ // initialize the input serde
+ serDe = (SerDe) ReflectionUtils.newInstance(Class.forName(serDeClass), job);
+ serDe.initialize(job, tbl);
+
+ // initialize the output serde
+ outputSerDe = (SerDe) ReflectionUtils.newInstance(Class.forName(outputSerDeClass), job);
+ outputSerDe.initialize(job, tbl);
+
+ // object inspector of the row
+ objectInspector = serDe.getObjectInspector();
+
+ // hyracks context
+ this.ctx = ctx;
+ this.outputColumnsOffset = outputColumnsOffset;
+
+ if (objectInspector instanceof LazySimpleStructObjectInspector) {
+ LazySimpleStructObjectInspector rowInspector = (LazySimpleStructObjectInspector) objectInspector;
+ List<? extends StructField> fieldRefs = rowInspector.getAllStructFieldRefs();
+ boolean lightWeightParsable = true;
+ for (StructField fieldRef : fieldRefs) {
+ Category category = fieldRef.getFieldObjectInspector().getCategory();
+ if (!(category == Category.PRIMITIVE)) {
+ lightWeightParsable = false;
+ break;
+ }
+ }
+ if (lightWeightParsable)
+ parser = new TextToBinaryTupleParser(this.outputColumnsOffset, this.objectInspector);
+ }
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ /**
+ * parse a input HDFS file split, the result is send to the writer
+ * one-frame-a-time
+ *
+ * @param split
+ * the HDFS file split
+ * @param writer
+ * the writer
+ * @throws HyracksDataException
+ * if there is sth. wrong in the ser/de
+ */
+ @Override
+ public void parse(FileSplit split, IFrameWriter writer) throws HyracksDataException {
+ try {
+ StructObjectInspector structInspector = (StructObjectInspector) objectInspector;
+
+ // create the reader, key, and value
+ RecordReader reader = inputFormat.getRecordReader(split, job, Reporter.NULL);
+ Object key = reader.createKey();
+ Object value = reader.createValue();
+
+ // allocate a new frame
+ ByteBuffer frame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender.reset(frame, true);
+
+ List<? extends StructField> fieldRefs = structInspector.getAllStructFieldRefs();
+ int size = 0;
+ for (int i = 0; i < outputColumnsOffset.length; i++)
+ if (outputColumnsOffset[i] >= 0)
+ size++;
+
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(size);
+ DataOutput dos = tb.getDataOutput();
+ StructField[] outputFieldRefs = new StructField[size];
+ Object[] outputFields = new Object[size];
+ for (int i = 0; i < outputColumnsOffset.length; i++)
+ if (outputColumnsOffset[i] >= 0)
+ outputFieldRefs[outputColumnsOffset[i]] = fieldRefs.get(i);
+
+ long serDeTime = 0;
+ while (reader.next(key, value)) {
+ long start = System.currentTimeMillis();
+ // reuse the tuple builder
+ tb.reset();
+ if (parser != null) {
+ Text text = (Text) value;
+ parser.parse(text.getBytes(), 0, text.getLength(), tb);
+ } else {
+ Object row = serDe.deserialize((Writable) value);
+ // write fields to the tuple builder one by one
+ int i = 0;
+ for (StructField fieldRef : fieldRefs) {
+ if (outputColumnsOffset[i] >= 0)
+ outputFields[outputColumnsOffset[i]] = structInspector.getStructFieldData(row, fieldRef);
+ i++;
+ }
+
+ i = 0;
+ for (Object field : outputFields) {
+ BytesWritable fieldWritable = (BytesWritable) outputSerDe.serialize(field,
+ outputFieldRefs[i].getFieldObjectInspector());
+ dos.write(fieldWritable.getBytes(), 0, fieldWritable.getSize());
+ tb.addFieldEndOffset();
+ i++;
+ }
+ }
+ long end = System.currentTimeMillis();
+ serDeTime += (end - start);
+
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ if (appender.getTupleCount() <= 0)
+ throw new IllegalStateException("zero tuples in a frame!");
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ System.out.println("serde time: " + serDeTime);
+ reader.close();
+ System.gc();
+
+ // flush the last frame
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ } catch (SerDeException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParserFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParserFactory.java
new file mode 100644
index 0000000..ccd18ee
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParserFactory.java
@@ -0,0 +1,103 @@
+package edu.uci.ics.hivesterix.runtime.operator.filescan;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazySerDe;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class HiveTupleParserFactory implements ITupleParserFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private int[] outputColumns;
+
+ private String outputSerDeClass = LazySerDe.class.getName();
+
+ private String inputSerDeClass;
+
+ private transient JobConf conf;
+
+ private Properties tbl;
+
+ private String confContent;
+
+ private String inputFormatClass;
+
+ public HiveTupleParserFactory(PartitionDesc desc, JobConf conf, int[] outputColumns) {
+ this.conf = conf;
+ tbl = desc.getProperties();
+ inputFormatClass = (String) tbl.getProperty("file.inputformat");
+ inputSerDeClass = (String) tbl.getProperty("serialization.lib");
+ this.outputColumns = outputColumns;
+
+ writeConfContent();
+ }
+
+ @Override
+ public ITupleParser createTupleParser(IHyracksTaskContext ctx) {
+ readConfContent();
+ try {
+ return new HiveTupleParser(inputFormatClass, inputSerDeClass, outputSerDeClass, tbl, conf, ctx, outputColumns);
+ } catch (Exception e) {
+ e.printStackTrace();
+ return null;
+ }
+ }
+
+ private void writeConfContent() {
+ File dir = new File("hadoop-conf-tmp");
+ if (!dir.exists()) {
+ dir.mkdir();
+ }
+
+ String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";
+ try {
+ DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(fileName)));
+ conf.writeXml(out);
+ out.close();
+
+ DataInputStream in = new DataInputStream(new FileInputStream(fileName));
+ StringBuffer buffer = new StringBuffer();
+ String line;
+ while ((line = in.readLine()) != null) {
+ buffer.append(line + "\n");
+ }
+ in.close();
+ confContent = buffer.toString();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ private void readConfContent() {
+ File dir = new File("hadoop-conf-tmp");
+ if (!dir.exists()) {
+ dir.mkdir();
+ }
+
+ String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";
+ try {
+ PrintWriter out = new PrintWriter((new OutputStreamWriter(new FileOutputStream(new File(fileName)))));
+ out.write(confContent);
+ out.close();
+
+ conf = new JobConf(fileName);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/Partition.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/Partition.java
new file mode 100644
index 0000000..db9dd78
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/Partition.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hivesterix.runtime.operator.filescan;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+
+public class Partition implements Serializable {
+
+ private String uri;
+ private long offset;
+ private long length;
+ private String[] locations;
+
+ public Partition() {
+ }
+
+ public Partition(FileSplit file) {
+ uri = file.getPath().toUri().toString();
+ offset = file.getStart();
+ length = file.getLength();
+ try {
+ locations = file.getLocations();
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ public FileSplit toFileSplit() {
+ return new FileSplit(new Path(uri), offset, length, locations);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
new file mode 100644
index 0000000..1abade8
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
@@ -0,0 +1,146 @@
+package edu.uci.ics.hivesterix.runtime.operator.filewrite;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
+import org.apache.hadoop.hive.ql.exec.RowSchema;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hivesterix.serde.lazy.LazyColumnar;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyColumnarObjectInspector;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public class HiveFileWritePushRuntime implements IPushRuntime {
+
+ /**
+ * frame tuple accessor to access byte buffer
+ */
+ private final FrameTupleAccessor accessor;
+
+ /**
+ * input object inspector
+ */
+ private final ObjectInspector inputInspector;
+
+ /**
+ * cachedInput
+ */
+ private final LazyColumnar cachedInput;
+
+ /**
+ * File sink operator of Hive
+ */
+ private final FileSinkDesc fileSink;
+
+ /**
+ * job configuration, which contain name node and other configuration
+ * information
+ */
+ private JobConf conf;
+
+ /**
+ * input object inspector
+ */
+ private final Schema inputSchema;
+
+ /**
+ * a copy of hive schema representation
+ */
+ private RowSchema rowSchema;
+
+ /**
+ * the Hive file sink operator
+ */
+ private FileSinkOperator fsOp;
+
+ /**
+ * cached tuple object reference
+ */
+ private FrameTupleReference tuple = new FrameTupleReference();
+
+ /**
+ * @param spec
+ * @param fsProvider
+ */
+ public HiveFileWritePushRuntime(IHyracksTaskContext context, RecordDescriptor inputRecordDesc, JobConf job,
+ FileSinkDesc fs, RowSchema schema, Schema oi) {
+ fileSink = fs;
+ fileSink.setGatherStats(false);
+
+ rowSchema = schema;
+ conf = job;
+ inputSchema = oi;
+
+ accessor = new FrameTupleAccessor(context.getFrameSize(), inputRecordDesc);
+ inputInspector = inputSchema.toObjectInspector();
+ cachedInput = new LazyColumnar((LazyColumnarObjectInspector) inputInspector);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ fsOp = (FileSinkOperator) OperatorFactory.get(fileSink, rowSchema);
+ fsOp.setChildOperators(null);
+ fsOp.setParentOperators(null);
+ conf.setClassLoader(this.getClass().getClassLoader());
+
+ ObjectInspector[] inspectors = new ObjectInspector[1];
+ inspectors[0] = inputInspector;
+ try {
+ fsOp.initialize(conf, inspectors);
+ fsOp.setExecContext(null);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int n = accessor.getTupleCount();
+ try {
+ for (int i = 0; i < n; ++i) {
+ tuple.reset(accessor, i);
+ cachedInput.init(tuple);
+ fsOp.process(cachedInput, 0);
+ }
+ } catch (HiveException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ fsOp.closeOp(false);
+ } catch (HiveException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HivePushRuntimeFactory.java b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HivePushRuntimeFactory.java
new file mode 100644
index 0000000..cdc5a36
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HivePushRuntimeFactory.java
@@ -0,0 +1,104 @@
+package edu.uci.ics.hivesterix.runtime.operator.filewrite;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.util.UUID;
+
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.RowSchema;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hivesterix.logical.expression.Schema;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class HivePushRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final RecordDescriptor inputRecordDesc;
+ private transient JobConf conf;
+ private final FileSinkDesc fileSink;
+ private final RowSchema outSchema;
+ private final Schema schema;
+
+ /**
+ * the content of the configuration
+ */
+ private String confContent;
+
+ public HivePushRuntimeFactory(RecordDescriptor inputRecordDesc, JobConf conf, FileSinkOperator fsp, Schema sch) {
+ this.inputRecordDesc = inputRecordDesc;
+ this.conf = conf;
+ this.fileSink = fsp.getConf();
+ outSchema = fsp.getSchema();
+ this.schema = sch;
+
+ writeConfContent();
+ }
+
+ @Override
+ public String toString() {
+ return "file write";
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(IHyracksTaskContext context) throws AlgebricksException {
+ if (conf == null)
+ readConfContent();
+
+ return new HiveFileWritePushRuntime(context, inputRecordDesc, conf, fileSink, outSchema, schema);
+ }
+
+ private void readConfContent() {
+ File dir = new File("hadoop-conf-tmp");
+ if (!dir.exists()) {
+ dir.mkdir();
+ }
+
+ String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";
+ try {
+ PrintWriter out = new PrintWriter((new OutputStreamWriter(new FileOutputStream(new File(fileName)))));
+ out.write(confContent);
+ out.close();
+ conf = new JobConf(fileName);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ private void writeConfContent() {
+ File dir = new File("hadoop-conf-tmp");
+ if (!dir.exists()) {
+ dir.mkdir();
+ }
+
+ String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";
+ try {
+ DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(fileName)));
+ conf.writeXml(out);
+ out.close();
+
+ DataInputStream in = new DataInputStream(new FileInputStream(fileName));
+ StringBuffer buffer = new StringBuffer();
+ String line;
+ while ((line = in.readLine()) != null) {
+ buffer.append(line + "\n");
+ }
+ in.close();
+ confContent = buffer.toString();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryComparatorFactoryProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryComparatorFactoryProvider.java
new file mode 100644
index 0000000..467ec0a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryComparatorFactoryProvider.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.hivesterix.runtime.provider;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveByteBinaryAscComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveByteBinaryDescComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveDoubleBinaryAscComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveDoubleBinaryDescComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveFloatBinaryAscComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveFloatBinaryDescComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveIntegerBinaryAscComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveIntegerBinaryDescComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveLongBinaryAscComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveLongBinaryDescComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveShortBinaryAscComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveShortBinaryDescComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveStringBinaryAscComparatorFactory;
+import edu.uci.ics.hivesterix.runtime.factory.comparator.HiveStringBinaryDescComparatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class HiveBinaryComparatorFactoryProvider implements IBinaryComparatorFactoryProvider {
+
+ public static final HiveBinaryComparatorFactoryProvider INSTANCE = new HiveBinaryComparatorFactoryProvider();
+
+ private HiveBinaryComparatorFactoryProvider() {
+ }
+
+ @Override
+ public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending)
+ throws AlgebricksException {
+ if (type.equals(TypeInfoFactory.intTypeInfo)) {
+ if (ascending)
+ return HiveIntegerBinaryAscComparatorFactory.INSTANCE;
+ else
+ return HiveIntegerBinaryDescComparatorFactory.INSTANCE;
+
+ } else if (type.equals(TypeInfoFactory.longTypeInfo)) {
+ if (ascending)
+ return HiveLongBinaryAscComparatorFactory.INSTANCE;
+ else
+ return HiveLongBinaryDescComparatorFactory.INSTANCE;
+
+ } else if (type.equals(TypeInfoFactory.floatTypeInfo)) {
+ if (ascending)
+ return HiveFloatBinaryAscComparatorFactory.INSTANCE;
+ else
+ return HiveFloatBinaryDescComparatorFactory.INSTANCE;
+
+ } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
+ if (ascending)
+ return HiveDoubleBinaryAscComparatorFactory.INSTANCE;
+ else
+ return HiveDoubleBinaryDescComparatorFactory.INSTANCE;
+ } else if (type.equals(TypeInfoFactory.shortTypeInfo)) {
+ if (ascending)
+ return HiveShortBinaryAscComparatorFactory.INSTANCE;
+ else
+ return HiveShortBinaryDescComparatorFactory.INSTANCE;
+ } else if (type.equals(TypeInfoFactory.stringTypeInfo)) {
+ if (ascending)
+ return HiveStringBinaryAscComparatorFactory.INSTANCE;
+ else
+ return HiveStringBinaryDescComparatorFactory.INSTANCE;
+ } else if (type.equals(TypeInfoFactory.byteTypeInfo) || type.equals(TypeInfoFactory.booleanTypeInfo)) {
+ if (ascending)
+ return HiveByteBinaryAscComparatorFactory.INSTANCE;
+ else
+ return HiveByteBinaryDescComparatorFactory.INSTANCE;
+ } else
+ throw new NotImplementedException();
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFactoryProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFactoryProvider.java
new file mode 100644
index 0000000..473eee1
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFactoryProvider.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hivesterix.runtime.provider;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.runtime.factory.hashfunction.HiveDoubleBinaryHashFunctionFactory;
+import edu.uci.ics.hivesterix.runtime.factory.hashfunction.HiveIntegerBinaryHashFunctionFactory;
+import edu.uci.ics.hivesterix.runtime.factory.hashfunction.HiveLongBinaryHashFunctionFactory;
+import edu.uci.ics.hivesterix.runtime.factory.hashfunction.HiveRawBinaryHashFunctionFactory;
+import edu.uci.ics.hivesterix.runtime.factory.hashfunction.HiveStingBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class HiveBinaryHashFunctionFactoryProvider implements IBinaryHashFunctionFactoryProvider {
+
+ public static final HiveBinaryHashFunctionFactoryProvider INSTANCE = new HiveBinaryHashFunctionFactoryProvider();
+
+ private HiveBinaryHashFunctionFactoryProvider() {
+ }
+
+ @Override
+ public IBinaryHashFunctionFactory getBinaryHashFunctionFactory(Object type) throws AlgebricksException {
+ if (type.equals(TypeInfoFactory.intTypeInfo)) {
+ return HiveIntegerBinaryHashFunctionFactory.INSTANCE;
+ } else if (type.equals(TypeInfoFactory.longTypeInfo)) {
+ return HiveLongBinaryHashFunctionFactory.INSTANCE;
+ } else if (type.equals(TypeInfoFactory.stringTypeInfo)) {
+ return HiveStingBinaryHashFunctionFactory.INSTANCE;
+ } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
+ return HiveDoubleBinaryHashFunctionFactory.INSTANCE;
+ } else {
+ return HiveRawBinaryHashFunctionFactory.INSTANCE;
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveNormalizedKeyComputerFactoryProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveNormalizedKeyComputerFactoryProvider.java
new file mode 100644
index 0000000..91bf3e5
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveNormalizedKeyComputerFactoryProvider.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.hivesterix.runtime.provider;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveDoubleAscNormalizedKeyComputerFactory;
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveDoubleDescNormalizedKeyComputerFactory;
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveIntegerAscNormalizedKeyComputerFactory;
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveIntegerDescNormalizedKeyComputerFactory;
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveLongAscNormalizedKeyComputerFactory;
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveLongDescNormalizedKeyComputerFactory;
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveStringAscNormalizedKeyComputerFactory;
+import edu.uci.ics.hivesterix.runtime.factory.normalize.HiveStringDescNormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class HiveNormalizedKeyComputerFactoryProvider implements INormalizedKeyComputerFactoryProvider {
+
+ public static final HiveNormalizedKeyComputerFactoryProvider INSTANCE = new HiveNormalizedKeyComputerFactoryProvider();
+
+ private HiveNormalizedKeyComputerFactoryProvider() {
+ }
+
+ @Override
+ public INormalizedKeyComputerFactory getNormalizedKeyComputerFactory(Object type, boolean ascending) {
+ if (ascending) {
+ if (type.equals(TypeInfoFactory.stringTypeInfo)) {
+ return new HiveStringAscNormalizedKeyComputerFactory();
+ } else if (type.equals(TypeInfoFactory.intTypeInfo)) {
+ return new HiveIntegerAscNormalizedKeyComputerFactory();
+ } else if (type.equals(TypeInfoFactory.longTypeInfo)) {
+ return new HiveLongAscNormalizedKeyComputerFactory();
+ } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
+ return new HiveDoubleAscNormalizedKeyComputerFactory();
+ } else {
+ return null;
+ }
+ } else {
+ if (type.equals(TypeInfoFactory.stringTypeInfo)) {
+ return new HiveStringDescNormalizedKeyComputerFactory();
+ } else if (type.equals(TypeInfoFactory.intTypeInfo)) {
+ return new HiveIntegerDescNormalizedKeyComputerFactory();
+ } else if (type.equals(TypeInfoFactory.longTypeInfo)) {
+ return new HiveLongDescNormalizedKeyComputerFactory();
+ } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
+ return new HiveDoubleDescNormalizedKeyComputerFactory();
+ } else {
+ return null;
+ }
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HivePrinterFactoryProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HivePrinterFactoryProvider.java
new file mode 100644
index 0000000..10c84d2
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HivePrinterFactoryProvider.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hivesterix.runtime.provider;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactoryProvider;
+
+public class HivePrinterFactoryProvider implements IPrinterFactoryProvider {
+
+ public static IPrinterFactoryProvider INSTANCE = new HivePrinterFactoryProvider();
+
+ @Override
+ public IPrinterFactory getPrinterFactory(Object type) throws AlgebricksException {
+ return null;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveSerializerDeserializerProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveSerializerDeserializerProvider.java
new file mode 100644
index 0000000..c8b5699
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveSerializerDeserializerProvider.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.hivesterix.runtime.provider;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+
+public class HiveSerializerDeserializerProvider implements ISerializerDeserializerProvider {
+
+ public static final HiveSerializerDeserializerProvider INSTANCE = new HiveSerializerDeserializerProvider();
+
+ private HiveSerializerDeserializerProvider() {
+ }
+
+ @Override
+ public ISerializerDeserializer getSerializerDeserializer(Object type) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ // return ARecordSerializerDeserializer.SCHEMALESS_INSTANCE;
+ return null;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveTypeTraitProvider.java b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveTypeTraitProvider.java
new file mode 100644
index 0000000..6ed1e63
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveTypeTraitProvider.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hivesterix.runtime.provider;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+
+public class HiveTypeTraitProvider implements ITypeTraitProvider, Serializable {
+
+ public static HiveTypeTraitProvider INSTANCE = new HiveTypeTraitProvider();
+
+ private HiveTypeTraitProvider() {
+
+ }
+
+ @Override
+ public ITypeTraits getTypeTrait(Object arg0) {
+ return new ITypeTraits() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public int getFixedLength() {
+ return -1;
+ }
+
+ @Override
+ public boolean isFixedLength() {
+ return false;
+ }
+
+ };
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/ByteArrayRef.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/ByteArrayRef.java
new file mode 100644
index 0000000..9bf74b7
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/ByteArrayRef.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+/**
+ * ByteArrayRef stores a reference to a byte array.
+ *
+ * The LazyObject hierarchy uses a reference to a single ByteArrayRef, so that
+ * it's much faster to switch to the next row and release the reference to the
+ * old row (so that the system can do garbage collection if needed).
+ */
+public class ByteArrayRef {
+
+ /**
+ * Stores the actual data.
+ */
+ byte[] data;
+
+ public byte[] getData() {
+ return data;
+ }
+
+ public void setData(byte[] data) {
+ this.data = data;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyArray.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyArray.java
new file mode 100644
index 0000000..b79cd86
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyArray.java
@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.RecordInfo;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyListObjectInspector;
+
+/**
+ * LazyArray is serialized as follows: start A b b b b b b end bytes[] ->
+ * |--------|---|---|---|---| ... |---|---|
+ *
+ * Section A is the null-bytes. Suppose the list has N elements, then there are
+ * (N+7)/8 bytes used as null-bytes. Each bit corresponds to an element and it
+ * indicates whether that element is null (0) or not null (1).
+ *
+ * After A, all b(s) represent the elements of the list. Each of them is again a
+ * LazyObject.
+ *
+ */
+
+public class LazyArray extends LazyNonPrimitive<LazyListObjectInspector> {
+
+ /**
+ * Whether the data is already parsed or not.
+ */
+ boolean parsed = false;
+ /**
+ * The length of the array. Only valid when the data is parsed.
+ */
+ int arraySize = 0;
+
+ /**
+ * The start positions and lengths of array elements. Only valid when the
+ * data is parsed.
+ */
+ int[] elementStart;
+ int[] elementLength;
+
+ /**
+ * Whether an element is initialized or not.
+ */
+ boolean[] elementInited;
+
+ /**
+ * Whether an element is null or not. Because length is 0 does not means the
+ * field is null. In particular, a 0-length string is not null.
+ */
+ boolean[] elementIsNull;
+
+ /**
+ * The elements of the array. Note that we call arrayElements[i].init(bytes,
+ * begin, length) only when that element is accessed.
+ */
+ LazyObject[] arrayElements;
+
+ /**
+ * Construct a LazyArray object with the ObjectInspector.
+ *
+ * @param oi
+ * the oi representing the type of this LazyArray
+ */
+ protected LazyArray(LazyListObjectInspector oi) {
+ super(oi);
+ }
+
+ /**
+ * Set the row data for this LazyArray.
+ *
+ * @see LazyObject#init(ByteArrayRef, int, int)
+ */
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ super.init(bytes, start, length);
+ parsed = false;
+ }
+
+ /**
+ * Enlarge the size of arrays storing information for the elements inside
+ * the array.
+ */
+ private void adjustArraySize(int newSize) {
+ if (elementStart == null || elementStart.length < newSize) {
+ elementStart = new int[newSize];
+ elementLength = new int[newSize];
+ elementInited = new boolean[newSize];
+ elementIsNull = new boolean[newSize];
+ arrayElements = new LazyObject[newSize];
+ }
+ }
+
+ VInt vInt = new LazyUtils.VInt();
+ RecordInfo recordInfo = new LazyUtils.RecordInfo();
+
+ /**
+ * Parse the bytes and fill elementStart, elementLength, elementInited and
+ * elementIsNull.
+ */
+ private void parse() {
+
+ // get the vlong that represents the map size
+ LazyUtils.readVInt(bytes, start, vInt);
+ arraySize = vInt.value;
+ if (0 == arraySize) {
+ parsed = true;
+ return;
+ }
+
+ // adjust arrays
+ adjustArraySize(arraySize);
+ // find out the null-bytes
+ int arryByteStart = start + vInt.length;
+ int nullByteCur = arryByteStart;
+ int nullByteEnd = arryByteStart + (arraySize + 7) / 8;
+ // the begin the real elements
+ int lastElementByteEnd = nullByteEnd;
+ // the list element object inspector
+ ObjectInspector listEleObjectInspector = ((ListObjectInspector) oi).getListElementObjectInspector();
+ // parsing elements one by one
+ for (int i = 0; i < arraySize; i++) {
+ elementIsNull[i] = true;
+ if ((bytes[nullByteCur] & (1 << (i % 8))) != 0) {
+ elementIsNull[i] = false;
+ LazyUtils.checkObjectByteInfo(listEleObjectInspector, bytes, lastElementByteEnd, recordInfo);
+ elementStart[i] = lastElementByteEnd + recordInfo.elementOffset;
+ elementLength[i] = recordInfo.elementSize;
+ lastElementByteEnd = elementStart[i] + elementLength[i];
+ }
+ // move onto the next null byte
+ if (7 == (i % 8)) {
+ nullByteCur++;
+ }
+ }
+
+ Arrays.fill(elementInited, 0, arraySize, false);
+ parsed = true;
+ }
+
+ /**
+ * Returns the actual primitive object at the index position inside the
+ * array represented by this LazyObject.
+ */
+ public Object getListElementObject(int index) {
+ if (!parsed) {
+ parse();
+ }
+ if (index < 0 || index >= arraySize) {
+ return null;
+ }
+ return uncheckedGetElement(index);
+ }
+
+ /**
+ * Get the element without checking out-of-bound index.
+ *
+ * @param index
+ * index to the array element
+ */
+ private Object uncheckedGetElement(int index) {
+
+ if (elementIsNull[index]) {
+ return null;
+ } else {
+ if (!elementInited[index]) {
+ elementInited[index] = true;
+ if (arrayElements[index] == null) {
+ arrayElements[index] = LazyFactory.createLazyObject((oi).getListElementObjectInspector());
+ }
+ arrayElements[index].init(bytes, elementStart[index], elementLength[index]);
+ }
+ }
+ return arrayElements[index].getObject();
+ }
+
+ /**
+ * Returns the array size.
+ */
+ public int getListLength() {
+ if (!parsed) {
+ parse();
+ }
+ return arraySize;
+ }
+
+ /**
+ * cachedList is reused every time getList is called. Different
+ * LazyBianryArray instances cannot share the same cachedList.
+ */
+ ArrayList<Object> cachedList;
+
+ /**
+ * Returns the List of actual primitive objects. Returns null for null
+ * array.
+ */
+ public List<Object> getList() {
+ if (!parsed) {
+ parse();
+ }
+ if (cachedList == null) {
+ cachedList = new ArrayList<Object>(arraySize);
+ } else {
+ cachedList.clear();
+ }
+ for (int index = 0; index < arraySize; index++) {
+ cachedList.add(uncheckedGetElement(index));
+ }
+ return cachedList;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyBoolean.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyBoolean.java
new file mode 100644
index 0000000..471a64a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyBoolean.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.io.BooleanWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyBooleanObjectInspector;
+
+/**
+ * LazyObject for storing a value of boolean.
+ *
+ * <p>
+ * Part of the code is adapted from Apache Harmony Project.
+ *
+ * As with the specification, this implementation relied on code laid out in <a
+ * href="http://www.hackersdelight.org/">Henry S. Warren, Jr.'s Hacker's
+ * Delight, (Addison Wesley, 2002)</a> as well as <a
+ * href="http://aggregate.org/MAGIC/">The Aggregate's Magic Algorithms</a>.
+ * </p>
+ *
+ */
+public class LazyBoolean extends LazyPrimitive<LazyBooleanObjectInspector, BooleanWritable> {
+
+ public LazyBoolean(LazyBooleanObjectInspector oi) {
+ super(oi);
+ data = new BooleanWritable();
+ }
+
+ public LazyBoolean(LazyBoolean copy) {
+ super(copy);
+ data = new BooleanWritable(copy.data.get());
+ }
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+
+ // a temporal hack
+ assert (1 == length);
+ byte val = bytes[start];
+ if (val == 0) {
+ data.set(false);
+ } else if (val == 1) {
+ data.set(true);
+ }
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyByte.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyByte.java
new file mode 100644
index 0000000..d57dcb8
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyByte.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.io.ByteWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyByteObjectInspector;
+
+/**
+ * LazyObject for storing a value of Byte.
+ *
+ * <p>
+ * Part of the code is adapted from Apache Harmony Project.
+ *
+ * As with the specification, this implementation relied on code laid out in <a
+ * href="http://www.hackersdelight.org/">Henry S. Warren, Jr.'s Hacker's
+ * Delight, (Addison Wesley, 2002)</a> as well as <a
+ * href="http://aggregate.org/MAGIC/">The Aggregate's Magic Algorithms</a>.
+ * </p>
+ *
+ */
+public class LazyByte extends LazyPrimitive<LazyByteObjectInspector, ByteWritable> {
+
+ public LazyByte(LazyByteObjectInspector oi) {
+ super(oi);
+ data = new ByteWritable();
+ }
+
+ public LazyByte(LazyByte copy) {
+ super(copy);
+ data = new ByteWritable(copy.data.get());
+ }
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+
+ assert (1 == length);
+ data.set(bytes[start]);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyColumnar.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyColumnar.java
new file mode 100644
index 0000000..2ca9ded
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyColumnar.java
@@ -0,0 +1,212 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyColumnarObjectInspector;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * LazyObject for storing a struct. The field of a struct can be primitive or
+ * non-primitive.
+ *
+ * LazyStruct does not deal with the case of a NULL struct. That is handled by
+ * the parent LazyObject.
+ */
+public class LazyColumnar extends LazyNonPrimitive<LazyColumnarObjectInspector> {
+
+ private static Log LOG = LogFactory.getLog(LazyColumnar.class.getName());
+
+ /**
+ * IFrameTupleReference: the backend of the struct
+ */
+ IFrameTupleReference tuple;
+
+ /**
+ * Whether the data is already parsed or not.
+ */
+ boolean reset;
+
+ /**
+ * The fields of the struct.
+ */
+ LazyObject[] fields;
+
+ /**
+ * Whether init() has been called on the field or not.
+ */
+ boolean[] fieldVisited;
+
+ /**
+ * whether it is the first time initialization
+ */
+ boolean start = true;
+
+ /**
+ * Construct a LazyStruct object with the ObjectInspector.
+ */
+ public LazyColumnar(LazyColumnarObjectInspector oi) {
+ super(oi);
+ }
+
+ /**
+ * Set the row data for this LazyStruct.
+ *
+ * @see LazyObject#init(ByteArrayRef, int, int)
+ */
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ super.init(bytes, start, length);
+ reset = false;
+ }
+
+ /**
+ * Parse the byte[] and fill each field.
+ */
+ private void parse() {
+
+ if (start) {
+ // initialize field array and reusable objects
+ List<? extends StructField> fieldRefs = ((StructObjectInspector) oi).getAllStructFieldRefs();
+
+ fields = new LazyObject[fieldRefs.size()];
+ for (int i = 0; i < fields.length; i++) {
+ fields[i] = LazyFactory.createLazyObject(fieldRefs.get(i).getFieldObjectInspector());
+ }
+ fieldVisited = new boolean[fields.length];
+ start = false;
+ }
+
+ Arrays.fill(fieldVisited, false);
+ reset = true;
+ }
+
+ /**
+ * Get one field out of the struct.
+ *
+ * If the field is a primitive field, return the actual object. Otherwise
+ * return the LazyObject. This is because PrimitiveObjectInspector does not
+ * have control over the object used by the user - the user simply directly
+ * use the Object instead of going through Object
+ * PrimitiveObjectInspector.get(Object).
+ *
+ * @param fieldID
+ * The field ID
+ * @return The field as a LazyObject
+ */
+ public Object getField(int fieldID) {
+ if (!reset) {
+ parse();
+ }
+ return uncheckedGetField(fieldID);
+ }
+
+ /**
+ * Get the field out of the row without checking parsed. This is called by
+ * both getField and getFieldsAsList.
+ *
+ * @param fieldID
+ * The id of the field starting from 0.
+ * @param nullSequence
+ * The sequence representing NULL value.
+ * @return The value of the field
+ */
+ private Object uncheckedGetField(int fieldID) {
+ // get the buffer
+ byte[] buffer = tuple.getFieldData(fieldID);
+ // get the offset of the field
+ int s1 = tuple.getFieldStart(fieldID);
+ int l1 = tuple.getFieldLength(fieldID);
+
+ if (!fieldVisited[fieldID]) {
+ fieldVisited[fieldID] = true;
+ fields[fieldID].init(buffer, s1, l1);
+ }
+ //if (fields[fieldID].getObject() == null) {
+ // throw new IllegalStateException("illegal field " + fieldID);
+ //}
+ return fields[fieldID].getObject();
+ }
+
+ ArrayList<Object> cachedList;
+
+ /**
+ * Get the values of the fields as an ArrayList.
+ *
+ * @return The values of the fields as an ArrayList.
+ */
+ public ArrayList<Object> getFieldsAsList() {
+ if (!reset) {
+ parse();
+ }
+ if (cachedList == null) {
+ cachedList = new ArrayList<Object>();
+ } else {
+ cachedList.clear();
+ }
+ for (int i = 0; i < fields.length; i++) {
+ cachedList.add(uncheckedGetField(i));
+ }
+ return cachedList;
+ }
+
+ @Override
+ public Object getObject() {
+ return this;
+ }
+
+ protected boolean getParsed() {
+ return reset;
+ }
+
+ protected void setParsed(boolean parsed) {
+ this.reset = parsed;
+ }
+
+ protected LazyObject[] getFields() {
+ return fields;
+ }
+
+ protected void setFields(LazyObject[] fields) {
+ this.fields = fields;
+ }
+
+ protected boolean[] getFieldInited() {
+ return fieldVisited;
+ }
+
+ protected void setFieldInited(boolean[] fieldInited) {
+ this.fieldVisited = fieldInited;
+ }
+
+ /**
+ * rebind a frametuplereference to the struct
+ */
+ public void init(IFrameTupleReference r) {
+ this.tuple = r;
+ reset = false;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyDouble.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyDouble.java
new file mode 100644
index 0000000..f7bb633
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyDouble.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.io.DoubleWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyDoubleObjectInspector;
+
+/**
+ * LazyObject for storing a value of Double.
+ *
+ */
+public class LazyDouble extends LazyPrimitive<LazyDoubleObjectInspector, DoubleWritable> {
+
+ public LazyDouble(LazyDoubleObjectInspector oi) {
+ super(oi);
+ data = new DoubleWritable();
+ }
+
+ public LazyDouble(LazyDouble copy) {
+ super(copy);
+ data = new DoubleWritable(copy.data.get());
+ }
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+ assert (8 == length);
+ data.set(Double.longBitsToDouble(LazyUtils.byteArrayToLong(bytes, start)));
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFactory.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFactory.java
new file mode 100644
index 0000000..a249d61
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFactory.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyColumnarObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyListObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyMapObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyStructObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyBooleanObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyByteObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyDoubleObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyFloatObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyIntObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyLongObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyShortObjectInspector;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyStringObjectInspector;
+
+/**
+ * LazyFactory.
+ *
+ */
+public final class LazyFactory {
+
+ /**
+ * Create a lazy binary primitive class given the type name.
+ */
+ public static LazyPrimitive<?, ?> createLazyPrimitiveClass(PrimitiveObjectInspector oi) {
+ PrimitiveCategory p = oi.getPrimitiveCategory();
+ switch (p) {
+ case BOOLEAN:
+ return new LazyBoolean((LazyBooleanObjectInspector) oi);
+ case BYTE:
+ return new LazyByte((LazyByteObjectInspector) oi);
+ case SHORT:
+ return new LazyShort((LazyShortObjectInspector) oi);
+ case INT:
+ return new LazyInteger((LazyIntObjectInspector) oi);
+ case LONG:
+ return new LazyLong((LazyLongObjectInspector) oi);
+ case FLOAT:
+ return new LazyFloat((LazyFloatObjectInspector) oi);
+ case DOUBLE:
+ return new LazyDouble((LazyDoubleObjectInspector) oi);
+ case STRING:
+ return new LazyString((LazyStringObjectInspector) oi);
+ default:
+ throw new RuntimeException("Internal error: no LazyObject for " + p);
+ }
+ }
+
+ /**
+ * Create a hierarchical LazyObject based on the given typeInfo.
+ */
+ public static LazyObject<? extends ObjectInspector> createLazyObject(ObjectInspector oi) {
+ ObjectInspector.Category c = oi.getCategory();
+ switch (c) {
+ case PRIMITIVE:
+ return createLazyPrimitiveClass((PrimitiveObjectInspector) oi);
+ case MAP:
+ return new LazyMap((LazyMapObjectInspector) oi);
+ case LIST:
+ return new LazyArray((LazyListObjectInspector) oi);
+ case STRUCT: // check whether it is a top-level struct
+ if (oi instanceof LazyStructObjectInspector)
+ return new LazyStruct((LazyStructObjectInspector) oi);
+ else
+ return new LazyColumnar((LazyColumnarObjectInspector) oi);
+ }
+
+ throw new RuntimeException("Hive LazySerDe Internal error.");
+ }
+
+ private LazyFactory() {
+ // prevent instantiation
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFloat.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFloat.java
new file mode 100644
index 0000000..26aa299
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFloat.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.io.FloatWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyFloatObjectInspector;
+
+/**
+ * LazyObject for storing a value of Double.
+ *
+ */
+public class LazyFloat extends LazyPrimitive<LazyFloatObjectInspector, FloatWritable> {
+
+ public LazyFloat(LazyFloatObjectInspector oi) {
+ super(oi);
+ data = new FloatWritable();
+ }
+
+ public LazyFloat(LazyFloat copy) {
+ super(copy);
+ data = new FloatWritable(copy.data.get());
+ }
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+
+ assert (4 == length);
+ data.set(Float.intBitsToFloat(LazyUtils.byteArrayToInt(bytes, start)));
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyInteger.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyInteger.java
new file mode 100644
index 0000000..5a0f0c1
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyInteger.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.io.IntWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyIntObjectInspector;
+
+/**
+ * LazyObject for storing a value of Integer.
+ *
+ * <p>
+ * Part of the code is adapted from Apache Harmony Project.
+ *
+ * As with the specification, this implementation relied on code laid out in <a
+ * href="http://www.hackersdelight.org/">Henry S. Warren, Jr.'s Hacker's
+ * Delight, (Addison Wesley, 2002)</a> as well as <a
+ * href="http://aggregate.org/MAGIC/">The Aggregate's Magic Algorithms</a>.
+ * </p>
+ *
+ */
+public class LazyInteger extends LazyPrimitive<LazyIntObjectInspector, IntWritable> {
+
+ public LazyInteger(LazyIntObjectInspector oi) {
+ super(oi);
+ data = new IntWritable();
+ }
+
+ public LazyInteger(LazyInteger copy) {
+ super(copy);
+ data = new IntWritable(copy.data.get());
+ }
+
+ /**
+ * The reusable vInt for decoding the integer.
+ */
+ VInt vInt = new LazyUtils.VInt();
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+
+ LazyUtils.readVInt(bytes, start, vInt);
+ assert (length == vInt.length);
+ if (length != vInt.length)
+ throw new IllegalStateException("parse int: length mismatch, expected " + vInt.length + " but get "
+ + length);
+ data.set(vInt.value);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyLong.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyLong.java
new file mode 100644
index 0000000..2d02e53
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyLong.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.io.LongWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VLong;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyLongObjectInspector;
+
+/**
+ * LazyObject for storing a value of Long.
+ *
+ * <p>
+ * Part of the code is adapted from Apache Harmony Project.
+ *
+ * As with the specification, this implementation relied on code laid out in <a
+ * href="http://www.hackersdelight.org/">Henry S. Warren, Jr.'s Hacker's
+ * Delight, (Addison Wesley, 2002)</a> as well as <a
+ * href="http://aggregate.org/MAGIC/">The Aggregate's Magic Algorithms</a>.
+ * </p>
+ *
+ */
+public class LazyLong extends LazyPrimitive<LazyLongObjectInspector, LongWritable> {
+
+ public LazyLong(LazyLongObjectInspector oi) {
+ super(oi);
+ data = new LongWritable();
+ }
+
+ public LazyLong(LazyLong copy) {
+ super(copy);
+ data = new LongWritable(copy.data.get());
+ }
+
+ /**
+ * The reusable vLong for decoding the long.
+ */
+ VLong vLong = new LazyUtils.VLong();
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+
+ LazyUtils.readVLong(bytes, start, vLong);
+ assert (length == vLong.length);
+ if (length != vLong.length)
+ throw new IllegalStateException("parse long: length mismatch");
+ data.set(vLong.value);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyMap.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyMap.java
new file mode 100644
index 0000000..c84a1d2
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyMap.java
@@ -0,0 +1,331 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
+import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.RecordInfo;
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyMapObjectInspector;
+
+/**
+ * LazyMap is serialized as follows: start A b c b c b c end bytes[] ->
+ * |--------|---|---|---|---| ... |---|---|
+ *
+ * Section A is the null-bytes. Suppose the map has N key-value pairs, then
+ * there are (N*2+7)/8 bytes used as null-bytes. Each bit corresponds to a key
+ * or a value and it indicates whether that key or value is null (0) or not null
+ * (1).
+ *
+ * After A, all the bytes are actual serialized data of the map, which are
+ * key-value pairs. b represent the keys and c represent the values. Each of
+ * them is again a LazyObject.
+ *
+ */
+
+public class LazyMap extends LazyNonPrimitive<LazyMapObjectInspector> {
+
+ private static Log LOG = LogFactory.getLog(LazyMap.class.getName());
+
+ /**
+ * Whether the data is already parsed or not.
+ */
+ boolean parsed;
+
+ /**
+ * The size of the map. Only valid when the data is parsed. -1 when the map
+ * is NULL.
+ */
+ int mapSize = 0;
+
+ /**
+ * The beginning position and length of key[i] and value[i]. Only valid when
+ * the data is parsed.
+ */
+ int[] keyStart;
+ int[] keyLength;
+ int[] valueStart;
+ int[] valueLength;
+ /**
+ * Whether valueObjects[i]/keyObjects[i] is initialized or not.
+ */
+ boolean[] keyInited;
+ boolean[] valueInited;
+
+ /**
+ * Whether valueObjects[i]/keyObjects[i] is null or not This could not be
+ * inferred from the length of the object. In particular, a 0-length string
+ * is not null.
+ */
+ boolean[] keyIsNull;
+ boolean[] valueIsNull;
+
+ /**
+ * The keys are stored in an array of LazyPrimitives.
+ */
+ LazyPrimitive<?, ?>[] keyObjects;
+ /**
+ * The values are stored in an array of LazyObjects. value[index] will start
+ * from KeyEnd[index] + 1, and ends before KeyStart[index+1] - 1.
+ */
+ LazyObject[] valueObjects;
+
+ protected LazyMap(LazyMapObjectInspector oi) {
+ super(oi);
+ }
+
+ /**
+ * Set the row data for this LazyMap.
+ *
+ * @see LazyObject#init(ByteArrayRef, int, int)
+ */
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ super.init(bytes, start, length);
+ parsed = false;
+ }
+
+ /**
+ * Adjust the size of arrays: keyStart, keyLength valueStart, valueLength
+ * keyInited, keyIsNull valueInited, valueIsNull.
+ */
+ protected void adjustArraySize(int newSize) {
+ if (keyStart == null || keyStart.length < newSize) {
+ keyStart = new int[newSize];
+ keyLength = new int[newSize];
+ valueStart = new int[newSize];
+ valueLength = new int[newSize];
+ keyInited = new boolean[newSize];
+ keyIsNull = new boolean[newSize];
+ valueInited = new boolean[newSize];
+ valueIsNull = new boolean[newSize];
+ keyObjects = new LazyPrimitive<?, ?>[newSize];
+ valueObjects = new LazyObject[newSize];
+ }
+ }
+
+ boolean nullMapKey = false;
+ VInt vInt = new LazyUtils.VInt();
+ RecordInfo recordInfo = new LazyUtils.RecordInfo();
+
+ /**
+ * Parse the byte[] and fill keyStart, keyLength, keyIsNull valueStart,
+ * valueLength and valueIsNull.
+ */
+ private void parse() {
+
+ // get the VInt that represents the map size
+ LazyUtils.readVInt(bytes, start, vInt);
+ mapSize = vInt.value;
+ if (0 == mapSize) {
+ parsed = true;
+ return;
+ }
+
+ // adjust arrays
+ adjustArraySize(mapSize);
+
+ // find out the null-bytes
+ int mapByteStart = start + vInt.length;
+ int nullByteCur = mapByteStart;
+ int nullByteEnd = mapByteStart + (mapSize * 2 + 7) / 8;
+ int lastElementByteEnd = nullByteEnd;
+
+ // parsing the keys and values one by one
+ for (int i = 0; i < mapSize; i++) {
+ // parse a key
+ keyIsNull[i] = true;
+ if ((bytes[nullByteCur] & (1 << ((i * 2) % 8))) != 0) {
+ keyIsNull[i] = false;
+ LazyUtils.checkObjectByteInfo(((MapObjectInspector) oi).getMapKeyObjectInspector(), bytes,
+ lastElementByteEnd, recordInfo);
+ keyStart[i] = lastElementByteEnd + recordInfo.elementOffset;
+ keyLength[i] = recordInfo.elementSize;
+ lastElementByteEnd = keyStart[i] + keyLength[i];
+ } else if (!nullMapKey) {
+ nullMapKey = true;
+ LOG.warn("Null map key encountered! Ignoring similar problems.");
+ }
+
+ // parse a value
+ valueIsNull[i] = true;
+ if ((bytes[nullByteCur] & (1 << ((i * 2 + 1) % 8))) != 0) {
+ valueIsNull[i] = false;
+ LazyUtils.checkObjectByteInfo(((MapObjectInspector) oi).getMapValueObjectInspector(), bytes,
+ lastElementByteEnd, recordInfo);
+ valueStart[i] = lastElementByteEnd + recordInfo.elementOffset;
+ valueLength[i] = recordInfo.elementSize;
+ lastElementByteEnd = valueStart[i] + valueLength[i];
+ }
+
+ // move onto the next null byte
+ if (3 == (i % 4)) {
+ nullByteCur++;
+ }
+ }
+
+ Arrays.fill(keyInited, 0, mapSize, false);
+ Arrays.fill(valueInited, 0, mapSize, false);
+ parsed = true;
+ }
+
+ /**
+ * Get the value object with the index without checking parsed.
+ *
+ * @param index
+ * The index into the array starting from 0
+ */
+ private LazyObject uncheckedGetValue(int index) {
+ if (valueIsNull[index]) {
+ return null;
+ }
+ if (!valueInited[index]) {
+ valueInited[index] = true;
+ if (valueObjects[index] == null) {
+ valueObjects[index] = LazyFactory.createLazyObject(((MapObjectInspector) oi)
+ .getMapValueObjectInspector());
+ }
+ valueObjects[index].init(bytes, valueStart[index], valueLength[index]);
+ }
+ return valueObjects[index];
+ }
+
+ /**
+ * Get the value in the map for the key.
+ *
+ * If there are multiple matches (which is possible in the serialized
+ * format), only the first one is returned.
+ *
+ * The most efficient way to get the value for the key is to serialize the
+ * key and then try to find it in the array. We do linear search because in
+ * most cases, user only wants to get one or two values out of the map, and
+ * the cost of building up a HashMap is substantially higher.
+ *
+ * @param key
+ * The key object that we are looking for.
+ * @return The corresponding value object, or NULL if not found
+ */
+ public Object getMapValueElement(Object key) {
+ if (!parsed) {
+ parse();
+ }
+ // search for the key
+ for (int i = 0; i < mapSize; i++) {
+ LazyPrimitive<?, ?> lazyKeyI = uncheckedGetKey(i);
+ if (lazyKeyI == null) {
+ continue;
+ }
+ // getWritableObject() will convert LazyPrimitive to actual
+ // primitive
+ // writable objects.
+ Object keyI = lazyKeyI.getWritableObject();
+ if (keyI == null) {
+ continue;
+ }
+ if (keyI.equals(key)) {
+ // Got a match, return the value
+ LazyObject v = uncheckedGetValue(i);
+ return v == null ? v : v.getObject();
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Get the key object with the index without checking parsed.
+ *
+ * @param index
+ * The index into the array starting from 0
+ */
+ private LazyPrimitive<?, ?> uncheckedGetKey(int index) {
+ if (keyIsNull[index]) {
+ return null;
+ }
+ if (!keyInited[index]) {
+ keyInited[index] = true;
+ if (keyObjects[index] == null) {
+ // Keys are always primitive
+ keyObjects[index] = LazyFactory
+ .createLazyPrimitiveClass((PrimitiveObjectInspector) ((MapObjectInspector) oi)
+ .getMapKeyObjectInspector());
+ }
+ keyObjects[index].init(bytes, keyStart[index], keyLength[index]);
+ }
+ return keyObjects[index];
+ }
+
+ /**
+ * cachedMap is reused for different calls to getMap(). But each LazyMap has
+ * a separate cachedMap so we won't overwrite the data by accident.
+ */
+ LinkedHashMap<Object, Object> cachedMap;
+
+ /**
+ * Return the map object representing this LazyMap. Note that the keyObjects
+ * will be Writable primitive objects.
+ *
+ * @return the map object
+ */
+ public Map<Object, Object> getMap() {
+ if (!parsed) {
+ parse();
+ }
+ if (cachedMap == null) {
+ // Use LinkedHashMap to provide deterministic order
+ cachedMap = new LinkedHashMap<Object, Object>();
+ } else {
+ cachedMap.clear();
+ }
+
+ // go through each element of the map
+ for (int i = 0; i < mapSize; i++) {
+ LazyPrimitive<?, ?> lazyKey = uncheckedGetKey(i);
+ if (lazyKey == null) {
+ continue;
+ }
+ Object key = lazyKey.getObject();
+ // do not overwrite if there are duplicate keys
+ if (key != null && !cachedMap.containsKey(key)) {
+ LazyObject lazyValue = uncheckedGetValue(i);
+ Object value = (lazyValue == null ? null : lazyValue.getObject());
+ cachedMap.put(key, value);
+ }
+ }
+ return cachedMap;
+ }
+
+ /**
+ * Get the size of the map represented by this LazyMap.
+ *
+ * @return The size of the map
+ */
+ public int getMapSize() {
+ if (!parsed) {
+ parse();
+ }
+ return mapSize;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyNonPrimitive.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyNonPrimitive.java
new file mode 100644
index 0000000..f7ae1e3
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyNonPrimitive.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * LazyPrimitive stores a primitive Object in a LazyObject.
+ */
+public abstract class LazyNonPrimitive<OI extends ObjectInspector> extends LazyObject<OI> {
+
+ protected byte[] bytes;
+ protected int start;
+ protected int length;
+
+ /**
+ * Create a LazyNonPrimitive object with the specified ObjectInspector.
+ *
+ * @param oi
+ * The ObjectInspector would have to have a hierarchy of
+ * LazyObjectInspectors with the leaf nodes being
+ * WritableObjectInspectors. It's used both for accessing the
+ * type hierarchy of the complex object, as well as getting meta
+ * information (separator, nullSequence, etc) when parsing the
+ * lazy object.
+ */
+ protected LazyNonPrimitive(OI oi) {
+ super(oi);
+ bytes = null;
+ start = 0;
+ length = 0;
+ }
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (bytes == null) {
+ throw new RuntimeException("bytes cannot be null!");
+ }
+ this.bytes = bytes;
+ this.start = start;
+ this.length = length;
+ assert start >= 0;
+ assert start + length <= bytes.length;
+ }
+
+ @Override
+ public Object getObject() {
+ return this;
+ }
+
+ @Override
+ public int hashCode() {
+ return LazyUtils.hashBytes(bytes, start, length);
+ }
+
+ @Override
+ public void init(IFrameTupleReference tuple) {
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyObject.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyObject.java
new file mode 100644
index 0000000..d71817b
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyObject.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * LazyObject stores an object in a range of bytes in a byte[].
+ *
+ * A LazyObject can represent any primitive object or hierarchical object like
+ * array, map or struct.
+ */
+public abstract class LazyObject<OI extends ObjectInspector> {
+
+ OI oi;
+
+ /**
+ * Create a LazyObject.
+ *
+ * @param oi
+ * Derived classes can access meta information about this Lazy
+ * Object (e.g, separator, nullSequence, escaper) from it.
+ */
+ protected LazyObject(OI oi) {
+ this.oi = oi;
+ }
+
+ /**
+ * Set the data for this LazyObject. We take ByteArrayRef instead of byte[]
+ * so that we will be able to drop the reference to byte[] by a single
+ * assignment. The ByteArrayRef object can be reused across multiple rows.
+ *
+ * @param bytes
+ * The wrapper of the byte[].
+ * @param start
+ * The start position inside the bytes.
+ * @param length
+ * The length of the data, starting from "start"
+ * @see ByteArrayRef
+ */
+ public abstract void init(byte[] bytes, int start, int length);
+
+ public abstract void init(IFrameTupleReference tuple);
+
+ /**
+ * If the LazyObject is a primitive Object, then deserialize it and return
+ * the actual primitive Object. Otherwise (array, map, struct), return this.
+ */
+ public abstract Object getObject();
+
+ @Override
+ public abstract int hashCode();
+
+ protected OI getInspector() {
+ return oi;
+ }
+
+ protected void setInspector(OI oi) {
+ this.oi = oi;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyPrimitive.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyPrimitive.java
new file mode 100644
index 0000000..8139c65
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyPrimitive.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * LazyPrimitive stores a primitive Object in a LazyObject.
+ */
+public abstract class LazyPrimitive<OI extends ObjectInspector, T extends Writable> extends LazyObject<OI> {
+
+ LazyPrimitive(OI oi) {
+ super(oi);
+ }
+
+ LazyPrimitive(LazyPrimitive<OI, T> copy) {
+ super(copy.oi);
+ isNull = copy.isNull;
+ }
+
+ T data;
+ boolean isNull = false;
+
+ /**
+ * Returns the primitive object represented by this LazyObject. This is
+ * useful because it can make sure we have "null" for null objects.
+ */
+ @Override
+ public Object getObject() {
+ return isNull ? null : this;
+ }
+
+ public T getWritableObject() {
+ return isNull ? null : data;
+ }
+
+ @Override
+ public String toString() {
+ return isNull ? "null" : data.toString();
+ }
+
+ @Override
+ public int hashCode() {
+ return isNull ? 0 : data.hashCode();
+ }
+
+ @Override
+ public void init(IFrameTupleReference tuple) {
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
new file mode 100644
index 0000000..a5d4b3b
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
@@ -0,0 +1,459 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde2.ByteStream;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * The LazySerDe class combines the lazy property of LazySimpleSerDe class and
+ * the binary property of BinarySortable class. Lazy means a field is not
+ * deserialized until required. Binary means a field is serialized in binary
+ * compact format.
+ */
+public class LazySerDe implements SerDe {
+
+ public static final Log LOG = LogFactory.getLog(LazySerDe.class.getName());
+
+ public LazySerDe() {
+ }
+
+ List<String> columnNames;
+ List<TypeInfo> columnTypes;
+
+ TypeInfo rowTypeInfo;
+ ObjectInspector cachedObjectInspector;
+
+ // The object for storing row data
+ LazyColumnar cachedLazyStruct;
+
+ /**
+ * Initialize the SerDe with configuration and table information.
+ */
+ @Override
+ public void initialize(Configuration conf, Properties tbl) throws SerDeException {
+ // Get column names and types
+ String columnNameProperty = tbl.getProperty(Constants.LIST_COLUMNS);
+ String columnTypeProperty = tbl.getProperty(Constants.LIST_COLUMN_TYPES);
+ if (columnNameProperty.length() == 0) {
+ columnNames = new ArrayList<String>();
+ } else {
+ columnNames = Arrays.asList(columnNameProperty.split(","));
+ }
+ if (columnTypeProperty.length() == 0) {
+ columnTypes = new ArrayList<TypeInfo>();
+ } else {
+ columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
+ }
+ assert (columnNames.size() == columnTypes.size());
+ // Create row related objects
+ rowTypeInfo = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes);
+ // Create the object inspector and the lazy binary struct object
+ cachedObjectInspector = LazyUtils.getLazyObjectInspectorFromTypeInfo(rowTypeInfo, true);
+ cachedLazyStruct = (LazyColumnar) LazyFactory.createLazyObject(cachedObjectInspector);
+ // output debug info
+ LOG.debug("LazySerDe initialized with: columnNames=" + columnNames + " columnTypes=" + columnTypes);
+ }
+
+ /**
+ * Returns the ObjectInspector for the row.
+ */
+ @Override
+ public ObjectInspector getObjectInspector() throws SerDeException {
+ return cachedObjectInspector;
+ }
+
+ /**
+ * Returns the Writable Class after serialization.
+ */
+ @Override
+ public Class<? extends Writable> getSerializedClass() {
+ return BytesWritable.class;
+ }
+
+ // The wrapper for byte array
+ ByteArrayRef byteArrayRef;
+
+ /**
+ * Deserialize a table record to a Lazy struct.
+ */
+ @Override
+ public Object deserialize(Writable field) throws SerDeException {
+ if (byteArrayRef == null) {
+ byteArrayRef = new ByteArrayRef();
+ }
+ if (field instanceof BytesWritable) {
+ BytesWritable b = (BytesWritable) field;
+ if (b.getSize() == 0) {
+ return null;
+ }
+ // For backward-compatibility with hadoop 0.17
+ byteArrayRef.setData(b.get());
+ cachedLazyStruct.init(byteArrayRef.getData(), 0, b.getSize());
+ } else if (field instanceof Text) {
+ Text t = (Text) field;
+ if (t.getLength() == 0) {
+ return null;
+ }
+ byteArrayRef.setData(t.getBytes());
+ cachedLazyStruct.init(byteArrayRef.getData(), 0, t.getLength());
+ } else {
+ throw new SerDeException(getClass().toString() + ": expects either BytesWritable or Text object!");
+ }
+ return cachedLazyStruct;
+ }
+
+ /**
+ * The reusable output buffer and serialize byte buffer.
+ */
+ BytesWritable serializeBytesWritable = new BytesWritable();
+ ByteStream.Output serializeByteStream = new ByteStream.Output();
+
+ /**
+ * Serialize an object to a byte buffer in a binary compact way.
+ */
+ @Override
+ public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException {
+ // make sure it is a struct record or not
+ serializeByteStream.reset();
+
+ if (objInspector.getCategory() != Category.STRUCT) {
+ // serialize the primitive object
+ serialize(serializeByteStream, obj, objInspector);
+ } else {
+ // serialize the row as a struct
+ serializeStruct(serializeByteStream, obj, (StructObjectInspector) objInspector);
+ }
+ // return the serialized bytes
+ serializeBytesWritable.set(serializeByteStream.getData(), 0, serializeByteStream.getCount());
+ return serializeBytesWritable;
+ }
+
+ boolean nullMapKey = false;
+
+ /**
+ * Serialize a struct object without writing the byte size. This function is
+ * shared by both row serialization and struct serialization.
+ *
+ * @param byteStream
+ * the byte stream storing the serialization data
+ * @param obj
+ * the struct object to serialize
+ * @param objInspector
+ * the struct object inspector
+ */
+ private void serializeStruct(Output byteStream, Object obj, StructObjectInspector soi) {
+ // do nothing for null struct
+ if (null == obj) {
+ return;
+ }
+ /*
+ * Interleave serializing one null byte and 8 struct fields in each
+ * round, in order to support data deserialization with different table
+ * schemas
+ */
+ List<? extends StructField> fields = soi.getAllStructFieldRefs();
+ int size = fields.size();
+ int lasti = 0;
+ byte nullByte = 0;
+ for (int i = 0; i < size; i++) {
+ // set bit to 1 if a field is not null
+ if (null != soi.getStructFieldData(obj, fields.get(i))) {
+ nullByte |= 1 << (i % 8);
+ }
+ // write the null byte every eight elements or
+ // if this is the last element and serialize the
+ // corresponding 8 struct fields at the same time
+ if (7 == i % 8 || i == size - 1) {
+ serializeByteStream.write(nullByte);
+ for (int j = lasti; j <= i; j++) {
+ serialize(serializeByteStream, soi.getStructFieldData(obj, fields.get(j)), fields.get(j)
+ .getFieldObjectInspector());
+ }
+ lasti = i + 1;
+ nullByte = 0;
+ }
+ }
+ }
+
+ /**
+ * A recursive function that serialize an object to a byte buffer based on
+ * its object inspector.
+ *
+ * @param byteStream
+ * the byte stream storing the serialization data
+ * @param obj
+ * the object to serialize
+ * @param objInspector
+ * the object inspector
+ */
+ private void serialize(Output byteStream, Object obj, ObjectInspector objInspector) {
+
+ // do nothing for null object
+ if (null == obj) {
+ return;
+ }
+
+ switch (objInspector.getCategory()) {
+ case PRIMITIVE: {
+ PrimitiveObjectInspector poi = (PrimitiveObjectInspector) objInspector;
+ switch (poi.getPrimitiveCategory()) {
+ case VOID: {
+ return;
+ }
+ case BOOLEAN: {
+ boolean v = ((BooleanObjectInspector) poi).get(obj);
+ byteStream.write((byte) (v ? 1 : 0));
+ return;
+ }
+ case BYTE: {
+ ByteObjectInspector boi = (ByteObjectInspector) poi;
+ byte v = boi.get(obj);
+ byteStream.write(v);
+ return;
+ }
+ case SHORT: {
+ ShortObjectInspector spoi = (ShortObjectInspector) poi;
+ short v = spoi.get(obj);
+ byteStream.write((byte) (v >> 8));
+ byteStream.write((byte) (v));
+ return;
+ }
+ case INT: {
+ IntObjectInspector ioi = (IntObjectInspector) poi;
+ int v = ioi.get(obj);
+ LazyUtils.writeVInt(byteStream, v);
+ return;
+ }
+ case LONG: {
+ LongObjectInspector loi = (LongObjectInspector) poi;
+ long v = loi.get(obj);
+ LazyUtils.writeVLong(byteStream, v);
+ return;
+ }
+ case FLOAT: {
+ FloatObjectInspector foi = (FloatObjectInspector) poi;
+ int v = Float.floatToIntBits(foi.get(obj));
+ byteStream.write((byte) (v >> 24));
+ byteStream.write((byte) (v >> 16));
+ byteStream.write((byte) (v >> 8));
+ byteStream.write((byte) (v));
+ return;
+ }
+ case DOUBLE: {
+ DoubleObjectInspector doi = (DoubleObjectInspector) poi;
+ long v = Double.doubleToLongBits(doi.get(obj));
+ byteStream.write((byte) (v >> 56));
+ byteStream.write((byte) (v >> 48));
+ byteStream.write((byte) (v >> 40));
+ byteStream.write((byte) (v >> 32));
+ byteStream.write((byte) (v >> 24));
+ byteStream.write((byte) (v >> 16));
+ byteStream.write((byte) (v >> 8));
+ byteStream.write((byte) (v));
+ return;
+ }
+ case STRING: {
+ StringObjectInspector soi = (StringObjectInspector) poi;
+ Text t = soi.getPrimitiveWritableObject(obj);
+ /* write byte size of the string which is a vint */
+ int length = t.getLength();
+ LazyUtils.writeVInt(byteStream, length);
+ /* write string itself */
+ byte[] data = t.getBytes();
+ byteStream.write(data, 0, length);
+ return;
+ }
+ default: {
+ throw new RuntimeException("Unrecognized type: " + poi.getPrimitiveCategory());
+ }
+ }
+ }
+ case LIST: {
+ ListObjectInspector loi = (ListObjectInspector) objInspector;
+ ObjectInspector eoi = loi.getListElementObjectInspector();
+
+ // 1/ reserve spaces for the byte size of the list
+ // which is a integer and takes four bytes
+ int byteSizeStart = byteStream.getCount();
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ int listStart = byteStream.getCount();
+
+ // 2/ write the size of the list as a VInt
+ int size = loi.getListLength(obj);
+ LazyUtils.writeVInt(byteStream, size);
+
+ // 3/ write the null bytes
+ byte nullByte = 0;
+ for (int eid = 0; eid < size; eid++) {
+ // set the bit to 1 if an element is not null
+ if (null != loi.getListElement(obj, eid)) {
+ nullByte |= 1 << (eid % 8);
+ }
+ // store the byte every eight elements or
+ // if this is the last element
+ if (7 == eid % 8 || eid == size - 1) {
+ byteStream.write(nullByte);
+ nullByte = 0;
+ }
+ }
+
+ // 4/ write element by element from the list
+ for (int eid = 0; eid < size; eid++) {
+ serialize(byteStream, loi.getListElement(obj, eid), eoi);
+ }
+
+ // 5/ update the list byte size
+ int listEnd = byteStream.getCount();
+ int listSize = listEnd - listStart;
+ byte[] bytes = byteStream.getData();
+ bytes[byteSizeStart] = (byte) (listSize >> 24);
+ bytes[byteSizeStart + 1] = (byte) (listSize >> 16);
+ bytes[byteSizeStart + 2] = (byte) (listSize >> 8);
+ bytes[byteSizeStart + 3] = (byte) (listSize);
+
+ return;
+ }
+ case MAP: {
+ MapObjectInspector moi = (MapObjectInspector) objInspector;
+ ObjectInspector koi = moi.getMapKeyObjectInspector();
+ ObjectInspector voi = moi.getMapValueObjectInspector();
+ Map<?, ?> map = moi.getMap(obj);
+
+ // 1/ reserve spaces for the byte size of the map
+ // which is a integer and takes four bytes
+ int byteSizeStart = byteStream.getCount();
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ int mapStart = byteStream.getCount();
+
+ // 2/ write the size of the map which is a VInt
+ int size = map.size();
+ LazyUtils.writeVInt(byteStream, size);
+
+ // 3/ write the null bytes
+ int b = 0;
+ byte nullByte = 0;
+ for (Map.Entry<?, ?> entry : map.entrySet()) {
+ // set the bit to 1 if a key is not null
+ if (null != entry.getKey()) {
+ nullByte |= 1 << (b % 8);
+ } else if (!nullMapKey) {
+ nullMapKey = true;
+ LOG.warn("Null map key encountered! Ignoring similar problems.");
+ }
+ b++;
+ // set the bit to 1 if a value is not null
+ if (null != entry.getValue()) {
+ nullByte |= 1 << (b % 8);
+ }
+ b++;
+ // write the byte to stream every 4 key-value pairs
+ // or if this is the last key-value pair
+ if (0 == b % 8 || b == size * 2) {
+ byteStream.write(nullByte);
+ nullByte = 0;
+ }
+ }
+
+ // 4/ write key-value pairs one by one
+ for (Map.Entry<?, ?> entry : map.entrySet()) {
+ serialize(byteStream, entry.getKey(), koi);
+ serialize(byteStream, entry.getValue(), voi);
+ }
+
+ // 5/ update the byte size of the map
+ int mapEnd = byteStream.getCount();
+ int mapSize = mapEnd - mapStart;
+ byte[] bytes = byteStream.getData();
+ bytes[byteSizeStart] = (byte) (mapSize >> 24);
+ bytes[byteSizeStart + 1] = (byte) (mapSize >> 16);
+ bytes[byteSizeStart + 2] = (byte) (mapSize >> 8);
+ bytes[byteSizeStart + 3] = (byte) (mapSize);
+
+ return;
+ }
+ case STRUCT: {
+ // 1/ reserve spaces for the byte size of the struct
+ // which is a integer and takes four bytes
+ int byteSizeStart = byteStream.getCount();
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ byteStream.write((byte) 0);
+ int structStart = byteStream.getCount();
+
+ // 2/ serialize the struct
+ serializeStruct(byteStream, obj, (StructObjectInspector) objInspector);
+
+ // 3/ update the byte size of the struct
+ int structEnd = byteStream.getCount();
+ int structSize = structEnd - structStart;
+ byte[] bytes = byteStream.getData();
+ bytes[byteSizeStart] = (byte) (structSize >> 24);
+ bytes[byteSizeStart + 1] = (byte) (structSize >> 16);
+ bytes[byteSizeStart + 2] = (byte) (structSize >> 8);
+ bytes[byteSizeStart + 3] = (byte) (structSize);
+
+ return;
+ }
+ default: {
+ throw new RuntimeException("Unrecognized type: " + objInspector.getCategory());
+ }
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyShort.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyShort.java
new file mode 100644
index 0000000..4588c58
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyShort.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyShortObjectInspector;
+
+/**
+ * LazyObject for storing a value of Short.
+ *
+ * <p>
+ * Part of the code is adapted from Apache Harmony Project.
+ *
+ * As with the specification, this implementation relied on code laid out in <a
+ * href="http://www.hackersdelight.org/">Henry S. Warren, Jr.'s Hacker's
+ * Delight, (Addison Wesley, 2002)</a> as well as <a
+ * href="http://aggregate.org/MAGIC/">The Aggregate's Magic Algorithms</a>.
+ * </p>
+ *
+ */
+public class LazyShort extends LazyPrimitive<LazyShortObjectInspector, ShortWritable> {
+
+ public LazyShort(LazyShortObjectInspector oi) {
+ super(oi);
+ data = new ShortWritable();
+ }
+
+ public LazyShort(LazyShort copy) {
+ super(copy);
+ data = new ShortWritable(copy.data.get());
+ }
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+
+ assert (2 == length);
+ data.set(LazyUtils.byteArrayToShort(bytes, start));
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyString.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyString.java
new file mode 100644
index 0000000..a11b1e1
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyString.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.VInt;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.LazyStringObjectInspector;
+
+/**
+ * LazyObject for storing a value of String.
+ */
+public class LazyString extends LazyPrimitive<LazyStringObjectInspector, Text> {
+
+ public LazyString(LazyStringObjectInspector oi) {
+ super(oi);
+ data = new Text();
+ }
+
+ public LazyString(LazyString copy) {
+ super(copy);
+ data = new Text(copy.data);
+ }
+
+ VInt vInt = new LazyUtils.VInt();
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ if (length == 0) {
+ isNull = true;
+ return;
+ } else
+ isNull = false;
+
+ // get the byte length of the string
+ LazyUtils.readVInt(bytes, start, vInt);
+ if (vInt.value + vInt.length != length)
+ throw new IllegalStateException("parse string: length mismatch, expected " + (vInt.value + vInt.length)
+ + " but get " + length);
+ assert (length - vInt.length > -1);
+ data.set(bytes, start + vInt.length, length - vInt.length);
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyStruct.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyStruct.java
new file mode 100644
index 0000000..3ab59b3
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyStruct.java
@@ -0,0 +1,236 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils.RecordInfo;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyStructObjectInspector;
+
+/**
+ * LazyStruct is serialized as follows: start A B A B A B end bytes[] ->
+ * |-----|---------|--- ... ---|-----|---------|
+ *
+ * Section A is one null-byte, corresponding to eight struct fields in Section
+ * B. Each bit indicates whether the corresponding field is null (0) or not null
+ * (1). Each field is a LazyObject.
+ *
+ * Following B, there is another section A and B. This pattern repeats until the
+ * all struct fields are serialized.
+ */
+public class LazyStruct extends LazyNonPrimitive<LazyStructObjectInspector> {
+
+ private static Log LOG = LogFactory.getLog(LazyStruct.class.getName());
+
+ /**
+ * Whether the data is already parsed or not.
+ */
+ boolean parsed;
+
+ /**
+ * The fields of the struct.
+ */
+ LazyObject[] fields;
+
+ /**
+ * Whether a field is initialized or not.
+ */
+ boolean[] fieldInited;
+
+ /**
+ * Whether a field is null or not. Because length is 0 does not means the
+ * field is null. In particular, a 0-length string is not null.
+ */
+ boolean[] fieldIsNull;
+
+ /**
+ * The start positions and lengths of struct fields. Only valid when the
+ * data is parsed.
+ */
+ int[] fieldStart;
+ int[] fieldLength;
+
+ /**
+ * Construct a LazyStruct object with an ObjectInspector.
+ */
+ protected LazyStruct(LazyStructObjectInspector oi) {
+ super(oi);
+ }
+
+ @Override
+ public void init(byte[] bytes, int start, int length) {
+ super.init(bytes, start, length);
+ parsed = false;
+ }
+
+ RecordInfo recordInfo = new LazyUtils.RecordInfo();
+ boolean missingFieldWarned = false;
+ boolean extraFieldWarned = false;
+
+ /**
+ * Parse the byte[] and fill fieldStart, fieldLength, fieldInited and
+ * fieldIsNull.
+ */
+ private void parse() {
+
+ List<? extends StructField> fieldRefs = ((StructObjectInspector) oi).getAllStructFieldRefs();
+
+ if (fields == null) {
+ fields = new LazyObject[fieldRefs.size()];
+ for (int i = 0; i < fields.length; i++) {
+ ObjectInspector insp = fieldRefs.get(i).getFieldObjectInspector();
+ fields[i] = insp == null ? null : LazyFactory.createLazyObject(insp);
+ }
+ fieldInited = new boolean[fields.length];
+ fieldIsNull = new boolean[fields.length];
+ fieldStart = new int[fields.length];
+ fieldLength = new int[fields.length];
+ }
+
+ /**
+ * Please note that one null byte is followed by eight fields, then more
+ * null byte and fields.
+ */
+
+ int fieldId = 0;
+ int structByteEnd = start + length;
+
+ byte nullByte = bytes[start];
+ int lastFieldByteEnd = start + 1;
+ // Go through all bytes in the byte[]
+ for (int i = 0; i < fields.length; i++) {
+ fieldIsNull[i] = true;
+ if ((nullByte & (1 << (i % 8))) != 0) {
+ fieldIsNull[i] = false;
+ LazyUtils.checkObjectByteInfo(fieldRefs.get(i).getFieldObjectInspector(), bytes, lastFieldByteEnd,
+ recordInfo);
+ fieldStart[i] = lastFieldByteEnd + recordInfo.elementOffset;
+ fieldLength[i] = recordInfo.elementSize;
+ lastFieldByteEnd = fieldStart[i] + fieldLength[i];
+ }
+
+ // count how many fields are there
+ if (lastFieldByteEnd <= structByteEnd) {
+ fieldId++;
+ }
+ // next byte is a null byte if there are more bytes to go
+ if (7 == (i % 8)) {
+ if (lastFieldByteEnd < structByteEnd) {
+ nullByte = bytes[lastFieldByteEnd];
+ lastFieldByteEnd++;
+ } else {
+ // otherwise all null afterwards
+ nullByte = 0;
+ lastFieldByteEnd++;
+ }
+ }
+ }
+
+ // Extra bytes at the end?
+ if (!extraFieldWarned && lastFieldByteEnd < structByteEnd) {
+ extraFieldWarned = true;
+ LOG.warn("Extra bytes detected at the end of the row! Ignoring similar " + "problems.");
+ }
+
+ // Missing fields?
+ if (!missingFieldWarned && lastFieldByteEnd > structByteEnd) {
+ missingFieldWarned = true;
+ LOG.warn("Missing fields! Expected " + fields.length + " fields but " + "only got " + fieldId
+ + "! Ignoring similar problems.");
+ }
+
+ Arrays.fill(fieldInited, false);
+ parsed = true;
+ }
+
+ /**
+ * Get one field out of the struct.
+ *
+ * If the field is a primitive field, return the actual object. Otherwise
+ * return the LazyObject. This is because PrimitiveObjectInspector does not
+ * have control over the object used by the user - the user simply directly
+ * use the Object instead of going through Object
+ * PrimitiveObjectInspector.get(Object).
+ *
+ * @param fieldID
+ * The field ID
+ * @return The field as a LazyObject
+ */
+ public Object getField(int fieldID) {
+ if (!parsed) {
+ parse();
+ }
+ return uncheckedGetField(fieldID);
+ }
+
+ /**
+ * Get the field out of the row without checking parsed. This is called by
+ * both getField and getFieldsAsList.
+ *
+ * @param fieldID
+ * The id of the field starting from 0.
+ * @return The value of the field
+ */
+ private Object uncheckedGetField(int fieldID) {
+ // Test the length first so in most cases we avoid doing a byte[]
+ // comparison.
+ if (fieldIsNull[fieldID]) {
+ return null;
+ }
+ if (!fieldInited[fieldID]) {
+ fieldInited[fieldID] = true;
+ fields[fieldID].init(bytes, fieldStart[fieldID], fieldLength[fieldID]);
+ }
+ return fields[fieldID].getObject();
+ }
+
+ ArrayList<Object> cachedList;
+
+ /**
+ * Get the values of the fields as an ArrayList.
+ *
+ * @return The values of the fields as an ArrayList.
+ */
+ public ArrayList<Object> getFieldsAsList() {
+ if (!parsed) {
+ parse();
+ }
+ if (cachedList == null) {
+ cachedList = new ArrayList<Object>();
+ } else {
+ cachedList.clear();
+ }
+ for (int i = 0; i < fields.length; i++) {
+ cachedList.add(uncheckedGetField(i));
+ }
+ return cachedList;
+ }
+
+ @Override
+ public Object getObject() {
+ return this;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyUtils.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyUtils.java
new file mode 100644
index 0000000..d7baf18
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyUtils.java
@@ -0,0 +1,509 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.WritableUtils;
+
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.LazyObjectInspectorFactory;
+import edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+
+/**
+ * LazyUtils.
+ *
+ */
+public final class LazyUtils {
+
+ private static Log LOG = LogFactory.getLog(LazyUtils.class.getName());
+
+ /**
+ * Convert the byte array to an int starting from the given offset. Refer to
+ * code by aeden on DZone Snippets:
+ *
+ * @param b
+ * the byte array
+ * @param offset
+ * the array offset
+ * @return the integer
+ */
+ public static int byteArrayToInt(byte[] b, int offset) {
+ int value = 0;
+ for (int i = 0; i < 4; i++) {
+ int shift = (4 - 1 - i) * 8;
+ value += (b[i + offset] & 0x000000FF) << shift;
+ }
+ return value;
+ }
+
+ /**
+ * Convert the byte array to a long starting from the given offset.
+ *
+ * @param b
+ * the byte array
+ * @param offset
+ * the array offset
+ * @return the long
+ */
+ public static long byteArrayToLong(byte[] b, int offset) {
+ long value = 0;
+ for (int i = 0; i < 8; i++) {
+ int shift = (8 - 1 - i) * 8;
+ value += ((long) (b[i + offset] & 0x00000000000000FF)) << shift;
+ }
+ return value;
+ }
+
+ /**
+ * Convert the byte array to a short starting from the given offset.
+ *
+ * @param b
+ * the byte array
+ * @param offset
+ * the array offset
+ * @return the short
+ */
+ public static short byteArrayToShort(byte[] b, int offset) {
+ short value = 0;
+ value += (b[offset] & 0x000000FF) << 8;
+ value += (b[offset + 1] & 0x000000FF);
+ return value;
+ }
+
+ /**
+ * Record is the unit that data is serialized in. A record includes two
+ * parts. The first part stores the size of the element and the second part
+ * stores the real element. size element record ->
+ * |----|-------------------------|
+ *
+ * A RecordInfo stores two information of a record, the size of the "size"
+ * part which is the element offset and the size of the element part which
+ * is element size.
+ */
+ public static class RecordInfo {
+ public RecordInfo() {
+ elementOffset = 0;
+ elementSize = 0;
+ }
+
+ public byte elementOffset;
+ public int elementSize;
+
+ @Override
+ public String toString() {
+ return "(" + elementOffset + ", " + elementSize + ")";
+ }
+ }
+
+ static VInt vInt = new LazyUtils.VInt();
+
+ /**
+ * Check a particular field and set its size and offset in bytes based on
+ * the field type and the bytes arrays.
+ *
+ * For void, boolean, byte, short, int, long, float and double, there is no
+ * offset and the size is fixed. For string, map, list, struct, the first
+ * four bytes are used to store the size. So the offset is 4 and the size is
+ * computed by concating the first four bytes together. The first four bytes
+ * are defined with respect to the offset in the bytes arrays.
+ *
+ * @param objectInspector
+ * object inspector of the field
+ * @param bytes
+ * bytes arrays store the table row
+ * @param offset
+ * offset of this field
+ * @param recordInfo
+ * modify this byteinfo object and return it
+ */
+ public static void checkObjectByteInfo(ObjectInspector objectInspector, byte[] bytes, int offset,
+ RecordInfo recordInfo) {
+ Category category = objectInspector.getCategory();
+ switch (category) {
+ case PRIMITIVE:
+ PrimitiveCategory primitiveCategory = ((PrimitiveObjectInspector) objectInspector)
+ .getPrimitiveCategory();
+ switch (primitiveCategory) {
+ case VOID:
+ recordInfo.elementOffset = 0;
+ recordInfo.elementSize = 0;
+ break;
+ case BOOLEAN:
+ case BYTE:
+ recordInfo.elementOffset = 0;
+ recordInfo.elementSize = 1;
+ break;
+ case SHORT:
+ recordInfo.elementOffset = 0;
+ recordInfo.elementSize = 2;
+ break;
+ case FLOAT:
+ recordInfo.elementOffset = 0;
+ recordInfo.elementSize = 4;
+ break;
+ case DOUBLE:
+ recordInfo.elementOffset = 0;
+ recordInfo.elementSize = 8;
+ break;
+ case INT:
+ recordInfo.elementOffset = 0;
+ recordInfo.elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
+ break;
+ case LONG:
+ recordInfo.elementOffset = 0;
+ recordInfo.elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
+ break;
+ case STRING:
+ // using vint instead of 4 bytes
+ LazyUtils.readVInt(bytes, offset, vInt);
+ recordInfo.elementOffset = vInt.length;
+ recordInfo.elementSize = vInt.value;
+ break;
+ default: {
+ throw new RuntimeException("Unrecognized primitive type: " + primitiveCategory);
+ }
+ }
+ break;
+ case LIST:
+ case MAP:
+ case STRUCT:
+ recordInfo.elementOffset = 4;
+ recordInfo.elementSize = LazyUtils.byteArrayToInt(bytes, offset);
+ break;
+ default: {
+ throw new RuntimeException("Unrecognized non-primitive type: " + category);
+ }
+ }
+ }
+
+ /**
+ * A zero-compressed encoded long.
+ */
+ public static class VLong {
+ public VLong() {
+ value = 0;
+ length = 0;
+ }
+
+ public long value;
+ public byte length;
+ };
+
+ /**
+ * Reads a zero-compressed encoded long from a byte array and returns it.
+ *
+ * @param bytes
+ * the byte array
+ * @param offset
+ * offset of the array to read from
+ * @param vlong
+ * storing the deserialized long and its size in byte
+ */
+ public static void readVLong(byte[] bytes, int offset, VLong vlong) {
+ byte firstByte = bytes[offset];
+ vlong.length = (byte) WritableUtils.decodeVIntSize(firstByte);
+ if (vlong.length == 1) {
+ vlong.value = firstByte;
+ return;
+ }
+ long i = 0;
+ for (int idx = 0; idx < vlong.length - 1; idx++) {
+ byte b = bytes[offset + 1 + idx];
+ i = i << 8;
+ i = i | (b & 0xFF);
+ }
+ vlong.value = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+ }
+
+ /**
+ * A zero-compressed encoded integer.
+ */
+ public static class VInt implements Serializable {
+ public VInt() {
+ value = 0;
+ length = 0;
+ }
+
+ public int value;
+ public byte length;
+ };
+
+ /**
+ * Reads a zero-compressed encoded int from a byte array and returns it.
+ *
+ * @param bytes
+ * the byte array
+ * @param offset
+ * offset of the array to read from
+ * @param vInt
+ * storing the deserialized int and its size in byte
+ */
+ public static void readVInt(byte[] bytes, int offset, VInt vInt) {
+ byte firstByte = bytes[offset];
+ vInt.length = (byte) WritableUtils.decodeVIntSize(firstByte);
+ if (vInt.length == 1) {
+ vInt.value = firstByte;
+ return;
+ }
+ int i = 0;
+ for (int idx = 0; idx < vInt.length - 1; idx++) {
+ byte b = bytes[offset + 1 + idx];
+ i = i << 8;
+ i = i | (b & 0xFF);
+ }
+ vInt.value = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1) : i);
+ }
+
+ /**
+ * Writes a zero-compressed encoded int to a byte array.
+ *
+ * @param byteStream
+ * the byte array/stream
+ * @param i
+ * the int
+ */
+ public static void writeVInt(Output byteStream, int i) {
+ writeVLong(byteStream, i);
+ }
+
+ /**
+ * Write a zero-compressed encoded long to a byte array.
+ *
+ * @param byteStream
+ * the byte array/stream
+ * @param l
+ * the long
+ */
+ public static void writeVLong(Output byteStream, long l) {
+ if (l >= -112 && l <= 127) {
+ byteStream.write((byte) l);
+ return;
+ }
+
+ int len = -112;
+ if (l < 0) {
+ l ^= -1L; // take one's complement'
+ len = -120;
+ }
+
+ long tmp = l;
+ while (tmp != 0) {
+ tmp = tmp >> 8;
+ len--;
+ }
+
+ byteStream.write((byte) len);
+
+ len = (len < -120) ? -(len + 120) : -(len + 112);
+
+ for (int idx = len; idx != 0; idx--) {
+ int shiftbits = (idx - 1) * 8;
+ long mask = 0xFFL << shiftbits;
+ byteStream.write((byte) ((l & mask) >> shiftbits));
+ }
+ }
+
+ static Map<TypeInfo, ObjectInspector> cachedLazyObjectInspector = new ConcurrentHashMap<TypeInfo, ObjectInspector>();
+
+ /**
+ * Returns the lazy binary object inspector that can be used to inspect an
+ * lazy binary object of that typeInfo
+ *
+ * For primitive types, we use the standard writable object inspector.
+ */
+ public static ObjectInspector getLazyObjectInspectorFromTypeInfo(TypeInfo typeInfo, boolean topLevel) {
+ if (typeInfo == null)
+ throw new IllegalStateException("illegal type null ");
+ ObjectInspector result = cachedLazyObjectInspector.get(typeInfo);
+ if (result == null) {
+ switch (typeInfo.getCategory()) {
+ case PRIMITIVE: {
+ result = PrimitiveObjectInspectorFactory
+ .getPrimitiveLazyObjectInspector(((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory());
+ break;
+ }
+ case LIST: {
+ ObjectInspector elementObjectInspector = getLazyObjectInspectorFromTypeInfo(
+ ((ListTypeInfo) typeInfo).getListElementTypeInfo(), false);
+ result = LazyObjectInspectorFactory.getLazyListObjectInspector(elementObjectInspector);
+ break;
+ }
+ case MAP: {
+ MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo;
+ ObjectInspector keyObjectInspector = getLazyObjectInspectorFromTypeInfo(
+ mapTypeInfo.getMapKeyTypeInfo(), false);
+ ObjectInspector valueObjectInspector = getLazyObjectInspectorFromTypeInfo(
+ mapTypeInfo.getMapValueTypeInfo(), false);
+ result = LazyObjectInspectorFactory.getLazyMapObjectInspector(keyObjectInspector,
+ valueObjectInspector);
+ break;
+ }
+ case STRUCT: {
+ StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo;
+ List<String> fieldNames = structTypeInfo.getAllStructFieldNames();
+ List<TypeInfo> fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos();
+ List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(fieldTypeInfos.size());
+
+ for (int i = 0; i < fieldTypeInfos.size(); i++) {
+ fieldObjectInspectors.add(getLazyObjectInspectorFromTypeInfo(fieldTypeInfos.get(i), false));
+ }
+
+ // if it is top level then create columnar
+ if (topLevel)
+ result = LazyObjectInspectorFactory.getLazyColumnarObjectInspector(fieldNames,
+ fieldObjectInspectors);
+ // if it is not top level then create struct
+ else
+ result = LazyObjectInspectorFactory.getLazyStructObjectInspector(fieldNames,
+ fieldObjectInspectors);
+
+ break;
+ }
+ default: {
+ result = null;
+ }
+ }
+ cachedLazyObjectInspector.put(typeInfo, result);
+ }
+ return result;
+ }
+
+ /**
+ * get top-level lazy object inspector
+ *
+ * @param fieldNames
+ * @param fieldTypeInfos
+ * @return
+ */
+ public static ObjectInspector getLazyObjectInspector(List<String> fieldNames, List<TypeInfo> fieldTypeInfos) {
+ List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(fieldTypeInfos.size());
+ for (int i = 0; i < fieldTypeInfos.size(); i++) {
+ fieldObjectInspectors.add(getLazyObjectInspectorFromTypeInfo(fieldTypeInfos.get(i), false));
+ }
+
+ return LazyObjectInspectorFactory.getLazyColumnarObjectInspector(fieldNames, fieldObjectInspectors);
+ }
+
+ private LazyUtils() {
+ // prevent instantiation
+ }
+
+ /**
+ * Returns -1 if the first byte sequence is lexicographically less than the
+ * second; returns +1 if the second byte sequence is lexicographically less
+ * than the first; otherwise return 0.
+ */
+ public static int compare(byte[] b1, int start1, int length1, byte[] b2, int start2, int length2) {
+
+ int min = Math.min(length1, length2);
+
+ for (int i = 0; i < min; i++) {
+ if (b1[start1 + i] == b2[start2 + i]) {
+ continue;
+ }
+ if (b1[start1 + i] < b2[start2 + i]) {
+ return -1;
+ } else {
+ return 1;
+ }
+ }
+
+ if (length1 < length2) {
+ return -1;
+ }
+ if (length1 > length2) {
+ return 1;
+ }
+ return 0;
+ }
+
+ public static int hashBytes(byte[] data, int start, int len) {
+ int hash = 1;
+ for (int i = start; i < len; i++) {
+ hash = (31 * hash) + data[i];
+ }
+ return hash;
+ }
+
+ /**
+ * Writes a zero-compressed encoded int to a byte array.
+ *
+ * @param byteStream
+ * the byte array/stream
+ * @param i
+ * the int
+ */
+ public static void writeVInt(DataOutput byteStream, int i) throws IOException {
+ writeVLong(byteStream, i);
+ }
+
+ /**
+ * Write a zero-compressed encoded long to a byte array.
+ *
+ * @param byteStream
+ * the byte array/stream
+ * @param l
+ * the long
+ */
+ public static void writeVLong(DataOutput byteStream, long l) throws IOException {
+ if (l >= -112 && l <= 127) {
+ byteStream.write((byte) l);
+ return;
+ }
+
+ int len = -112;
+ if (l < 0) {
+ l ^= -1L; // take one's complement'
+ len = -120;
+ }
+
+ long tmp = l;
+ while (tmp != 0) {
+ tmp = tmp >> 8;
+ len--;
+ }
+
+ byteStream.write((byte) len);
+
+ len = (len < -120) ? -(len + 120) : -(len + 112);
+
+ for (int idx = len; idx != 0; idx--) {
+ int shiftbits = (idx - 1) * 8;
+ long mask = 0xFFL << shiftbits;
+ byteStream.write((byte) ((l & mask) >> shiftbits));
+ }
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyColumnarObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyColumnarObjectInspector.java
new file mode 100644
index 0000000..b1ca622
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyColumnarObjectInspector.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyColumnar;
+
+/**
+ * ObjectInspector for LazyColumnar.
+ *
+ * @see LazyColumnar
+ */
+public class LazyColumnarObjectInspector extends StandardStructObjectInspector implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ public LazyColumnarObjectInspector(List<String> structFieldNames, List<ObjectInspector> structFieldObjectInspectors) {
+ super(structFieldNames, structFieldObjectInspectors);
+ }
+
+ public LazyColumnarObjectInspector(List<StructField> fields) {
+ super(fields);
+ }
+
+ @Override
+ public Object getStructFieldData(Object data, StructField fieldRef) {
+ if (data == null) {
+ return null;
+ }
+ LazyColumnar struct = (LazyColumnar) data;
+ MyField f = (MyField) fieldRef;
+
+ int fieldID = f.getFieldID();
+ assert (fieldID >= 0 && fieldID < fields.size());
+
+ Object column = struct.getField(fieldID);
+ return column;
+ }
+
+ @Override
+ public List<Object> getStructFieldsDataAsList(Object data) {
+ if (data == null) {
+ return null;
+ }
+ LazyColumnar struct = (LazyColumnar) data;
+ return struct.getFieldsAsList();
+ }
+
+ public String toString() {
+ String str = "";
+ for (MyField f : fields) {
+ str += f.getFieldName() + ":" + f.getFieldObjectInspector().getTypeName() + " ";
+ }
+ return str;
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyListObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyListObjectInspector.java
new file mode 100644
index 0000000..aaa5d66
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyListObjectInspector.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardListObjectInspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyArray;
+
+/**
+ * ObjectInspector for LazyList.
+ */
+public class LazyListObjectInspector extends StandardListObjectInspector {
+
+ protected LazyListObjectInspector(ObjectInspector listElementObjectInspector) {
+ super(listElementObjectInspector);
+ }
+
+ @Override
+ public List<?> getList(Object data) {
+ if (data == null) {
+ return null;
+ }
+ LazyArray array = (LazyArray) data;
+ return array.getList();
+ }
+
+ @Override
+ public Object getListElement(Object data, int index) {
+ if (data == null) {
+ return null;
+ }
+ LazyArray array = (LazyArray) data;
+ return array.getListElementObject(index);
+ }
+
+ @Override
+ public int getListLength(Object data) {
+ if (data == null) {
+ return -1;
+ }
+ LazyArray array = (LazyArray) data;
+ return array.getListLength();
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyMapObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyMapObjectInspector.java
new file mode 100644
index 0000000..1b0c412
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyMapObjectInspector.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector;
+
+import java.util.Map;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardMapObjectInspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyMap;
+
+/**
+ * ObjectInspector for LazyMap.
+ *
+ * @see LazyMap
+ */
+public class LazyMapObjectInspector extends StandardMapObjectInspector {
+
+ protected LazyMapObjectInspector(ObjectInspector mapKeyObjectInspector, ObjectInspector mapValueObjectInspector) {
+ super(mapKeyObjectInspector, mapValueObjectInspector);
+ }
+
+ @Override
+ public Map<?, ?> getMap(Object data) {
+ if (data == null) {
+ return null;
+ }
+ return ((LazyMap) data).getMap();
+ }
+
+ @Override
+ public int getMapSize(Object data) {
+ if (data == null) {
+ return -1;
+ }
+ return ((LazyMap) data).getMapSize();
+ }
+
+ @Override
+ public Object getMapValueElement(Object data, Object key) {
+ if (data == null) {
+ return -1;
+ }
+ return ((LazyMap) data).getMapValueElement(key);
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyObjectInspectorFactory.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyObjectInspectorFactory.java
new file mode 100644
index 0000000..f57f6d9
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyObjectInspectorFactory.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.hivesterix.serde.lazy.objectinspector;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+/**
+ * ObjectInspectorFactory is the primary way to create new ObjectInspector
+ * instances.
+ *
+ * SerDe classes should call the static functions in this library to create an
+ * ObjectInspector to return to the caller of SerDe2.getObjectInspector().
+ *
+ * The reason of having caches here is that ObjectInspectors do not have an
+ * internal state - so ObjectInspectors with the same construction parameters
+ * should result in exactly the same ObjectInspector.
+ */
+
+public final class LazyObjectInspectorFactory {
+
+ static ConcurrentHashMap<ArrayList<Object>, LazyColumnarObjectInspector> cachedLazyColumnarObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyColumnarObjectInspector>();
+
+ static ConcurrentHashMap<ArrayList<Object>, LazyStructObjectInspector> cachedLazyStructObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyStructObjectInspector>();
+
+ static ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector> cachedLazyListObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector>();
+
+ static ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector> cachedLazyMapObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector>();
+
+ public static LazyColumnarObjectInspector getLazyColumnarObjectInspector(List<String> structFieldNames,
+ List<ObjectInspector> structFieldObjectInspectors) {
+ ArrayList<Object> signature = new ArrayList<Object>();
+ signature.add(structFieldNames);
+ signature.add(structFieldObjectInspectors);
+ LazyColumnarObjectInspector result = cachedLazyColumnarObjectInspector.get(signature);
+ if (result == null) {
+ result = new LazyColumnarObjectInspector(structFieldNames, structFieldObjectInspectors);
+ cachedLazyColumnarObjectInspector.put(signature, result);
+ }
+ return result;
+ }
+
+ public static LazyStructObjectInspector getLazyStructObjectInspector(List<String> structFieldNames,
+ List<ObjectInspector> structFieldObjectInspectors) {
+ ArrayList<Object> signature = new ArrayList<Object>();
+ signature.add(structFieldNames);
+ signature.add(structFieldObjectInspectors);
+ LazyStructObjectInspector result = cachedLazyStructObjectInspector.get(signature);
+ if (result == null) {
+ result = new LazyStructObjectInspector(structFieldNames, structFieldObjectInspectors);
+ cachedLazyStructObjectInspector.put(signature, result);
+ }
+ return result;
+ }
+
+ public static LazyListObjectInspector getLazyListObjectInspector(ObjectInspector listElementInspector) {
+ ArrayList<Object> signature = new ArrayList<Object>();
+ signature.add(listElementInspector);
+ LazyListObjectInspector result = cachedLazyListObjectInspector.get(signature);
+ if (result == null) {
+ result = new LazyListObjectInspector(listElementInspector);
+ cachedLazyListObjectInspector.put(signature, result);
+ }
+ return result;
+ }
+
+ public static LazyMapObjectInspector getLazyMapObjectInspector(ObjectInspector keyInspector,
+ ObjectInspector valueInspector) {
+ ArrayList<Object> signature = new ArrayList<Object>();
+ signature.add(keyInspector);
+ signature.add(valueInspector);
+ LazyMapObjectInspector result = cachedLazyMapObjectInspector.get(signature);
+ if (result == null) {
+ result = new LazyMapObjectInspector(keyInspector, valueInspector);
+ cachedLazyMapObjectInspector.put(signature, result);
+ }
+ return result;
+ }
+
+ private LazyObjectInspectorFactory() {
+ // prevent instantiation
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyStructObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyStructObjectInspector.java
new file mode 100644
index 0000000..ad70d4c
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyStructObjectInspector.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyStruct;
+
+/**
+ * ObjectInspector for LazyStruct.
+ *
+ * @see LazyStruct
+ */
+public class LazyStructObjectInspector extends StandardStructObjectInspector {
+
+ protected LazyStructObjectInspector(List<String> structFieldNames, List<ObjectInspector> structFieldObjectInspectors) {
+ super(structFieldNames, structFieldObjectInspectors);
+ }
+
+ protected LazyStructObjectInspector(List<StructField> fields) {
+ super(fields);
+ }
+
+ @Override
+ public Object getStructFieldData(Object data, StructField fieldRef) {
+ if (data == null) {
+ return null;
+ }
+ LazyStruct struct = (LazyStruct) data;
+ MyField f = (MyField) fieldRef;
+
+ int fieldID = f.getFieldID();
+ assert (fieldID >= 0 && fieldID < fields.size());
+
+ return struct.getField(fieldID);
+ }
+
+ @Override
+ public List<Object> getStructFieldsDataAsList(Object data) {
+ if (data == null) {
+ return null;
+ }
+ LazyStruct struct = (LazyStruct) data;
+ return struct.getFieldsAsList();
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/AbstractPrimitiveLazyObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/AbstractPrimitiveLazyObjectInspector.java
new file mode 100644
index 0000000..6c19a73
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/AbstractPrimitiveLazyObjectInspector.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyPrimitive;
+
+/**
+ * An AbstractPrimitiveLazyObjectInspector for a LazyPrimitive object.
+ */
+public abstract class AbstractPrimitiveLazyObjectInspector<T extends Writable> extends AbstractPrimitiveObjectInspector {
+
+ protected AbstractPrimitiveLazyObjectInspector(PrimitiveTypeEntry typeEntry) {
+ super(typeEntry);
+ }
+
+ @Override
+ public T getPrimitiveWritableObject(Object o) {
+ if(o==null)
+ System.out.println("sth. wrong");
+ return o == null ? null : ((LazyPrimitive<?, T>) o).getWritableObject();
+ }
+
+ @Override
+ public boolean preferWritable() {
+ return true;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyBooleanObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyBooleanObjectInspector.java
new file mode 100644
index 0000000..7927c1e
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyBooleanObjectInspector.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.BooleanWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyBoolean;
+
+/**
+ * A WritableBooleanObjectInspector inspects a BooleanWritable Object.
+ */
+public class LazyBooleanObjectInspector extends AbstractPrimitiveLazyObjectInspector<BooleanWritable> implements
+ BooleanObjectInspector {
+
+ LazyBooleanObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.booleanTypeEntry);
+ }
+
+ @Override
+ public boolean get(Object o) {
+ return getPrimitiveWritableObject(o).get();
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyBoolean((LazyBoolean) o);
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : Boolean.valueOf(get(o));
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyByteObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyByteObjectInspector.java
new file mode 100644
index 0000000..10a881c
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyByteObjectInspector.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.ByteWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyByte;
+
+/**
+ * A WritableByteObjectInspector inspects a ByteWritable Object.
+ */
+public class LazyByteObjectInspector extends AbstractPrimitiveLazyObjectInspector<ByteWritable> implements
+ ByteObjectInspector {
+
+ LazyByteObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.byteTypeEntry);
+ }
+
+ @Override
+ public byte get(Object o) {
+ return getPrimitiveWritableObject(o).get();
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyByte((LazyByte) o);
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : Byte.valueOf(get(o));
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyDoubleObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyDoubleObjectInspector.java
new file mode 100644
index 0000000..9f98b56
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyDoubleObjectInspector.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.DoubleWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyDouble;
+
+/**
+ * A WritableDoubleObjectInspector inspects a DoubleWritable Object.
+ */
+public class LazyDoubleObjectInspector extends AbstractPrimitiveLazyObjectInspector<DoubleWritable> implements
+ DoubleObjectInspector {
+
+ LazyDoubleObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.doubleTypeEntry);
+ }
+
+ @Override
+ public double get(Object o) {
+ return getPrimitiveWritableObject(o).get();
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyDouble((LazyDouble) o);
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : Double.valueOf(get(o));
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyFloatObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyFloatObjectInspector.java
new file mode 100644
index 0000000..bf3e9a2
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyFloatObjectInspector.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.FloatWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyFloat;
+
+/**
+ * A FloatObjectInspector inspects a FloatWritable Object.
+ */
+public class LazyFloatObjectInspector extends AbstractPrimitiveLazyObjectInspector<FloatWritable> implements
+ FloatObjectInspector {
+
+ LazyFloatObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.floatTypeEntry);
+ }
+
+ @Override
+ public float get(Object o) {
+ return getPrimitiveWritableObject(o).get();
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyFloat((LazyFloat) o);
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : Float.valueOf(get(o));
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyIntObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyIntObjectInspector.java
new file mode 100644
index 0000000..87bcb0d
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyIntObjectInspector.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.IntWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyInteger;
+
+/**
+ * A WritableIntObjectInspector inspects a IntWritable Object.
+ */
+public class LazyIntObjectInspector extends AbstractPrimitiveLazyObjectInspector<IntWritable> implements
+ IntObjectInspector {
+
+ LazyIntObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.intTypeEntry);
+ }
+
+ @Override
+ public int get(Object o) {
+ return getPrimitiveWritableObject(o).get();
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyInteger((LazyInteger) o);
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : Integer.valueOf(get(o));
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyLongObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyLongObjectInspector.java
new file mode 100644
index 0000000..06b5d3c
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyLongObjectInspector.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.LongWritable;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyLong;
+
+/**
+ * A WritableLongObjectInspector inspects a LongWritable Object.
+ */
+public class LazyLongObjectInspector extends AbstractPrimitiveLazyObjectInspector<LongWritable> implements
+ LongObjectInspector {
+
+ LazyLongObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.longTypeEntry);
+ }
+
+ @Override
+ public long get(Object o) {
+ return getPrimitiveWritableObject(o).get();
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyLong((LazyLong) o);
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : Long.valueOf(get(o));
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
new file mode 100644
index 0000000..c13b547
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import java.util.ArrayList;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+
+/**
+ * LazyPrimitiveObjectInspectorFactory is the primary way to create new
+ * ObjectInspector instances.
+ *
+ * SerDe classes should call the static functions in this library to create an
+ * ObjectInspector to return to the caller of SerDe2.getObjectInspector().
+ *
+ * The reason of having caches here is that ObjectInspector is because
+ * ObjectInspectors do not have an internal state - so ObjectInspectors with the
+ * same construction parameters should result in exactly the same
+ * ObjectInspector.
+ */
+public final class LazyPrimitiveObjectInspectorFactory {
+
+ public static final LazyBooleanObjectInspector LAZY_BOOLEAN_OBJECT_INSPECTOR = new LazyBooleanObjectInspector();
+ public static final LazyByteObjectInspector LAZY_BYTE_OBJECT_INSPECTOR = new LazyByteObjectInspector();
+ public static final LazyShortObjectInspector LAZY_SHORT_OBJECT_INSPECTOR = new LazyShortObjectInspector();
+ public static final LazyIntObjectInspector LAZY_INT_OBJECT_INSPECTOR = new LazyIntObjectInspector();
+ public static final LazyLongObjectInspector LAZY_LONG_OBJECT_INSPECTOR = new LazyLongObjectInspector();
+ public static final LazyFloatObjectInspector LAZY_FLOAT_OBJECT_INSPECTOR = new LazyFloatObjectInspector();
+ public static final LazyDoubleObjectInspector LAZY_DOUBLE_OBJECT_INSPECTOR = new LazyDoubleObjectInspector();
+ public static final LazyVoidObjectInspector LAZY_VOID_OBJECT_INSPECTOR = new LazyVoidObjectInspector();
+
+ static ConcurrentHashMap<ArrayList<Object>, LazyStringObjectInspector> cachedLazyStringObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyStringObjectInspector>();
+
+ public static LazyStringObjectInspector getLazyStringObjectInspector(boolean escaped, byte escapeChar) {
+ ArrayList<Object> signature = new ArrayList<Object>();
+ signature.add(Boolean.valueOf(escaped));
+ signature.add(Byte.valueOf(escapeChar));
+ LazyStringObjectInspector result = cachedLazyStringObjectInspector.get(signature);
+ if (result == null) {
+ result = new LazyStringObjectInspector(escaped, escapeChar);
+ cachedLazyStringObjectInspector.put(signature, result);
+ }
+ return result;
+ }
+
+ public static AbstractPrimitiveLazyObjectInspector<?> getLazyObjectInspector(PrimitiveCategory primitiveCategory,
+ boolean escaped, byte escapeChar) {
+
+ switch (primitiveCategory) {
+ case BOOLEAN:
+ return LAZY_BOOLEAN_OBJECT_INSPECTOR;
+ case BYTE:
+ return LAZY_BYTE_OBJECT_INSPECTOR;
+ case SHORT:
+ return LAZY_SHORT_OBJECT_INSPECTOR;
+ case INT:
+ return LAZY_INT_OBJECT_INSPECTOR;
+ case LONG:
+ return LAZY_LONG_OBJECT_INSPECTOR;
+ case FLOAT:
+ return LAZY_FLOAT_OBJECT_INSPECTOR;
+ case DOUBLE:
+ return LAZY_DOUBLE_OBJECT_INSPECTOR;
+ case STRING:
+ return getLazyStringObjectInspector(escaped, escapeChar);
+ case VOID:
+ return LAZY_VOID_OBJECT_INSPECTOR;
+ default:
+ throw new RuntimeException("Internal error: Cannot find ObjectInspector " + " for " + primitiveCategory);
+ }
+ }
+
+ private LazyPrimitiveObjectInspectorFactory() {
+ // prevent instantiation
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyShortObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyShortObjectInspector.java
new file mode 100644
index 0000000..b02d9bc
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyShortObjectInspector.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyShort;
+
+/**
+ * A WritableShortObjectInspector inspects a ShortWritable Object.
+ */
+public class LazyShortObjectInspector extends AbstractPrimitiveLazyObjectInspector<ShortWritable> implements
+ ShortObjectInspector {
+
+ LazyShortObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.shortTypeEntry);
+ }
+
+ @Override
+ public short get(Object o) {
+ return getPrimitiveWritableObject(o).get();
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyShort((LazyShort) o);
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : Short.valueOf(get(o));
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyStringObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyStringObjectInspector.java
new file mode 100644
index 0000000..4d649dc
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyStringObjectInspector.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyString;
+
+/**
+ * A WritableStringObjectInspector inspects a Text Object.
+ */
+public class LazyStringObjectInspector extends AbstractPrimitiveLazyObjectInspector<Text> implements
+ StringObjectInspector {
+
+ boolean escaped;
+ byte escapeChar;
+
+ LazyStringObjectInspector(boolean escaped, byte escapeChar) {
+ super(PrimitiveObjectInspectorUtils.stringTypeEntry);
+ this.escaped = escaped;
+ this.escapeChar = escapeChar;
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new LazyString((LazyString) o);
+ }
+
+ @Override
+ public Text getPrimitiveWritableObject(Object o) {
+ return o == null ? null : ((LazyString) o).getWritableObject();
+ }
+
+ @Override
+ public String getPrimitiveJavaObject(Object o) {
+ return o == null ? null : ((LazyString) o).getWritableObject().toString();
+ }
+
+ public boolean isEscaped() {
+ return escaped;
+ }
+
+ public byte getEscapeChar() {
+ return escapeChar;
+ }
+
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyVoidObjectInspector.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyVoidObjectInspector.java
new file mode 100644
index 0000000..c916191
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyVoidObjectInspector.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.VoidObjectInspector;
+import org.apache.hadoop.io.NullWritable;
+
+/**
+ * A WritableVoidObjectInspector inspects a NullWritable Object.
+ */
+public class LazyVoidObjectInspector extends AbstractPrimitiveLazyObjectInspector<NullWritable> implements
+ VoidObjectInspector {
+
+ LazyVoidObjectInspector() {
+ super(PrimitiveObjectInspectorUtils.voidTypeEntry);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o;
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ throw new RuntimeException("Internal error: cannot create Void object.");
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/PrimitiveObjectInspectorFactory.java b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
new file mode 100644
index 0000000..dbb766a
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 edu.uci.ics.hivesterix.serde.lazy.objectinspector.primitive;
+
+import java.util.HashMap;
+
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+
+/**
+ * PrimitiveObjectInspectorFactory is the primary way to create new
+ * PrimitiveObjectInspector instances.
+ *
+ * The reason of having caches here is that ObjectInspector is because
+ * ObjectInspectors do not have an internal state - so ObjectInspectors with the
+ * same construction parameters should result in exactly the same
+ * ObjectInspector.
+ */
+public final class PrimitiveObjectInspectorFactory {
+
+ public static final LazyBooleanObjectInspector LazyBooleanObjectInspector = new LazyBooleanObjectInspector();
+ public static final LazyByteObjectInspector LazyByteObjectInspector = new LazyByteObjectInspector();
+ public static final LazyShortObjectInspector LazyShortObjectInspector = new LazyShortObjectInspector();
+ public static final LazyIntObjectInspector LazyIntObjectInspector = new LazyIntObjectInspector();
+ public static final LazyLongObjectInspector LazyLongObjectInspector = new LazyLongObjectInspector();
+ public static final LazyFloatObjectInspector LazyFloatObjectInspector = new LazyFloatObjectInspector();
+ public static final LazyDoubleObjectInspector LazyDoubleObjectInspector = new LazyDoubleObjectInspector();
+ public static final LazyStringObjectInspector LazyStringObjectInspector = new LazyStringObjectInspector(false,
+ (byte) '\\');
+ public static final LazyVoidObjectInspector LazyVoidObjectInspector = new LazyVoidObjectInspector();
+
+ private static HashMap<PrimitiveCategory, AbstractPrimitiveLazyObjectInspector<?>> cachedPrimitiveLazyInspectorCache = new HashMap<PrimitiveCategory, AbstractPrimitiveLazyObjectInspector<?>>();
+
+ static {
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.BOOLEAN, LazyBooleanObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.BYTE, LazyByteObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.SHORT, LazyShortObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.INT, LazyIntObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.LONG, LazyLongObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.FLOAT, LazyFloatObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.DOUBLE, LazyDoubleObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.STRING, LazyStringObjectInspector);
+ cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.VOID, LazyVoidObjectInspector);
+ }
+
+ /**
+ * Returns the PrimitiveWritableObjectInspector for the PrimitiveCategory.
+ *
+ * @param primitiveCategory
+ */
+ public static AbstractPrimitiveLazyObjectInspector<?> getPrimitiveLazyObjectInspector(
+ PrimitiveCategory primitiveCategory) {
+ AbstractPrimitiveLazyObjectInspector<?> result = cachedPrimitiveLazyInspectorCache.get(primitiveCategory);
+ if (result == null) {
+ throw new RuntimeException("Internal error: Cannot find ObjectInspector " + " for " + primitiveCategory);
+ }
+ return result;
+ }
+
+ private PrimitiveObjectInspectorFactory() {
+ // prevent instantiation
+ }
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/parser/IHiveParser.java b/src/main/java/edu/uci/ics/hivesterix/serde/parser/IHiveParser.java
new file mode 100644
index 0000000..7830c52
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/parser/IHiveParser.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hivesterix.serde.parser;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public interface IHiveParser {
+ /**
+ * parse one hive rwo into
+ *
+ * @param row
+ * @param objectInspector
+ * @param tb
+ */
+ public void parse(byte[] data, int start, int length, ArrayTupleBuilder tb) throws IOException;
+}
diff --git a/src/main/java/edu/uci/ics/hivesterix/serde/parser/TextToBinaryTupleParser.java b/src/main/java/edu/uci/ics/hivesterix/serde/parser/TextToBinaryTupleParser.java
new file mode 100644
index 0000000..38e1b36
--- /dev/null
+++ b/src/main/java/edu/uci/ics/hivesterix/serde/parser/TextToBinaryTupleParser.java
@@ -0,0 +1,174 @@
+package edu.uci.ics.hivesterix.serde.parser;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
+import org.apache.hadoop.hive.serde2.lazy.LazyLong;
+import org.apache.hadoop.hive.serde2.lazy.LazyShort;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hivesterix.serde.lazy.LazyUtils;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class TextToBinaryTupleParser implements IHiveParser {
+ private int[] invertedIndex;
+ private int[] fieldEnds;
+ private int lastNecessaryFieldIndex;
+ private LazySimpleStructObjectInspector inputObjectInspector;
+ private List<? extends StructField> fieldRefs;
+
+ public TextToBinaryTupleParser(int[] outputColumnsOffset, ObjectInspector structInspector) {
+ int size = 0;
+ for (int i = 0; i < outputColumnsOffset.length; i++)
+ if (outputColumnsOffset[i] >= 0)
+ size++;
+ invertedIndex = new int[size];
+ for (int i = 0; i < outputColumnsOffset.length; i++)
+ if (outputColumnsOffset[i] >= 0) {
+ invertedIndex[outputColumnsOffset[i]] = i;
+ lastNecessaryFieldIndex = i;
+ }
+ fieldEnds = new int[outputColumnsOffset.length];
+ for (int i = 0; i < fieldEnds.length; i++)
+ fieldEnds[i] = 0;
+ inputObjectInspector = (LazySimpleStructObjectInspector) structInspector;
+ fieldRefs = inputObjectInspector.getAllStructFieldRefs();
+ }
+
+ @Override
+ public void parse(byte[] bytes, int start, int length, ArrayTupleBuilder tb) throws IOException {
+ byte separator = inputObjectInspector.getSeparator();
+ boolean lastColumnTakesRest = inputObjectInspector.getLastColumnTakesRest();
+ boolean isEscaped = inputObjectInspector.isEscaped();
+ byte escapeChar = inputObjectInspector.getEscapeChar();
+ DataOutput output = tb.getDataOutput();
+
+ int structByteEnd = start + length - 1;
+ int fieldId = 0;
+ int fieldByteEnd = start;
+
+ // Go through all bytes in the byte[]
+ while (fieldByteEnd <= structByteEnd && fieldId <= lastNecessaryFieldIndex) {
+ if (fieldByteEnd == structByteEnd || bytes[fieldByteEnd] == separator) {
+ // Reached the end of a field?
+ if (lastColumnTakesRest && fieldId == fieldEnds.length - 1) {
+ fieldByteEnd = structByteEnd;
+ }
+ fieldEnds[fieldId] = fieldByteEnd;
+ if (fieldId == fieldEnds.length - 1 || fieldByteEnd == structByteEnd) {
+ // for the case of null fields
+ for (int i = fieldId; i < fieldEnds.length; i++) {
+ fieldEnds[i] = fieldByteEnd;
+ }
+ break;
+ }
+ fieldByteEnd++;
+ fieldId++;
+ } else {
+ if (isEscaped && bytes[fieldByteEnd] == escapeChar && fieldByteEnd + 1 < structByteEnd) {
+ // ignore the char after escape_char
+ fieldByteEnd += 2;
+ } else {
+ fieldByteEnd++;
+ }
+ }
+ }
+
+ for (int i = 0; i < invertedIndex.length; i++) {
+ int index = invertedIndex[i];
+ StructField fieldRef = fieldRefs.get(index);
+ ObjectInspector inspector = fieldRef.getFieldObjectInspector();
+ Category category = inspector.getCategory();
+ int fieldStart = index == 0 ? 0 : fieldEnds[index - 1] + 1;
+ int fieldEnd = fieldEnds[index];
+ if (bytes[fieldEnd] == separator)
+ fieldEnd--;
+ int fieldLen = fieldEnd - fieldStart + 1;
+ switch (category) {
+ case PRIMITIVE:
+ PrimitiveObjectInspector poi = (PrimitiveObjectInspector) inspector;
+ switch (poi.getPrimitiveCategory()) {
+ case VOID: {
+ break;
+ }
+ case BOOLEAN: {
+ output.write(bytes[fieldStart]);
+ break;
+ }
+ case BYTE: {
+ output.write(bytes[fieldStart]);
+ break;
+ }
+ case SHORT: {
+ short v = LazyShort.parseShort(bytes, fieldStart, fieldLen);
+ output.write((byte) (v >> 8));
+ output.write((byte) (v));
+ break;
+ }
+ case INT: {
+ int v = LazyInteger.parseInt(bytes, fieldStart, fieldLen);
+ LazyUtils.writeVInt(output, v);
+ break;
+ }
+ case LONG: {
+ long v = LazyLong.parseLong(bytes, fieldStart, fieldLen);
+ LazyUtils.writeVLong(output, v);
+ break;
+ }
+ case FLOAT: {
+ float value = Float.parseFloat(Text.decode(bytes, fieldStart, fieldLen));
+ int v = Float.floatToIntBits(value);
+ output.write((byte) (v >> 24));
+ output.write((byte) (v >> 16));
+ output.write((byte) (v >> 8));
+ output.write((byte) (v));
+ break;
+ }
+ case DOUBLE: {
+ try {
+ double value = Double.parseDouble(Text.decode(bytes, fieldStart, fieldLen));
+ long v = Double.doubleToLongBits(value);
+ output.write((byte) (v >> 56));
+ output.write((byte) (v >> 48));
+ output.write((byte) (v >> 40));
+ output.write((byte) (v >> 32));
+ output.write((byte) (v >> 24));
+ output.write((byte) (v >> 16));
+ output.write((byte) (v >> 8));
+ output.write((byte) (v));
+ } catch (NumberFormatException e) {
+ throw e;
+ }
+ break;
+ }
+ case STRING: {
+ LazyUtils.writeVInt(output, fieldLen);
+ output.write(bytes, fieldStart, fieldLen);
+ break;
+ }
+ default: {
+ throw new RuntimeException("Unrecognized type: " + poi.getPrimitiveCategory());
+ }
+ }
+ break;
+ case STRUCT:
+ throw new NotImplementedException("Unrecognized type: struct ");
+ case LIST:
+ throw new NotImplementedException("Unrecognized type: struct ");
+ case MAP:
+ throw new NotImplementedException("Unrecognized type: struct ");
+ case UNION:
+ throw new NotImplementedException("Unrecognized type: struct ");
+ }
+ tb.addFieldEndOffset();
+ }
+ }
+}
diff --git a/src/main/java/org/apache/hadoop/hive/ql/Driver.java b/src/main/java/org/apache/hadoop/hive/ql/Driver.java
new file mode 100644
index 0000000..de21cf9
--- /dev/null
+++ b/src/main/java/org/apache/hadoop/hive/ql/Driver.java
@@ -0,0 +1,1311 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql;
+
+import java.io.DataInput;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.exec.ConditionalTask;
+import org.apache.hadoop.hive.ql.exec.ExecDriver;
+import org.apache.hadoop.hive.ql.exec.FetchTask;
+import org.apache.hadoop.hive.ql.exec.MapRedTask;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.StatsTask;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.exec.TaskResult;
+import org.apache.hadoop.hive.ql.exec.TaskRunner;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.history.HiveHistory.Keys;
+import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
+import org.apache.hadoop.hive.ql.hooks.Hook;
+import org.apache.hadoop.hive.ql.hooks.HookContext;
+import org.apache.hadoop.hive.ql.hooks.PostExecute;
+import org.apache.hadoop.hive.ql.hooks.PreExecute;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLock;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockManagerCtx;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData;
+import org.apache.hadoop.hive.ql.lockmgr.LockException;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.metadata.DummyPartition;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.ErrorMsg;
+import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext;
+import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContextImpl;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseException;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
+import org.apache.hadoop.hive.ql.plan.ConditionalResolver;
+import org.apache.hadoop.hive.ql.plan.ConditionalResolverMergeFiles;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.ql.processors.CommandProcessor;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
+import org.apache.hadoop.hive.serde2.ByteStream;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hivesterix.runtime.exec.HyracksExecutionEngine;
+import edu.uci.ics.hivesterix.runtime.exec.IExecutionEngine;
+
+public class Driver implements CommandProcessor {
+
+ static final private Log LOG = LogFactory.getLog(Driver.class.getName());
+ static final private LogHelper console = new LogHelper(LOG);
+
+ // hive-sterix
+ private IExecutionEngine engine;
+ private boolean hivesterix = false;
+ private PrintWriter planPrinter;
+
+ private int maxRows = 100;
+ ByteStream.Output bos = new ByteStream.Output();
+
+ private HiveConf conf;
+ private DataInput resStream;
+ private Context ctx;
+ private QueryPlan plan;
+ private Schema schema;
+ private HiveLockManager hiveLockMgr;
+
+ private String errorMessage;
+ private String SQLState;
+
+ // A limit on the number of threads that can be launched
+ private int maxthreads;
+ private final int sleeptime = 2000;
+
+ protected int tryCount = Integer.MAX_VALUE;
+
+ private int checkLockManager() {
+ boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
+ if (supportConcurrency && (hiveLockMgr == null)) {
+ try {
+ setLockManager();
+ } catch (SemanticException e) {
+ errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
+ SQLState = ErrorMsg.findSQLState(e.getMessage());
+ console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return (12);
+ }
+ }
+ return (0);
+ }
+
+ private void setLockManager() throws SemanticException {
+ boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
+ if (supportConcurrency) {
+ String lockMgr = conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER);
+ if ((lockMgr == null) || (lockMgr.isEmpty())) {
+ throw new SemanticException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg());
+ }
+
+ try {
+ hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(conf.getClassByName(lockMgr), conf);
+ hiveLockMgr.setContext(new HiveLockManagerCtx(conf));
+ } catch (Exception e) {
+ throw new SemanticException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + e.getMessage());
+ }
+ }
+ }
+
+ public void init() {
+ Operator.resetId();
+ }
+
+ /**
+ * Return the status information about the Map-Reduce cluster
+ */
+ public ClusterStatus getClusterStatus() throws Exception {
+ ClusterStatus cs;
+ try {
+ JobConf job = new JobConf(conf, ExecDriver.class);
+ JobClient jc = new JobClient(job);
+ cs = jc.getClusterStatus();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
+ }
+ LOG.info("Returning cluster status: " + cs.toString());
+ return cs;
+ }
+
+ public Schema getSchema() {
+ return schema;
+ }
+
+ /**
+ * Get a Schema with fields represented with native Hive types
+ */
+ public static Schema getSchema(BaseSemanticAnalyzer sem, HiveConf conf) {
+ Schema schema = null;
+
+ // If we have a plan, prefer its logical result schema if it's
+ // available; otherwise, try digging out a fetch task; failing that,
+ // give up.
+ if (sem == null) {
+ // can't get any info without a plan
+ } else if (sem.getResultSchema() != null) {
+ List<FieldSchema> lst = sem.getResultSchema();
+ schema = new Schema(lst, null);
+ } else if (sem.getFetchTask() != null) {
+ FetchTask ft = sem.getFetchTask();
+ TableDesc td = ft.getTblDesc();
+ // partitioned tables don't have tableDesc set on the FetchTask.
+ // Instead
+ // they have a list of PartitionDesc objects, each with a table
+ // desc.
+ // Let's
+ // try to fetch the desc for the first partition and use it's
+ // deserializer.
+ if (td == null && ft.getWork() != null && ft.getWork().getPartDesc() != null) {
+ if (ft.getWork().getPartDesc().size() > 0) {
+ td = ft.getWork().getPartDesc().get(0).getTableDesc();
+ }
+ }
+
+ if (td == null) {
+ LOG.info("No returning schema.");
+ } else {
+ String tableName = "result";
+ List<FieldSchema> lst = null;
+ try {
+ lst = MetaStoreUtils.getFieldsFromDeserializer(tableName, td.getDeserializer());
+ } catch (Exception e) {
+ LOG.warn("Error getting schema: " + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ }
+ if (lst != null) {
+ schema = new Schema(lst, null);
+ }
+ }
+ }
+ if (schema == null) {
+ schema = new Schema();
+ }
+ LOG.info("Returning Hive schema: " + schema);
+ return schema;
+ }
+
+ /**
+ * Get a Schema with fields represented with Thrift DDL types
+ */
+ public Schema getThriftSchema() throws Exception {
+ Schema schema;
+ try {
+ schema = getSchema();
+ if (schema != null) {
+ List<FieldSchema> lst = schema.getFieldSchemas();
+ // Go over the schema and convert type to thrift type
+ if (lst != null) {
+ for (FieldSchema f : lst) {
+ f.setType(MetaStoreUtils.typeToThriftType(f.getType()));
+ }
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
+ }
+ LOG.info("Returning Thrift schema: " + schema);
+ return schema;
+ }
+
+ /**
+ * Return the maximum number of rows returned by getResults
+ */
+ public int getMaxRows() {
+ return maxRows;
+ }
+
+ /**
+ * Set the maximum number of rows returned by getResults
+ */
+ public void setMaxRows(int maxRows) {
+ this.maxRows = maxRows;
+ }
+
+ public boolean hasReduceTasks(List<Task<? extends Serializable>> tasks) {
+ if (tasks == null) {
+ return false;
+ }
+
+ boolean hasReduce = false;
+ for (Task<? extends Serializable> task : tasks) {
+ if (task.hasReduce()) {
+ return true;
+ }
+
+ hasReduce = (hasReduce || hasReduceTasks(task.getChildTasks()));
+ }
+ return hasReduce;
+ }
+
+ /**
+ * for backwards compatibility with current tests
+ */
+ public Driver(HiveConf conf) {
+ this.conf = conf;
+
+ // hivesterix
+ engine = new HyracksExecutionEngine(conf);
+ }
+
+ public Driver() {
+ if (SessionState.get() != null) {
+ conf = SessionState.get().getConf();
+ }
+
+ // hivesterix
+ engine = new HyracksExecutionEngine(conf);
+ }
+
+ // hivesterix: plan printer
+ public Driver(HiveConf conf, PrintWriter planPrinter) {
+ this.conf = conf;
+ engine = new HyracksExecutionEngine(conf, planPrinter);
+ this.planPrinter = planPrinter;
+ }
+
+ public void clear() {
+ this.hivesterix = false;
+ }
+
+ /**
+ * Compile a new query. Any currently-planned query associated with this
+ * Driver is discarded.
+ *
+ * @param command
+ * The SQL query to compile.
+ */
+ public int compile(String command) {
+ if (plan != null) {
+ close();
+ plan = null;
+ }
+
+ TaskFactory.resetId();
+
+ try {
+ command = new VariableSubstitution().substitute(conf, command);
+ ctx = new Context(conf);
+
+ ParseDriver pd = new ParseDriver();
+ ASTNode tree = pd.parse(command, ctx);
+ tree = ParseUtils.findRootNonNullToken(tree);
+
+ BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
+ List<AbstractSemanticAnalyzerHook> saHooks = getSemanticAnalyzerHooks();
+
+ // Do semantic analysis and plan generation
+ if (saHooks != null) {
+ HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
+ hookCtx.setConf(conf);
+ for (AbstractSemanticAnalyzerHook hook : saHooks) {
+ tree = hook.preAnalyze(hookCtx, tree);
+ }
+ sem.analyze(tree, ctx);
+ for (AbstractSemanticAnalyzerHook hook : saHooks) {
+ hook.postAnalyze(hookCtx, sem.getRootTasks());
+ }
+ } else {
+ sem.analyze(tree, ctx);
+ }
+
+ LOG.info("Semantic Analysis Completed");
+
+ // validate the plan
+ sem.validate();
+
+ plan = new QueryPlan(command, sem);
+ // initialize FetchTask right here
+ if (plan.getFetchTask() != null) {
+ plan.getFetchTask().initialize(conf, plan, null);
+ }
+
+ // get the output schema
+ schema = getSchema(sem, conf);
+
+ // test Only - serialize the query plan and deserialize it
+ if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
+
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+
+ String queryPlanFileName = ctx.getLocalScratchDir(true) + Path.SEPARATOR_CHAR + "queryplan.xml";
+ LOG.info("query plan = " + queryPlanFileName);
+ queryPlanFileName = new Path(queryPlanFileName).toUri().getPath();
+
+ // serialize the queryPlan
+ FileOutputStream fos = new FileOutputStream(queryPlanFileName);
+ Utilities.serializeQueryPlan(plan, fos);
+ fos.close();
+
+ // deserialize the queryPlan
+ FileInputStream fis = new FileInputStream(queryPlanFileName);
+ QueryPlan newPlan = Utilities.deserializeQueryPlan(fis, conf);
+ fis.close();
+
+ // Use the deserialized plan
+ plan = newPlan;
+ }
+
+ // initialize FetchTask right here
+ if (plan.getFetchTask() != null) {
+ plan.getFetchTask().initialize(conf, plan, null);
+ }
+
+ // do the authorization check
+ if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
+ try {
+ // doAuthorization(sem);
+ } catch (AuthorizationException authExp) {
+ console.printError("Authorization failed:" + authExp.getMessage()
+ + ". Use show grant to get more details.");
+ return 403;
+ }
+ }
+
+ // hyracks run
+ if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
+ hivesterix = true;
+ return engine.compileJob(sem.getRootTasks());
+ }
+
+ return 0;
+ } catch (SemanticException e) {
+ errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
+ SQLState = ErrorMsg.findSQLState(e.getMessage());
+ console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return (10);
+ } catch (ParseException e) {
+ errorMessage = "FAILED: Parse Error: " + e.getMessage();
+ SQLState = ErrorMsg.findSQLState(e.getMessage());
+ console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return (11);
+ } catch (Exception e) {
+ errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
+ SQLState = ErrorMsg.findSQLState(e.getMessage());
+ console.printError(errorMessage + "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return (12);
+ }
+ }
+
+ private void doAuthorization(BaseSemanticAnalyzer sem) throws HiveException, AuthorizationException {
+ HashSet<ReadEntity> inputs = sem.getInputs();
+ HashSet<WriteEntity> outputs = sem.getOutputs();
+ SessionState ss = SessionState.get();
+ HiveOperation op = ss.getHiveOperation();
+ Hive db = sem.getDb();
+ if (op != null) {
+ if (op.equals(HiveOperation.CREATETABLE_AS_SELECT) || op.equals(HiveOperation.CREATETABLE)) {
+ ss.getAuthorizer().authorize(db.getDatabase(db.getCurrentDatabase()), null,
+ HiveOperation.CREATETABLE_AS_SELECT.getOutputRequiredPrivileges());
+ } else {
+ // if (op.equals(HiveOperation.IMPORT)) {
+ // ImportSemanticAnalyzer isa = (ImportSemanticAnalyzer) sem;
+ // if (!isa.existsTable()) {
+ ss.getAuthorizer().authorize(db.getDatabase(db.getCurrentDatabase()), null,
+ HiveOperation.CREATETABLE_AS_SELECT.getOutputRequiredPrivileges());
+ // }
+ // }
+ }
+ if (outputs != null && outputs.size() > 0) {
+ for (WriteEntity write : outputs) {
+
+ if (write.getType() == WriteEntity.Type.PARTITION) {
+ Partition part = db.getPartition(write.getTable(), write.getPartition().getSpec(), false);
+ if (part != null) {
+ ss.getAuthorizer().authorize(write.getPartition(), null, op.getOutputRequiredPrivileges());
+ continue;
+ }
+ }
+
+ if (write.getTable() != null) {
+ ss.getAuthorizer().authorize(write.getTable(), null, op.getOutputRequiredPrivileges());
+ }
+ }
+
+ }
+ }
+
+ if (inputs != null && inputs.size() > 0) {
+
+ Map<Table, List<String>> tab2Cols = new HashMap<Table, List<String>>();
+ Map<Partition, List<String>> part2Cols = new HashMap<Partition, List<String>>();
+
+ Map<String, Boolean> tableUsePartLevelAuth = new HashMap<String, Boolean>();
+ for (ReadEntity read : inputs) {
+ if (read.getPartition() != null) {
+ Table tbl = read.getTable();
+ String tblName = tbl.getTableName();
+ if (tableUsePartLevelAuth.get(tblName) == null) {
+ boolean usePartLevelPriv = (tbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE") != null && ("TRUE"
+ .equalsIgnoreCase(tbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))));
+ if (usePartLevelPriv) {
+ tableUsePartLevelAuth.put(tblName, Boolean.TRUE);
+ } else {
+ tableUsePartLevelAuth.put(tblName, Boolean.FALSE);
+ }
+ }
+ }
+ }
+
+ if (op.equals(HiveOperation.CREATETABLE_AS_SELECT) || op.equals(HiveOperation.QUERY)) {
+ SemanticAnalyzer querySem = (SemanticAnalyzer) sem;
+ ParseContext parseCtx = querySem.getParseContext();
+ Map<TableScanOperator, Table> tsoTopMap = parseCtx.getTopToTable();
+
+ for (Map.Entry<String, Operator<? extends Serializable>> topOpMap : querySem.getParseContext()
+ .getTopOps().entrySet()) {
+ Operator<? extends Serializable> topOp = topOpMap.getValue();
+ if (topOp instanceof TableScanOperator && tsoTopMap.containsKey(topOp)) {
+ TableScanOperator tableScanOp = (TableScanOperator) topOp;
+ Table tbl = tsoTopMap.get(tableScanOp);
+ List<Integer> neededColumnIds = tableScanOp.getNeededColumnIDs();
+ List<FieldSchema> columns = tbl.getCols();
+ List<String> cols = new ArrayList<String>();
+ if (neededColumnIds != null && neededColumnIds.size() > 0) {
+ for (int i = 0; i < neededColumnIds.size(); i++) {
+ cols.add(columns.get(neededColumnIds.get(i)).getName());
+ }
+ } else {
+ for (int i = 0; i < columns.size(); i++) {
+ cols.add(columns.get(i).getName());
+ }
+ }
+ if (tbl.isPartitioned() && tableUsePartLevelAuth.get(tbl.getTableName())) {
+ String alias_id = topOpMap.getKey();
+ PrunedPartitionList partsList = PartitionPruner.prune(parseCtx.getTopToTable().get(topOp),
+ parseCtx.getOpToPartPruner().get(topOp), parseCtx.getConf(), alias_id,
+ parseCtx.getPrunedPartitions());
+ Set<Partition> parts = new HashSet<Partition>();
+ parts.addAll(partsList.getConfirmedPartns());
+ parts.addAll(partsList.getUnknownPartns());
+ for (Partition part : parts) {
+ List<String> existingCols = part2Cols.get(part);
+ if (existingCols == null) {
+ existingCols = new ArrayList<String>();
+ }
+ existingCols.addAll(cols);
+ part2Cols.put(part, existingCols);
+ }
+ } else {
+ List<String> existingCols = tab2Cols.get(tbl);
+ if (existingCols == null) {
+ existingCols = new ArrayList<String>();
+ }
+ existingCols.addAll(cols);
+ tab2Cols.put(tbl, existingCols);
+ }
+ }
+ }
+ }
+
+ // cache the results for table authorization
+ Set<String> tableAuthChecked = new HashSet<String>();
+ for (ReadEntity read : inputs) {
+ Table tbl = null;
+ if (read.getPartition() != null) {
+ tbl = read.getPartition().getTable();
+ // use partition level authorization
+ if (tableUsePartLevelAuth.get(tbl.getTableName())) {
+ List<String> cols = part2Cols.get(read.getPartition());
+ if (cols != null && cols.size() > 0) {
+ ss.getAuthorizer().authorize(read.getPartition().getTable(), read.getPartition(), cols,
+ op.getInputRequiredPrivileges(), null);
+ } else {
+ ss.getAuthorizer().authorize(read.getPartition(), op.getInputRequiredPrivileges(), null);
+ }
+ continue;
+ }
+ } else if (read.getTable() != null) {
+ tbl = read.getTable();
+ }
+
+ // if we reach here, it means it needs to do a table
+ // authorization
+ // check, and the table authorization may already happened
+ // because of other
+ // partitions
+ if (tbl != null && !tableAuthChecked.contains(tbl.getTableName())) {
+ List<String> cols = tab2Cols.get(tbl);
+ if (cols != null && cols.size() > 0) {
+ ss.getAuthorizer().authorize(tbl, null, cols, op.getInputRequiredPrivileges(), null);
+ } else {
+ ss.getAuthorizer().authorize(tbl, op.getInputRequiredPrivileges(), null);
+ }
+ tableAuthChecked.add(tbl.getTableName());
+ }
+ }
+
+ }
+ }
+
+ /**
+ * @return The current query plan associated with this Driver, if any.
+ */
+ public QueryPlan getPlan() {
+ return plan;
+ }
+
+ /**
+ * @param t
+ * The table to be locked
+ * @param p
+ * The partition to be locked
+ * @param mode
+ * The mode of the lock (SHARED/EXCLUSIVE) Get the list of
+ * objects to be locked. If a partition needs to be locked (in
+ * any mode), all its parents should also be locked in SHARED
+ * mode.
+ **/
+ private List<HiveLockObj> getLockObjects(Table t, Partition p, HiveLockMode mode) throws SemanticException {
+ List<HiveLockObj> locks = new LinkedList<HiveLockObj>();
+
+ HiveLockObjectData lockData = new HiveLockObjectData(plan.getQueryId(), String.valueOf(System
+ .currentTimeMillis()), "IMPLICIT");
+
+ if (t != null) {
+ locks.add(new HiveLockObj(new HiveLockObject(t, lockData), mode));
+ mode = HiveLockMode.SHARED;
+ locks.add(new HiveLockObj(new HiveLockObject(t.getDbName(), lockData), mode));
+ return locks;
+ }
+
+ if (p != null) {
+ if (!(p instanceof DummyPartition)) {
+ locks.add(new HiveLockObj(new HiveLockObject(p, lockData), mode));
+ }
+
+ // All the parents are locked in shared mode
+ mode = HiveLockMode.SHARED;
+
+ // For dummy partitions, only partition name is needed
+ String name = p.getName();
+
+ if (p instanceof DummyPartition) {
+ name = p.getName().split("@")[2];
+ }
+
+ String partName = name;
+ String partialName = "";
+ String[] partns = name.split("/");
+ int len = p instanceof DummyPartition ? partns.length : partns.length - 1;
+ for (int idx = 0; idx < len; idx++) {
+ String partn = partns[idx];
+ partialName += partn;
+ try {
+ locks.add(new HiveLockObj(new HiveLockObject(new DummyPartition(p.getTable(), p.getTable()
+ .getDbName() + "/" + p.getTable().getTableName() + "/" + partialName), lockData), mode));
+ partialName += "/";
+ } catch (HiveException e) {
+ throw new SemanticException(e.getMessage());
+ }
+ }
+
+ locks.add(new HiveLockObj(new HiveLockObject(p.getTable(), lockData), mode));
+ locks.add(new HiveLockObj(new HiveLockObject(p.getTable().getDbName(), lockData), mode));
+ }
+ return locks;
+ }
+
+ /**
+ * Acquire read and write locks needed by the statement. The list of objects
+ * to be locked are obtained from he inputs and outputs populated by the
+ * compiler. The lock acuisition scheme is pretty simple. If all the locks
+ * cannot be obtained, error out. Deadlock is avoided by making sure that
+ * the locks are lexicographically sorted.
+ **/
+ public int acquireReadWriteLocks() {
+ try {
+ int sleepTime = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES) * 1000;
+ int numRetries = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_NUMRETRIES);
+
+ boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
+ if (!supportConcurrency) {
+ return 0;
+ }
+
+ List<HiveLockObj> lockObjects = new ArrayList<HiveLockObj>();
+
+ // Sort all the inputs, outputs.
+ // If a lock needs to be acquired on any partition, a read lock
+ // needs to be acquired on all
+ // its parents also
+ for (ReadEntity input : plan.getInputs()) {
+ if (input.getType() == ReadEntity.Type.TABLE) {
+ lockObjects.addAll(getLockObjects(input.getTable(), null, HiveLockMode.SHARED));
+ } else {
+ lockObjects.addAll(getLockObjects(null, input.getPartition(), HiveLockMode.SHARED));
+ }
+ }
+
+ for (WriteEntity output : plan.getOutputs()) {
+ if (output.getTyp() == WriteEntity.Type.TABLE) {
+ lockObjects.addAll(getLockObjects(output.getTable(), null,
+ output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED));
+ } else if (output.getTyp() == WriteEntity.Type.PARTITION) {
+ lockObjects.addAll(getLockObjects(null, output.getPartition(), HiveLockMode.EXCLUSIVE));
+ }
+ // In case of dynamic queries, it is possible to have incomplete
+ // dummy partitions
+ else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) {
+ lockObjects.addAll(getLockObjects(null, output.getPartition(), HiveLockMode.SHARED));
+ }
+ }
+
+ if (lockObjects.isEmpty() && !ctx.isNeedLockMgr()) {
+ return 0;
+ }
+
+ int ret = checkLockManager();
+ if (ret != 0) {
+ return ret;
+ }
+
+ HiveLockObjectData lockData = new HiveLockObjectData(plan.getQueryId(), String.valueOf(System
+ .currentTimeMillis()), "IMPLICIT");
+
+ // Lock the database also
+ try {
+ Hive db = Hive.get(conf);
+ lockObjects.add(new HiveLockObj(new HiveLockObject(db.getCurrentDatabase(), lockData),
+ HiveLockMode.SHARED));
+ } catch (HiveException e) {
+ throw new SemanticException(e.getMessage());
+ }
+
+ ctx.setHiveLockMgr(hiveLockMgr);
+ List<HiveLock> hiveLocks = null;
+
+ int tryNum = 1;
+ do {
+
+ // ctx.getHiveLockMgr();
+ // hiveLocks = ctx.getHiveLockMgr().lock(lockObjects, false);
+
+ if (hiveLocks != null) {
+ break;
+ }
+
+ tryNum++;
+ try {
+ Thread.sleep(sleepTime);
+ } catch (InterruptedException e) {
+ }
+ } while (tryNum < numRetries);
+
+ if (hiveLocks == null) {
+ throw new SemanticException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg());
+ } else {
+ ctx.setHiveLocks(hiveLocks);
+ }
+
+ return (0);
+ } catch (SemanticException e) {
+ errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage();
+ SQLState = ErrorMsg.findSQLState(e.getMessage());
+ console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return (10);
+ } catch (Exception e) {
+ errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage();
+ SQLState = ErrorMsg.findSQLState(e.getMessage());
+ console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return (10);
+ }
+ }
+
+ /**
+ * Release all the locks acquired implicitly by the statement. Note that the
+ * locks acquired with 'keepAlive' set to True are not released.
+ **/
+ private void releaseLocks() {
+ if (ctx != null && ctx.getHiveLockMgr() != null) {
+ try {
+ ctx.getHiveLockMgr().close();
+ ctx.setHiveLocks(null);
+ } catch (LockException e) {
+ }
+ }
+ }
+
+ /**
+ * @param hiveLocks
+ * list of hive locks to be released Release all the locks
+ * specified. If some of the locks have already been released,
+ * ignore them
+ **/
+ private void releaseLocks(List<HiveLock> hiveLocks) {
+ if (hiveLocks != null) {
+ ctx.getHiveLockMgr().releaseLocks(hiveLocks);
+ }
+ ctx.setHiveLocks(null);
+ }
+
+ public CommandProcessorResponse run(String command) {
+ errorMessage = null;
+ SQLState = null;
+
+ int ret = compile(command);
+ if (ret != 0) {
+ // releaseLocks(ctx.getHiveLocks());
+ return new CommandProcessorResponse(ret, errorMessage, SQLState);
+ }
+
+ // ret = acquireReadWriteLocks();
+ if (ret != 0) {
+ // releaseLocks(ctx.getHiveLocks());
+ return new CommandProcessorResponse(ret, errorMessage, SQLState);
+ }
+
+ ret = execute();
+ if (ret != 0) {
+ // releaseLocks(ctx.getHiveLocks());
+ return new CommandProcessorResponse(ret, errorMessage, SQLState);
+ }
+
+ // releaseLocks(ctx.getHiveLocks());
+ return new CommandProcessorResponse(ret);
+ }
+
+ private List<AbstractSemanticAnalyzerHook> getSemanticAnalyzerHooks() throws Exception {
+ ArrayList<AbstractSemanticAnalyzerHook> saHooks = new ArrayList<AbstractSemanticAnalyzerHook>();
+ String pestr = conf.getVar(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK);
+ if (pestr == null) {
+ return saHooks;
+ }
+ pestr = pestr.trim();
+ if (pestr.equals("")) {
+ return saHooks;
+ }
+
+ String[] peClasses = pestr.split(",");
+
+ for (String peClass : peClasses) {
+ try {
+ AbstractSemanticAnalyzerHook hook = HiveUtils.getSemanticAnalyzerHook(conf, peClass);
+ saHooks.add(hook);
+ } catch (HiveException e) {
+ console.printError("Pre Exec Hook Class not found:" + e.getMessage());
+ throw e;
+ }
+ }
+
+ return saHooks;
+ }
+
+ private List<Hook> getPreExecHooks() throws Exception {
+ ArrayList<Hook> pehooks = new ArrayList<Hook>();
+ String pestr = conf.getVar(HiveConf.ConfVars.PREEXECHOOKS);
+ pestr = pestr.trim();
+ if (pestr.equals("")) {
+ return pehooks;
+ }
+
+ String[] peClasses = pestr.split(",");
+
+ for (String peClass : peClasses) {
+ try {
+ pehooks.add((Hook) Class.forName(peClass.trim(), true, JavaUtils.getClassLoader()).newInstance());
+ } catch (ClassNotFoundException e) {
+ console.printError("Pre Exec Hook Class not found:" + e.getMessage());
+ throw e;
+ }
+ }
+
+ return pehooks;
+ }
+
+ private List<Hook> getPostExecHooks() throws Exception {
+ ArrayList<Hook> pehooks = new ArrayList<Hook>();
+ String pestr = conf.getVar(HiveConf.ConfVars.POSTEXECHOOKS);
+ pestr = pestr.trim();
+ if (pestr.equals("")) {
+ return pehooks;
+ }
+
+ String[] peClasses = pestr.split(",");
+
+ for (String peClass : peClasses) {
+ try {
+ pehooks.add((Hook) Class.forName(peClass.trim(), true, JavaUtils.getClassLoader()).newInstance());
+ } catch (ClassNotFoundException e) {
+ console.printError("Post Exec Hook Class not found:" + e.getMessage());
+ throw e;
+ }
+ }
+
+ return pehooks;
+ }
+
+ public int execute() {
+ // execute hivesterix plan
+ if (hivesterix) {
+ hivesterix = false;
+ int ret = engine.executeJob();
+ if (ret != 0)
+ return ret;
+ }
+
+ boolean noName = StringUtils.isEmpty(conf.getVar(HiveConf.ConfVars.HADOOPJOBNAME));
+ int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH);
+
+ String queryId = plan.getQueryId();
+ String queryStr = plan.getQueryStr();
+
+ conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
+ conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, queryStr);
+ maxthreads = HiveConf.getIntVar(conf, HiveConf.ConfVars.EXECPARALLETHREADNUMBER);
+
+ try {
+ LOG.info("Starting command: " + queryStr);
+
+ plan.setStarted();
+
+ if (SessionState.get() != null) {
+ SessionState.get().getHiveHistory().startQuery(queryStr, conf.getVar(HiveConf.ConfVars.HIVEQUERYID));
+ SessionState.get().getHiveHistory().logPlanProgress(plan);
+ }
+ resStream = null;
+
+ HookContext hookContext = new HookContext(plan, conf);
+
+ for (Hook peh : getPreExecHooks()) {
+ if (peh instanceof ExecuteWithHookContext) {
+ ((ExecuteWithHookContext) peh).run(hookContext);
+ } else if (peh instanceof PreExecute) {
+ ((PreExecute) peh).run(SessionState.get(), plan.getInputs(), plan.getOutputs(), ShimLoader
+ .getHadoopShims().getUGIForConf(conf));
+ }
+ }
+
+ int jobs = Utilities.getMRTasks(plan.getRootTasks()).size();
+ if (jobs > 0) {
+ console.printInfo("Total MapReduce jobs = " + jobs);
+ }
+ if (SessionState.get() != null) {
+ SessionState.get().getHiveHistory()
+ .setQueryProperty(queryId, Keys.QUERY_NUM_TASKS, String.valueOf(jobs));
+ SessionState.get().getHiveHistory().setIdToTableMap(plan.getIdToTableNameMap());
+ }
+ String jobname = Utilities.abbreviate(queryStr, maxlen - 6);
+
+ // A runtime that launches runnable tasks as separate Threads
+ // through
+ // TaskRunners
+ // As soon as a task isRunnable, it is put in a queue
+ // At any time, at most maxthreads tasks can be running
+ // The main thread polls the TaskRunners to check if they have
+ // finished.
+
+ Queue<Task<? extends Serializable>> runnable = new LinkedList<Task<? extends Serializable>>();
+ Map<TaskResult, TaskRunner> running = new HashMap<TaskResult, TaskRunner>();
+
+ DriverContext driverCxt = new DriverContext(runnable, ctx);
+
+ // Add root Tasks to runnable
+
+ for (Task<? extends Serializable> tsk : plan.getRootTasks()) {
+ driverCxt.addToRunnable(tsk);
+ }
+
+ // Loop while you either have tasks running, or tasks queued up
+
+ while (running.size() != 0 || runnable.peek() != null) {
+ // Launch upto maxthreads tasks
+ while (runnable.peek() != null && running.size() < maxthreads) {
+ Task<? extends Serializable> tsk = runnable.remove();
+ console.printInfo("executing task " + tsk.getName());
+ launchTask(tsk, queryId, noName, running, jobname, jobs, driverCxt);
+ }
+
+ // poll the Tasks to see which one completed
+ TaskResult tskRes = pollTasks(running.keySet());
+ TaskRunner tskRun = running.remove(tskRes);
+ Task<? extends Serializable> tsk = tskRun.getTask();
+ hookContext.addCompleteTask(tskRun);
+
+ int exitVal = tskRes.getExitVal();
+ if (exitVal != 0) {
+ Task<? extends Serializable> backupTask = tsk.getAndInitBackupTask();
+ if (backupTask != null) {
+ errorMessage = "FAILED: Execution Error, return code " + exitVal + " from "
+ + tsk.getClass().getName();
+ console.printError(errorMessage);
+
+ errorMessage = "ATTEMPT: Execute BackupTask: " + backupTask.getClass().getName();
+ console.printError(errorMessage);
+
+ // add backup task to runnable
+ if (DriverContext.isLaunchable(backupTask)) {
+ driverCxt.addToRunnable(backupTask);
+ }
+ continue;
+
+ } else {
+ // TODO: This error messaging is not very informative.
+ // Fix that.
+ errorMessage = "FAILED: Execution Error, return code " + exitVal + " from "
+ + tsk.getClass().getName();
+ SQLState = "08S01";
+ console.printError(errorMessage);
+ if (running.size() != 0) {
+ taskCleanup();
+ }
+ // in case we decided to run everything in local mode,
+ // restore the
+ // the jobtracker setting to its initial value
+ ctx.restoreOriginalTracker();
+ return 9;
+ }
+ }
+
+ if (SessionState.get() != null) {
+ SessionState.get().getHiveHistory()
+ .setTaskProperty(queryId, tsk.getId(), Keys.TASK_RET_CODE, String.valueOf(exitVal));
+ SessionState.get().getHiveHistory().endTask(queryId, tsk);
+ }
+
+ if (tsk.getChildTasks() != null) {
+ for (Task<? extends Serializable> child : tsk.getChildTasks()) {
+ // hivesterix: don't check launchable condition
+ // if (DriverContext.isLaunchable(child)) {
+ driverCxt.addToRunnable(child);
+ // }
+ }
+ }
+ }
+
+ // in case we decided to run everything in local mode, restore the
+ // the jobtracker setting to its initial value
+ ctx.restoreOriginalTracker();
+
+ // remove incomplete outputs.
+ // Some incomplete outputs may be added at the beginning, for eg:
+ // for dynamic partitions.
+ // remove them
+ HashSet<WriteEntity> remOutputs = new HashSet<WriteEntity>();
+ for (WriteEntity output : plan.getOutputs()) {
+ if (!output.isComplete()) {
+ remOutputs.add(output);
+ }
+ }
+
+ for (WriteEntity output : remOutputs) {
+ plan.getOutputs().remove(output);
+ }
+
+ // Get all the post execution hooks and execute them.
+ for (Hook peh : getPostExecHooks()) {
+ if (peh instanceof ExecuteWithHookContext) {
+ ((ExecuteWithHookContext) peh).run(hookContext);
+ } else if (peh instanceof PostExecute) {
+ ((PostExecute) peh)
+ .run(SessionState.get(), plan.getInputs(), plan.getOutputs(),
+ (SessionState.get() != null ? SessionState.get().getLineageState().getLineageInfo()
+ : null), ShimLoader.getHadoopShims().getUGIForConf(conf));
+ }
+ }
+
+ if (SessionState.get() != null) {
+ SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE, String.valueOf(0));
+ SessionState.get().getHiveHistory().printRowCount(queryId);
+ }
+ } catch (Exception e) {
+ if (SessionState.get() != null) {
+ SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE, String.valueOf(12));
+ }
+ // TODO: do better with handling types of Exception here
+ errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
+ SQLState = "08S01";
+ console.printError(errorMessage + "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return (12);
+ } finally {
+ if (SessionState.get() != null) {
+ SessionState.get().getHiveHistory().endQuery(queryId);
+ }
+ if (noName) {
+ conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, "");
+ }
+ }
+ plan.setDone();
+
+ if (SessionState.get() != null) {
+ try {
+ SessionState.get().getHiveHistory().logPlanProgress(plan);
+ } catch (Exception e) {
+ }
+ }
+ console.printInfo("OK");
+
+ return (0);
+ }
+
+ /**
+ * Launches a new task
+ *
+ * @param tsk
+ * task being launched
+ * @param queryId
+ * Id of the query containing the task
+ * @param noName
+ * whether the task has a name set
+ * @param running
+ * map from taskresults to taskrunners
+ * @param jobname
+ * name of the task, if it is a map-reduce job
+ * @param jobs
+ * number of map-reduce jobs
+ * @param curJobNo
+ * the sequential number of the next map-reduce job
+ * @return the updated number of last the map-reduce job launched
+ */
+
+ public void launchTask(Task<? extends Serializable> tsk, String queryId, boolean noName,
+ Map<TaskResult, TaskRunner> running, String jobname, int jobs, DriverContext cxt) {
+
+ if (SessionState.get() != null) {
+ SessionState.get().getHiveHistory().startTask(queryId, tsk, tsk.getClass().getName());
+ }
+ if (tsk.isMapRedTask() && !(tsk instanceof ConditionalTask)) {
+ if (noName) {
+ conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, jobname + "(" + tsk.getId() + ")");
+ }
+ cxt.incCurJobNo(1);
+ console.printInfo("Launching Job " + cxt.getCurJobNo() + " out of " + jobs);
+ }
+ tsk.initialize(conf, plan, cxt);
+ TaskResult tskRes = new TaskResult();
+ TaskRunner tskRun = new TaskRunner(tsk, tskRes);
+
+ // HiveConf.getBoolVar(conf, HiveConf.ConfVars.EXECPARALLEL) &&
+ // Launch Task: hivesterix tweak
+ if (tsk instanceof MapRedTask || tsk instanceof StatsTask) {
+ // Launch it in the parallel mode, as a separate thread only for MR
+ // tasks
+ tskRes.setRunning(false);
+ tskRes.setExitVal(0);
+ } else if (tsk instanceof ConditionalTask) {
+ ConditionalTask condTask = (ConditionalTask) tsk;
+ ConditionalResolver crs = condTask.getResolver();
+ if (crs instanceof ConditionalResolverMergeFiles) {
+ tskRes.setRunning(false);
+ tskRes.setExitVal(0);
+
+ List<Task<? extends Serializable>> children = condTask.getListTasks();
+ for (Task<? extends Serializable> child : children)
+ if (child instanceof MapRedTask)
+ cxt.addToRunnable(child);
+ }
+ } else {
+ tskRun.runSequential();
+ }
+ running.put(tskRes, tskRun);
+ return;
+ }
+
+ /**
+ * Cleans up remaining tasks in case of failure
+ */
+
+ public void taskCleanup() {
+ // The currently existing Shutdown hooks will be automatically called,
+ // killing the map-reduce processes.
+ // The non MR processes will be killed as well.
+ System.exit(9);
+ }
+
+ /**
+ * Polls running tasks to see if a task has ended.
+ *
+ * @param results
+ * Set of result objects for running tasks
+ * @return The result object for any completed/failed task
+ */
+
+ public TaskResult pollTasks(Set<TaskResult> results) {
+ Iterator<TaskResult> resultIterator = results.iterator();
+ while (true) {
+ while (resultIterator.hasNext()) {
+ TaskResult tskRes = resultIterator.next();
+ if (tskRes.isRunning() == false) {
+ return tskRes;
+ }
+ }
+
+ // In this loop, nothing was found
+ // Sleep 10 seconds and restart
+ try {
+ Thread.sleep(sleeptime);
+ } catch (InterruptedException ie) {
+ // Do Nothing
+ ;
+ }
+ resultIterator = results.iterator();
+ }
+ }
+
+ public boolean getResults(ArrayList<String> res) throws IOException {
+ if (plan != null && plan.getFetchTask() != null) {
+ FetchTask ft = plan.getFetchTask();
+ ft.setMaxRows(maxRows);
+ return ft.fetch(res);
+ }
+
+ if (resStream == null) {
+ resStream = ctx.getStream();
+ }
+ if (resStream == null) {
+ return false;
+ }
+
+ int numRows = 0;
+ String row = null;
+
+ while (numRows < maxRows) {
+ if (resStream == null) {
+ if (numRows > 0) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ bos.reset();
+ Utilities.StreamStatus ss;
+ try {
+ ss = Utilities.readColumn(resStream, bos);
+ if (bos.getCount() > 0) {
+ row = new String(bos.getData(), 0, bos.getCount(), "UTF-8");
+ } else if (ss == Utilities.StreamStatus.TERMINATED) {
+ row = new String();
+ }
+
+ if (row != null) {
+ numRows++;
+ res.add(row);
+ }
+ } catch (IOException e) {
+ console.printError("FAILED: Unexpected IO exception : " + e.getMessage());
+ res = null;
+ return false;
+ }
+
+ if (ss == Utilities.StreamStatus.EOF) {
+ resStream = ctx.getStream();
+ }
+ }
+ return true;
+ }
+
+ public int close() {
+ try {
+ if (plan != null) {
+ FetchTask fetchTask = plan.getFetchTask();
+ if (null != fetchTask) {
+ try {
+ fetchTask.clearFetch();
+ } catch (Exception e) {
+ LOG.debug(" Exception while clearing the Fetch task ", e);
+ }
+ }
+ }
+ if (ctx != null) {
+ ctx.clear();
+ }
+ if (null != resStream) {
+ try {
+ ((FSDataInputStream) resStream).close();
+ } catch (Exception e) {
+ LOG.debug(" Exception while closing the resStream ", e);
+ }
+ }
+ } catch (Exception e) {
+ console.printError("FAILED: Hive Internal Error: " + Utilities.getNameMessage(e) + "\n"
+ + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return 13;
+ }
+
+ return 0;
+ }
+
+ public void destroy() {
+ releaseLocks();
+ }
+
+ public org.apache.hadoop.hive.ql.plan.api.Query getQueryPlan() throws IOException {
+ return plan.getQueryPlan();
+ }
+
+ public int getTryCount() {
+ return tryCount;
+ }
+
+ public void setTryCount(int tryCount) {
+ this.tryCount = tryCount;
+ }
+}
diff --git a/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
new file mode 100644
index 0000000..4a05927
--- /dev/null
+++ b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql.udf.generic;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.util.StringUtils;
+
+import edu.uci.ics.hivesterix.runtime.evaluator.BufferSerDeUtil;
+import edu.uci.ics.hivesterix.runtime.evaluator.SerializableBuffer;
+
+/**
+ * GenericUDAFAverage.
+ *
+ */
+@Description(name = "avg", value = "_FUNC_(x) - Returns the mean of a set of numbers")
+public class GenericUDAFAverage extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFAverage.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length != 1) {
+ throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected.");
+ }
+
+ if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ case STRING:
+ return new GenericUDAFAverageEvaluator();
+ case BOOLEAN:
+ default:
+ throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ }
+
+ /**
+ * GenericUDAFAverageEvaluator.
+ *
+ */
+ public static class GenericUDAFAverageEvaluator extends GenericUDAFEvaluator {
+
+ // For PARTIAL1 and COMPLETE
+ PrimitiveObjectInspector inputOI;
+
+ // For PARTIAL2 and FINAL
+ StructObjectInspector soi;
+ StructField countField;
+ StructField sumField;
+ LongObjectInspector countFieldOI;
+ DoubleObjectInspector sumFieldOI;
+
+ // For PARTIAL1 and PARTIAL2
+ Object[] partialResult;
+
+ // For FINAL and COMPLETE
+ DoubleWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+
+ // init input
+ if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ } else {
+ soi = (StructObjectInspector) parameters[0];
+ countField = soi.getStructFieldRef("count");
+ sumField = soi.getStructFieldRef("sum");
+ countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
+ sumFieldOI = (DoubleObjectInspector) sumField.getFieldObjectInspector();
+ }
+
+ // init output
+ if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+ // The output of a partial aggregation is a struct containing
+ // a "long" count and a "double" sum.
+
+ ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+ foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ ArrayList<String> fname = new ArrayList<String>();
+ fname.add("count");
+ fname.add("sum");
+ partialResult = new Object[2];
+ partialResult[0] = new LongWritable(0);
+ partialResult[1] = new DoubleWritable(0);
+ return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+
+ } else {
+ result = new DoubleWritable(0);
+ return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+ }
+ }
+
+ static class AverageAgg implements SerializableBuffer {
+ long count;
+ double sum;
+
+ @Override
+ public void deSerializeAggBuffer(byte[] data, int start, int len) {
+ count = BufferSerDeUtil.getLong(data, start);
+ start += 8;
+ sum = BufferSerDeUtil.getDouble(data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(byte[] data, int start, int len) {
+ BufferSerDeUtil.writeLong(count, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(sum, data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(DataOutput output) throws IOException {
+ output.writeLong(count);
+ output.writeDouble(sum);
+ }
+ };
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ AverageAgg result = new AverageAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ AverageAgg myagg = (AverageAgg) agg;
+ myagg.count = 0;
+ myagg.sum = 0;
+ }
+
+ boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ Object p = parameters[0];
+ if (p != null) {
+ AverageAgg myagg = (AverageAgg) agg;
+ try {
+ double v = PrimitiveObjectInspectorUtils.getDouble(p, inputOI);
+ myagg.count++;
+ myagg.sum += v;
+ } catch (NumberFormatException e) {
+ if (!warned) {
+ warned = true;
+ LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
+ LOG.warn(getClass().getSimpleName() + " ignoring similar exceptions.");
+ }
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ AverageAgg myagg = (AverageAgg) agg;
+ ((LongWritable) partialResult[0]).set(myagg.count);
+ ((DoubleWritable) partialResult[1]).set(myagg.sum);
+ return partialResult;
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ AverageAgg myagg = (AverageAgg) agg;
+ Object partialCount = soi.getStructFieldData(partial, countField);
+ Object partialSum = soi.getStructFieldData(partial, sumField);
+ myagg.count += countFieldOI.get(partialCount);
+ myagg.sum += sumFieldOI.get(partialSum);
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ AverageAgg myagg = (AverageAgg) agg;
+ if (myagg.count == 0) {
+ return null;
+ } else {
+ result.set(myagg.sum / myagg.count);
+ return result;
+ }
+ }
+ }
+
+}
diff --git a/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
new file mode 100644
index 0000000..6652afc
--- /dev/null
+++ b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
@@ -0,0 +1,398 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql.udf.generic;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.LongWritable;
+
+import edu.uci.ics.hivesterix.runtime.evaluator.BufferSerDeUtil;
+import edu.uci.ics.hivesterix.runtime.evaluator.SerializableBuffer;
+
+/**
+ * Compute the Pearson correlation coefficient corr(x, y), using the following
+ * stable one-pass method, based on: "Formulas for Robust, One-Pass Parallel
+ * Computation of Covariances and Arbitrary-Order Statistical Moments", Philippe
+ * Pebay, Sandia Labs and
+ * "The Art of Computer Programming, volume 2: Seminumerical Algorithms", Donald
+ * Knuth.
+ *
+ * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg> my_n =
+ * my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n - mx_(n-1))*(y_n
+ * - my_n) : <covariance * n> vx_n = vx_(n-1) + (x_n - mx_n)(x_n - mx_(n-1)):
+ * <variance * n> vy_n = vy_(n-1) + (y_n - my_n)(y_n - my_(n-1)): <variance * n>
+ *
+ * Merge: c_(A,B) = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/(n_A+n_B)
+ * vx_(A,B) = vx_A + vx_B + (mx_A - mx_B)*(mx_A - mx_B)*n_A*n_B/(n_A+n_B)
+ * vy_(A,B) = vy_A + vy_B + (my_A - my_B)*(my_A - my_B)*n_A*n_B/(n_A+n_B)
+ *
+ */
+@Description(name = "corr", value = "_FUNC_(x,y) - Returns the Pearson coefficient of correlation\n"
+ + "between a set of number pairs", extended = "The function takes as arguments any pair of numeric types and returns a double.\n"
+ + "Any pair with a NULL is ignored. If the function is applied to an empty set or\n"
+ + "a singleton set, NULL will be returned. Otherwise, it computes the following:\n"
+ + " COVAR_POP(x,y)/(STDDEV_POP(x)*STDDEV_POP(y))\n"
+ + "where neither x nor y is null,\n"
+ + "COVAR_POP is the population covariance,\n" + "and STDDEV_POP is the population standard deviation.")
+public class GenericUDAFCorrelation extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFCorrelation.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length != 2) {
+ throw new UDFArgumentTypeException(parameters.length - 1, "Exactly two arguments are expected.");
+ }
+
+ if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+
+ if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(1, "Only primitive type arguments are accepted but "
+ + parameters[1].getTypeName() + " is passed.");
+ }
+
+ switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ switch (((PrimitiveTypeInfo) parameters[1]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ return new GenericUDAFCorrelationEvaluator();
+ case STRING:
+ case BOOLEAN:
+ default:
+ throw new UDFArgumentTypeException(1, "Only numeric type arguments are accepted but "
+ + parameters[1].getTypeName() + " is passed.");
+ }
+ case STRING:
+ case BOOLEAN:
+ default:
+ throw new UDFArgumentTypeException(0, "Only numeric type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ }
+
+ /**
+ * Evaluate the Pearson correlation coefficient using a stable one-pass
+ * algorithm, based on work by Philippe Pébay and Donald Knuth.
+ *
+ * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg>
+ * my_n = my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n -
+ * mx_(n-1))*(y_n - my_n) : <covariance * n> vx_n = vx_(n-1) + (x_n -
+ * mx_n)(x_n - mx_(n-1)): <variance * n> vy_n = vy_(n-1) + (y_n - my_n)(y_n
+ * - my_(n-1)): <variance * n>
+ *
+ * Merge: c_X = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/n_X vx_(A,B)
+ * = vx_A + vx_B + (mx_A - mx_B)*(mx_A - mx_B)*n_A*n_B/(n_A+n_B) vy_(A,B) =
+ * vy_A + vy_B + (my_A - my_B)*(my_A - my_B)*n_A*n_B/(n_A+n_B)
+ *
+ */
+ public static class GenericUDAFCorrelationEvaluator extends GenericUDAFEvaluator {
+
+ // For PARTIAL1 and COMPLETE
+ private PrimitiveObjectInspector xInputOI;
+ private PrimitiveObjectInspector yInputOI;
+
+ // For PARTIAL2 and FINAL
+ private StructObjectInspector soi;
+ private StructField countField;
+ private StructField xavgField;
+ private StructField yavgField;
+ private StructField xvarField;
+ private StructField yvarField;
+ private StructField covarField;
+ private LongObjectInspector countFieldOI;
+ private DoubleObjectInspector xavgFieldOI;
+ private DoubleObjectInspector yavgFieldOI;
+ private DoubleObjectInspector xvarFieldOI;
+ private DoubleObjectInspector yvarFieldOI;
+ private DoubleObjectInspector covarFieldOI;
+
+ // For PARTIAL1 and PARTIAL2
+ private Object[] partialResult;
+
+ // For FINAL and COMPLETE
+ private DoubleWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ super.init(m, parameters);
+
+ // init input
+ if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+ assert (parameters.length == 2);
+ xInputOI = (PrimitiveObjectInspector) parameters[0];
+ yInputOI = (PrimitiveObjectInspector) parameters[1];
+ } else {
+ assert (parameters.length == 1);
+ soi = (StructObjectInspector) parameters[0];
+
+ countField = soi.getStructFieldRef("count");
+ xavgField = soi.getStructFieldRef("xavg");
+ yavgField = soi.getStructFieldRef("yavg");
+ xvarField = soi.getStructFieldRef("xvar");
+ yvarField = soi.getStructFieldRef("yvar");
+ covarField = soi.getStructFieldRef("covar");
+
+ countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
+ xavgFieldOI = (DoubleObjectInspector) xavgField.getFieldObjectInspector();
+ yavgFieldOI = (DoubleObjectInspector) yavgField.getFieldObjectInspector();
+ xvarFieldOI = (DoubleObjectInspector) xvarField.getFieldObjectInspector();
+ yvarFieldOI = (DoubleObjectInspector) yvarField.getFieldObjectInspector();
+ covarFieldOI = (DoubleObjectInspector) covarField.getFieldObjectInspector();
+ }
+
+ // init output
+ if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+ // The output of a partial aggregation is a struct containing
+ // a long count, two double averages, two double variances,
+ // and a double covariance.
+
+ ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+
+ foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+
+ ArrayList<String> fname = new ArrayList<String>();
+ fname.add("count");
+ fname.add("xavg");
+ fname.add("yavg");
+ fname.add("xvar");
+ fname.add("yvar");
+ fname.add("covar");
+
+ partialResult = new Object[6];
+ partialResult[0] = new LongWritable(0);
+ partialResult[1] = new DoubleWritable(0);
+ partialResult[2] = new DoubleWritable(0);
+ partialResult[3] = new DoubleWritable(0);
+ partialResult[4] = new DoubleWritable(0);
+ partialResult[5] = new DoubleWritable(0);
+
+ return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+
+ } else {
+ setResult(new DoubleWritable(0));
+ return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+ }
+ }
+
+ static class StdAgg implements SerializableBuffer {
+ long count; // number n of elements
+ double xavg; // average of x elements
+ double yavg; // average of y elements
+ double xvar; // n times the variance of x elements
+ double yvar; // n times the variance of y elements
+ double covar; // n times the covariance
+
+ @Override
+ public void deSerializeAggBuffer(byte[] data, int start, int len) {
+ count = BufferSerDeUtil.getLong(data, start);
+ start += 8;
+ xavg = BufferSerDeUtil.getDouble(data, start);
+ start += 8;
+ yavg = BufferSerDeUtil.getDouble(data, start);
+ start += 8;
+ xvar = BufferSerDeUtil.getDouble(data, start);
+ start += 8;
+ yvar = BufferSerDeUtil.getDouble(data, start);
+ start += 8;
+ covar = BufferSerDeUtil.getDouble(data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(byte[] data, int start, int len) {
+ BufferSerDeUtil.writeLong(count, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(xavg, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(yavg, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(xvar, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(yvar, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(covar, data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(DataOutput output) throws IOException {
+ output.writeLong(count);
+ output.writeDouble(xavg);
+ output.writeDouble(yavg);
+ output.writeDouble(xvar);
+ output.writeDouble(yvar);
+ output.writeDouble(covar);
+ }
+ };
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ StdAgg result = new StdAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+ myagg.count = 0;
+ myagg.xavg = 0;
+ myagg.yavg = 0;
+ myagg.xvar = 0;
+ myagg.yvar = 0;
+ myagg.covar = 0;
+ }
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 2);
+ Object px = parameters[0];
+ Object py = parameters[1];
+ if (px != null && py != null) {
+ StdAgg myagg = (StdAgg) agg;
+ double vx = PrimitiveObjectInspectorUtils.getDouble(px, xInputOI);
+ double vy = PrimitiveObjectInspectorUtils.getDouble(py, yInputOI);
+ double xavgOld = myagg.xavg;
+ double yavgOld = myagg.yavg;
+ myagg.count++;
+ myagg.xavg += (vx - xavgOld) / myagg.count;
+ myagg.yavg += (vy - yavgOld) / myagg.count;
+ if (myagg.count > 1) {
+ myagg.covar += (vx - xavgOld) * (vy - myagg.yavg);
+ myagg.xvar += (vx - xavgOld) * (vx - myagg.xavg);
+ myagg.yvar += (vy - yavgOld) * (vy - myagg.yavg);
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+ ((LongWritable) partialResult[0]).set(myagg.count);
+ ((DoubleWritable) partialResult[1]).set(myagg.xavg);
+ ((DoubleWritable) partialResult[2]).set(myagg.yavg);
+ ((DoubleWritable) partialResult[3]).set(myagg.xvar);
+ ((DoubleWritable) partialResult[4]).set(myagg.yvar);
+ ((DoubleWritable) partialResult[5]).set(myagg.covar);
+ return partialResult;
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ StdAgg myagg = (StdAgg) agg;
+
+ Object partialCount = soi.getStructFieldData(partial, countField);
+ Object partialXAvg = soi.getStructFieldData(partial, xavgField);
+ Object partialYAvg = soi.getStructFieldData(partial, yavgField);
+ Object partialXVar = soi.getStructFieldData(partial, xvarField);
+ Object partialYVar = soi.getStructFieldData(partial, yvarField);
+ Object partialCovar = soi.getStructFieldData(partial, covarField);
+
+ long nA = myagg.count;
+ long nB = countFieldOI.get(partialCount);
+
+ if (nA == 0) {
+ // Just copy the information since there is nothing so far
+ myagg.count = countFieldOI.get(partialCount);
+ myagg.xavg = xavgFieldOI.get(partialXAvg);
+ myagg.yavg = yavgFieldOI.get(partialYAvg);
+ myagg.xvar = xvarFieldOI.get(partialXVar);
+ myagg.yvar = yvarFieldOI.get(partialYVar);
+ myagg.covar = covarFieldOI.get(partialCovar);
+ }
+
+ if (nA != 0 && nB != 0) {
+ // Merge the two partials
+ double xavgA = myagg.xavg;
+ double yavgA = myagg.yavg;
+ double xavgB = xavgFieldOI.get(partialXAvg);
+ double yavgB = yavgFieldOI.get(partialYAvg);
+ double xvarB = xvarFieldOI.get(partialXVar);
+ double yvarB = yvarFieldOI.get(partialYVar);
+ double covarB = covarFieldOI.get(partialCovar);
+
+ myagg.count += nB;
+ myagg.xavg = (xavgA * nA + xavgB * nB) / myagg.count;
+ myagg.yavg = (yavgA * nA + yavgB * nB) / myagg.count;
+ myagg.xvar += xvarB + (xavgA - xavgB) * (xavgA - xavgB) * myagg.count;
+ myagg.yvar += yvarB + (yavgA - yavgB) * (yavgA - yavgB) * myagg.count;
+ myagg.covar += covarB + (xavgA - xavgB) * (yavgA - yavgB) * ((double) (nA * nB) / myagg.count);
+ }
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+
+ if (myagg.count < 2) { // SQL standard - return null for zero or one
+ // pair
+ return null;
+ } else {
+ getResult().set(myagg.covar / java.lang.Math.sqrt(myagg.xvar) / java.lang.Math.sqrt(myagg.yvar));
+ return getResult();
+ }
+ }
+
+ public void setResult(DoubleWritable result) {
+ this.result = result;
+ }
+
+ public DoubleWritable getResult() {
+ return result;
+ }
+ }
+
+}
diff --git a/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
new file mode 100644
index 0000000..e4081f5
--- /dev/null
+++ b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql.udf.generic;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.LongWritable;
+
+import edu.uci.ics.hivesterix.runtime.evaluator.BufferSerDeUtil;
+import edu.uci.ics.hivesterix.runtime.evaluator.SerializableBuffer;
+
+/**
+ * This class implements the COUNT aggregation function as in SQL.
+ */
+@Description(name = "count", value = "_FUNC_(*) - Returns the total number of retrieved rows, including "
+ + "rows containing NULL values.\n"
+
+ + "_FUNC_(expr) - Returns the number of rows for which the supplied " + "expression is non-NULL.\n"
+
+ + "_FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for "
+ + "which the supplied expression(s) are unique and non-NULL.")
+public class GenericUDAFCount implements GenericUDAFResolver2 {
+
+ private static final Log LOG = LogFactory.getLog(GenericUDAFCount.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ // This method implementation is preserved for backward compatibility.
+ return new GenericUDAFCountEvaluator();
+ }
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(GenericUDAFParameterInfo paramInfo) throws SemanticException {
+
+ TypeInfo[] parameters = paramInfo.getParameters();
+
+ if (parameters.length == 0) {
+ if (!paramInfo.isAllColumns()) {
+ throw new UDFArgumentException("Argument expected");
+ }
+ assert !paramInfo.isDistinct() : "DISTINCT not supported with *";
+ } else {
+ if (parameters.length > 1 && !paramInfo.isDistinct()) {
+ throw new UDFArgumentException("DISTINCT keyword must be specified");
+ }
+ assert !paramInfo.isAllColumns() : "* not supported in expression list";
+ }
+
+ return new GenericUDAFCountEvaluator().setCountAllColumns(paramInfo.isAllColumns());
+ }
+
+ /**
+ * GenericUDAFCountEvaluator.
+ *
+ */
+ public static class GenericUDAFCountEvaluator extends GenericUDAFEvaluator {
+ private boolean countAllColumns = false;
+ private LongObjectInspector partialCountAggOI;
+ private LongWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ super.init(m, parameters);
+ partialCountAggOI = PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+ result = new LongWritable(0);
+ return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+ }
+
+ private GenericUDAFCountEvaluator setCountAllColumns(boolean countAllCols) {
+ countAllColumns = countAllCols;
+ return this;
+ }
+
+ /** class for storing count value. */
+ static class CountAgg implements SerializableBuffer {
+ long value;
+
+ @Override
+ public void deSerializeAggBuffer(byte[] data, int start, int len) {
+ value = BufferSerDeUtil.getLong(data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(byte[] data, int start, int len) {
+ BufferSerDeUtil.writeLong(value, data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(DataOutput output) throws IOException {
+ output.writeLong(value);
+ }
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ CountAgg buffer = new CountAgg();
+ reset(buffer);
+ return buffer;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ ((CountAgg) agg).value = 0;
+ }
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ // parameters == null means the input table/split is empty
+ if (parameters == null) {
+ return;
+ }
+ if (countAllColumns) {
+ assert parameters.length == 0;
+ ((CountAgg) agg).value++;
+ } else {
+ assert parameters.length > 0;
+ boolean countThisRow = true;
+ for (Object nextParam : parameters) {
+ if (nextParam == null) {
+ countThisRow = false;
+ break;
+ }
+ }
+ if (countThisRow) {
+ ((CountAgg) agg).value++;
+ }
+ }
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ long p = partialCountAggOI.get(partial);
+ ((CountAgg) agg).value += p;
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ result.set(((CountAgg) agg).value);
+ return result;
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ return terminate(agg);
+ }
+ }
+}
diff --git a/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
new file mode 100644
index 0000000..0453c3b
--- /dev/null
+++ b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
@@ -0,0 +1,350 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql.udf.generic;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.LongWritable;
+
+import edu.uci.ics.hivesterix.runtime.evaluator.BufferSerDeUtil;
+import edu.uci.ics.hivesterix.runtime.evaluator.SerializableBuffer;
+
+/**
+ * Compute the covariance covar_pop(x, y), using the following one-pass method
+ * (ref. "Formulas for Robust, One-Pass Parallel Computation of Covariances and
+ * Arbitrary-Order Statistical Moments", Philippe Pebay, Sandia Labs):
+ *
+ * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg> my_n =
+ * my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n - mx_(n-1))*(y_n
+ * - my_n) : <covariance * n>
+ *
+ * Merge: c_X = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/n_X
+ *
+ */
+@Description(name = "covariance,covar_pop", value = "_FUNC_(x,y) - Returns the population covariance of a set of number pairs", extended = "The function takes as arguments any pair of numeric types and returns a double.\n"
+ + "Any pair with a NULL is ignored. If the function is applied to an empty set, NULL\n"
+ + "will be returned. Otherwise, it computes the following:\n"
+ + " (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/COUNT(x,y)\n" + "where neither x nor y is null.")
+public class GenericUDAFCovariance extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFCovariance.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length != 2) {
+ throw new UDFArgumentTypeException(parameters.length - 1, "Exactly two arguments are expected.");
+ }
+
+ if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+
+ if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(1, "Only primitive type arguments are accepted but "
+ + parameters[1].getTypeName() + " is passed.");
+ }
+
+ switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ switch (((PrimitiveTypeInfo) parameters[1]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ return new GenericUDAFCovarianceEvaluator();
+ case STRING:
+ case BOOLEAN:
+ default:
+ throw new UDFArgumentTypeException(1, "Only numeric or string type arguments are accepted but "
+ + parameters[1].getTypeName() + " is passed.");
+ }
+ case STRING:
+ case BOOLEAN:
+ default:
+ throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ }
+
+ /**
+ * Evaluate the variance using the algorithm described in
+ * http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance,
+ * presumably by Pébay, Philippe (2008), in "Formulas for Robust, One-Pass
+ * Parallel Computation of Covariances and Arbitrary-Order Statistical
+ * Moments", Technical Report SAND2008-6212, Sandia National Laboratories,
+ * http://infoserve.sandia.gov/sand_doc/2008/086212.pdf
+ *
+ * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg>
+ * my_n = my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n -
+ * mx_(n-1))*(y_n - my_n) : <covariance * n>
+ *
+ * Merge: c_X = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/n_X
+ *
+ * This one-pass algorithm is stable.
+ *
+ */
+ public static class GenericUDAFCovarianceEvaluator extends GenericUDAFEvaluator {
+
+ // For PARTIAL1 and COMPLETE
+ private PrimitiveObjectInspector xInputOI;
+ private PrimitiveObjectInspector yInputOI;
+
+ // For PARTIAL2 and FINAL
+ private StructObjectInspector soi;
+ private StructField countField;
+ private StructField xavgField;
+ private StructField yavgField;
+ private StructField covarField;
+ private LongObjectInspector countFieldOI;
+ private DoubleObjectInspector xavgFieldOI;
+ private DoubleObjectInspector yavgFieldOI;
+ private DoubleObjectInspector covarFieldOI;
+
+ // For PARTIAL1 and PARTIAL2
+ private Object[] partialResult;
+
+ // For FINAL and COMPLETE
+ private DoubleWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ super.init(m, parameters);
+
+ // init input
+ if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+ assert (parameters.length == 2);
+ xInputOI = (PrimitiveObjectInspector) parameters[0];
+ yInputOI = (PrimitiveObjectInspector) parameters[1];
+ } else {
+ assert (parameters.length == 1);
+ soi = (StructObjectInspector) parameters[0];
+
+ countField = soi.getStructFieldRef("count");
+ xavgField = soi.getStructFieldRef("xavg");
+ yavgField = soi.getStructFieldRef("yavg");
+ covarField = soi.getStructFieldRef("covar");
+
+ countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
+ xavgFieldOI = (DoubleObjectInspector) xavgField.getFieldObjectInspector();
+ yavgFieldOI = (DoubleObjectInspector) yavgField.getFieldObjectInspector();
+ covarFieldOI = (DoubleObjectInspector) covarField.getFieldObjectInspector();
+ }
+
+ // init output
+ if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+ // The output of a partial aggregation is a struct containing
+ // a long count, two double averages, and a double covariance.
+
+ ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+
+ foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+
+ ArrayList<String> fname = new ArrayList<String>();
+ fname.add("count");
+ fname.add("xavg");
+ fname.add("yavg");
+ fname.add("covar");
+
+ partialResult = new Object[4];
+ partialResult[0] = new LongWritable(0);
+ partialResult[1] = new DoubleWritable(0);
+ partialResult[2] = new DoubleWritable(0);
+ partialResult[3] = new DoubleWritable(0);
+
+ return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+
+ } else {
+ setResult(new DoubleWritable(0));
+ return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+ }
+ }
+
+ static class StdAgg implements SerializableBuffer {
+ long count; // number n of elements
+ double xavg; // average of x elements
+ double yavg; // average of y elements
+ double covar; // n times the covariance
+
+ @Override
+ public void deSerializeAggBuffer(byte[] data, int start, int len) {
+ count = BufferSerDeUtil.getLong(data, start);
+ start += 8;
+ xavg = BufferSerDeUtil.getDouble(data, start);
+ start += 8;
+ yavg = BufferSerDeUtil.getDouble(data, start);
+ start += 8;
+ covar = BufferSerDeUtil.getDouble(data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(byte[] data, int start, int len) {
+ BufferSerDeUtil.writeLong(count, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(xavg, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(yavg, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(covar, data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(DataOutput output) throws IOException {
+ output.writeLong(count);
+ output.writeDouble(xavg);
+ output.writeDouble(yavg);
+ output.writeDouble(covar);
+ }
+ };
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ StdAgg result = new StdAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+ myagg.count = 0;
+ myagg.xavg = 0;
+ myagg.yavg = 0;
+ myagg.covar = 0;
+ }
+
+ private boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 2);
+ Object px = parameters[0];
+ Object py = parameters[1];
+ if (px != null && py != null) {
+ StdAgg myagg = (StdAgg) agg;
+ double vx = PrimitiveObjectInspectorUtils.getDouble(px, xInputOI);
+ double vy = PrimitiveObjectInspectorUtils.getDouble(py, yInputOI);
+ myagg.count++;
+ myagg.yavg = myagg.yavg + (vy - myagg.yavg) / myagg.count;
+ if (myagg.count > 1) {
+ myagg.covar += (vx - myagg.xavg) * (vy - myagg.yavg);
+ }
+ myagg.xavg = myagg.xavg + (vx - myagg.xavg) / myagg.count;
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+ ((LongWritable) partialResult[0]).set(myagg.count);
+ ((DoubleWritable) partialResult[1]).set(myagg.xavg);
+ ((DoubleWritable) partialResult[2]).set(myagg.yavg);
+ ((DoubleWritable) partialResult[3]).set(myagg.covar);
+ return partialResult;
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ StdAgg myagg = (StdAgg) agg;
+
+ Object partialCount = soi.getStructFieldData(partial, countField);
+ Object partialXAvg = soi.getStructFieldData(partial, xavgField);
+ Object partialYAvg = soi.getStructFieldData(partial, yavgField);
+ Object partialCovar = soi.getStructFieldData(partial, covarField);
+
+ long nA = myagg.count;
+ long nB = countFieldOI.get(partialCount);
+
+ if (nA == 0) {
+ // Just copy the information since there is nothing so far
+ myagg.count = countFieldOI.get(partialCount);
+ myagg.xavg = xavgFieldOI.get(partialXAvg);
+ myagg.yavg = yavgFieldOI.get(partialYAvg);
+ myagg.covar = covarFieldOI.get(partialCovar);
+ }
+
+ if (nA != 0 && nB != 0) {
+ // Merge the two partials
+ double xavgA = myagg.xavg;
+ double yavgA = myagg.yavg;
+ double xavgB = xavgFieldOI.get(partialXAvg);
+ double yavgB = yavgFieldOI.get(partialYAvg);
+ double covarB = covarFieldOI.get(partialCovar);
+
+ myagg.count += nB;
+ myagg.xavg = (xavgA * nA + xavgB * nB) / myagg.count;
+ myagg.yavg = (yavgA * nA + yavgB * nB) / myagg.count;
+ myagg.covar += covarB + (xavgA - xavgB) * (yavgA - yavgB) * ((double) (nA * nB) / myagg.count);
+ }
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+
+ if (myagg.count == 0) { // SQL standard - return null for zero
+ // elements
+ return null;
+ } else {
+ getResult().set(myagg.covar / (myagg.count));
+ return getResult();
+ }
+ }
+
+ public void setResult(DoubleWritable result) {
+ this.result = result;
+ }
+
+ public DoubleWritable getResult() {
+ return result;
+ }
+ }
+
+}
diff --git a/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
new file mode 100644
index 0000000..23b453f
--- /dev/null
+++ b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
@@ -0,0 +1,275 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql.udf.generic;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.util.StringUtils;
+
+import edu.uci.ics.hivesterix.runtime.evaluator.BufferSerDeUtil;
+import edu.uci.ics.hivesterix.runtime.evaluator.SerializableBuffer;
+
+/**
+ * GenericUDAFSum.
+ *
+ */
+@Description(name = "sum", value = "_FUNC_(x) - Returns the sum of a set of numbers")
+public class GenericUDAFSum extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFSum.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length != 1) {
+ throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected.");
+ }
+
+ if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ return new GenericUDAFSumLong();
+ case FLOAT:
+ case DOUBLE:
+ case STRING:
+ return new GenericUDAFSumDouble();
+ case BOOLEAN:
+ default:
+ throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ }
+
+ /**
+ * GenericUDAFSumDouble.
+ *
+ */
+ public static class GenericUDAFSumDouble extends GenericUDAFEvaluator {
+ private PrimitiveObjectInspector inputOI;
+ private DoubleWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+ result = new DoubleWritable(0);
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+ }
+
+ /** class for storing double sum value. */
+ static class SumDoubleAgg implements SerializableBuffer {
+ boolean empty;
+ double sum;
+
+ @Override
+ public void deSerializeAggBuffer(byte[] data, int start, int len) {
+ empty = BufferSerDeUtil.getBoolean(data, start);
+ start += 1;
+ sum = BufferSerDeUtil.getDouble(data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(byte[] data, int start, int len) {
+ BufferSerDeUtil.writeBoolean(empty, data, start);
+ start += 1;
+ BufferSerDeUtil.writeDouble(sum, data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(DataOutput output) throws IOException {
+ output.writeBoolean(empty);
+ output.writeDouble(sum);
+ }
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ SumDoubleAgg result = new SumDoubleAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ SumDoubleAgg myagg = (SumDoubleAgg) agg;
+ myagg.empty = true;
+ myagg.sum = 0;
+ }
+
+ boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ try {
+ merge(agg, parameters[0]);
+ } catch (NumberFormatException e) {
+ if (!warned) {
+ warned = true;
+ LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
+ LOG.warn(getClass().getSimpleName() + " ignoring similar exceptions.");
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ return terminate(agg);
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ SumDoubleAgg myagg = (SumDoubleAgg) agg;
+ myagg.empty = false;
+ myagg.sum += PrimitiveObjectInspectorUtils.getDouble(partial, inputOI);
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ SumDoubleAgg myagg = (SumDoubleAgg) agg;
+ if (myagg.empty) {
+ return null;
+ }
+ result.set(myagg.sum);
+ return result;
+ }
+
+ }
+
+ /**
+ * GenericUDAFSumLong.
+ *
+ */
+ public static class GenericUDAFSumLong extends GenericUDAFEvaluator {
+ private PrimitiveObjectInspector inputOI;
+ private LongWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+ result = new LongWritable(0);
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+ }
+
+ /** class for storing double sum value. */
+ static class SumLongAgg implements SerializableBuffer {
+ boolean empty;
+ long sum;
+
+ @Override
+ public void deSerializeAggBuffer(byte[] data, int start, int len) {
+ empty = BufferSerDeUtil.getBoolean(data, start);
+ start += 1;
+ sum = BufferSerDeUtil.getLong(data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(byte[] data, int start, int len) {
+ BufferSerDeUtil.writeBoolean(empty, data, start);
+ start += 1;
+ BufferSerDeUtil.writeLong(sum, data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(DataOutput output) throws IOException {
+ output.writeBoolean(empty);
+ output.writeLong(sum);
+ }
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ SumLongAgg result = new SumLongAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ SumLongAgg myagg = (SumLongAgg) agg;
+ myagg.empty = true;
+ myagg.sum = 0;
+ }
+
+ private boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ try {
+ merge(agg, parameters[0]);
+ } catch (NumberFormatException e) {
+ if (!warned) {
+ warned = true;
+ LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ return terminate(agg);
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ SumLongAgg myagg = (SumLongAgg) agg;
+ myagg.sum += PrimitiveObjectInspectorUtils.getLong(partial, inputOI);
+ myagg.empty = false;
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ SumLongAgg myagg = (SumLongAgg) agg;
+ if (myagg.empty) {
+ return null;
+ }
+ result.set(myagg.sum);
+ return result;
+ }
+
+ }
+
+}
diff --git a/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
new file mode 100644
index 0000000..d91551d
--- /dev/null
+++ b/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
@@ -0,0 +1,312 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql.udf.generic;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.util.StringUtils;
+
+import edu.uci.ics.hivesterix.runtime.evaluator.BufferSerDeUtil;
+import edu.uci.ics.hivesterix.runtime.evaluator.SerializableBuffer;
+
+/**
+ * Compute the variance. This class is extended by: GenericUDAFVarianceSample
+ * GenericUDAFStd GenericUDAFStdSample
+ *
+ */
+@Description(name = "variance,var_pop", value = "_FUNC_(x) - Returns the variance of a set of numbers")
+public class GenericUDAFVariance extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFVariance.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+ if (parameters.length != 1) {
+ throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected.");
+ }
+
+ if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ case STRING:
+ return new GenericUDAFVarianceEvaluator();
+ case BOOLEAN:
+ default:
+ throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ }
+
+ /**
+ * Evaluate the variance using the algorithm described by Chan, Golub, and
+ * LeVeque in
+ * "Algorithms for computing the sample variance: analysis and recommendations"
+ * The American Statistician, 37 (1983) pp. 242--247.
+ *
+ * variance = variance1 + variance2 + n/(m*(m+n)) * pow(((m/n)*t1 - t2),2)
+ *
+ * where: - variance is sum[x-avg^2] (this is actually n times the variance)
+ * and is updated at every step. - n is the count of elements in chunk1 - m
+ * is the count of elements in chunk2 - t1 = sum of elements in chunk1, t2 =
+ * sum of elements in chunk2.
+ *
+ * This algorithm was proven to be numerically stable by J.L. Barlow in
+ * "Error analysis of a pairwise summation algorithm to compute sample variance"
+ * Numer. Math, 58 (1991) pp. 583--590
+ *
+ */
+ public static class GenericUDAFVarianceEvaluator extends GenericUDAFEvaluator {
+
+ // For PARTIAL1 and COMPLETE
+ private PrimitiveObjectInspector inputOI;
+
+ // For PARTIAL2 and FINAL
+ private StructObjectInspector soi;
+ private StructField countField;
+ private StructField sumField;
+ private StructField varianceField;
+ private LongObjectInspector countFieldOI;
+ private DoubleObjectInspector sumFieldOI;
+ private DoubleObjectInspector varianceFieldOI;
+
+ // For PARTIAL1 and PARTIAL2
+ private Object[] partialResult;
+
+ // For FINAL and COMPLETE
+ private DoubleWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+
+ // init input
+ if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ } else {
+ soi = (StructObjectInspector) parameters[0];
+
+ countField = soi.getStructFieldRef("count");
+ sumField = soi.getStructFieldRef("sum");
+ varianceField = soi.getStructFieldRef("variance");
+
+ countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
+ sumFieldOI = (DoubleObjectInspector) sumField.getFieldObjectInspector();
+ varianceFieldOI = (DoubleObjectInspector) varianceField.getFieldObjectInspector();
+ }
+
+ // init output
+ if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+ // The output of a partial aggregation is a struct containing
+ // a long count and doubles sum and variance.
+
+ ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+
+ foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+ foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+
+ ArrayList<String> fname = new ArrayList<String>();
+ fname.add("count");
+ fname.add("sum");
+ fname.add("variance");
+
+ partialResult = new Object[3];
+ partialResult[0] = new LongWritable(0);
+ partialResult[1] = new DoubleWritable(0);
+ partialResult[2] = new DoubleWritable(0);
+
+ return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+
+ } else {
+ setResult(new DoubleWritable(0));
+ return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+ }
+ }
+
+ static class StdAgg implements SerializableBuffer {
+ long count; // number of elements
+ double sum; // sum of elements
+ double variance; // sum[x-avg^2] (this is actually n times the
+ // variance)
+
+ @Override
+ public void deSerializeAggBuffer(byte[] data, int start, int len) {
+ count = BufferSerDeUtil.getLong(data, start);
+ start += 8;
+ sum = BufferSerDeUtil.getDouble(data, start);
+ start += 8;
+ variance = BufferSerDeUtil.getDouble(data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(byte[] data, int start, int len) {
+ BufferSerDeUtil.writeLong(count, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(sum, data, start);
+ start += 8;
+ BufferSerDeUtil.writeDouble(variance, data, start);
+ }
+
+ @Override
+ public void serializeAggBuffer(DataOutput output) throws IOException {
+ output.writeLong(count);
+ output.writeDouble(sum);
+ output.writeDouble(variance);
+ }
+ };
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ StdAgg result = new StdAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+ myagg.count = 0;
+ myagg.sum = 0;
+ myagg.variance = 0;
+ }
+
+ private boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ Object p = parameters[0];
+ if (p != null) {
+ StdAgg myagg = (StdAgg) agg;
+ try {
+ double v = PrimitiveObjectInspectorUtils.getDouble(p, inputOI);
+ myagg.count++;
+ myagg.sum += v;
+ if (myagg.count > 1) {
+ double t = myagg.count * v - myagg.sum;
+ myagg.variance += (t * t) / ((double) myagg.count * (myagg.count - 1));
+ }
+ } catch (NumberFormatException e) {
+ if (!warned) {
+ warned = true;
+ LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
+ LOG.warn(getClass().getSimpleName() + " ignoring similar exceptions.");
+ }
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+ ((LongWritable) partialResult[0]).set(myagg.count);
+ ((DoubleWritable) partialResult[1]).set(myagg.sum);
+ ((DoubleWritable) partialResult[2]).set(myagg.variance);
+ return partialResult;
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ StdAgg myagg = (StdAgg) agg;
+
+ Object partialCount = soi.getStructFieldData(partial, countField);
+ Object partialSum = soi.getStructFieldData(partial, sumField);
+ Object partialVariance = soi.getStructFieldData(partial, varianceField);
+
+ long n = myagg.count;
+ long m = countFieldOI.get(partialCount);
+
+ if (n == 0) {
+ // Just copy the information since there is nothing so far
+ myagg.variance = sumFieldOI.get(partialVariance);
+ myagg.count = countFieldOI.get(partialCount);
+ myagg.sum = sumFieldOI.get(partialSum);
+ }
+
+ if (m != 0 && n != 0) {
+ // Merge the two partials
+
+ double a = myagg.sum;
+ double b = sumFieldOI.get(partialSum);
+
+ myagg.count += m;
+ myagg.sum += b;
+ double t = (m / (double) n) * a - b;
+ myagg.variance += sumFieldOI.get(partialVariance) + ((n / (double) m) / ((double) n + m)) * t * t;
+ }
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ StdAgg myagg = (StdAgg) agg;
+
+ if (myagg.count == 0) { // SQL standard - return null for zero
+ // elements
+ return null;
+ } else {
+ if (myagg.count > 1) {
+ getResult().set(myagg.variance / (myagg.count));
+ } else { // for one element the variance is always 0
+ getResult().set(0);
+ }
+ return getResult();
+ }
+ }
+
+ public void setResult(DoubleWritable result) {
+ this.result = result;
+ }
+
+ public DoubleWritable getResult() {
+ return result;
+ }
+ }
+
+}
diff --git a/src/main/scripts/run.cmd b/src/main/scripts/run.cmd
new file mode 100755
index 0000000..b8eb4a0
--- /dev/null
+++ b/src/main/scripts/run.cmd
@@ -0,0 +1,63 @@
+@ECHO OFF
+SETLOCAL
+
+:: Licensed to the Apache Software Foundation (ASF) under one or more
+:: contributor license agreements. See the NOTICE file distributed with
+:: this work for additional information regarding copyright ownership.
+:: The ASF licenses this file to You 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.
+
+:: JAVA classpath
+:: Use the local variable CLASSPATH to add custom entries (e.g. JDBC drivers) to
+:: the classpath. Separate multiple paths with ":". Enclose the value
+:: in double quotes. Adding additional files or locations on separate
+:: lines makes things clearer.
+:: Note: If under running under cygwin use "/cygdrive/c/..." for "C:/..."
+:: Example:
+::
+:: Set the CLASSPATH to a jar file and a directory. Note that
+:: "classes dir" is a directory of class files with a space in the name.
+::
+:: CLASSPATH="usr/local/Product1/lib/product.jar"
+:: CLASSPATH="${CLASSPATH}:../MyProject/classes dir"
+::
+SET CLASSPATH="@classpath@"
+
+:: JVM parameters
+:: If you want to modify the default parameters (e.g. maximum heap size -Xmx)
+:: for the Java virtual machine set the local variable JVM_PARAMETERS below
+:: Example:
+:: JVM_PARAMETERS=-Xms100M -Xmx200M
+::
+:: Below are the JVM parameters needed to do remote debugging using Intellij
+:: IDEA. Uncomment and then do: JVM_PARAMETERS="$IDEA_REMOTE_DEBUG_PARAMS"
+:: IDEA_REMOTE_DEBUG_PARAMS="-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"
+::
+:: JVM_PARAMETERS=
+
+:: ---------------------------------------------------------------------------
+:: Default configuration. Do not modify below this line.
+:: ---------------------------------------------------------------------------
+:: Application specific parameters
+
+SET MAIN_CLASS=@main.class@
+SET JVM_PARAMS=@jvm.params@
+SET PROGRAM_PARAMS=@program.params@
+
+:: Try to find java virtual machine
+IF NOT DEFINED JAVA (
+ IF NOT DEFINED JAVA_HOME SET JAVA="java.exe"
+ IF DEFINED JAVA_HOME SET JAVA="%JAVA_HOME%\bin\java.exe"
+)
+
+:: Run program
+%JAVA% %JVM_PARAMS% %JVM_PARAMETERS% -classpath %CLASSPATH% %MAIN_CLASS% %PROGRAM_PARAMS% %*
diff --git a/src/main/scripts/run.sh b/src/main/scripts/run.sh
new file mode 100755
index 0000000..a998626
--- /dev/null
+++ b/src/main/scripts/run.sh
@@ -0,0 +1,81 @@
+#!/bin/sh
+# JAVA classpath
+# Use the local variable CLASSPATH to add custom entries (e.g. JDBC drivers) to
+# the classpath. Separate multiple paths with ":". Enclose the value
+# in double quotes. Adding additional files or locations on separate
+# lines makes things clearer.
+# Note: If under running under cygwin use "/cygdrive/c/..." for "C:/..."
+# Example:
+#
+# Set the CLASSPATH to a jar file and a directory. Note that
+# "classes dir" is a directory of class files with a space in the name.
+#
+# CLASSPATH="usr/local/Product1/lib/product.jar"
+# CLASSPATH="${CLASSPATH}:../MyProject/classes dir"
+#
+CLASSPATH="@classpath@"
+
+# JVM parameters
+# If you want to modify the default parameters (e.g. maximum heap size -Xmx)
+# for the Java virtual machine set the local variable JVM_PARAMETERS below
+# Example:
+# JVM_PARAMETERS=-Xms100M -Xmx200M
+#
+# Below are the JVM parameters needed to do remote debugging using Intellij
+# IDEA. Uncomment and then do: JVM_PARAMETERS="$IDEA_REMOTE_DEBUG_PARAMS"
+# IDEA_REMOTE_DEBUG_PARAMS="-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"
+#
+# JVM_PARAMETERS=
+
+#run with shared memory setup
+#if [ -n "${RUN_SHARED_MEM}"]; then
+# JVM_PARAMETERS="${JVM_PARAMETERS} -Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_shmem,server=n,address=javadebug,suspend=y"
+#fi
+
+# ---------------------------------------------------------------------------
+# Default configuration. Do not modify below this line.
+# ---------------------------------------------------------------------------
+# Application specific parameters
+
+MAIN_CLASS="@main.class@"
+JVM_PARAMS="@jvm.params@"
+PROGRAM_PARAMS="@program.params@"
+
+# Cygwin support. $cygwin _must_ be set to either true or false.
+case "`uname`" in
+ CYGWIN*) cygwin=true ;;
+ *) cygwin=false ;;
+esac
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin; then
+ [ -n "$JAVA_HOME" ] &&
+ JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+ [ -n "$CLASSPATH" ] &&
+ CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# Try to find java virtual machine
+if [ -z "${JAVA}" ]; then
+ if [ -z "${JAVA_HOME}" ]; then
+ JAVA=java
+ else
+ JAVA=${JAVA_HOME}/bin/java
+ fi
+fi
+
+# Try to find directory where this script is located
+COMMAND="${PWD}/$0"
+if [ ! -f "${COMMAND}" ]; then
+ COMMAND="$0"
+fi
+BASEDIR=`expr "${COMMAND}" : '\(.*\)/\.*'`
+
+# For Cygwin, switch paths to Windows format before running java
+if $cygwin; then
+# JAVA=`cygpath --path --windows "$JAVA"`
+ CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
+fi
+
+# Run program
+${JAVA} ${JVM_PARAMS} ${JVM_PARAMETERS} -classpath "${CLASSPATH}" ${MAIN_CLASS} ${PROGRAM_PARAMS} $*