migrate hivesterix to depend on hive-0.11.0
diff --git a/.gitignore b/.gitignore
index 503b769..6c81ef8 100644
--- a/.gitignore
+++ b/.gitignore
@@ -17,3 +17,4 @@
metastore_db
teststore
output
+tmp
diff --git a/hivesterix/hivesterix-common/pom.xml b/hivesterix/hivesterix-common/pom.xml
index 7741193..28305d7 100644
--- a/hivesterix/hivesterix-common/pom.xml
+++ b/hivesterix/hivesterix-common/pom.xml
@@ -63,9 +63,16 @@
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-exec</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hivesterix-serde</artifactId>
+ <version>0.2.7-SNAPSHOT</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
diff --git a/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionTranslator.java b/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionTranslator.java
index 783004c..2ec3811 100644
--- a/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionTranslator.java
+++ b/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionTranslator.java
@@ -12,212 +12,217 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-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.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.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());
- }
- }
-}
+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.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.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.serde.lazy.LazyUtils;
+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());
+ }
+
+ List<ObjectInspector> originalParameterOIs = new ArrayList<ObjectInspector>();
+ for (TypeInfo type : originalParameterTypeInfos) {
+ originalParameterOIs.add(LazyUtils.getLazyObjectInspectorFromTypeInfo(type, false));
+ }
+ GenericUDAFEvaluator eval = FunctionRegistry.getGenericUDAFEvaluator(
+ aggregateDesc.getGenericUDAFName(), originalParameterOIs, 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/hivesterix/hivesterix-dist/pom.xml b/hivesterix/hivesterix-dist/pom.xml
index 8ecdfe1..83de061 100644
--- a/hivesterix/hivesterix-dist/pom.xml
+++ b/hivesterix/hivesterix-dist/pom.xml
@@ -25,260 +25,12 @@
<dependencies>
<dependency>
- <groupId>javax.servlet</groupId>
- <artifactId>servlet-api</artifactId>
- <version>2.5</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.8.1</version>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>args4j</groupId>
- <artifactId>args4j</artifactId>
- <version>2.0.12</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.json</groupId>
- <artifactId>json</artifactId>
- <version>20090211</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.eclipse.jetty</groupId>
- <artifactId>jetty-server</artifactId>
- <version>8.0.0.M1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.eclipse.jetty</groupId>
- <artifactId>jetty-servlet</artifactId>
- <version>8.0.0.M1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>jline</groupId>
- <artifactId>jline</artifactId>
- <version>0.9.94</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-core</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-connectionpool</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-enhancer</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-rdbms</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-dbcp</groupId>
- <artifactId>commons-dbcp</artifactId>
- <version>1.4</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-pool</groupId>
- <artifactId>commons-pool</artifactId>
- <version>1.5.4</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-collections</groupId>
- <artifactId>commons-collections</artifactId>
- <version>3.2.1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-lang</groupId>
- <artifactId>commons-lang</artifactId>
- <version>2.4</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>javax</groupId>
- <artifactId>jdo2-api</artifactId>
- <version>2.3-ec</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>com.facebook</groupId>
- <artifactId>libfb303</artifactId>
- <version>0.5.0</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.thrift</groupId>
- <artifactId>libthrift</artifactId>
- <version>0.5.0</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>cli</artifactId>
- <version>1.2</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache</groupId>
- <artifactId>log4j</artifactId>
- <version>1.2.15</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.antlr</groupId>
- <artifactId>antlr-runtime</artifactId>
- <version>3.0.1</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-cli</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-common</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-exec</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-hwi</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-jdbc</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-metastore</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-service</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-shims</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
- <artifactId>hive-serde</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- <version>1.6.1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-cli</groupId>
- <artifactId>commons-cli</artifactId>
- <version>1.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
- <version>1.6.1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-logging</groupId>
- <artifactId>commons-logging</artifactId>
- <version>1.1.1</version>
- <type>jar</type>
- <classifier>api</classifier>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>com.google.guava</groupId>
- <artifactId>guava</artifactId>
- <version>r06</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.antlr</groupId>
- <artifactId>stringtemplate</artifactId>
- <version>3.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.derby</groupId>
- <artifactId>derby</artifactId>
- <version>10.8.1.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>0.20.2</version>
@@ -286,6 +38,69 @@
<scope>compile</scope>
</dependency>
<dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-cli</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-common</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-hwi</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-jdbc</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-metastore</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-service</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-shims</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-serde</artifactId>
+ <version>0.11.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hivesterix-translator</artifactId>
<version>0.2.7-SNAPSHOT</version>
diff --git a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
index d3bcaca..bfab157 100644
--- a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
+++ b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
@@ -36,16 +36,20 @@
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.MoveTask;
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.exec.Utilities;
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.OperatorDesc;
import org.apache.hadoop.hive.ql.plan.PartitionDesc;
import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.mapred.Reporter;
import edu.uci.ics.hivesterix.common.config.ConfUtil;
import edu.uci.ics.hivesterix.logical.expression.HiveExpressionTypeComputer;
@@ -214,7 +218,6 @@
// get all leave Ops
getLeaves(rootOps, leaveOps);
-
HiveAlgebricksTranslator translator = new HiveAlgebricksTranslator();
try {
translator.translate(rootOps, null, aliasToPath);
@@ -377,6 +380,12 @@
// remove map-reduce branches in condition task
ConditionalTask condition = (ConditionalTask) task;
List<Task<? extends Serializable>> branches = condition.getListTasks();
+ for (Task branch : branches) {
+ if (branch instanceof MoveTask) {
+ //return articulateMapReduceOperators(branch, rootOps, aliasToPath, rootTasks);
+ return null;
+ }
+ }
for (int i = branches.size() - 1; i >= 0; i--) {
Task branch = branches.get(i);
if (branch instanceof MapRedTask) {
@@ -396,7 +405,7 @@
MapRedTask mrtask = (MapRedTask) task;
MapredWork work = (MapredWork) mrtask.getWork();
- HashMap<String, Operator<? extends Serializable>> operators = work.getAliasToWork();
+ HashMap<String, Operator<? extends OperatorDesc>> operators = work.getAliasToWork();
Set entries = operators.entrySet();
Iterator<Entry<String, Operator>> iterator = entries.iterator();
@@ -414,7 +423,7 @@
// get map local work
MapredLocalWork localWork = work.getMapLocalWork();
if (localWork != null) {
- HashMap<String, Operator<? extends Serializable>> localOperators = localWork.getAliasToWork();
+ HashMap<String, Operator<? extends OperatorDesc>> localOperators = localWork.getAliasToWork();
Set localEntries = localOperators.entrySet();
Iterator<Entry<String, Operator>> localIterator = localEntries.iterator();
@@ -479,9 +488,9 @@
for (Operator childMap : childMapOps) {
if (childMap instanceof TableScanOperator) {
TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
- if (topDesc == null)
+ if (topDesc == null || topDesc.getAlias() == null) {
mapChildren.add(childMap);
- else {
+ } else {
rootOps.add(childMap);
}
} else {
@@ -501,9 +510,14 @@
}
i = 0;
for (Operator child : mapChildren) {
- if (child.getParentOperators() == null || child.getParentOperators().size() == 0)
+ if (child.getParentOperators() == null || child.getParentOperators().size() == 0) {
child.setParentOperators(new ArrayList<Operator>());
- child.getParentOperators().add(leafs.get(i));
+ }
+ if (i < leafs.size()) {
+ if (child.getParentOperators().size()==0) {
+ child.getParentOperators().add(leafs.get(i));
+ }
+ }
i++;
}
}
@@ -603,10 +617,10 @@
String specPath = desc.getDirName();
DynamicPartitionCtx dpCtx = desc.getDynPartCtx();
// for 0.7.0
- fsOp.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx);
+ //fsOp.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx);
// for 0.8.0
- // Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx,
- // desc);
+ //Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx, desc);
+ Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx, desc, Reporter.NULL);
}
}
}
diff --git a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
index 4ef74e9..a7ccb5a 100644
--- a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1,17 +1,3 @@
-/*
- * Copyright 2009-2013 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.
- */
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@@ -42,11 +28,13 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
@@ -62,6 +50,7 @@
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.MoveTask;
import org.apache.hadoop.hive.ql.exec.Operator;
import org.apache.hadoop.hive.ql.exec.StatsTask;
import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -86,23 +75,22 @@
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.log.PerfLogger;
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.ImportSemanticAnalyzer;
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;
@@ -112,6 +100,7 @@
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.OperatorDesc;
import org.apache.hadoop.hive.ql.plan.TableDesc;
import org.apache.hadoop.hive.ql.processors.CommandProcessor;
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -127,15 +116,18 @@
import edu.uci.ics.hivesterix.runtime.exec.HyracksExecutionEngine;
import edu.uci.ics.hivesterix.runtime.exec.IExecutionEngine;
-@SuppressWarnings({ "deprecation", "unused" })
+@SuppressWarnings({ "deprecation", "unchecked" })
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 Set<Task> executedConditionalTsks = new HashSet<Task>();
+
+ static final private Log LOG = LogFactory.getLog(Driver.class.getName());
+ static final private LogHelper console = new LogHelper(LOG);
+
+ private static final Object compileMonitor = new Object();
private int maxRows = 100;
ByteStream.Output bos = new ByteStream.Output();
@@ -152,23 +144,57 @@
// A limit on the number of threads that can be launched
private int maxthreads;
- private final int sleeptime = 2000;
-
+ private static final int SLEEP_TIME = 2000;
protected int tryCount = Integer.MAX_VALUE;
- private int checkLockManager() {
+ /**
+ * for backwards compatibility with current tests
+ */
+ public Driver(HiveConf conf) {
+ this.conf = 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);
+ }
+
+ public void clear() {
+ this.hivesterix = false;
+ this.executedConditionalTsks.clear();
+ }
+
+ private boolean checkLockManager() {
boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
- if (supportConcurrency && (hiveLockMgr == null)) {
+ if (!supportConcurrency) {
+ return false;
+ }
+ if ((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 false;
}
}
- return (0);
+ // the reason that we set the lock manager for the cxt here is because each
+ // query has its own ctx object. The hiveLockMgr is shared accross the
+ // same instance of Driver, which can run multiple queries.
+ ctx.setHiveLockMgr(hiveLockMgr);
+ return hiveLockMgr != null;
}
private void setLockManager() throws SemanticException {
@@ -183,6 +209,16 @@
hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(conf.getClassByName(lockMgr), conf);
hiveLockMgr.setContext(new HiveLockManagerCtx(conf));
} catch (Exception e) {
+ // set hiveLockMgr to null just in case this invalid manager got set to
+ // next query's ctx.
+ if (hiveLockMgr != null) {
+ try {
+ hiveLockMgr.close();
+ } catch (LockException e1) {
+ //nothing can do here
+ }
+ hiveLockMgr = null;
+ }
throw new SemanticException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + e.getMessage());
}
}
@@ -230,10 +266,8 @@
} 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.
+ // 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.
@@ -320,59 +354,102 @@
}
/**
- * 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);
- }
-
- public void clear() {
- this.hivesterix = false;
- }
-
- /**
- * Compile a new query. Any currently-planned query associated with this
- * Driver is discarded.
+ * 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) {
+ return compile(command, true);
+ }
+
+ /**
+ * Hold state variables specific to each query being executed, that may not
+ * be consistent in the overall SessionState
+ */
+ private static class QueryState {
+ private HiveOperation op;
+ private String cmd;
+ private boolean init = false;
+
+ /**
+ * Initialize the queryState with the query state variables
+ */
+ public void init(HiveOperation op, String cmd) {
+ this.op = op;
+ this.cmd = cmd;
+ this.init = true;
+ }
+
+ public boolean isInitialized() {
+ return this.init;
+ }
+
+ public HiveOperation getOp() {
+ return this.op;
+ }
+
+ public String getCmd() {
+ return this.cmd;
+ }
+ }
+
+ public void saveSession(QueryState qs) {
+ SessionState oldss = SessionState.get();
+ if (oldss != null && oldss.getHiveOperation() != null) {
+ qs.init(oldss.getHiveOperation(), oldss.getCmd());
+ }
+ }
+
+ public void restoreSession(QueryState qs) {
+ SessionState ss = SessionState.get();
+ if (ss != null && qs != null && qs.isInitialized()) {
+ ss.setCmd(qs.getCmd());
+ ss.setCommandType(qs.getOp());
+ }
+ }
+
+ /**
+ * Compile a new query, but potentially reset taskID counter. Not resetting task counter
+ * is useful for generating re-entrant QL queries.
+ *
+ * @param command
+ * The HiveQL query to compile
+ * @param resetTaskIds
+ * Resets taskID counter if true.
+ * @return 0 for ok
+ */
+ public int compile(String command, boolean resetTaskIds) {
+ PerfLogger perfLogger = PerfLogger.getPerfLogger();
+ perfLogger.PerfLogBegin(LOG, PerfLogger.COMPILE);
+
+ //holder for parent command type/string when executing reentrant queries
+ QueryState queryState = new QueryState();
+
if (plan != null) {
close();
plan = null;
}
- TaskFactory.resetId();
+ if (resetTaskIds) {
+ TaskFactory.resetId();
+ }
+ saveSession(queryState);
try {
command = new VariableSubstitution().substitute(conf, command);
ctx = new Context(conf);
+ ctx.setTryCount(getTryCount());
+ ctx.setCmd(command);
+ ctx.setHDFSCleanup(true);
ParseDriver pd = new ParseDriver();
ASTNode tree = pd.parse(command, ctx);
tree = ParseUtils.findRootNonNullToken(tree);
BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
- List<AbstractSemanticAnalyzerHook> saHooks = getSemanticAnalyzerHooks();
+ List<AbstractSemanticAnalyzerHook> saHooks = getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK,
+ AbstractSemanticAnalyzerHook.class);
// Do semantic analysis and plan generation
if (saHooks != null) {
@@ -382,6 +459,7 @@
tree = hook.preAnalyze(hookCtx, tree);
}
sem.analyze(tree, ctx);
+ hookCtx.update(sem);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
hook.postAnalyze(hookCtx, sem.getRootTasks());
}
@@ -394,19 +472,10 @@
// 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);
+ plan = new QueryPlan(command, sem, perfLogger.getStartTime(PerfLogger.DRIVER_RUN));
// test Only - serialize the query plan and deserialize it
- if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
-
- Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ if ("true".equalsIgnoreCase(System.getProperty("test.serialize.qplan"))) {
String queryPlanFileName = ctx.getLocalScratchDir(true) + Path.SEPARATOR_CHAR + "queryplan.xml";
LOG.info("query plan = " + queryPlanFileName);
@@ -431,17 +500,24 @@
plan.getFetchTask().initialize(conf, plan, null);
}
- // do the authorization check
+ // get the output schema
+ schema = getSchema(sem, conf);
+
+ //do the authorization check
if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
try {
- // doAuthorization(sem);
+ perfLogger.PerfLogBegin(LOG, PerfLogger.DO_AUTHORIZATION);
+ doAuthorization(sem);
} catch (AuthorizationException authExp) {
console.printError("Authorization failed:" + authExp.getMessage()
+ ". Use show grant to get more details.");
return 403;
+ } finally {
+ perfLogger.PerfLogEnd(LOG, PerfLogger.DO_AUTHORIZATION);
}
}
+ //restore state after we're done executing a specific query
// hyracks run
if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
int engineRet = engine.compileJob(sem.getRootTasks());
@@ -450,21 +526,19 @@
}
}
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);
+ ErrorMsg error = ErrorMsg.getErrorMsg(e.getMessage());
+ errorMessage = "FAILED: " + e.getClass().getSimpleName();
+ if (error != ErrorMsg.GENERIC_ERROR) {
+ errorMessage += " [Error " + error.getErrorCode() + "]:";
+ }
+ errorMessage += " " + e.getMessage();
+ SQLState = error.getSQLState();
+ console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ return error.getErrorCode();
+ } finally {
+ perfLogger.PerfLogEnd(LOG, PerfLogger.COMPILE);
+ restoreSession(queryState);
}
}
@@ -479,13 +553,13 @@
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 (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) {
@@ -513,8 +587,8 @@
Map<String, Boolean> tableUsePartLevelAuth = new HashMap<String, Boolean>();
for (ReadEntity read : inputs) {
- if (read.getPartition() != null) {
- Table tbl = read.getTable();
+ Table tbl = read.getTable();
+ if ((read.getPartition() != null) || (tbl.isPartitioned())) {
String tblName = tbl.getTableName();
if (tableUsePartLevelAuth.get(tblName) == null) {
boolean usePartLevelPriv = (tbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE") != null && ("TRUE"
@@ -533,9 +607,9 @@
ParseContext parseCtx = querySem.getParseContext();
Map<TableScanOperator, Table> tsoTopMap = parseCtx.getTopToTable();
- for (Map.Entry<String, Operator<? extends Serializable>> topOpMap : querySem.getParseContext()
+ for (Map.Entry<String, Operator<? extends OperatorDesc>> topOpMap : querySem.getParseContext()
.getTopOps().entrySet()) {
- Operator<? extends Serializable> topOp = topOpMap.getValue();
+ Operator<? extends OperatorDesc> topOp = topOpMap.getValue();
if (topOp instanceof TableScanOperator && tsoTopMap.containsKey(topOp)) {
TableScanOperator tableScanOp = (TableScanOperator) topOp;
Table tbl = tsoTopMap.get(tableScanOp);
@@ -551,7 +625,10 @@
cols.add(columns.get(i).getName());
}
}
- if (tbl.isPartitioned() && tableUsePartLevelAuth.get(tbl.getTableName())) {
+ //map may not contain all sources, since input list may have been optimized out
+ //or non-existent tho such sources may still be referenced by the TableScanOperator
+ //if it's null then the partition probably doesn't exist so let's use table permission
+ if (tbl.isPartitioned() && tableUsePartLevelAuth.get(tbl.getTableName()) == Boolean.TRUE) {
String alias_id = topOpMap.getKey();
PrunedPartitionList partsList = PartitionPruner.prune(parseCtx.getTopToTable().get(topOp),
parseCtx.getOpToPartPruner().get(topOp), parseCtx.getConf(), alias_id,
@@ -582,30 +659,28 @@
// cache the results for table authorization
Set<String> tableAuthChecked = new HashSet<String>();
for (ReadEntity read : inputs) {
- Table tbl = null;
+ Table tbl = read.getTable();
if (read.getPartition() != null) {
- tbl = read.getPartition().getTable();
+ Partition partition = read.getPartition();
+ tbl = partition.getTable();
// use partition level authorization
- if (tableUsePartLevelAuth.get(tbl.getTableName())) {
- List<String> cols = part2Cols.get(read.getPartition());
+ if (tableUsePartLevelAuth.get(tbl.getTableName()) == Boolean.TRUE) {
+ List<String> cols = part2Cols.get(partition);
if (cols != null && cols.size() > 0) {
- ss.getAuthorizer().authorize(read.getPartition().getTable(), read.getPartition(), cols,
+ ss.getAuthorizer().authorize(partition.getTable(), partition, cols,
op.getInputRequiredPrivileges(), null);
} else {
- ss.getAuthorizer().authorize(read.getPartition(), op.getInputRequiredPrivileges(), null);
+ ss.getAuthorizer().authorize(partition, 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
+ // 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())) {
+ if (tbl != null && !tableAuthChecked.contains(tbl.getTableName())
+ && !(tableUsePartLevelAuth.get(tbl.getTableName()) == Boolean.TRUE)) {
List<String> cols = tab2Cols.get(tbl);
if (cols != null && cols.size() > 0) {
ss.getAuthorizer().authorize(tbl, null, cols, op.getInputRequiredPrivileges(), null);
@@ -632,16 +707,15 @@
* @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.
+ * 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");
+ .currentTimeMillis()), "IMPLICIT", plan.getQueryStr());
if (t != null) {
locks.add(new HiveLockObj(new HiveLockObject(t, lockData), mode));
@@ -665,16 +739,20 @@
name = p.getName().split("@")[2];
}
- String partName = name;
String partialName = "";
String[] partns = name.split("/");
int len = p instanceof DummyPartition ? partns.length : partns.length - 1;
+ Map<String, String> partialSpec = new LinkedHashMap<String, String>();
for (int idx = 0; idx < len; idx++) {
String partn = partns[idx];
partialName += partn;
+ String[] nameValue = partn.split("=");
+ assert (nameValue.length == 2);
+ partialSpec.put(nameValue[0], nameValue[1]);
try {
locks.add(new HiveLockObj(new HiveLockObject(new DummyPartition(p.getTable(), p.getTable()
- .getDbName() + "/" + p.getTable().getTableName() + "/" + partialName), lockData), mode));
+ .getDbName() + "/" + p.getTable().getTableName() + "/" + partialName, partialSpec),
+ lockData), mode));
partialName += "/";
} catch (HiveException e) {
throw new SemanticException(e.getMessage());
@@ -688,17 +766,16 @@
}
/**
- * 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.
+ * 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);
+ PerfLogger perfLogger = PerfLogger.getPerfLogger();
+ perfLogger.PerfLogBegin(LOG, PerfLogger.ACQUIRE_READ_WRITE_LOCKS);
+ try {
boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
if (!supportConcurrency) {
return 0;
@@ -707,8 +784,7 @@
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
+ // 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) {
@@ -719,16 +795,21 @@
}
for (WriteEntity output : plan.getOutputs()) {
+ List<HiveLockObj> lockObj = null;
if (output.getTyp() == WriteEntity.Type.TABLE) {
- lockObjects.addAll(getLockObjects(output.getTable(), null,
- output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED));
+ lockObj = 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));
+ lockObj = getLockObjects(null, output.getPartition(), HiveLockMode.EXCLUSIVE);
}
- // In case of dynamic queries, it is possible to have incomplete
- // dummy partitions
+ // 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));
+ lockObj = getLockObjects(null, output.getPartition(), HiveLockMode.SHARED);
+ }
+
+ if (lockObj != null) {
+ lockObjects.addAll(lockObj);
+ ctx.getOutputLockObjects().put(output, lockObj);
}
}
@@ -736,13 +817,8 @@
return 0;
}
- int ret = checkLockManager();
- if (ret != 0) {
- return ret;
- }
-
HiveLockObjectData lockData = new HiveLockObjectData(plan.getQueryId(), String.valueOf(System
- .currentTimeMillis()), "IMPLICIT");
+ .currentTimeMillis()), "IMPLICIT", plan.getQueryStr());
// Lock the database also
try {
@@ -753,25 +829,7 @@
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);
+ List<HiveLock> hiveLocks = ctx.getHiveLockMgr().lock(lockObjects, false);
if (hiveLocks == null) {
throw new SemanticException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg());
@@ -785,138 +843,207 @@
SQLState = ErrorMsg.findSQLState(e.getMessage());
console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
return (10);
- } catch (Exception e) {
+ } catch (LockException 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) {
- }
+ } finally {
+ perfLogger.PerfLogEnd(LOG, PerfLogger.ACQUIRE_READ_WRITE_LOCKS);
}
}
/**
* @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
+ * 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) {
+ PerfLogger perfLogger = PerfLogger.getPerfLogger();
+ perfLogger.PerfLogBegin(LOG, PerfLogger.RELEASE_LOCKS);
+
if (hiveLocks != null) {
ctx.getHiveLockMgr().releaseLocks(hiveLocks);
}
ctx.setHiveLocks(null);
+
+ perfLogger.PerfLogEnd(LOG, PerfLogger.RELEASE_LOCKS);
}
- public CommandProcessorResponse run(String command) {
+ public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
errorMessage = null;
SQLState = null;
- int ret = compile(command);
+ if (!validateConfVariables()) {
+ return new CommandProcessorResponse(12, errorMessage, SQLState);
+ }
+
+ HiveDriverRunHookContext hookContext = new HiveDriverRunHookContextImpl(conf, command);
+ // Get all the driver run hooks and pre-execute them.
+ List<HiveDriverRunHook> driverRunHooks;
+ try {
+ driverRunHooks = getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS, HiveDriverRunHook.class);
+ for (HiveDriverRunHook driverRunHook : driverRunHooks) {
+ driverRunHook.preDriverRun(hookContext);
+ }
+ } 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 new CommandProcessorResponse(12, errorMessage, SQLState);
+ }
+
+ // Reset the perf logger
+ PerfLogger perfLogger = PerfLogger.getPerfLogger(true);
+ perfLogger.PerfLogBegin(LOG, PerfLogger.DRIVER_RUN);
+ perfLogger.PerfLogBegin(LOG, PerfLogger.TIME_TO_SUBMIT);
+
+ int ret;
+ synchronized (compileMonitor) {
+ ret = compile(command);
+ }
if (ret != 0) {
- // releaseLocks(ctx.getHiveLocks());
+ releaseLocks(ctx.getHiveLocks());
return new CommandProcessorResponse(ret, errorMessage, SQLState);
}
- // ret = acquireReadWriteLocks();
- if (ret != 0) {
- // releaseLocks(ctx.getHiveLocks());
- return new CommandProcessorResponse(ret, errorMessage, SQLState);
+ boolean requireLock = false;
+ boolean ckLock = checkLockManager();
+
+ if (ckLock) {
+ boolean lockOnlyMapred = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_LOCK_MAPRED_ONLY);
+ if (lockOnlyMapred) {
+ Queue<Task<? extends Serializable>> taskQueue = new LinkedList<Task<? extends Serializable>>();
+ taskQueue.addAll(plan.getRootTasks());
+ while (taskQueue.peek() != null) {
+ Task<? extends Serializable> tsk = taskQueue.remove();
+ requireLock = requireLock || tsk.requireLock();
+ if (requireLock) {
+ break;
+ }
+ if (tsk instanceof ConditionalTask) {
+ taskQueue.addAll(((ConditionalTask) tsk).getListTasks());
+ }
+ if (tsk.getChildTasks() != null) {
+ taskQueue.addAll(tsk.getChildTasks());
+ }
+ // does not add back up task here, because back up task should be the same
+ // type of the original task.
+ }
+ } else {
+ requireLock = true;
+ }
+ }
+
+ if (requireLock) {
+ ret = acquireReadWriteLocks();
+ if (ret != 0) {
+ releaseLocks(ctx.getHiveLocks());
+ return new CommandProcessorResponse(ret, errorMessage, SQLState);
+ }
}
ret = execute();
if (ret != 0) {
- // releaseLocks(ctx.getHiveLocks());
+ //if needRequireLock is false, the release here will do nothing because there is no lock
+ releaseLocks(ctx.getHiveLocks());
return new CommandProcessorResponse(ret, errorMessage, SQLState);
}
- // releaseLocks(ctx.getHiveLocks());
+ //if needRequireLock is false, the release here will do nothing because there is no lock
+ releaseLocks(ctx.getHiveLocks());
+
+ perfLogger.PerfLogEnd(LOG, PerfLogger.DRIVER_RUN);
+ perfLogger.close(LOG, plan);
+
+ // Take all the driver run hooks and post-execute them.
+ try {
+ for (HiveDriverRunHook driverRunHook : driverRunHooks) {
+ driverRunHook.postDriverRun(hookContext);
+ }
+ } 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 new CommandProcessorResponse(12, errorMessage, SQLState);
+ }
+
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;
+ /**
+ * Validate configuration variables.
+ *
+ * @return
+ */
+ private boolean validateConfVariables() {
+ boolean valid = true;
+ if ((!conf.getBoolVar(HiveConf.ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES))
+ && ((conf.getBoolVar(HiveConf.ConfVars.HADOOPMAPREDINPUTDIRRECURSIVE))
+ || (conf.getBoolVar(HiveConf.ConfVars.HIVEOPTLISTBUCKETING)) || ((conf
+ .getBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_UNION_REMOVE))))) {
+ errorMessage = "FAILED: Hive Internal Error: " + ErrorMsg.SUPPORT_DIR_MUST_TRUE_FOR_LIST_BUCKETING.getMsg();
+ SQLState = ErrorMsg.findSQLState(errorMessage);
+ console.printError(errorMessage + "\n");
+ valid = false;
}
- 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;
+ return valid;
}
- 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;
+ /**
+ * Returns a set of hooks specified in a configuration variable.
+ * See getHooks(HiveConf.ConfVars hookConfVar, Class<T> clazz)
+ *
+ * @param hookConfVar
+ * @return
+ * @throws Exception
+ */
+ private List<Hook> getHooks(HiveConf.ConfVars hookConfVar) throws Exception {
+ return getHooks(hookConfVar, Hook.class);
+ }
+
+ /**
+ * Returns the hooks specified in a configuration variable. The hooks are returned in a list in
+ * the order they were specified in the configuration variable.
+ *
+ * @param hookConfVar
+ * The configuration variable specifying a comma separated list of the hook
+ * class names.
+ * @param clazz
+ * The super type of the hooks.
+ * @return A list of the hooks cast as the type specified in clazz, in the order
+ * they are listed in the value of hookConfVar
+ * @throws Exception
+ */
+ private <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar, Class<T> clazz) throws Exception {
+
+ List<T> hooks = new ArrayList<T>();
+ String csHooks = conf.getVar(hookConfVar);
+ if (csHooks == null) {
+ return hooks;
}
- String[] peClasses = pestr.split(",");
+ csHooks = csHooks.trim();
+ if (csHooks.equals("")) {
+ return hooks;
+ }
- for (String peClass : peClasses) {
+ String[] hookClasses = csHooks.split(",");
+
+ for (String hookClass : hookClasses) {
try {
- pehooks.add((Hook) Class.forName(peClass.trim(), true, JavaUtils.getClassLoader()).newInstance());
+ T hook = (T) Class.forName(hookClass.trim(), true, JavaUtils.getClassLoader()).newInstance();
+ hooks.add(hook);
} catch (ClassNotFoundException e) {
- console.printError("Pre Exec Hook Class not found:" + e.getMessage());
+ console.printError(hookConfVar.varname + " Class not found:" + e.getMessage());
throw e;
}
}
- return pehooks;
+ return hooks;
}
- 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() {
+ public int execute() throws CommandNeedRetryException {
// execute hivesterix plan
if (hivesterix) {
hivesterix = false;
@@ -925,6 +1052,9 @@
return ret;
}
+ PerfLogger perfLogger = PerfLogger.getPerfLogger();
+ perfLogger.PerfLogBegin(LOG, PerfLogger.DRIVER_EXECUTE);
+
boolean noName = StringUtils.isEmpty(conf.getVar(HiveConf.ConfVars.HADOOPJOBNAME));
int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH);
@@ -933,6 +1063,10 @@
conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, queryStr);
+
+ conf.set("mapreduce.workflow.id", "hive_" + queryId);
+ conf.set("mapreduce.workflow.name", queryStr);
+
maxthreads = HiveConf.getIntVar(conf, HiveConf.ConfVars.EXECPARALLETHREADNUMBER);
try {
@@ -946,14 +1080,23 @@
}
resStream = null;
- HookContext hookContext = new HookContext(plan, conf);
+ HookContext hookContext = new HookContext(plan, conf, ctx.getPathToCS());
+ hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);
- for (Hook peh : getPreExecHooks()) {
+ for (Hook peh : getHooks(HiveConf.ConfVars.PREEXECHOOKS)) {
if (peh instanceof ExecuteWithHookContext) {
+ perfLogger.PerfLogBegin(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
+
((ExecuteWithHookContext) peh).run(hookContext);
+
+ perfLogger.PerfLogEnd(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
} else if (peh instanceof PreExecute) {
+ perfLogger.PerfLogBegin(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
+
((PreExecute) peh).run(SessionState.get(), plan.getInputs(), plan.getOutputs(), ShimLoader
.getHadoopShims().getUGIForConf(conf));
+
+ perfLogger.PerfLogEnd(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
}
}
@@ -968,32 +1111,36 @@
}
String jobname = Utilities.abbreviate(queryStr, maxlen - 6);
- // A runtime that launches runnable tasks as separate Threads
- // through
+ // 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.
+ // The main thread polls the TaskRunners to check if they have finished.
- Queue<Task<? extends Serializable>> runnable = new LinkedList<Task<? extends Serializable>>();
+ Queue<Task<? extends Serializable>> runnable = new ConcurrentLinkedQueue<Task<? extends Serializable>>();
Map<TaskResult, TaskRunner> running = new HashMap<TaskResult, TaskRunner>();
DriverContext driverCxt = new DriverContext(runnable, ctx);
+ ctx.setHDFSCleanup(true);
+
+ SessionState.get().setLastMapRedStatsList(new ArrayList<MapRedStats>());
+ SessionState.get().setStackTraces(new HashMap<String, List<List<String>>>());
+ SessionState.get().setLocalMapRedErrors(new HashMap<String, List<String>>());
// Add root Tasks to runnable
-
for (Task<? extends Serializable> tsk : plan.getRootTasks()) {
+ // This should never happen, if it does, it's a bug with the potential to produce
+ // incorrect results.
+ assert tsk.getParentTasks() == null || tsk.getParentTasks().isEmpty();
driverCxt.addToRunnable(tsk);
}
+ perfLogger.PerfLogEnd(LOG, PerfLogger.TIME_TO_SUBMIT);
// 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);
}
@@ -1005,12 +1152,24 @@
int exitVal = tskRes.getExitVal();
if (exitVal != 0) {
+ if (tsk.ifRetryCmdWhenFail()) {
+ if (!running.isEmpty()) {
+ taskCleanup(running);
+ }
+ // in case we decided to run everything in local mode, restore the
+ // the jobtracker setting to its initial value
+ ctx.restoreOriginalTracker();
+ throw new CommandNeedRetryException();
+ }
Task<? extends Serializable> backupTask = tsk.getAndInitBackupTask();
if (backupTask != null) {
errorMessage = "FAILED: Execution Error, return code " + exitVal + " from "
+ tsk.getClass().getName();
+ ErrorMsg em = ErrorMsg.getErrorMsg(exitVal);
+ if (em != null) {
+ errorMessage += ". " + em.getMsg();
+ }
console.printError(errorMessage);
-
errorMessage = "ATTEMPT: Execute BackupTask: " + backupTask.getClass().getName();
console.printError(errorMessage);
@@ -1021,20 +1180,31 @@
continue;
} else {
- // TODO: This error messaging is not very informative.
- // Fix that.
+ hookContext.setHookType(HookContext.HookType.ON_FAILURE_HOOK);
+ // Get all the failure execution hooks and execute them.
+ for (Hook ofh : getHooks(HiveConf.ConfVars.ONFAILUREHOOKS)) {
+ perfLogger.PerfLogBegin(LOG, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
+
+ ((ExecuteWithHookContext) ofh).run(hookContext);
+
+ perfLogger.PerfLogEnd(LOG, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
+ }
+
errorMessage = "FAILED: Execution Error, return code " + exitVal + " from "
+ tsk.getClass().getName();
+ ErrorMsg em = ErrorMsg.getErrorMsg(exitVal);
+ if (em != null) {
+ errorMessage += ". " + em.getMsg();
+ }
SQLState = "08S01";
console.printError(errorMessage);
- if (running.size() != 0) {
- taskCleanup();
+ if (!running.isEmpty()) {
+ taskCleanup(running);
}
- // in case we decided to run everything in local mode,
- // restore the
+ // in case we decided to run everything in local mode, restore the
// the jobtracker setting to its initial value
ctx.restoreOriginalTracker();
- return 9;
+ return exitVal;
}
}
@@ -1047,9 +1217,9 @@
if (tsk.getChildTasks() != null) {
for (Task<? extends Serializable> child : tsk.getChildTasks()) {
// hivesterix: don't check launchable condition
- // if (DriverContext.isLaunchable(child)) {
+ //if(DriverContext.isLaunchable(tsk)){
driverCxt.addToRunnable(child);
- // }
+ //}
}
}
}
@@ -1059,8 +1229,7 @@
ctx.restoreOriginalTracker();
// remove incomplete outputs.
- // Some incomplete outputs may be added at the beginning, for eg:
- // for dynamic partitions.
+ // 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()) {
@@ -1073,15 +1242,24 @@
plan.getOutputs().remove(output);
}
+ hookContext.setHookType(HookContext.HookType.POST_EXEC_HOOK);
// Get all the post execution hooks and execute them.
- for (Hook peh : getPostExecHooks()) {
+ for (Hook peh : getHooks(HiveConf.ConfVars.POSTEXECHOOKS)) {
if (peh instanceof ExecuteWithHookContext) {
+ perfLogger.PerfLogBegin(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
+
((ExecuteWithHookContext) peh).run(hookContext);
+
+ perfLogger.PerfLogEnd(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
} else if (peh instanceof PostExecute) {
+ perfLogger.PerfLogBegin(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
+
((PostExecute) peh)
.run(SessionState.get(), plan.getInputs(), plan.getOutputs(),
(SessionState.get() != null ? SessionState.get().getLineageState().getLineageInfo()
: null), ShimLoader.getHadoopShims().getUGIForConf(conf));
+
+ perfLogger.PerfLogEnd(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
}
}
@@ -1089,7 +1267,10 @@
SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE, String.valueOf(0));
SessionState.get().getHiveHistory().printRowCount(queryId);
}
+ } catch (CommandNeedRetryException e) {
+ throw e;
} catch (Exception e) {
+ ctx.restoreOriginalTracker();
if (SessionState.get() != null) {
SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE, String.valueOf(12));
}
@@ -1105,6 +1286,18 @@
if (noName) {
conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, "");
}
+ perfLogger.PerfLogEnd(LOG, PerfLogger.DRIVER_EXECUTE);
+
+ if (SessionState.get().getLastMapRedStatsList() != null
+ && SessionState.get().getLastMapRedStatsList().size() > 0) {
+ long totalCpu = 0;
+ console.printInfo("MapReduce Jobs Launched: ");
+ for (int i = 0; i < SessionState.get().getLastMapRedStatsList().size(); i++) {
+ console.printInfo("Job " + i + ": " + SessionState.get().getLastMapRedStatsList().get(i));
+ totalCpu += SessionState.get().getLastMapRedStatsList().get(i).getCpuMSec();
+ }
+ console.printInfo("Total MapReduce CPU Time Spent: " + Utilities.formatMsecToStr(totalCpu));
+ }
}
plan.setDone();
@@ -1134,14 +1327,12 @@
* 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
+ * @param cxt
+ * the driver context
*/
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());
}
@@ -1149,6 +1340,8 @@
if (noName) {
conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, jobname + "(" + tsk.getId() + ")");
}
+ conf.set("mapreduce.workflow.node.name", tsk.getId());
+ Utilities.setWorkflowAdjacencies(conf, plan);
cxt.incCurJobNo(1);
console.printInfo("Launching Job " + cxt.getCurJobNo() + " out of " + jobs);
}
@@ -1156,7 +1349,13 @@
TaskResult tskRes = new TaskResult();
TaskRunner tskRun = new TaskRunner(tsk, tskRes);
- // HiveConf.getBoolVar(conf, HiveConf.ConfVars.EXECPARALLEL) &&
+ // Launch Task
+ //if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.EXECPARALLEL) && tsk.isMapRedTask()) {
+ // Launch it in the parallel mode, as a separate thread only for MR tasks
+ // tskRun.start();
+ //} else {
+ // tskRun.runSequential();
+ //}
// Launch Task: hivesterix tweak
if (tsk instanceof MapRedTask || tsk instanceof StatsTask) {
// Launch it in the parallel mode, as a separate thread only for MR
@@ -1169,11 +1368,27 @@
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);
+ if (!executedConditionalTsks.contains(tsk)) {
+ List<Task<? extends Serializable>> children = condTask.getListTasks();
+ Task<? extends Serializable> selectedBranch = null;
+ for (Task<? extends Serializable> branch : children) {
+ if (branch instanceof MoveTask) {
+ selectedBranch = branch;
+ break;
+ }
+ }
+ if (selectedBranch == null) {
+ for (int i = children.size() - 1; i >= 0; i--) {
+ Task<? extends Serializable> child = children.get(i);
+ if (child instanceof MapRedTask) {
+ selectedBranch = child;
+ break;
+ }
+ }
+ }
+ executedConditionalTsks.add(tsk);
+ cxt.addToRunnable(selectedBranch);
+ }
}
} else {
tskRun.runSequential();
@@ -1185,12 +1400,18 @@
/**
* 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);
+ public void taskCleanup(Map<TaskResult, TaskRunner> running) {
+ for (Map.Entry<TaskResult, TaskRunner> entry : running.entrySet()) {
+ if (entry.getKey().isRunning()) {
+ Task<?> task = entry.getValue().getTask();
+ try {
+ task.shutdown();
+ } catch (Exception e) {
+ console.printError("Exception on shutting down task " + task.getId() + ": " + e);
+ }
+ }
+ }
+ running.clear();
}
/**
@@ -1214,7 +1435,7 @@
// In this loop, nothing was found
// Sleep 10 seconds and restart
try {
- Thread.sleep(sleeptime);
+ Thread.sleep(SLEEP_TIME);
} catch (InterruptedException ie) {
// Do Nothing
;
@@ -1223,7 +1444,7 @@
}
}
- public boolean getResults(ArrayList<String> res) throws IOException {
+ public boolean getResults(ArrayList<String> res) throws IOException, CommandNeedRetryException {
if (plan != null && plan.getFetchTask() != null) {
FetchTask ft = plan.getFetchTask();
ft.setMaxRows(maxRows);
@@ -1276,6 +1497,14 @@
return true;
}
+ public int getTryCount() {
+ return tryCount;
+ }
+
+ public void setTryCount(int tryCount) {
+ this.tryCount = tryCount;
+ }
+
public int close() {
try {
if (plan != null) {
@@ -1308,18 +1537,21 @@
}
public void destroy() {
- releaseLocks();
+ if (ctx != null) {
+ releaseLocks(ctx.getHiveLocks());
+ }
+
+ if (hiveLockMgr != null) {
+ try {
+ hiveLockMgr.close();
+ } catch (LockException e) {
+ LOG.warn("Exception in closing hive lock manager. "
+ + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ }
+ }
}
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/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
index 2d5191d..1b96259 100644
--- a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
+++ b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
@@ -68,6 +68,7 @@
@Override
public GenericUDAFEvaluator getEvaluator(GenericUDAFParameterInfo paramInfo) throws SemanticException {
+ @SuppressWarnings("deprecation")
TypeInfo[] parameters = paramInfo.getParameters();
if (parameters.length == 0) {
diff --git a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
index 0fea4b9..e26f477 100644
--- a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
+++ b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
@@ -36,7 +36,7 @@
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
-import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde.serdeConstants;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
/**
@@ -65,15 +65,18 @@
return result;
}
- public static final TypeInfo voidTypeInfo = getPrimitiveTypeInfo(Constants.VOID_TYPE_NAME);
- public static final TypeInfo booleanTypeInfo = getPrimitiveTypeInfo(Constants.BOOLEAN_TYPE_NAME);
- public static final TypeInfo intTypeInfo = getPrimitiveTypeInfo(Constants.INT_TYPE_NAME);
- public static final TypeInfo longTypeInfo = getPrimitiveTypeInfo(Constants.BIGINT_TYPE_NAME);
- public static final TypeInfo stringTypeInfo = getPrimitiveTypeInfo(Constants.STRING_TYPE_NAME);
- public static final TypeInfo floatTypeInfo = getPrimitiveTypeInfo(Constants.FLOAT_TYPE_NAME);
- public static final TypeInfo doubleTypeInfo = getPrimitiveTypeInfo(Constants.DOUBLE_TYPE_NAME);
- public static final TypeInfo byteTypeInfo = getPrimitiveTypeInfo(Constants.TINYINT_TYPE_NAME);
- public static final TypeInfo shortTypeInfo = getPrimitiveTypeInfo(Constants.SMALLINT_TYPE_NAME);
+ public static final TypeInfo voidTypeInfo = getPrimitiveTypeInfo(serdeConstants.VOID_TYPE_NAME);
+ public static final TypeInfo booleanTypeInfo = getPrimitiveTypeInfo(serdeConstants.BOOLEAN_TYPE_NAME);
+ public static final TypeInfo intTypeInfo = getPrimitiveTypeInfo(serdeConstants.INT_TYPE_NAME);
+ public static final TypeInfo longTypeInfo = getPrimitiveTypeInfo(serdeConstants.BIGINT_TYPE_NAME);
+ public static final TypeInfo stringTypeInfo = getPrimitiveTypeInfo(serdeConstants.STRING_TYPE_NAME);
+ public static final TypeInfo floatTypeInfo = getPrimitiveTypeInfo(serdeConstants.FLOAT_TYPE_NAME);
+ public static final TypeInfo doubleTypeInfo = getPrimitiveTypeInfo(serdeConstants.DOUBLE_TYPE_NAME);
+ public static final TypeInfo byteTypeInfo = getPrimitiveTypeInfo(serdeConstants.TINYINT_TYPE_NAME);
+ public static final TypeInfo shortTypeInfo = getPrimitiveTypeInfo(serdeConstants.SMALLINT_TYPE_NAME);
+ public static final TypeInfo timestampTypeInfo = getPrimitiveTypeInfo(serdeConstants.TIMESTAMP_TYPE_NAME);
+ public static final TypeInfo binaryTypeInfo = getPrimitiveTypeInfo(serdeConstants.BINARY_TYPE_NAME);
+ public static final TypeInfo decimalTypeInfo = getPrimitiveTypeInfo(serdeConstants.DECIMAL_TYPE_NAME);
public static final TypeInfo unknownTypeInfo = getPrimitiveTypeInfo("unknown");
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java
index b024269..5fdfb3a 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java
@@ -12,77 +12,77 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hivesterix.test.runtimefunction;
-
-import java.io.File;
-import java.util.List;
-
-import junit.framework.Test;
-import junit.framework.TestResult;
-import edu.uci.ics.hivesterix.test.base.AbstractTestSuiteClass;
-
-public class RuntimeFunctionTestSuite extends AbstractTestSuiteClass {
-
- private static final String PATH_TO_QUERIES = "src/test/resources/runtimefunctionts/queries/";
- private static final String PATH_TO_RESULTS = "src/test/resources/runtimefunctionts/results/";
- private static final String PATH_TO_IGNORES = "src/test/resources/runtimefunctionts/ignore.txt";
-
- private static final String FILE_EXTENSION_OF_RESULTS = "result";
-
- public static Test suite() throws Exception {
- List<String> ignores = getIgnoreList(PATH_TO_IGNORES);
- File testData = new File(PATH_TO_QUERIES);
- File[] queries = testData.listFiles();
- RuntimeFunctionTestSuite testSuite = new RuntimeFunctionTestSuite();
-
- // set hdfs and hyracks cluster, and load test data to hdfs
- try {
- testSuite.setup();
- testSuite.loadData();
- } catch (Exception e) {
- e.printStackTrace();
- throw new IllegalStateException(e.getMessage());
- }
-
- for (File qFile : queries) {
- if (isIgnored(qFile.getName(), ignores))
- continue;
-
- if (qFile.isFile()) {
- String resultFileName = hiveExtToResExt(qFile.getName());
- File rFile = new File(PATH_TO_RESULTS + resultFileName);
- testSuite.addTest(new RuntimeFunctionTestCase(qFile, rFile));
- }
- }
- return testSuite;
- }
-
- private static String hiveExtToResExt(String fname) {
- int dot = fname.lastIndexOf('.');
- return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
- }
-
- /**
- * Runs the tests and collects their result in a TestResult.
- */
- @Override
- public void run(TestResult result) {
-
- int testCount = countTestCases();
- for (int i = 0; i < testCount; i++) {
- Test each = this.testAt(i);
- if (result.shouldStop())
- break;
- runTest(each, result);
- }
-
- // cleanup hdfs and hyracks cluster
- try {
- cleanup();
- } catch (Exception e) {
- e.printStackTrace();
- throw new IllegalStateException(e.getMessage());
- }
- }
-
-}
+package edu.uci.ics.hivesterix.test.runtimefunction;
+
+import java.io.File;
+import java.util.List;
+
+import junit.framework.Test;
+import junit.framework.TestResult;
+import edu.uci.ics.hivesterix.test.base.AbstractTestSuiteClass;
+
+public class RuntimeFunctionTestSuite extends AbstractTestSuiteClass {
+
+ private static final String PATH_TO_QUERIES = "src/test/resources/runtimefunctionts/queries/";
+ private static final String PATH_TO_RESULTS = "src/test/resources/runtimefunctionts/results/";
+ private static final String PATH_TO_IGNORES = "src/test/resources/runtimefunctionts/ignore.txt";
+
+ private static final String FILE_EXTENSION_OF_RESULTS = "result";
+
+ public static Test suite() throws Exception {
+ List<String> ignores = getIgnoreList(PATH_TO_IGNORES);
+ File testData = new File(PATH_TO_QUERIES);
+ File[] queries = testData.listFiles();
+ RuntimeFunctionTestSuite testSuite = new RuntimeFunctionTestSuite();
+
+ // set hdfs and hyracks cluster, and load test data to hdfs
+ try {
+ testSuite.setup();
+ testSuite.loadData();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new IllegalStateException(e.getMessage());
+ }
+
+ for (File qFile : queries) {
+ if (isIgnored(qFile.getName(), ignores))
+ continue;
+
+ if (qFile.isFile()) {
+ String resultFileName = hiveExtToResExt(qFile.getName());
+ File rFile = new File(PATH_TO_RESULTS + resultFileName);
+ testSuite.addTest(new RuntimeFunctionTestCase(qFile, rFile));
+ }
+ }
+ return testSuite;
+ }
+
+ private static String hiveExtToResExt(String fname) {
+ int dot = fname.lastIndexOf('.');
+ return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
+ }
+
+ /**
+ * Runs the tests and collects their result in a TestResult.
+ */
+ @Override
+ public void run(TestResult result) {
+
+ int testCount = countTestCases();
+ for (int i = 0; i < testCount; i++) {
+ Test each = this.testAt(i);
+ if (result.shouldStop())
+ break;
+ runTest(each, result);
+ }
+
+ // cleanup hdfs and hyracks cluster
+ try {
+ cleanup();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new IllegalStateException(e.getMessage());
+ }
+ }
+
+}
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q10_returned_item.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q10_returned_item.hive
index 3f1214a..bb07665 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q10_returned_item.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q10_returned_item.hive
@@ -13,9 +13,6 @@
-- create the result table
create table q10_returned_item (c_custkey int, c_name string, revenue double, c_acctbal string, n_name string, c_address string, c_phone string, c_comment string);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1024000000;
-
-- the query
insert overwrite table q10_returned_item
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q12_shipping.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q12_shipping.hive
index 062f7b9..ae2fa3a 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q12_shipping.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q12_shipping.hive
@@ -9,8 +9,6 @@
-- create the result table
create table q12_shipping(l_shipmode string, high_line_count double, low_line_count double);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1225000000;
-- the query
insert overwrite table q12_shipping
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q14_promotion_effect.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q14_promotion_effect.hive
index 988f400..4644d23 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q14_promotion_effect.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q14_promotion_effect.hive
@@ -9,9 +9,6 @@
-- create the result table
create table q14_promotion_effect(promo_revenue double);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1040000000;
-
-- the query
insert overwrite table q14_promotion_effect
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q15_top_supplier.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q15_top_supplier.hive
index 04064ed..8fa333e 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q15_top_supplier.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q15_top_supplier.hive
@@ -14,8 +14,6 @@
create table q15_top_supplier(s_suppkey int, s_name string, s_address string, s_phone string, total_revenue double);
-set mapred.min.split.size=536870912;
-
-- the query
insert overwrite table revenue
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q18_large_volume_customer.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q18_large_volume_customer.hive
index 76d0475..c617c26 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q18_large_volume_customer.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q18_large_volume_customer.hive
@@ -13,8 +13,6 @@
create table q18_tmp(l_orderkey int, t_sum_quantity double);
create table q18_large_volume_customer(c_name string, c_custkey int, o_orderkey int, o_orderdate string, o_totalprice double, sum_quantity double);
-set mapred.min.split.size=268435456;
-set hive.exec.reducers.bytes.per.reducer=1164000000;
-- the query
insert overwrite table q18_tmp
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q19_discounted_revenue.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q19_discounted_revenue.hive
index fd330cd..a7a0a0a 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q19_discounted_revenue.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q19_discounted_revenue.hive
@@ -9,8 +9,6 @@
-- create the result table
create table q19_discounted_revenue(revenue double);
-set mapred.min.split.size=268435456;
-set hive.exec.reducers.bytes.per.reducer=1040000000;
-- the query
insert overwrite table q19_discounted_revenue
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q1_pricing_summary_report.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q1_pricing_summary_report.hive
index a002068..af64a4f 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q1_pricing_summary_report.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q1_pricing_summary_report.hive
@@ -7,8 +7,6 @@
-- create the target table
CREATE TABLE q1_pricing_summary_report ( L_RETURNFLAG STRING, L_LINESTATUS STRING, SUM_QTY DOUBLE, SUM_BASE_PRICE DOUBLE, SUM_DISC_PRICE DOUBLE, SUM_CHARGE DOUBLE, AVE_QTY DOUBLE, AVE_PRICE DOUBLE, AVE_DISC DOUBLE, COUNT_ORDER INT);
-set mapred.min.split.size=536870912;
-
-- the query
INSERT OVERWRITE TABLE q1_pricing_summary_report
SELECT
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q20_potential_part_promotion.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q20_potential_part_promotion.hive
index 63297e6..3149962 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q20_potential_part_promotion.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q20_potential_part_promotion.hive
@@ -22,7 +22,6 @@
create table q20_tmp4(ps_suppkey int);
create table q20_potential_part_promotion(s_name string, s_address string);
-set mapred.min.split.size=536870912;
-- the query
insert overwrite table q20_tmp1
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q3_shipping_priority.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q3_shipping_priority.hive
index 0049eb3..67f6dc4 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q3_shipping_priority.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q3_shipping_priority.hive
@@ -11,9 +11,6 @@
-- create the target table
create table q3_shipping_priority (l_orderkey int, revenue double, o_orderdate string, o_shippriority int);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1024000000;
-
-- the query
Insert overwrite table q3_shipping_priority
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q4_order_priority.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q4_order_priority.hive
index aa828e9..efbcff2 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q4_order_priority.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q4_order_priority.hive
@@ -11,7 +11,6 @@
CREATE TABLE q4_order_priority_tmp (O_ORDERKEY INT);
CREATE TABLE q4_order_priority (O_ORDERPRIORITY STRING, ORDER_COUNT INT);
-set mapred.min.split.size=536870912;
-- the query
INSERT OVERWRITE TABLE q4_order_priority_tmp
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q5_local_supplier_volume.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q5_local_supplier_volume.hive
index a975ce1..091f000 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q5_local_supplier_volume.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q5_local_supplier_volume.hive
@@ -17,7 +17,6 @@
-- create the target table
create table q5_local_supplier_volume (N_NAME STRING, REVENUE DOUBLE);
-set mapred.min.split.size=536870912;
-- the query
insert overwrite table q5_local_supplier_volume
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q7_volume_shipping.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q7_volume_shipping.hive
index 3dfb22a..444644f 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q7_volume_shipping.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q7_volume_shipping.hive
@@ -17,8 +17,6 @@
create table q7_volume_shipping (supp_nation string, cust_nation string, l_year int, revenue double);
create table q7_volume_shipping_tmp(supp_nation string, cust_nation string, s_nationkey int, c_nationkey int);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1225000000;
-- the query
insert overwrite table q7_volume_shipping_tmp
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q9_product_type_profit.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q9_product_type_profit.hive
index 586779c..a9bb58b 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q9_product_type_profit.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/q9_product_type_profit.hive
@@ -17,8 +17,6 @@
-- create the result table
create table q9_product_type_profit (nation string, o_year string, sum_profit double);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1024000000;
-- the query
insert overwrite table q9_product_type_profit
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/u7_multi_join.hive b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/u7_multi_join.hive
index 2891c56..70b3538 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/u7_multi_join.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/queries/u7_multi_join.hive
@@ -6,4 +6,7 @@
create external table orders (O_ORDERKEY INT, O_CUSTKEY INT, O_ORDERSTATUS STRING, O_TOTALPRICE DOUBLE, O_ORDERDATE STRING, O_ORDERPRIORITY STRING, O_CLERK STRING, O_SHIPPRIORITY INT, O_COMMENT STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' STORED AS TEXTFILE LOCATION '/tpch/orders';
create external table customer (C_CUSTKEY INT, C_NAME STRING, C_ADDRESS STRING, C_NATIONKEY INT, C_PHONE STRING, C_ACCTBAL DOUBLE, C_MKTSEGMENT STRING, C_COMMENT STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' STORED AS TEXTFILE LOCATION '/tpch/customer';
-select l_linenumber, o_orderkey, o_totalprice, o_orderdate, o_shippriority from customer c join orders o on c.c_custkey = o.o_custkey join lineitem l on o.o_orderkey = l.l_orderkey where c.c_custkey<5 and o.o_totalprice<30000;
+select l_linenumber, o_orderkey, o_totalprice, o_orderdate, o_shippriority from
+ customer c join orders o on c.c_custkey = o.o_custkey
+ join lineitem l on o.o_orderkey = l.l_orderkey
+ where c.c_custkey<5 and o.o_totalprice<30000;
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan
index 05b3718..2629f1a 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan
@@ -82,7 +82,7 @@
-- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
project ([$$30, $$29])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$33, 1994-01-01], function-call: algebricks:ge, Args:[%0->$$33, 1993-10-01], function-call: algebricks:lt, Args:[%0->$$33, 1994-01-01]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$33, 1993-10-01], function-call: algebricks:lt, Args:[%0->$$33, 1994-01-01]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan
index 5c240e2..623f894 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan
@@ -38,7 +38,7 @@
-- HASH_PARTITION_EXCHANGE [$$10] |PARTITIONED|
project ([$$10, $$24])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$22, 1995-01-01], function-call: algebricks:ge, Args:[%0->$$22, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$20, %0->$$21], function-call: algebricks:lt, Args:[%0->$$21, %0->$$22], function-call: algebricks:lt, Args:[%0->$$20, %0->$$21], function-call: algebricks:ge, Args:[%0->$$22, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$22, 1995-01-01], function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$24, MAIL], function-call: algebricks:eq, Args:[%0->$$24, SHIP]], function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$24, MAIL], function-call: algebricks:eq, Args:[%0->$$24, SHIP]]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$21, %0->$$22], function-call: algebricks:lt, Args:[%0->$$20, %0->$$21], function-call: algebricks:ge, Args:[%0->$$22, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$22, 1995-01-01], function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$24, MAIL], function-call: algebricks:eq, Args:[%0->$$24, SHIP]]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan
index 19bcd24..55bd51b 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan
@@ -10,17 +10,17 @@
-- STABLE_SORT [$$21(DESC), $$20(DESC)] |LOCAL|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$$20 := %0->$$28]) decor ([]) {
- aggregate [$$21] <- [function-call: hive:count(FINAL), Args:[%0->$$27]]
+ group by ([$$20 := %0->$$26]) decor ([]) {
+ aggregate [$$21] <- [function-call: hive:count(FINAL), Args:[%0->$$25]]
-- AGGREGATE |LOCAL|
nested tuple source
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- EXTERNAL_GROUP_BY[$$28] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$26] |PARTITIONED|
exchange
- -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
- group by ([$$28 := %0->$$19]) decor ([]) {
- aggregate [$$27] <- [function-call: hive:count(PARTIAL1), Args:[1]]
+ -- HASH_PARTITION_EXCHANGE [$$26] |PARTITIONED|
+ group by ([$$26 := %0->$$19]) decor ([]) {
+ aggregate [$$25] <- [function-call: hive:count(PARTIAL1), Args:[1]]
-- AGGREGATE |LOCAL|
nested tuple source
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -32,49 +32,40 @@
-- STREAM_PROJECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- group by ([$$18 := %0->$$26]) decor ([]) {
- aggregate [$$19] <- [function-call: hive:count(FINAL), Args:[%0->$$25]]
+ group by ([$$18 := %0->$$10]) decor ([]) {
+ aggregate [$$19] <- [function-call: hive:count(COMPLETE), Args:[%0->$$1]]
-- AGGREGATE |LOCAL|
nested tuple source
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- EXTERNAL_GROUP_BY[$$26] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$10] |PARTITIONED|
exchange
- -- HASH_PARTITION_EXCHANGE [$$26] |PARTITIONED|
- group by ([$$26 := %0->$$10]) decor ([]) {
- aggregate [$$25] <- [function-call: hive:count(PARTIAL1), Args:[%0->$$1]]
- -- AGGREGATE |LOCAL|
- nested tuple source
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- EXTERNAL_GROUP_BY[$$10] |LOCAL|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$10, $$1])
+ -- STREAM_PROJECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$10, $$1])
- -- STREAM_PROJECT |PARTITIONED|
+ left outer join (function-call: algebricks:eq, Args:[%0->$$2, %0->$$10])
+ -- HYBRID_HASH_JOIN [$$10][$$2] |PARTITIONED|
exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- left outer join (function-call: algebricks:eq, Args:[%0->$$2, %0->$$10])
- -- HYBRID_HASH_JOIN [$$10][$$2] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$10] |PARTITIONED|
+ data-scan [$$10]<-[$$10, $$11, $$12, $$13, $$14, $$15, $$16, $$17] <- default.customer
+ -- DATASOURCE_SCAN |PARTITIONED|
exchange
- -- HASH_PARTITION_EXCHANGE [$$10] |PARTITIONED|
- data-scan [$$10]<-[$$10, $$11, $$12, $$13, $$14, $$15, $$16, $$17] <- default.customer
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
- project ([$$2, $$1])
- -- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:not, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$9, %special%requests%]])
- -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
+ project ([$$2, $$1])
+ -- STREAM_PROJECT |PARTITIONED|
+ select (function-call: algebricks:not, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$9, %special%requests%]])
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan [$$1, $$2, $$9]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.orders
+ -- DATASOURCE_SCAN |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan [$$1, $$2, $$9]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.orders
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan
index 21b90bd..35055a4 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan
@@ -34,7 +34,7 @@
-- HASH_PARTITION_EXCHANGE [$$11] |PARTITIONED|
project ([$$11, $$15, $$16])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$20, 1995-10-01], function-call: algebricks:ge, Args:[%0->$$20, 1995-09-01], function-call: algebricks:lt, Args:[%0->$$20, 1995-10-01]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$20, 1995-09-01], function-call: algebricks:lt, Args:[%0->$$20, 1995-10-01]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan
index a5bd27a..a4db8a7 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan
@@ -24,7 +24,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
project ([$$3, $$6, $$7])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$11, 1996-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1996-04-01], function-call: algebricks:ge, Args:[%0->$$11, 1996-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1996-04-01]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$11, 1996-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1996-04-01]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan
index 9835346..d1c0578 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan
@@ -38,7 +38,7 @@
-- HYBRID_HASH_JOIN [$$2][$$11] |PARTITIONED|
exchange
-- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:not, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$6, MEDIUM POLISHED%]], function-call: algebricks:neq, Args:[%0->$$5, Brand#45], function-call: algebricks:not, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$6, MEDIUM POLISHED%]]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:neq, Args:[%0->$$5, Brand#45], function-call: algebricks:not, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$6, MEDIUM POLISHED%]]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan
index a827007..714a56e 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan
@@ -84,7 +84,7 @@
-- HASH_PARTITION_EXCHANGE [$$3] |PARTITIONED|
project ([$$3])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$9, MED BOX], function-call: algebricks:eq, Args:[%0->$$6, Brand#23], function-call: algebricks:eq, Args:[%0->$$9, MED BOX]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$6, Brand#23], function-call: algebricks:eq, Args:[%0->$$9, MED BOX]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan
index ea47ea0..d0e1630 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan
@@ -90,7 +90,7 @@
-- HASH_PARTITION_EXCHANGE [$$1] |PARTITIONED|
project ([$$1])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:gt, Args:[%0->$$2, 300])
+ select (function-call: algebricks:gt, Args:[%0->$$2, 300.0])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan
index 1827729..25d1696 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan
@@ -22,7 +22,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
project ([$$15, $$16])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$4, Brand#12], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$7, SM CASE||SM BOX||SM PACK||SM PKG]], function-call: algebricks:ge, Args:[%0->$$14, 1]], function-call: algebricks:le, Args:[%0->$$14, 11]], function-call: algebricks:ge, Args:[%0->$$6, 1]], function-call: algebricks:le, Args:[%0->$$6, 5]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$24, AIR||AIR REG]], function-call: algebricks:eq, Args:[%0->$$23, DELIVER IN PERSON]], function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$4, Brand#23], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$7, MED BAG||MED BOX||MED PKG||MED PACK]], function-call: algebricks:ge, Args:[%0->$$14, 10]], function-call: algebricks:le, Args:[%0->$$14, 20]], function-call: algebricks:ge, Args:[%0->$$6, 1]], function-call: algebricks:le, Args:[%0->$$6, 10]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$24, AIR||AIR REG]], function-call: algebricks:eq, Args:[%0->$$23, DELIVER IN PERSON]]], function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$4, Brand#34], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$7, LG CASE||LG BOX||LG PACK||LG PKG]], function-call: algebricks:ge, Args:[%0->$$14, 20]], function-call: algebricks:le, Args:[%0->$$14, 30]], function-call: algebricks:ge, Args:[%0->$$6, 1]], function-call: algebricks:le, Args:[%0->$$6, 15]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$24, AIR||AIR REG]], function-call: algebricks:eq, Args:[%0->$$23, DELIVER IN PERSON]]])
+ select (function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$4, Brand#12], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$7, SM CASE||SM BOX||SM PACK||SM PKG]], function-call: algebricks:ge, Args:[%0->$$14, 1.0]], function-call: algebricks:le, Args:[%0->$$14, 11.0]], function-call: algebricks:ge, Args:[%0->$$6, 1]], function-call: algebricks:le, Args:[%0->$$6, 5]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$24, AIR||AIR REG]], function-call: algebricks:eq, Args:[%0->$$23, DELIVER IN PERSON]], function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$4, Brand#23], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$7, MED BAG||MED BOX||MED PKG||MED PACK]], function-call: algebricks:ge, Args:[%0->$$14, 10.0]], function-call: algebricks:le, Args:[%0->$$14, 20.0]], function-call: algebricks:ge, Args:[%0->$$6, 1]], function-call: algebricks:le, Args:[%0->$$6, 10]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$24, AIR||AIR REG]], function-call: algebricks:eq, Args:[%0->$$23, DELIVER IN PERSON]]], function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$4, Brand#34], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$7, LG CASE||LG BOX||LG PACK||LG PKG]], function-call: algebricks:ge, Args:[%0->$$14, 20.0]], function-call: algebricks:le, Args:[%0->$$14, 30.0]], function-call: algebricks:ge, Args:[%0->$$6, 1]], function-call: algebricks:le, Args:[%0->$$6, 15]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFRegExp, Args:[%0->$$24, AIR||AIR REG]], function-call: algebricks:eq, Args:[%0->$$23, DELIVER IN PERSON]]])
-- STREAM_SELECT |UNPARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan
index eddfca5..cebc0eb 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan
@@ -50,7 +50,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
project ([$$2, $$3, $$5])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$11, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1995-01-01], function-call: algebricks:ge, Args:[%0->$$11, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1995-01-01]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$11, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1995-01-01]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan
index cc47cf3..fd20869 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan
@@ -150,9 +150,9 @@
-- HYBRID_HASH_JOIN [$$18][$$36] |PARTITIONED|
exchange
-- HASH_PARTITION_EXCHANGE [$$18] |PARTITIONED|
- project ([$$16, $$18])
+ project ([$$18, $$16])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:gt, Args:[%0->$$28, %0->$$27], function-call: algebricks:gt, Args:[%0->$$28, %0->$$27]])
+ select (function-call: algebricks:gt, Args:[%0->$$28, %0->$$27])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan
index 151f34d..1698365 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan
@@ -10,7 +10,7 @@
-- HASH_PARTITION_EXCHANGE [$$1] |PARTITIONED|
project ([$$1, $$3])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$5, %BRASS], function-call: algebricks:eq, Args:[%0->$$6, 15], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$5, %BRASS]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$6, 15], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$5, %BRASS]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan
index 435fd7c..6929f74 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan
@@ -70,7 +70,7 @@
-- HASH_PARTITION_EXCHANGE [$$2] |PARTITIONED|
project ([$$2, $$7])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$6, 1993-10-01], function-call: algebricks:ge, Args:[%0->$$6, 1993-07-01], function-call: algebricks:lt, Args:[%0->$$6, 1993-10-01]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$6, 1993-07-01], function-call: algebricks:lt, Args:[%0->$$6, 1993-10-01]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan
index 177d24c..3161a61 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan
@@ -106,7 +106,7 @@
-- HASH_PARTITION_EXCHANGE [$$9] |PARTITIONED|
project ([$$9, $$10])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$13, 1995-01-01], function-call: algebricks:ge, Args:[%0->$$13, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$13, 1995-01-01]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$13, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$13, 1995-01-01]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan
index cd9ffcd..92deaea 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan
@@ -22,7 +22,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
project ([$$6, $$7])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$11, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1995-01-01], function-call: algebricks:ge, Args:[%0->$$7, 0.05], function-call: algebricks:le, Args:[%0->$$7, 0.07], function-call: algebricks:lt, Args:[%0->$$5, 24], function-call: algebricks:ge, Args:[%0->$$11, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1995-01-01], function-call: algebricks:ge, Args:[%0->$$7, 0.05], function-call: algebricks:le, Args:[%0->$$7, 0.07], function-call: algebricks:lt, Args:[%0->$$5, 24]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$11, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$11, 1995-01-01], function-call: algebricks:ge, Args:[%0->$$7, 0.05], function-call: algebricks:le, Args:[%0->$$7, 0.07], function-call: algebricks:lt, Args:[%0->$$5, 24.0]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan
index 39f8301..c3081a5 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan
@@ -148,7 +148,7 @@
-- HYBRID_HASH_JOIN [$$20][$$36] |PARTITIONED|
exchange
-- HASH_PARTITION_EXCHANGE [$$20] |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:le, Args:[%0->$$30, 1996-12-31], function-call: algebricks:ge, Args:[%0->$$30, 1995-01-01], function-call: algebricks:le, Args:[%0->$$30, 1996-12-31]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$30, 1995-01-01], function-call: algebricks:le, Args:[%0->$$30, 1996-12-31]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan
index b807a24..6b1ea41 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan
@@ -138,7 +138,7 @@
-- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
project ([$$38, $$37, $$41])
-- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$41, 1996-12-31], function-call: algebricks:ge, Args:[%0->$$41, 1995-01-01]])
+ select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$41, 1995-01-01], function-call: algebricks:lt, Args:[%0->$$41, 1996-12-31]])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u5_lateral_view.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u5_lateral_view.plan
index 48e624e..2cbea4a 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u5_lateral_view.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u5_lateral_view.plan
@@ -6,7 +6,7 @@
-- UNNEST |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan [$$1, $$2, $$3]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7] <- default.supplier
+ data-scan [$$2, $$3, $$1]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7] <- default.supplier
-- DATASOURCE_SCAN |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u6_limit.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u6_limit.plan
index b5ed12f..1fb6117 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u6_limit.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u6_limit.plan
@@ -14,7 +14,7 @@
-- STABLE_SORT [$$4(ASC)] |LOCAL|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- select (function-call: algebricks:lt, Args:[%0->$$4, 10000])
+ select (function-call: algebricks:lt, Args:[%0->$$4, 10000.0])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan
index ab55181..b5f1dc2 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan
@@ -16,37 +16,35 @@
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange
-- HASH_PARTITION_EXCHANGE [$$25] |PARTITIONED|
- project ([$$32, $$25, $$29, $$28])
+ project ([$$25, $$28, $$29, $$32])
-- STREAM_PROJECT |PARTITIONED|
- project ([$$25, $$17, $$28, $$29, $$32])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (function-call: algebricks:eq, Args:[%0->$$26, %0->$$17])
- -- HYBRID_HASH_JOIN [$$26][$$17] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$26] |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$28, 30000], function-call: algebricks:lt, Args:[%0->$$28, 30000]])
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (function-call: algebricks:eq, Args:[%0->$$26, %0->$$17])
+ -- HYBRID_HASH_JOIN [$$26][$$17] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$26] |PARTITIONED|
+ select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$28, 30000.0], function-call: algebricks:lt, Args:[%0->$$26, 5]])
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan [$$32, $$25, $$26, $$29, $$28]<-[$$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33] <- default.orders
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$17] |PARTITIONED|
+ project ([$$17])
+ -- STREAM_PROJECT |PARTITIONED|
+ select (function-call: algebricks:lt, Args:[%0->$$17, 5])
-- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan [$$32, $$25, $$26, $$29, $$28]<-[$$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33] <- default.orders
+ data-scan [$$17]<-[$$17, $$18, $$19, $$20, $$21, $$22, $$23, $$24] <- default.customer
-- DATASOURCE_SCAN |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$17] |PARTITIONED|
- project ([$$17])
- -- STREAM_PROJECT |PARTITIONED|
- select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$17, 5], function-call: algebricks:lt, Args:[%0->$$17, 5]])
- -- STREAM_SELECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan [$$17]<-[$$17, $$18, $$19, $$20, $$21, $$22, $$23, $$24] <- default.customer
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
index a4ee677..c726bfa 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
@@ -42,6 +42,20 @@
</property>
<property>
+ <name>hive.auto.convert.join.noconditionaltask</name>
+ <value>false</value>
+ <description>Whether Hive enable the optimization about converting common join into mapjoin based on the input file
+ size. If this paramater is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the
+ specified size, the join is directly converted to a mapjoin (there is no conditional task).
+ </description>
+ </property>
+
+ <property>
+ <name>hive.auto.convert.join</name>
+ <value>false</value>
+ </property>
+
+ <property>
<name>hive.hyracks.connectorpolicy</name>
<value>SEND_SIDE_MAT_PIPELINING</value>
</property>
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties
index a7d8d9c..f886a44 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties
@@ -43,7 +43,7 @@
# Note that the ConsoleHandler also has a separate level
# setting to limit messages printed to the console.
-.level= WARNING
+.level= INFO
# .level= INFO
# .level= FINE
# .level = FINEST
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q10_returned_item.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q10_returned_item.hive
index 3f1214a..bb07665 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q10_returned_item.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q10_returned_item.hive
@@ -13,9 +13,6 @@
-- create the result table
create table q10_returned_item (c_custkey int, c_name string, revenue double, c_acctbal string, n_name string, c_address string, c_phone string, c_comment string);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1024000000;
-
-- the query
insert overwrite table q10_returned_item
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q12_shipping.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q12_shipping.hive
index 062f7b9..8546365 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q12_shipping.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q12_shipping.hive
@@ -9,9 +9,6 @@
-- create the result table
create table q12_shipping(l_shipmode string, high_line_count double, low_line_count double);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1225000000;
-
-- the query
insert overwrite table q12_shipping
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q14_promotion_effect.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q14_promotion_effect.hive
index 988f400..4644d23 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q14_promotion_effect.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q14_promotion_effect.hive
@@ -9,9 +9,6 @@
-- create the result table
create table q14_promotion_effect(promo_revenue double);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1040000000;
-
-- the query
insert overwrite table q14_promotion_effect
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q15_top_supplier.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q15_top_supplier.hive
index 04064ed..8fa333e 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q15_top_supplier.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q15_top_supplier.hive
@@ -14,8 +14,6 @@
create table q15_top_supplier(s_suppkey int, s_name string, s_address string, s_phone string, total_revenue double);
-set mapred.min.split.size=536870912;
-
-- the query
insert overwrite table revenue
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q18_large_volume_customer.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q18_large_volume_customer.hive
index ac2902c..d1eaacc 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q18_large_volume_customer.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q18_large_volume_customer.hive
@@ -13,9 +13,6 @@
create table q18_tmp(l_orderkey int, t_sum_quantity double);
create table q18_large_volume_customer(c_name string, c_custkey int, o_orderkey int, o_orderdate string, o_totalprice double, sum_quantity double);
-set mapred.min.split.size=268435456;
-set hive.exec.reducers.bytes.per.reducer=1164000000;
-
-- the query
insert overwrite table q18_tmp
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q19_discounted_revenue.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q19_discounted_revenue.hive
index 2002e1e..6badfcf 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q19_discounted_revenue.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q19_discounted_revenue.hive
@@ -9,9 +9,6 @@
-- create the result table
create table q19_discounted_revenue(revenue double);
-set mapred.min.split.size=268435456;
-set hive.exec.reducers.bytes.per.reducer=1040000000;
-
-- the query
insert overwrite table q19_discounted_revenue
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q1_pricing_summary_report.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q1_pricing_summary_report.hive
index a002068..af64a4f 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q1_pricing_summary_report.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q1_pricing_summary_report.hive
@@ -7,8 +7,6 @@
-- create the target table
CREATE TABLE q1_pricing_summary_report ( L_RETURNFLAG STRING, L_LINESTATUS STRING, SUM_QTY DOUBLE, SUM_BASE_PRICE DOUBLE, SUM_DISC_PRICE DOUBLE, SUM_CHARGE DOUBLE, AVE_QTY DOUBLE, AVE_PRICE DOUBLE, AVE_DISC DOUBLE, COUNT_ORDER INT);
-set mapred.min.split.size=536870912;
-
-- the query
INSERT OVERWRITE TABLE q1_pricing_summary_report
SELECT
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q20_potential_part_promotion.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q20_potential_part_promotion.hive
index 2bb90ea..32181bf 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q20_potential_part_promotion.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q20_potential_part_promotion.hive
@@ -23,8 +23,6 @@
create table q20_tmp4(ps_suppkey int);
create table q20_potential_part_promotion(s_name string, s_address string);
-set mapred.min.split.size=536870912;
-
-- the query
insert overwrite table q20_tmp1
select distinct p_partkey
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q3_shipping_priority.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q3_shipping_priority.hive
index 0049eb3..67f6dc4 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q3_shipping_priority.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q3_shipping_priority.hive
@@ -11,9 +11,6 @@
-- create the target table
create table q3_shipping_priority (l_orderkey int, revenue double, o_orderdate string, o_shippriority int);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1024000000;
-
-- the query
Insert overwrite table q3_shipping_priority
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q4_order_priority.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q4_order_priority.hive
index aa828e9..efbcff2 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q4_order_priority.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q4_order_priority.hive
@@ -11,7 +11,6 @@
CREATE TABLE q4_order_priority_tmp (O_ORDERKEY INT);
CREATE TABLE q4_order_priority (O_ORDERPRIORITY STRING, ORDER_COUNT INT);
-set mapred.min.split.size=536870912;
-- the query
INSERT OVERWRITE TABLE q4_order_priority_tmp
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q5_local_supplier_volume.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q5_local_supplier_volume.hive
index 9af2dd2..838a1e8 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q5_local_supplier_volume.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q5_local_supplier_volume.hive
@@ -17,7 +17,6 @@
-- create the target table
create table q5_local_supplier_volume (N_NAME STRING, REVENUE DOUBLE);
-set mapred.min.split.size=536870912;
-- the query
insert overwrite table q5_local_supplier_volume
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q7_volume_shipping.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q7_volume_shipping.hive
index 2678f80..12ae8ae 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q7_volume_shipping.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q7_volume_shipping.hive
@@ -17,9 +17,6 @@
create table q7_volume_shipping (supp_nation string, cust_nation string, l_year int, revenue double);
create table q7_volume_shipping_tmp(supp_nation string, cust_nation string, s_nationkey int, c_nationkey int);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1225000000;
-
-- the query
insert overwrite table q7_volume_shipping_tmp
select
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q9_product_type_profit.hive b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q9_product_type_profit.hive
index 2e5b4a1..c491997 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q9_product_type_profit.hive
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/queries/q9_product_type_profit.hive
@@ -17,9 +17,6 @@
-- create the result table
create table q9_product_type_profit (nation string, o_year string, sum_profit double);
-set mapred.min.split.size=536870912;
-set hive.exec.reducers.bytes.per.reducer=1024000000;
-
-- the query
insert overwrite table q9_product_type_profit
select
diff --git a/hivesterix/hivesterix-runtime/pom.xml b/hivesterix/hivesterix-runtime/pom.xml
index e4f5416..6d075ba 100644
--- a/hivesterix/hivesterix-runtime/pom.xml
+++ b/hivesterix/hivesterix-runtime/pom.xml
@@ -24,12 +24,10 @@
</parent>
<dependencies>
- <dependency>
- <groupId>javax.servlet</groupId>
- <artifactId>servlet-api</artifactId>
- <version>2.5</version>
- <type>jar</type>
- <scope>compile</scope>
+ <dependency>
+ <groupId>sqlline</groupId>
+ <artifactId>sqlline</artifactId>
+ <version>1_0_2</version>
</dependency>
<dependency>
<groupId>junit</groupId>
@@ -38,250 +36,65 @@
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>args4j</groupId>
- <artifactId>args4j</artifactId>
- <version>2.0.12</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.json</groupId>
- <artifactId>json</artifactId>
- <version>20090211</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.eclipse.jetty</groupId>
- <artifactId>jetty-server</artifactId>
- <version>8.0.0.M1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.eclipse.jetty</groupId>
- <artifactId>jetty-servlet</artifactId>
- <version>8.0.0.M1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>jline</groupId>
- <artifactId>jline</artifactId>
- <version>0.9.94</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-core</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-connectionpool</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-enhancer</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.datanucleus</groupId>
- <artifactId>datanucleus-rdbms</artifactId>
- <version>2.0.3</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-dbcp</groupId>
- <artifactId>commons-dbcp</artifactId>
- <version>1.4</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-pool</groupId>
- <artifactId>commons-pool</artifactId>
- <version>1.5.4</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-collections</groupId>
- <artifactId>commons-collections</artifactId>
- <version>3.2.1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-lang</groupId>
- <artifactId>commons-lang</artifactId>
- <version>2.4</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>javax</groupId>
- <artifactId>jdo2-api</artifactId>
- <version>2.3-ec</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>com.facebook</groupId>
- <artifactId>libfb303</artifactId>
- <version>0.5.0</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.thrift</groupId>
- <artifactId>libthrift</artifactId>
- <version>0.5.0</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>cli</artifactId>
- <version>1.2</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache</groupId>
- <artifactId>log4j</artifactId>
- <version>1.2.15</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.antlr</groupId>
- <artifactId>antlr-runtime</artifactId>
- <version>3.0.1</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-cli</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-exec</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-hwi</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-metastore</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-service</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-shims</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-serde</artifactId>
- <version>0.7.0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- <version>1.6.1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-cli</groupId>
- <artifactId>commons-cli</artifactId>
- <version>1.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
- <version>1.6.1</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>commons-logging</groupId>
- <artifactId>commons-logging</artifactId>
- <version>1.1.1</version>
- <type>jar</type>
- <classifier>api</classifier>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>com.google.guava</groupId>
- <artifactId>guava</artifactId>
- <version>r06</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.antlr</groupId>
- <artifactId>stringtemplate</artifactId>
- <version>3.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.derby</groupId>
- <artifactId>derby</artifactId>
- <version>10.8.1.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hbase</groupId>
- <artifactId>hbase</artifactId>
- <version>0.90.3</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
@@ -384,5 +197,19 @@
<id>hyracks-public-release</id>
<url>http://obelix.ics.uci.edu/nexus/content/repositories/hyracks-public-releases</url>
</repository>
+ <repository>
+ <releases>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>warn</checksumPolicy>
+ </releases>
+ <snapshots>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>fail</checksumPolicy>
+ </snapshots>
+ <id>plugins-release</id>
+ <url>http://repo.springsource.org/plugins-release</url>
+ </repository>
</repositories>
</project>
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
index a5177c9..dd4fbe7 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
@@ -34,6 +34,7 @@
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+@SuppressWarnings("deprecation")
public abstract class AbstractExpressionEvaluator implements ICopyEvaluator {
private List<ICopyEvaluator> children;
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
index d061b23..87d2221 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
@@ -36,6 +36,7 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyAggregateFunction;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+@SuppressWarnings("deprecation")
public class AggregationFunctionEvaluator implements ICopyAggregateFunction {
/**
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
index f4b77b8..3f1cc27 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
@@ -35,6 +35,7 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunction;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+@SuppressWarnings("deprecation")
public class AggregatuibFunctionSerializableEvaluator implements ICopySerializableAggregateFunction {
/**
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
index d91b806..b511d87 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
@@ -35,6 +35,7 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyUnnestingFunction;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+@SuppressWarnings("deprecation")
public class UDTFFunctionEvaluator implements ICopyUnnestingFunction, Collector {
/**
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
index 09f0cb6..d65dc24 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
@@ -12,370 +12,383 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-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.ExpressionTranslator;
-import edu.uci.ics.hivesterix.runtime.evaluator.AggregationFunctionEvaluator;
-import edu.uci.ics.hivesterix.runtime.jobgen.Schema;
-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;
- }
-}
+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.ExpressionTranslator;
+import edu.uci.ics.hivesterix.runtime.evaluator.AggregationFunctionEvaluator;
+import edu.uci.ics.hivesterix.runtime.jobgen.Schema;
+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.hivesterix.serde.lazy.LazyUtils;
+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;
+
+@SuppressWarnings("deprecation")
+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 {
+ /**
+ * list of object inspectors correlated to types
+ */
+ List<ObjectInspector> oiListForTypes = new ArrayList<ObjectInspector>();
+ for (TypeInfo type : types) {
+ oiListForTypes.add(LazyUtils.getLazyObjectInspectorFromTypeInfo(type, false));
+ }
+
+ 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, oiListForTypes, 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, oiListForTypes, 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/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
index 71d11c0..c1ee814 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
@@ -39,12 +39,14 @@
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.hivesterix.serde.lazy.LazyUtils;
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;
+@SuppressWarnings("deprecation")
public class AggregationFunctionSerializableFactory implements ICopySerializableAggregateFunctionFactory {
private static final long serialVersionUID = 1L;
@@ -190,10 +192,19 @@
String s = Utilities.serializeExpression(input);
parametersSerialization.add(s);
}
+
}
@Override
public synchronized ICopySerializableAggregateFunction createAggregateFunction() throws AlgebricksException {
+ /**
+ * list of object inspectors correlated to types
+ */
+ List<ObjectInspector> oiListForTypes = new ArrayList<ObjectInspector>();
+ for (TypeInfo type : types) {
+ oiListForTypes.add(LazyUtils.getLazyObjectInspectorFromTypeInfo(type, false));
+ }
+
if (parametersOrigin == null) {
Configuration config = new Configuration();
config.setClassLoader(this.getClass().getClassLoader());
@@ -328,7 +339,8 @@
udafComplete = udafsComplete.get(threadId);
if (udafComplete == null) {
try {
- udafComplete = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
+ udafComplete = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, oiListForTypes, distinct,
+ false);
} catch (HiveException e) {
throw new AlgebricksException(e);
}
@@ -352,7 +364,8 @@
udafPartial = udafsPartial.get(threadId);
if (udafPartial == null) {
try {
- udafPartial = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
+ udafPartial = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, oiListForTypes, distinct,
+ false);
} catch (HiveException e) {
throw new AlgebricksException(e);
}
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
index e89a4c4..4bbb21f 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
@@ -12,156 +12,176 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-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.runtime.jobgen.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;
-
-@SuppressWarnings("deprecation")
-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 {
-
- }
-
-}
+package edu.uci.ics.hivesterix.runtime.operator.filewrite;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.exec.Utilities;
+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.hive.shims.ShimLoader;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hivesterix.runtime.jobgen.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;
+
+@SuppressWarnings("deprecation")
+public class HiveFileWritePushRuntime implements IPushRuntime {
+ private final static Logger LOGGER = Logger.getLogger(HiveFileWritePushRuntime.class.getName());
+
+ /**
+ * 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);
+ createTempDir();
+ } 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 {
+
+ }
+
+ private void createTempDir() throws IOException {
+ FileSinkDesc fdesc = fsOp.getConf();
+ String tempDir = fdesc.getDirName();
+ if (tempDir != null) {
+ Path tempPath = Utilities.toTempPath(new Path(tempDir));
+ FileSystem fs = tempPath.getFileSystem(conf);
+ if (!fs.exists(tempPath)) {
+ try {
+ fs.mkdirs(tempPath);
+ ShimLoader.getHadoopShims().fileSystemDeleteOnExit(fs, tempPath);
+ } catch (IOException e) {
+ //if the dir already exists, that should be fine; so log a warning msg
+ LOGGER.warning("create tmp result directory fails.");
+ }
+ }
+ }
+ }
+
+}
diff --git a/hivesterix/hivesterix-serde/pom.xml b/hivesterix/hivesterix-serde/pom.xml
index 97c9174..5b02ab3 100644
--- a/hivesterix/hivesterix-serde/pom.xml
+++ b/hivesterix/hivesterix-serde/pom.xml
@@ -42,16 +42,16 @@
<dependencies>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-serde</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
@@ -63,12 +63,12 @@
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hivesterix-common</artifactId>
- <version>0.2.7-SNAPSHOT</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <version>0.2.7-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
diff --git a/hivesterix/hivesterix-serde/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java b/hivesterix/hivesterix-serde/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
index b5d64e8..2bbb1d5 100644
--- a/hivesterix/hivesterix-serde/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
+++ b/hivesterix/hivesterix-serde/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
@@ -46,6 +46,7 @@
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.SerDeStats;
import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -74,6 +75,7 @@
* deserialized until required. Binary means a field is serialized in binary
* compact format.
*/
+@SuppressWarnings("deprecation")
public class LazySerDe implements SerDe {
public static final Log LOG = LogFactory.getLog(LazySerDe.class.getName());
@@ -140,7 +142,6 @@
/**
* Deserialize a table record to a Lazy struct.
*/
- @SuppressWarnings("deprecation")
@Override
public Object deserialize(Writable field) throws SerDeException {
if (byteArrayRef == null) {
@@ -471,4 +472,9 @@
}
}
}
+
+ @Override
+ public SerDeStats getSerDeStats() {
+ return null;
+ }
}
diff --git a/hivesterix/hivesterix-translator/pom.xml b/hivesterix/hivesterix-translator/pom.xml
index 8a24d5e..b4c4454 100644
--- a/hivesterix/hivesterix-translator/pom.xml
+++ b/hivesterix/hivesterix-translator/pom.xml
@@ -42,9 +42,9 @@
<dependencies>
<dependency>
- <groupId>org.apache.hadoop.hive</groupId>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-exec</artifactId>
- <version>0.7.0</version>
+ <version>0.11.0</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
diff --git a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
index f32d85b..76cc51d 100644
--- a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
+++ b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
@@ -12,810 +12,809 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-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.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.hivesterix.runtime.jobgen.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.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;
-
-@SuppressWarnings("rawtypes")
-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)) {
- 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 s
- *
- * @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) {
- @SuppressWarnings("unchecked")
- 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()) {
- throw new IllegalStateException("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) {
- throw new IllegalStateException(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);
- }
- }
-}
+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.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.hivesterix.runtime.jobgen.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.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;
+
+@SuppressWarnings("rawtypes")
+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);
+ }
+
+ public 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)) {
+ 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 s
+ *
+ * @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) {
+ @SuppressWarnings("unchecked")
+ 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()) {
+ // throw new IllegalStateException("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) {
+ throw new IllegalStateException(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/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
index f4161a4..aa1837c 100644
--- a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
+++ b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
@@ -12,113 +12,145 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-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.plan.visitor.base.DefaultVisitor;
-import edu.uci.ics.hivesterix.logical.plan.visitor.base.Translator;
-import edu.uci.ics.hivesterix.runtime.jobgen.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.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.
- */
-
-@SuppressWarnings("rawtypes")
-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);
- ILogicalOperator parentOperator;
- ILogicalExpression unnestArg;
- if (parent0 instanceof UDTFOperator) {
- 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 {
- 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;
- }
-
-}
+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.ColumnInfo;
+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.LateralViewJoinDesc;
+import org.apache.hadoop.hive.ql.plan.UDTFDesc;
+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.hivesterix.runtime.jobgen.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.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.ProjectOperator;
+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.
+ */
+
+@SuppressWarnings("rawtypes")
+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;
+ }
+
+ ILogicalOperator parentOperator = null;
+ ILogicalExpression unnestArg = null;
+ List<LogicalVariable> projectVariables = new ArrayList<LogicalVariable>();
+ for (Mutable<ILogicalOperator> parentLOpRef : parents) {
+ VariableUtilities.getLiveVariables(parentLOpRef.getValue(), projectVariables);
+ }
+ for (Operator parentOp : operator.getParentOperators()) {
+ if (parentOp instanceof UDTFOperator) {
+ int index = operator.getParentOperators().indexOf(parentOp);
+ List<LogicalVariable> unnestVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(parents.get(index).getValue(), unnestVars);
+ unnestArg = new VariableReferenceExpression(unnestVars.get(0));
+ parentOperator = parents.get(index).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(parents.get(0).getValue(), outputVars);
+ outputVars.add(var);
+ ILogicalOperator inputProjectOperator = new ProjectOperator(projectVariables);
+ currentOperator.getInputs().add(new MutableObject<ILogicalOperator>(inputProjectOperator));
+ inputProjectOperator.getInputs().addAll(parentOperator.getInputs());
+
+ parents.clear();
+ udtf = null;
+ List<ColumnInfo> inputSchema = operator.getSchema().getSignature();
+ rewriteOperatorDesc(outputVars, operator.getConf(), inputSchema, t);
+ //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;
+ }
+
+ private void rewriteOperatorDesc(List<LogicalVariable> variables, LateralViewJoinDesc desc,
+ List<ColumnInfo> schema, Translator t) {
+ List<String> outputFieldNames = desc.getOutputInternalColNames();
+ for (int i = 0; i < variables.size(); i++) {
+ LogicalVariable var = variables.get(i);
+ String fieldName = outputFieldNames.get(i);
+ String tabAlias = findTabAlias(fieldName, schema);
+ fieldName = tabAlias + "." + fieldName;
+ if (fieldName.indexOf("$$") < 0) {
+ //outputFieldNames.set(i, var.toString());
+ t.updateVariable(fieldName, var);
+ }
+ }
+ }
+
+ private String findTabAlias(String fieldName, List<ColumnInfo> schema) {
+ for (int i = 0; i < schema.size(); i++) {
+ ColumnInfo column = schema.get(i);
+ if (column.getInternalName().equals(fieldName)) {
+ return column.getTabAlias();
+ }
+ }
+ return "null";
+ }
+
+}
diff --git a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
index 186b291..3ed9786 100644
--- a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
+++ b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
@@ -14,36 +14,36 @@
*/
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;
+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.plan.OperatorDesc;
+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;
@SuppressWarnings("rawtypes")
public class MapJoinVisitor extends DefaultVisitor {
@@ -56,7 +56,7 @@
@Override
public Mutable<ILogicalOperator> visit(MapJoinOperator operator,
Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {
- List<Operator<? extends Serializable>> joinSrc = operator.getParentOperators();
+ List<Operator<? extends OperatorDesc>> joinSrc = operator.getParentOperators();
List<Mutable<ILogicalOperator>> parents = opMap.get(operator);
if (parents == null) {
parents = new ArrayList<Mutable<ILogicalOperator>>();
diff --git a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
index 74cebaa..25abdec 100644
--- a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
+++ b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
@@ -69,9 +69,9 @@
@Override
public Mutable<ILogicalOperator> visit(TableScanOperator operator,
- Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) throws AlgebricksException {
+ Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) throws AlgebricksException {
TableScanDesc desc = (TableScanDesc) operator.getConf();
- if (desc == null) {
+ if (desc == null || desc.getAlias()==null) {
List<LogicalVariable> schema = new ArrayList<LogicalVariable>();
VariableUtilities.getLiveVariables(AlgebricksParentOperator.getValue(), schema);
t.rewriteOperatorOutputSchema(schema, operator);
@@ -124,7 +124,6 @@
@Override
public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator,
Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) {
-
if (hiveOperator.getChildOperators() != null && hiveOperator.getChildOperators().size() > 0)
return null;
diff --git a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
index 1cb5121..c710f3f 100644
--- a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
+++ b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
@@ -14,21 +14,21 @@
*/
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;
+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 {
@@ -46,8 +46,8 @@
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);
+ VariableUtilities.getLiveVariables(parents.get(0).getValue(), leftVars);
+ VariableUtilities.getLiveVariables(parents.get(1).getValue(), rightVars);
List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> triples = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
List<LogicalVariable> unionVars = new ArrayList<LogicalVariable>();
diff --git a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
index 32b0f66..5b6ac50 100644
--- a/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
+++ b/hivesterix/hivesterix-translator/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
@@ -12,173 +12,181 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-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.runtime.jobgen.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;
-
-@SuppressWarnings("rawtypes")
-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);
-
-}
+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.runtime.jobgen.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;
+
+@SuppressWarnings("rawtypes")
+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);
+
+ /**
+ * update a name-variable binding
+ *
+ * @param fieldName
+ * @param variable
+ */
+ public void updateVariable(String fieldName, LogicalVariable variable);
+
+ /**
+ * 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);
+
+}