Merge branch 'asterix_lsm_stabilization' of https://code.google.com/p/asterixdb into yingyi/asterix_beta_fix
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index f14fff8..587cbc5 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -36,6 +36,7 @@
import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
+import edu.uci.ics.asterix.optimizer.rules.NestedSubplanToJoinRule;
import edu.uci.ics.asterix.optimizer.rules.PullPositionalVariableFromUnnestRule;
import edu.uci.ics.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
import edu.uci.ics.asterix.optimizer.rules.PushAggregateIntoGroupbyRule;
@@ -163,6 +164,7 @@
public final static List<IAlgebraicRewriteRule> buildLoadFieldsRuleCollection() {
List<IAlgebraicRewriteRule> fieldLoads = new LinkedList<IAlgebraicRewriteRule>();
fieldLoads.add(new LoadRecordFieldsRule());
+ fieldLoads.add(new NestedSubplanToJoinRule());
fieldLoads.add(new PushFieldAccessRule());
// fieldLoads.add(new ByNameToByHandleFieldAccessRule()); -- disabled
fieldLoads.add(new ByNameToByIndexFieldAccessRule());
@@ -192,6 +194,7 @@
consolidation.add(new CountVarToCountOneRule());
consolidation.add(new RemoveUnusedAssignAndAggregateRule());
consolidation.add(new RemoveRedundantGroupByDecorVars());
+ consolidation.add(new NestedSubplanToJoinRule());
return consolidation;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
index 2dce5f6..5cff07b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
@@ -24,6 +24,7 @@
import edu.uci.ics.asterix.aql.util.FunctionUtils;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -120,11 +121,8 @@
// assign
AbstractFunctionCallExpression cast = new ScalarFunctionCallExpression(
FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_RECORD));
- ARecordType[] types = new ARecordType[2];
- types[0] = requiredRecordType;
- types[1] = inputRecordType;
cast.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputRecordVar)));
- cast.setOpaqueParameters(types);
+ TypeComputerUtilities.setRequiredAndInputTypes(cast, requiredRecordType, inputRecordType);
LogicalVariable newAssignVar = context.newVar();
AssignOperator newAssignOperator = new AssignOperator(newAssignVar, new MutableObject<ILogicalExpression>(cast));
newAssignOperator.getInputs().add(new MutableObject<ILogicalOperator>(op3));
@@ -142,6 +140,10 @@
newInserDeleteOperator.getInputs().add(new MutableObject<ILogicalOperator>(projectOperator));
insertDeleteOperator.getInputs().clear();
op1.getInputs().get(0).setValue(newInserDeleteOperator);
+
+ context.computeAndSetTypeEnvironmentForOperator(newAssignOperator);
+ context.computeAndSetTypeEnvironmentForOperator(projectOperator);
+ context.computeAndSetTypeEnvironmentForOperator(newInserDeleteOperator);
return true;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index c99e4bc..6985753 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -64,10 +64,16 @@
}
FunctionIdentifier fid = null;
+ /** find the record variable */
+ InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
+ ILogicalExpression recordExpr = insertOp.getPayloadExpression().getValue();
+ List<LogicalVariable> recordVar = new ArrayList<LogicalVariable>();
+ /** assume the payload is always a single variable expression */
+ recordExpr.getUsedVariables(recordVar);
+
/** op2 is the assign operator which extract primary keys from the record variable */
AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
- List<LogicalVariable> recordVar = new ArrayList<LogicalVariable>();
- VariableUtilities.getUsedVariables(op2, recordVar);
+
if (recordVar.size() == 0) {
/**
* For the case primary key-assignment expressions are constant expressions,
@@ -92,7 +98,6 @@
AssignOperator assignOp2 = (AssignOperator) op2;
recordVar.addAll(assignOp2.getVariables());
}
- InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
AqlDataSource datasetSource = (AqlDataSource) insertOp.getDataSource();
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
String dataverseName = datasetSource.getId().getDataverseName();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
index 3aae2dd..f42782b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
@@ -42,7 +42,7 @@
/**
* Statically cast a constant from its type to a specified required type, in a
* recursive way. It enables: 1. bag-based fields in a record, 2. bidirectional
- * cast of a open field and a matched closed field, and 3. put in null fields
+ * cast of an open field and a matched closed field, and 3. put in null fields
* when necessary. It should be fired before the constant folding rule.
* This rule is not responsible for type casting between primitive types.
* Here is an example: A record { "hobby": {{"music", "coding"}}, "id": "001",
@@ -90,9 +90,6 @@
InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op2;
if (insertDeleteOp.getOperation() == InsertDeleteOperator.Kind.DELETE)
return false;
- AbstractLogicalOperator assignOp = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
- if (assignOp.getOperatorTag() != LogicalOperatorTag.ASSIGN)
- return false;
/**
* get required record type
*/
@@ -101,21 +98,21 @@
IAType[] schemaTypes = (IAType[]) dataSource.getSchemaTypes();
IAType requiredRecordType = schemaTypes[schemaTypes.length - 1];
- AssignOperator topAssignOperator = (AssignOperator) assignOp;
List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
- VariableUtilities.getUsedVariables(topAssignOperator, usedVariables);
+ insertDeleteOperator.getPayloadExpression().getValue().getUsedVariables(usedVariables);
// the used variable should contain the record that will be inserted
// but it will not fail in many cases even if the used variable set is
// empty
if (usedVariables.size() == 0)
return false;
+
oldRecordVariable = usedVariables.get(0);
LogicalVariable inputRecordVar = usedVariables.get(0);
- IVariableTypeEnvironment env = topAssignOperator.computeOutputTypeEnvironment(context);
+ IVariableTypeEnvironment env = insertDeleteOperator.computeOutputTypeEnvironment(context);
IAType inputRecordType = (IAType) env.getVarType(inputRecordVar);
- AbstractLogicalOperator currentOperator = assignOp;
+ AbstractLogicalOperator currentOperator = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
/**
* find the assign operator for the "input record" to the insert_delete
* operator
@@ -123,6 +120,7 @@
do {
context.addToDontApplySet(this, currentOperator);
if (currentOperator.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assignOp = (AssignOperator) currentOperator;
producedVariables.clear();
VariableUtilities.getProducedVariables(currentOperator, producedVariables);
int position = producedVariables.indexOf(oldRecordVariable);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
new file mode 100644
index 0000000..376efc4
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+
+package edu.uci.ics.asterix.optimizer.rules;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class NestedSubplanToJoinRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getValue()))
+ return false;
+ context.addToDontApplySet(this, opRef.getValue());
+
+ ILogicalOperator op1 = opRef.getValue();
+ if (op1.getInputs().size() == 0) {
+ return false;
+ }
+
+ boolean rewritten = false;
+ for (int index = 0; index < op1.getInputs().size(); index++) {
+ AbstractLogicalOperator child = (AbstractLogicalOperator) op1.getInputs().get(index).getValue();
+ if (child.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ continue;
+ }
+
+ AbstractOperatorWithNestedPlans subplan = (AbstractOperatorWithNestedPlans) child;
+ Set<LogicalVariable> freeVars = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSubplans(subplan, freeVars);
+ if (!freeVars.isEmpty()) {
+ continue;
+ }
+
+ //we only deals with the first subplan for now
+ ILogicalOperator subplanInput = subplan.getInputs().get(0).getValue();
+ List<ILogicalPlan> nestedPlans = subplan.getNestedPlans();
+ ILogicalPlan nestedPlan = nestedPlans.get(0);
+ List<Mutable<ILogicalOperator>> nestedRoots = nestedPlan.getRoots();
+
+ //expend the input and roots into a DAG of nested loop joins
+ Mutable<ILogicalExpression> expr = new MutableObject<ILogicalExpression>(ConstantExpression.TRUE);
+ Mutable<ILogicalOperator> nestedRootRef = nestedRoots.get(0);
+ ILogicalOperator join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(subplanInput),
+ nestedRootRef);
+
+ //rewrite the nested tuple source to be empty tuple source
+ rewriteNestedTupleSource(nestedRootRef);
+
+ for (int i = 1; i < nestedRoots.size(); i++) {
+ join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(join), nestedRoots.get(i));
+ }
+ op1.getInputs().get(index).setValue(join);
+ context.computeAndSetTypeEnvironmentForOperator(join);
+ rewritten = true;
+ }
+ return rewritten;
+ }
+
+ private void rewriteNestedTupleSource(Mutable<ILogicalOperator> nestedRootRef) {
+ AbstractLogicalOperator nestedRoot = (AbstractLogicalOperator) nestedRootRef.getValue();
+ if (nestedRoot.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ nestedRootRef.setValue(new EmptyTupleSourceOperator());
+ }
+ List<Mutable<ILogicalOperator>> inputs = nestedRoot.getInputs();
+ for (Mutable<ILogicalOperator> input : inputs) {
+ rewriteNestedTupleSource(input);
+ }
+ }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
index c5a1cb0..bee8c40 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
@@ -111,19 +111,7 @@
// The assign now just "renames" the variable to make sure the upstream plan still works.
srcAssignExprRef.setValue(new VariableReferenceExpression(aggVar));
-
- // Create a new assign for a TRUE variable.
- LogicalVariable trueVar = context.newVar();
- AssignOperator trueAssignOp = new AssignOperator(trueVar, new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
- ILogicalOperator aggInput = aggOp.getInputs().get(0).getValue();
- aggOp.getInputs().get(0).setValue(trueAssignOp);
- trueAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(aggInput));
-
- // Set partitioning variable.
- aggOp.setPartitioningVariable(trueVar);
-
- context.computeAndSetTypeEnvironmentForOperator(trueAssignOp);
context.computeAndSetTypeEnvironmentForOperator(aggOp);
context.computeAndSetTypeEnvironmentForOperator(assignOp);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index 3ba3e96..59c9ed0 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -22,6 +22,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -39,10 +40,12 @@
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.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.functions.IFunctionInfo;
/**
* This class is utility to do type cast.
@@ -124,6 +127,11 @@
*/
public static boolean rewriteFuncExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
IVariableTypeEnvironment env) throws AlgebricksException {
+ /**
+ * sanity check: if there are list(ordered or unordered)/record variable expressions in the funcExpr, we will not do STATIC type casting
+ * because they are not "statically cast-able".
+ * instead, the record will be dynamically casted at the runtime
+ */
if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR) {
if (reqType.equals(BuiltinType.ANY)) {
reqType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
@@ -399,32 +407,88 @@
for (int i = 0; i < openFields.length; i++) {
if (openFields[i]) {
arguments.add(originalArguments.get(2 * i));
- Mutable<ILogicalExpression> fExprRef = originalArguments.get(2 * i + 1);
- ILogicalExpression argExpr = fExprRef.getValue();
-
+ Mutable<ILogicalExpression> expRef = originalArguments.get(2 * i + 1);
+ ILogicalExpression argExpr = expRef.getValue();
+ List<LogicalVariable> parameterVars = new ArrayList<LogicalVariable>();
+ argExpr.getUsedVariables(parameterVars);
// we need to handle open fields recursively by their default
// types
// for list, their item type is any
// for record, their
- if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ boolean castInjected = false;
+ if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+ || argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
IAType reqFieldType = inputFieldTypes[i];
+ // do not enforce nested type in the case of no-used variables
if (inputFieldTypes[i].getTypeTag() == ATypeTag.RECORD) {
reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+ if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+ //inject dynamic type casting
+ injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_RECORD),
+ reqFieldType, inputFieldTypes[i], expRef, argExpr);
+ castInjected = true;
+ }
}
if (inputFieldTypes[i].getTypeTag() == ATypeTag.ORDEREDLIST) {
reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+ if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+ //inject dynamic type casting
+ injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
+ reqFieldType, inputFieldTypes[i], expRef, argExpr);
+ castInjected = true;
+ }
}
if (inputFieldTypes[i].getTypeTag() == ATypeTag.UNORDEREDLIST) {
reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+ if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+ //inject dynamic type casting
+ injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
+ reqFieldType, inputFieldTypes[i], expRef, argExpr);
+ castInjected = true;
+ }
}
- if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null) {
- ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
- rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
+ if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ //recursively rewrite function arguments
+ if (!castInjected
+ && TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null
+ && reqFieldType != null) {
+ //rewrite arg
+ ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+ rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
+ }
+ if (castInjected
+ && TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null
+ && reqFieldType != null) {
+ //rewrite the arg expression inside the dynamic cast
+ ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+ rewriteFuncExpr(argFunc, inputFieldTypes[i], inputFieldTypes[i], env);
+ }
}
}
- arguments.add(fExprRef);
+ arguments.add(expRef);
}
}
}
+ /**
+ * Inject a dynamic cast function wrapping an existing expression
+ *
+ * @param funcInfo
+ * the cast function
+ * @param reqType
+ * the required type
+ * @param inputType
+ * the original type
+ * @param exprRef
+ * the expression reference
+ * @param argExpr
+ * the original expression
+ */
+ private static void injectCastFunction(IFunctionInfo funcInfo, IAType reqType, IAType inputType,
+ Mutable<ILogicalExpression> exprRef, ILogicalExpression argExpr) {
+ ScalarFunctionCallExpression cast = new ScalarFunctionCallExpression(funcInfo);
+ cast.getArguments().add(new MutableObject<ILogicalExpression>(argExpr));
+ exprRef.setValue(cast);
+ TypeComputerUtilities.setRequiredAndInputTypes(cast, reqType, inputType);
+ }
}
diff --git a/asterix-app/src/test/resources/logging.properties b/asterix-app/src/test/resources/logging.properties
index 6f8cd00..f04eb3de 100644
--- a/asterix-app/src/test/resources/logging.properties
+++ b/asterix-app/src/test/resources/logging.properties
@@ -60,7 +60,8 @@
# For example, set the com.xyz.foo logger to only log SEVERE
# messages:
+
edu.uci.ics.asterix.test.level = INFO
#edu.uci.ics.asterix.level = FINE
-#edu.uci.ics.algebricks.level = FINE
+#edu.uci.ics.hyracks.algebricks.level = FINE
#edu.uci.ics.hyracks.level = INFO
diff --git a/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql b/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql
new file mode 100644
index 0000000..f1b2678
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql
@@ -0,0 +1,44 @@
+/*
+ * Description : This test case is to verify the fix for issue353
+ : https://code.google.com/p/asterixdb/issues/detail?id=353
+ * Expected Res : Success
+ * Date : 28th April 2013
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use dataverse tpch;
+
+create type LineItemType as closed {
+ l_orderkey: int32,
+ l_partkey: int32,
+ l_suppkey: int32,
+ l_linenumber: int32,
+ l_quantity: double,
+ l_extendedprice: double,
+ l_discount: double,
+ l_tax: double,
+ l_returnflag: string,
+ l_linestatus: string,
+ l_shipdate: string,
+ l_commitdate: string,
+ l_receiptdate: string,
+ l_shipinstruct: string,
+ l_shipmode: string,
+ l_comment: string
+}
+
+create dataset LineItem(LineItemType)
+ primary key l_orderkey, l_linenumber;
+
+write output to nc1:"/tmp/push_limit.adm";
+
+for $l in dataset('LineItem')
+limit 2
+return {
+"l_returnflag": $l.l_returnflag,
+"l_linestatus": $l.l_linestatus,
+"l_shipmode": $l.l_shipmode
+}
+
diff --git a/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan b/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan
new file mode 100644
index 0000000..829a245
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- STREAM_LIMIT |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$6(ASC), $$7(ASC) ] |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
index e69de29..e6a3879 100644
--- a/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
@@ -0,0 +1,2 @@
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql
new file mode 100644
index 0000000..59e814a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql
@@ -0,0 +1,22 @@
+/*
+ * Description : This test case is to verify the fix for issue196
+ : https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date : 5th May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type testtype1 as open {
+id : int32
+}
+
+create type testtype2 as open {
+id : int32
+}
+
+create dataset t1(testtype1) primary key id;
+create dataset t2(testtype2) primary key id;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql
new file mode 100644
index 0000000..18756f9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql
@@ -0,0 +1,20 @@
+/*
+ * Description : This test case is to verify the fix for issue196
+ : https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date : 5th May 2013
+ */
+
+use dataverse test;
+
+insert into dataset t1({"id":24});
+insert into dataset t1({"id":23});
+insert into dataset t1({"id":21});
+insert into dataset t1({"id":44});
+insert into dataset t1({"id":64});
+
+insert into dataset t2({"id":24});
+insert into dataset t2({"id":23});
+insert into dataset t2({"id":21});
+insert into dataset t2({"id":44});
+insert into dataset t2({"id":64});
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql
new file mode 100644
index 0000000..43d1980
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description : This test case is to verify the fix for issue196
+ : https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date : 5th May 2013
+ */
+
+use dataverse test;
+
+let $a := (for $l in dataset('t1') order by $l.id return $l)
+let $b := (for $m in dataset('t2') order by $m.id return $m)
+return {"a":$a,"b":$b}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql
new file mode 100644
index 0000000..a49658e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql
@@ -0,0 +1,47 @@
+/*
+ * Description : This test case is to verify the fix for issue343. It is a more general case.
+ : https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date : 30th April 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+ number: int32,
+ street: string,
+ city: string
+}
+
+create type AllType as open {
+ id: int32,
+ name: string,
+ age: float,
+ salary: double,
+ married: boolean,
+ interests: {{string}},
+ children: [string],
+ address: AddressType,
+ dob: date,
+ time: time,
+ datetime: datetime,
+ duration: duration,
+ location2d: point,
+ location3d: point3d,
+ line: line,
+ polygon: polygon,
+ circle: circle
+}
+
+create type MyListType as open{
+ id: int32,
+ mylist: [string]
+}
+
+create dataset All(AllType)
+ primary key id;
+
+create dataset MyList(MyListType)
+ primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql
new file mode 100644
index 0000000..39b85f3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql
@@ -0,0 +1,21 @@
+/*
+ * Description : This test case is to verify the fix for issue343. It is a more general case.
+ : https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date : 30th April 2013
+ */
+
+use dataverse test;
+
+insert into dataset MyList (
+ {
+ "id" : 1,
+ "mylist": ["blah"]
+ }
+);
+
+insert into dataset All (
+for $m in dataset MyList
+let $record:= { "id": 13, "name": string("Nancy"), "age": 32.5f, "salary": 12.000 ,"married": boolean("true"), "interests": {{"reading", "writing"}}, "children": ["Brad", "Scott"], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30Z"), "datetime": datetime("-1951-12-27T12:20:30"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.00,44.00"), "location3d": point3d("44.00,13.00,41.00"), "line" : line("10.1,11.1 10.2,11.2"), "polygon" : polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle" : circle("10.1,11.1 10.2"), "mylist" : $m.mylist }
+return $record
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql
new file mode 100644
index 0000000..e17a6e0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description : This test case is to verify the fix for issue343. It is a more general case.
+ : https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date : 30th April 2013
+ */
+
+use dataverse test;
+
+for $x in dataset All
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql
new file mode 100644
index 0000000..bd2ab1a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql
@@ -0,0 +1,41 @@
+/*
+ * Description : This test case is to verify the fix for issue343
+ : https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date : 30th April 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+ number: int32,
+ street: string,
+ city: string
+}
+
+create type AllType as open {
+ id: int32,
+ name: string,
+ age: float,
+ salary: double,
+ married: boolean,
+ interests: {{string}},
+ children: [string],
+ address: AddressType,
+ dob: date,
+ time: time,
+ datetime: datetime,
+ duration: duration,
+ location2d: point,
+ location3d: point3d,
+ line: line,
+ polygon: polygon,
+ circle: circle
+}
+
+create dataset All(AllType)
+ primary key id;
+
+
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql
new file mode 100644
index 0000000..eaafddd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql
@@ -0,0 +1,14 @@
+/*
+ * Description : This test case is to verify the fix for issue343
+ : https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date : 30th April 2013
+ */
+
+use dataverse test;
+
+insert into dataset All (
+let $addedList := ["blah"]
+let $record:= { "id": 13, "name": string("Nancy"), "age": 32.5f, "salary": 12.000 ,"married": boolean("true"), "interests": {{"reading", "writing"}}, "children": ["Brad", "Scott"], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30Z"), "datetime": datetime("-1951-12-27T12:20:30"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.00,44.00"), "location3d": point3d("44.00,13.00,41.00"), "line" : line("10.1,11.1 10.2,11.2"), "polygon" : polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle" : circle("10.1,11.1 10.2"), "mylist" : $addedList }
+return $record
+)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql
new file mode 100644
index 0000000..9dfde51
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description : This test case is to verify the fix for issue343
+ : https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date : 30th April 2013
+ */
+
+use dataverse test;
+
+for $x in dataset All
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql
new file mode 100644
index 0000000..85f2160
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description : This test case is to verify the fix for issue350
+ : https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date : 28th April 2013
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TweetMessageType as open {
+tweetid: string
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql
new file mode 100644
index 0000000..372d7ca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql
@@ -0,0 +1,50 @@
+/*
+ * Description : This test case is to verify the fix for issue350
+ : https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+load dataset TweetMessages
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/twitter/tw_messages.adm"),("format"="adm"));
+
+insert into dataset TweetMessages
+(
+ {"tweetid":"13",
+ "user":
+ {"screen-name":"NathanGiesen@211",
+ "lang":"en",
+ "friends_count":39345,
+ "statuses_count":479,
+ "name":"Nathan Giesen",
+ "followers_count":49420,
+ "hobbies":["basket weaving","mud wrestling"]
+ },
+ "sender-location":point("47.44,80.65"),
+ "send-time":datetime("2008-04-26T10:10:35"),
+ "referred-topics":{{"tweeting"}},
+ "message-text":"tweety tweet, my fellow tweeters!"
+ }
+);
+
+insert into dataset TweetMessages
+(
+ {"tweetid":"15",
+ "user":
+ {"screen-name":"Jason17",
+ "lang":"en",
+ "friends_count":393,
+ "statuses_count":47,
+ "name":"Nathan Giesen",
+ "followers_count":420,
+ "hobbies":["swimming"]
+ },
+ "sender-location":point("49.44,80.65"),
+ "send-time":datetime("2009-04-26T10:10:35"),
+ "referred-topics":{{"nothing"}},
+ "message-text":"Nothing to say!"
+ }
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql
new file mode 100644
index 0000000..298940a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description : This test case is to verify the fix for issue350
+ : https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+for $tm in dataset TweetMessages
+where (every $h in $tm.user.hobbies satisfies $h = "basket weaving")
+order by $tm.tweetid
+return $tm;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql
new file mode 100644
index 0000000..85f2160
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description : This test case is to verify the fix for issue350
+ : https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date : 28th April 2013
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TweetMessageType as open {
+tweetid: string
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql
new file mode 100644
index 0000000..372d7ca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql
@@ -0,0 +1,50 @@
+/*
+ * Description : This test case is to verify the fix for issue350
+ : https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+load dataset TweetMessages
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/twitter/tw_messages.adm"),("format"="adm"));
+
+insert into dataset TweetMessages
+(
+ {"tweetid":"13",
+ "user":
+ {"screen-name":"NathanGiesen@211",
+ "lang":"en",
+ "friends_count":39345,
+ "statuses_count":479,
+ "name":"Nathan Giesen",
+ "followers_count":49420,
+ "hobbies":["basket weaving","mud wrestling"]
+ },
+ "sender-location":point("47.44,80.65"),
+ "send-time":datetime("2008-04-26T10:10:35"),
+ "referred-topics":{{"tweeting"}},
+ "message-text":"tweety tweet, my fellow tweeters!"
+ }
+);
+
+insert into dataset TweetMessages
+(
+ {"tweetid":"15",
+ "user":
+ {"screen-name":"Jason17",
+ "lang":"en",
+ "friends_count":393,
+ "statuses_count":47,
+ "name":"Nathan Giesen",
+ "followers_count":420,
+ "hobbies":["swimming"]
+ },
+ "sender-location":point("49.44,80.65"),
+ "send-time":datetime("2009-04-26T10:10:35"),
+ "referred-topics":{{"nothing"}},
+ "message-text":"Nothing to say!"
+ }
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql
new file mode 100644
index 0000000..79a620b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description : This test case is to verify the fix for issue350
+ : https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+for $tm in dataset TweetMessages
+where (some $h in $tm.user.hobbies satisfies $h = "basket weaving")
+return $tm;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm
new file mode 100644
index 0000000..f93766f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm
@@ -0,0 +1 @@
+{ "a": [ { "id": 21 }, { "id": 23 }, { "id": 24 }, { "id": 44 }, { "id": 64 } ], "b": [ { "id": 21 }, { "id": 23 }, { "id": 24 }, { "id": 44 }, { "id": 64 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
new file mode 100644
index 0000000..5196a0f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
@@ -0,0 +1 @@
+{ "id": 13, "name": "Nancy", "age": 32.5f, "salary": 12.0d, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
new file mode 100644
index 0000000..5196a0f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
@@ -0,0 +1 @@
+{ "id": 13, "name": "Nancy", "age": 32.5f, "salary": 12.0d, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
new file mode 100644
index 0000000..6466feb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
@@ -0,0 +1,10 @@
+{ "tweetid": "1", "tweetid-copy": "1", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstein", "followers_count": 3311368 }, "sender-location": point("42.13,80.43"), "send-time": datetime("2005-12-05T21:06:41.000Z"), "send-time-copy": datetime("2005-12-05T21:06:41.000Z"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " love samsung the plan is amazing" }
+{ "tweetid": "10", "tweetid-copy": "10", "user": { "screen-name": "Rolldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstful", "followers_count": 3311368 }, "sender-location": point("46.94,93.98"), "send-time": datetime("2011-04-07T14:08:46.000Z"), "send-time-copy": datetime("2011-04-07T14:08:46.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is good" }
+{ "tweetid": "2", "tweetid-copy": "2", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "David Eckhardstein", "followers_count": 3311368 }, "sender-location": point("28.86,70.44"), "send-time": datetime("2007-08-15T06:44:17.000Z"), "send-time-copy": datetime("2007-08-15T06:44:17.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint its voice-clarity is mind-blowing" }
+{ "tweetid": "3", "tweetid-copy": "3", "user": { "screen-name": "RollandEckhard#500", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Hetfield", "followers_count": 3311368 }, "sender-location": point("39.84,86.48"), "send-time": datetime("2008-12-24T00:07:04.000Z"), "send-time-copy": datetime("2008-12-24T00:07:04.000Z"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is terrible:(" }
+{ "tweetid": "4", "tweetid-copy": "4", "user": { "screen-name": "RollandEckhardstein#221", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstinz", "followers_count": 3311368 }, "sender-location": point("27.67,87.32"), "send-time": datetime("2007-02-05T16:39:13.000Z"), "send-time-copy": datetime("2007-02-05T16:39:13.000Z"), "referred-topics": {{ "t-mobile", "customer-service" }}, "message-text": " love t-mobile its customer-service is mind-blowing" }
+{ "tweetid": "5", "tweetid-copy": "5", "user": { "screen-name": "RollandEcstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardst", "followers_count": 3311368 }, "sender-location": point("27.3,92.77"), "send-time": datetime("2010-09-12T06:15:28.000Z"), "send-time-copy": datetime("2010-09-12T06:15:28.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile the customization is amazing:)" }
+{ "tweetid": "6", "tweetid-copy": "6", "user": { "screen-name": "Rollkhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Kirk Hammette ", "followers_count": 3311368 }, "sender-location": point("45.62,84.78"), "send-time": datetime("2012-01-23T06:23:13.000Z"), "send-time-copy": datetime("2012-01-23T06:23:13.000Z"), "referred-topics": {{ "iphone", "network" }}, "message-text": " like iphone its network is awesome:)" }
+{ "tweetid": "7", "tweetid-copy": "7", "user": { "screen-name": "andEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland khardstein", "followers_count": 3311368 }, "sender-location": point("44.12,81.46"), "send-time": datetime("2012-02-17T17:30:26.000Z"), "send-time-copy": datetime("2012-02-17T17:30:26.000Z"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " hate t-mobile the network is bad" }
+{ "tweetid": "8", "tweetid-copy": "8", "user": { "screen-name": "Rolltein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Ron Eckhardstein", "followers_count": 3311368 }, "sender-location": point("36.86,90.71"), "send-time": datetime("2009-03-12T13:18:04.000Z"), "send-time-copy": datetime("2009-03-12T13:18:04.000Z"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " dislike at&t its touch-screen is OMG" }
+{ "tweetid": "9", "tweetid-copy": "9", "user": { "screen-name": "Roldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckdstein", "followers_count": 3311368 }, "sender-location": point("29.07,97.05"), "send-time": datetime("2012-08-15T20:19:46.000Z"), "send-time-copy": datetime("2012-08-15T20:19:46.000Z"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " hate verizon its speed is bad" }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
new file mode 100644
index 0000000..e5ac8ab
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
@@ -0,0 +1 @@
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420, "hobbies": [ "basket weaving", "mud wrestling" ] }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000Z"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index c79efac..6cb0ecc 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2676,6 +2676,31 @@
<output-dir compare="Text">query-proposal</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue350">
+ <output-dir compare="Text">query-issue350</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue350-2">
+ <output-dir compare="Text">query-issue350-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue343">
+ <output-dir compare="Text">query-issue343</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue343-2">
+ <output-dir compare="Text">query-issue343-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-closed">
+ <compilation-unit name="query-issue196">
+ <output-dir compare="Text">query-issue196</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="quantifiers">
<test-case FilePath="quantifiers">
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index 3cad3be..2be7fbd1 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -27,6 +27,7 @@
import edu.uci.ics.asterix.om.typecomputer.impl.BinaryBooleanOrNullFunctionTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringBoolOrNullTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringStringOrNullTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.CastListResultTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.CastRecordResultTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.ClosedRecordConstructorResultType;
import edu.uci.ics.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
@@ -462,6 +463,8 @@
"inject-failure", 2);
public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"cast-record", 1);
+ public final static FunctionIdentifier CAST_LIST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "cast-list", 1);
// Spatial and temporal type accessors
public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR = new FunctionIdentifier(
@@ -775,6 +778,7 @@
add(REG_EXP, ABooleanTypeComputer.INSTANCE);
add(INJECT_FAILURE, InjectFailureTypeComputer.INSTANCE);
add(CAST_RECORD, CastRecordResultTypeComputer.INSTANCE);
+ add(CAST_LIST, CastListResultTypeComputer.INSTANCE);
add(TID, AInt32TypeComputer.INSTANCE);
add(TIME_CONSTRUCTOR, OptionalATimeTypeComputer.INSTANCE);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
index ab32b6b..84c9e6c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
@@ -93,6 +93,7 @@
@Override
public void set(byte[] b, int s, int len) {
reset();
+ super.set(b, s, len);
int numberOfitems = AInt32SerializerDeserializer.getInt(b, s + 6);
int itemOffset;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java
new file mode 100644
index 0000000..d3e0699
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class CastListResultTypeComputer implements IResultTypeComputer {
+
+ public static final CastListResultTypeComputer INSTANCE = new CastListResultTypeComputer();
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+ ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) expression;
+ return TypeComputerUtilities.getRequiredType(funcExpr);
+ }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java
new file mode 100644
index 0000000..fb4e2e4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.pointables.PointableAllocator;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.pointables.cast.ACastVisitor;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+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.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class CastListDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new CastListDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+ private AbstractCollectionType reqType;
+ private AbstractCollectionType inputType;
+
+ public void reset(AbstractCollectionType reqType, AbstractCollectionType inputType) {
+ this.reqType = reqType;
+ this.inputType = inputType;
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return AsterixBuiltinFunctions.CAST_LIST;
+ }
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+ final ICopyEvaluatorFactory recordEvalFactory = args[0];
+
+ return new ICopyEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ final DataOutput out = output.getDataOutput();
+ final ArrayBackedValueStorage recordBuffer = new ArrayBackedValueStorage();
+ final ICopyEvaluator recEvaluator = recordEvalFactory.createEvaluator(recordBuffer);
+
+ return new ICopyEvaluator() {
+ // pointable allocator
+ private PointableAllocator allocator = new PointableAllocator();
+ final IVisitablePointable recAccessor = allocator.allocateListValue(inputType);
+ final IVisitablePointable resultAccessor = allocator.allocateListValue(reqType);
+ final ACastVisitor castVisitor = new ACastVisitor();
+ final Triple<IVisitablePointable, IAType, Boolean> arg = new Triple<IVisitablePointable, IAType, Boolean>(
+ resultAccessor, reqType, Boolean.FALSE);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ try {
+ recordBuffer.reset();
+ recEvaluator.evaluate(tuple);
+ recAccessor.set(recordBuffer);
+ recAccessor.accept(castVisitor, arg);
+ out.write(resultAccessor.getByteArray(), resultAccessor.getStartOffset(),
+ resultAccessor.getLength());
+ } catch (Exception ioe) {
+ throw new AlgebricksException(ioe);
+ }
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 84142be..20d70d6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -41,6 +41,7 @@
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.AUnionType;
import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.runtime.aggregates.collections.ListifyAggregateDescriptor;
@@ -67,6 +68,11 @@
import edu.uci.ics.asterix.runtime.aggregates.std.SumAggregateDescriptor;
import edu.uci.ics.asterix.runtime.aggregates.stream.EmptyStreamAggregateDescriptor;
import edu.uci.ics.asterix.runtime.aggregates.stream.NonEmptyStreamAggregateDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalDayAccessor;
import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalHourAccessor;
import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalIntervalEndAccessor;
@@ -76,11 +82,6 @@
import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalMonthAccessor;
import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalSecondAccessor;
import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalYearAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
import edu.uci.ics.asterix.runtime.evaluators.common.CreateMBREvalFactory;
import edu.uci.ics.asterix.runtime.evaluators.common.FieldAccessByIndexEvalFactory;
import edu.uci.ics.asterix.runtime.evaluators.common.FunctionManagerImpl;
@@ -111,6 +112,7 @@
import edu.uci.ics.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.CastListDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.CastRecordDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.ClosedRecordConstructorDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.CodePointToStringDescriptor;
@@ -170,10 +172,6 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.StartsWithDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.StringConcatDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.StringEndWithDescrtiptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.StringEqualDescriptor;
@@ -189,6 +187,10 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.Substring2Descriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringAfterDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringBeforeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDateDurationDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDatetimeDurationDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddTimeDurationDescriptor;
@@ -215,13 +217,13 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalEndsDecriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalMeetsDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalMetByDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MillisecondsOfDayTimeDurationDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MonthsOfYearMonthDurationDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalOverlappedByDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MillisecondsOfDayTimeDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MonthsOfYearMonthDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDateDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDatetimeDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractTimeDescriptor;
@@ -462,6 +464,7 @@
temp.add(SwitchCaseDescriptor.FACTORY);
temp.add(RegExpDescriptor.FACTORY);
temp.add(InjectFailureDescriptor.FACTORY);
+ temp.add(CastListDescriptor.FACTORY);
temp.add(CastRecordDescriptor.FACTORY);
temp.add(NotNullDescriptor.FACTORY);
@@ -689,10 +692,17 @@
}
}
if (fd.getIdentifier().equals(AsterixBuiltinFunctions.CAST_RECORD)) {
- ARecordType rt = (ARecordType) TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) expr);
- ARecordType it = (ARecordType) TypeComputerUtilities.getInputType((AbstractFunctionCallExpression) expr);
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ ARecordType rt = (ARecordType) TypeComputerUtilities.getRequiredType(funcExpr);
+ ARecordType it = (ARecordType) context.getType(funcExpr.getArguments().get(0).getValue());
((CastRecordDescriptor) fd).reset(rt, it);
}
+ if (fd.getIdentifier().equals(AsterixBuiltinFunctions.CAST_LIST)) {
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ AbstractCollectionType rt = (AbstractCollectionType) TypeComputerUtilities.getRequiredType(funcExpr);
+ AbstractCollectionType it = (AbstractCollectionType) context.getType(funcExpr.getArguments().get(0).getValue());
+ ((CastListDescriptor) fd).reset(rt, it);
+ }
if (fd.getIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)) {
ARecordType rt = (ARecordType) context.getType(expr);
((OpenRecordConstructorDescriptor) fd).reset(rt,
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index c70cdf8..415ee33 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -22,6 +22,8 @@
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
import edu.uci.ics.asterix.runtime.evaluators.common.AsterixListAccessor;
import edu.uci.ics.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -73,12 +75,20 @@
private ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
private ICopyEvaluator argEval = listEvalFactory.createEvaluator(inputVal);
private int itemIndex;
+ private boolean metNull = false;
@Override
public void init(IFrameTupleReference tuple) throws AlgebricksException {
try {
+ metNull = false;
inputVal.reset();
argEval.evaluate(tuple);
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+ .deserialize(inputVal.getByteArray()[0]);
+ if (typeTag == ATypeTag.NULL) {
+ metNull = true;
+ return;
+ }
listAccessor.reset(inputVal.getByteArray(), 0);
itemIndex = 0;
} catch (AsterixException e) {
@@ -89,10 +99,12 @@
@Override
public boolean step() throws AlgebricksException {
try {
- if (itemIndex < listAccessor.size()) {
- listAccessor.writeItem(itemIndex, out);
- ++itemIndex;
- return true;
+ if (!metNull) {
+ if (itemIndex < listAccessor.size()) {
+ listAccessor.writeItem(itemIndex, out);
+ ++itemIndex;
+ return true;
+ }
}
} catch (IOException e) {
throw new AlgebricksException(e);