You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by vi...@apache.org on 2012/06/20 14:36:52 UTC

svn commit: r1352067 - in /incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main: java/org/apache/vxquery/compiler/algebricks/ java/org/apache/vxquery/exceptions/ java/org/apache/vxquery/functions/ java/org/apache/vxquery/runtime/function...

Author: vinayakb
Date: Wed Jun 20 12:36:51 2012
New Revision: 1352067

URL: http://svn.apache.org/viewvc?rev=1352067&view=rev
Log:
Added opext:sequence and opext:iterate in support of FLWOR.

Added:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluator.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluatorFactory.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/IterateUnnestingEvaluatorFactory.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/SequenceAggregateEvaluatorFactory.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/TreatScalarEvaluatorFactory.java
Modified:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionJobGen.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionRuntimeProvider.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/exceptions/SystemException.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/Function.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-fn-defns.xsl
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-op-defns.xsl

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionJobGen.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionJobGen.java?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionJobGen.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionJobGen.java Wed Jun 20 12:36:51 2012
@@ -12,14 +12,20 @@ import edu.uci.ics.hyracks.algebricks.co
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyAggregateFunction;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyAggregateFunctionFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyRunningAggregateFunctionFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyUnnestingFunction;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyUnnestingFunctionFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
@@ -43,7 +49,8 @@ public class VXQueryExpressionJobGen imp
     public ICopyAggregateFunctionFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
-        return null;
+        IAggregateEvaluatorFactory aef = erp.createAggregateFunctionFactory(expr, env, inputSchemas, context);
+        return new AggregateCopyEvaluatorFactoryAdapter(aef);
     }
 
     @Override
@@ -64,26 +71,27 @@ public class VXQueryExpressionJobGen imp
     public ICopyUnnestingFunctionFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
-        return null;
+        IUnnestingEvaluatorFactory uef = erp.createUnnestingFunctionFactory(expr, env, inputSchemas, context);
+        return new UnnestCopyEvaluatorFactoryAdapter(uef);
     }
 
     private static final class ScalarCopyEvaluatorFactoryAdapter implements ICopyEvaluatorFactory {
         private static final long serialVersionUID = 1L;
 
-        private final IScalarEvaluatorFactory ef;
+        private final IScalarEvaluatorFactory sef;
 
-        public ScalarCopyEvaluatorFactoryAdapter(IScalarEvaluatorFactory ef) {
-            this.ef = ef;
+        public ScalarCopyEvaluatorFactoryAdapter(IScalarEvaluatorFactory sef) {
+            this.sef = sef;
         }
 
         @Override
         public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
-            final IScalarEvaluator e = ef.createScalarEvaluator();
+            final IScalarEvaluator se = sef.createScalarEvaluator();
             final IPointable p = VoidPointable.FACTORY.createPointable();
             return new ICopyEvaluator() {
                 @Override
                 public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
-                    e.evaluate(tuple, p);
+                    se.evaluate(tuple, p);
                     try {
                         output.getDataOutput().write(p.getByteArray(), p.getStartOffset(), p.getLength());
                     } catch (IOException ex) {
@@ -93,4 +101,85 @@ public class VXQueryExpressionJobGen imp
             };
         }
     }
+
+    private static final class AggregateCopyEvaluatorFactoryAdapter implements ICopyAggregateFunctionFactory {
+        private static final long serialVersionUID = 1L;
+
+        private final IAggregateEvaluatorFactory aef;
+
+        public AggregateCopyEvaluatorFactoryAdapter(IAggregateEvaluatorFactory aef) {
+            this.aef = aef;
+        }
+
+        @Override
+        public ICopyAggregateFunction createAggregateFunction(final IDataOutputProvider output)
+                throws AlgebricksException {
+            final IAggregateEvaluator ae = aef.createAggregateEvaluator();
+            final IPointable p = VoidPointable.FACTORY.createPointable();
+            return new ICopyAggregateFunction() {
+
+                @Override
+                public void step(IFrameTupleReference tuple) throws AlgebricksException {
+                    ae.step(tuple);
+                }
+
+                @Override
+                public void init() throws AlgebricksException {
+                    ae.init();
+                }
+
+                @Override
+                public void finishPartial() throws AlgebricksException {
+                    finish();
+                }
+
+                @Override
+                public void finish() throws AlgebricksException {
+                    ae.finish(p);
+                    try {
+                        output.getDataOutput().write(p.getByteArray(), p.getStartOffset(), p.getLength());
+                    } catch (IOException ex) {
+                        throw new AlgebricksException(ex);
+                    }
+                }
+            };
+        }
+    }
+
+    private static final class UnnestCopyEvaluatorFactoryAdapter implements ICopyUnnestingFunctionFactory {
+        private static final long serialVersionUID = 1L;
+
+        private final IUnnestingEvaluatorFactory uef;
+
+        public UnnestCopyEvaluatorFactoryAdapter(IUnnestingEvaluatorFactory uef) {
+            this.uef = uef;
+        }
+
+        @Override
+        public ICopyUnnestingFunction createUnnestingFunction(final IDataOutputProvider output)
+                throws AlgebricksException {
+            final IUnnestingEvaluator ue = uef.createUnnestingEvaluator();
+            final IPointable p = VoidPointable.FACTORY.createPointable();
+            return new ICopyUnnestingFunction() {
+                @Override
+                public boolean step() throws AlgebricksException {
+                    boolean status = ue.step(p);
+                    if (status) {
+                        try {
+                            output.getDataOutput().write(p.getByteArray(), p.getStartOffset(), p.getLength());
+                        } catch (IOException ex) {
+                            throw new AlgebricksException(ex);
+                        }
+                        return true;
+                    }
+                    return false;
+                }
+
+                @Override
+                public void init(IFrameTupleReference tuple) throws AlgebricksException {
+                    ue.init(tuple);
+                }
+            };
+        }
+    }
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionRuntimeProvider.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionRuntimeProvider.java?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionRuntimeProvider.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/algebricks/VXQueryExpressionRuntimeProvider.java Wed Jun 20 12:36:51 2012
@@ -44,12 +44,8 @@ public class VXQueryExpressionRuntimePro
                 ScalarFunctionCallExpression fcExpr = (ScalarFunctionCallExpression) expr;
                 Function fn = (Function) fcExpr.getFunctionInfo();
 
-                List<Mutable<ILogicalExpression>> args = fcExpr.getArguments();
-                IScalarEvaluatorFactory[] argFactories = new IScalarEvaluatorFactory[args.size()];
-                for (int i = 0; i < argFactories.length; ++i) {
-                    Mutable<ILogicalExpression> arg = args.get(i);
-                    argFactories[i] = createEvaluatorFactory(arg.getValue(), env, inputSchemas, context);
-                }
+                IScalarEvaluatorFactory[] argFactories = createArgumentEvaluatorFactories(env, inputSchemas, context,
+                        fcExpr.getArguments());
 
                 try {
                     return fn.createScalarEvaluatorFactory(argFactories);
@@ -60,11 +56,30 @@ public class VXQueryExpressionRuntimePro
         throw new UnsupportedOperationException("Cannot create runtime for " + expr.getExpressionTag());
     }
 
+    private IScalarEvaluatorFactory[] createArgumentEvaluatorFactories(IVariableTypeEnvironment env,
+            IOperatorSchema[] inputSchemas, JobGenContext context, List<Mutable<ILogicalExpression>> args)
+            throws AlgebricksException {
+        IScalarEvaluatorFactory[] argFactories = new IScalarEvaluatorFactory[args.size()];
+        for (int i = 0; i < argFactories.length; ++i) {
+            Mutable<ILogicalExpression> arg = args.get(i);
+            argFactories[i] = createEvaluatorFactory(arg.getValue(), env, inputSchemas, context);
+        }
+        return argFactories;
+    }
+
     @Override
     public IAggregateEvaluatorFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
-        return null;
+        Function fn = (Function) expr.getFunctionInfo();
+
+        IScalarEvaluatorFactory[] argFactories = createArgumentEvaluatorFactories(env, inputSchemas, context,
+                expr.getArguments());
+        try {
+            return fn.createAggregateEvaluatorFactory(argFactories);
+        } catch (SystemException e) {
+            throw new AlgebricksException(e);
+        }
     }
 
     @Override
@@ -85,6 +100,14 @@ public class VXQueryExpressionRuntimePro
     public IUnnestingEvaluatorFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
-        return null;
+        Function fn = (Function) expr.getFunctionInfo();
+
+        IScalarEvaluatorFactory[] argFactories = createArgumentEvaluatorFactories(env, inputSchemas, context,
+                expr.getArguments());
+        try {
+            return fn.createUnnestingEvaluatorFactory(argFactories);
+        } catch (SystemException e) {
+            throw new AlgebricksException(e);
+        }
     }
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/exceptions/SystemException.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/exceptions/SystemException.java?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/exceptions/SystemException.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/exceptions/SystemException.java Wed Jun 20 12:36:51 2012
@@ -17,22 +17,26 @@
 package org.apache.vxquery.exceptions;
 
 import java.text.MessageFormat;
+import java.util.Arrays;
 
 import org.apache.vxquery.util.SourceLocation;
 
 public class SystemException extends Exception {
     private static final long serialVersionUID = 1L;
 
-    private ErrorCode code;    
-    
+    private ErrorCode code;
+
     private static String message(ErrorCode code, SourceLocation loc) {
-        return code + ": " + (loc == null ? "" : loc + " ") + code.getDescription();
+        String description = code.getDescription();
+        return code + ": " + (loc == null ? "" : loc + " ") + (description != null ? description : "");
     }
-    
+
     private static String message(ErrorCode code, SourceLocation loc, Object... params) {
-        return code + ": " + (loc == null ? "" : loc + " ") + MessageFormat.format(code.getDescription(), params);
+        String description = code.getDescription();
+        return code + ": " + (loc == null ? "" : loc + " ")
+                + (description != null ? MessageFormat.format(description, params) : Arrays.deepToString(params));
     }
-    
+
     public SystemException(ErrorCode code) {
         super(message(code, null));
         this.code = code;
@@ -47,12 +51,12 @@ public class SystemException extends Exc
         super(message(code, null), cause);
         this.code = code;
     }
-    
+
     public SystemException(ErrorCode code, Throwable cause, Object... params) {
         super(message(code, null, params), cause);
         this.code = code;
     }
-    
+
     public SystemException(ErrorCode code, SourceLocation loc) {
         super(message(code, loc));
         this.code = code;

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/Function.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/Function.java?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/Function.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/Function.java Wed Jun 20 12:36:51 2012
@@ -18,7 +18,6 @@ package org.apache.vxquery.functions;
 
 import javax.xml.namespace.QName;
 
-import org.apache.vxquery.exceptions.DefaultSystemExceptionFactory;
 import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
 
@@ -52,15 +51,17 @@ public abstract class Function implement
     public abstract boolean useContextImplicitly();
 
     public IScalarEvaluatorFactory createScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) throws SystemException {
-        throw DefaultSystemExceptionFactory.INSTANCE.createException(ErrorCode.SYSE0001);
+        throw new SystemException(ErrorCode.SYSE0001, "No IScalarEvaluatorFactory runtime for " + fid.getName());
     }
 
-    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) throws SystemException {
-        throw DefaultSystemExceptionFactory.INSTANCE.createException(ErrorCode.SYSE0001);
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args)
+            throws SystemException {
+        throw new SystemException(ErrorCode.SYSE0001, "No IAggregateEvaluatorFactory runtime for " + fid.getName());
     }
 
-    public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args) throws SystemException {
-        throw DefaultSystemExceptionFactory.INSTANCE.createException(ErrorCode.SYSE0001);
+    public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args)
+            throws SystemException {
+        throw new SystemException(ErrorCode.SYSE0001, "No IUnnestingEvaluatorFactory runtime for " + fid.getName());
     }
 
     public QName getName() {

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml Wed Jun 20 12:36:51 2012
@@ -815,12 +815,14 @@
     <operator name="opext:sequence">
         <param name="arg" type="item()*"/>
         <return type="item()*"/>
+        <runtime type="aggregate" class="org.apache.vxquery.runtime.functions.sequence.SequenceAggregateEvaluatorFactory"/>
     </operator>
 
     <!-- opext:iterate($arg as item()*) as item() -->
     <operator name="opext:iterate">
         <param name="arg" type="item()*"/>
         <return type="item()"/>
+        <runtime type="unnesting" class="org.apache.vxquery.runtime.functions.sequence.IterateUnnestingEvaluatorFactory"/>
     </operator>
 
     <!-- opext:validate-lax($arg as item()*) as item()* -->
@@ -848,6 +850,7 @@
         <param name="arg" type="item()*"/>
         <param name="type" type="xs:int"/>
         <return type="item()*"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.type.TreatScalarEvaluatorFactory"/>
     </operator>
 
     <!-- opext:cast($arg as item()*, $type as xsext:type) as item()* -->

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluator.java?rev=1352067&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluator.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluator.java Wed Jun 20 12:36:51 2012
@@ -0,0 +1,37 @@
+package org.apache.vxquery.runtime.functions.base;
+
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.exceptions.SystemException;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractTaggedValueArgumentAggregateEvaluator implements IAggregateEvaluator {
+    private final IScalarEvaluator[] args;
+
+    private final TaggedValuePointable[] tvps;
+
+    public AbstractTaggedValueArgumentAggregateEvaluator(IScalarEvaluator[] args) {
+        this.args = args;
+        tvps = new TaggedValuePointable[args.length];
+        for (int i = 0; i < tvps.length; ++i) {
+            tvps[i] = new TaggedValuePointable();
+        }
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple) throws AlgebricksException {
+        for (int i = 0; i < args.length; ++i) {
+            args[i].evaluate(tuple, tvps[i]);
+        }
+        try {
+            step(tvps);
+        } catch (SystemException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    protected abstract void step(TaggedValuePointable[] args) throws SystemException;
+}
\ No newline at end of file

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluatorFactory.java?rev=1352067&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluatorFactory.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentAggregateEvaluatorFactory.java Wed Jun 20 12:36:51 2012
@@ -0,0 +1,28 @@
+package org.apache.vxquery.runtime.functions.base;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+
+public abstract class AbstractTaggedValueArgumentAggregateEvaluatorFactory implements IAggregateEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final IScalarEvaluatorFactory[] args;
+
+    public AbstractTaggedValueArgumentAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        this.args = args;
+    }
+
+    @Override
+    public final IAggregateEvaluator createAggregateEvaluator() throws AlgebricksException {
+        IScalarEvaluator[] es = new IScalarEvaluator[args.length];
+        for (int i = 0; i < es.length; ++i) {
+            es[i] = args[i].createScalarEvaluator();
+        }
+        return createEvaluator(es);
+    }
+
+    protected abstract IAggregateEvaluator createEvaluator(IScalarEvaluator[] args) throws AlgebricksException;
+}
\ No newline at end of file

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java?rev=1352067&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java Wed Jun 20 12:36:51 2012
@@ -0,0 +1,32 @@
+package org.apache.vxquery.runtime.functions.base;
+
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractTaggedValueArgumentUnnestingEvaluator implements IUnnestingEvaluator {
+    private final IScalarEvaluator[] args;
+
+    protected final TaggedValuePointable[] tvps;
+
+    public AbstractTaggedValueArgumentUnnestingEvaluator(IScalarEvaluator[] args) {
+        this.args = args;
+        tvps = new TaggedValuePointable[args.length];
+        for (int i = 0; i < tvps.length; ++i) {
+            tvps[i] = new TaggedValuePointable();
+        }
+    }
+
+    @Override
+    public final void init(IFrameTupleReference tuple) throws AlgebricksException {
+        for (int i = 0; i < args.length; ++i) {
+            args[i].evaluate(tuple, tvps[i]);
+        }
+        init(tvps);
+    }
+
+    protected abstract void init(TaggedValuePointable[] args);
+}
\ No newline at end of file

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java?rev=1352067&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java Wed Jun 20 12:36:51 2012
@@ -0,0 +1,28 @@
+package org.apache.vxquery.runtime.functions.base;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+
+public abstract class AbstractTaggedValueArgumentUnnestingEvaluatorFactory implements IUnnestingEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final IScalarEvaluatorFactory[] args;
+
+    public AbstractTaggedValueArgumentUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        this.args = args;
+    }
+
+    @Override
+    public final IUnnestingEvaluator createUnnestingEvaluator() throws AlgebricksException {
+        IScalarEvaluator[] es = new IScalarEvaluator[args.length];
+        for (int i = 0; i < es.length; ++i) {
+            es[i] = args[i].createScalarEvaluator();
+        }
+        return createEvaluator(es);
+    }
+
+    protected abstract IUnnestingEvaluator createEvaluator(IScalarEvaluator[] args) throws AlgebricksException;
+}
\ No newline at end of file

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/IterateUnnestingEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/IterateUnnestingEvaluatorFactory.java?rev=1352067&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/IterateUnnestingEvaluatorFactory.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/IterateUnnestingEvaluatorFactory.java Wed Jun 20 12:36:51 2012
@@ -0,0 +1,59 @@
+package org.apache.vxquery.runtime.functions.sequence;
+
+import org.apache.vxquery.datamodel.accessors.SequencePointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluator;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluatorFactory;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+
+public class IterateUnnestingEvaluatorFactory extends AbstractTaggedValueArgumentUnnestingEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public IterateUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IUnnestingEvaluator createEvaluator(IScalarEvaluator[] args) throws AlgebricksException {
+        final SequencePointable seqp = new SequencePointable();
+        return new AbstractTaggedValueArgumentUnnestingEvaluator(args) {
+            private int index;
+            private int seqLength;
+
+            @Override
+            public boolean step(IPointable result) throws AlgebricksException {
+                TaggedValuePointable tvp = tvps[0];
+                if (tvp.getTag() != ValueTag.SEQUENCE_TAG) {
+                    if (index == 0) {
+                        result.set(tvp);
+                        ++index;
+                        return true;
+                    }
+                } else {
+                    if (index < seqLength) {
+                        seqp.getEntry(index, result);
+                        ++index;
+                        return true;
+                    }
+                }
+                return false;
+            }
+
+            @Override
+            protected void init(TaggedValuePointable[] args) {
+                index = 0;
+                TaggedValuePointable tvp = tvps[0];
+                if (tvp.getTag() == ValueTag.SEQUENCE_TAG) {
+                    tvp.getValue(seqp);
+                    seqLength = seqp.getEntryCount();
+                }
+            }
+        };
+    }
+}
\ No newline at end of file

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/SequenceAggregateEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/SequenceAggregateEvaluatorFactory.java?rev=1352067&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/SequenceAggregateEvaluatorFactory.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/sequence/SequenceAggregateEvaluatorFactory.java Wed Jun 20 12:36:51 2012
@@ -0,0 +1,108 @@
+package org.apache.vxquery.runtime.functions.sequence;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.vxquery.datamodel.accessors.SequencePointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentAggregateEvaluator;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentAggregateEvaluatorFactory;
+import org.apache.vxquery.util.GrowableIntArray;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class SequenceAggregateEvaluatorFactory extends AbstractTaggedValueArgumentAggregateEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public SequenceAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IAggregateEvaluator createEvaluator(IScalarEvaluator[] args) throws AlgebricksException {
+        final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+        final GrowableIntArray slots = new GrowableIntArray();
+        final ArrayBackedValueStorage dataArea = new ArrayBackedValueStorage();
+        final SequencePointable seq = new SequencePointable();
+        final VoidPointable p = (VoidPointable) VoidPointable.FACTORY.createPointable();
+        return new AbstractTaggedValueArgumentAggregateEvaluator(args) {
+            @Override
+            public void init() throws AlgebricksException {
+                abvs.reset();
+                slots.clear();
+                dataArea.reset();
+            }
+
+            @Override
+            public void finishPartial() throws AlgebricksException {
+
+            }
+
+            @Override
+            public void finish(IPointable result) throws AlgebricksException {
+                if (slots.getSize() != 1) {
+                    try {
+                        assembleResult(abvs, slots, dataArea);
+                    } catch (SystemException e) {
+                        throw new AlgebricksException(e);
+                    }
+                    result.set(abvs);
+                } else {
+                    result.set(dataArea);
+                }
+            }
+
+            @Override
+            protected void step(TaggedValuePointable[] args) throws SystemException {
+                TaggedValuePointable tvp = args[0];
+                if (tvp.getTag() == ValueTag.SEQUENCE_TAG) {
+                    tvp.getValue(seq);
+                    int seqLen = seq.getEntryCount();
+                    for (int j = 0; j < seqLen; ++j) {
+                        seq.getEntry(j, p);
+                        addItem(p);
+                    }
+                } else {
+                    addItem(tvp);
+                }
+            }
+
+            private void assembleResult(ArrayBackedValueStorage abvs, GrowableIntArray slots,
+                    ArrayBackedValueStorage dataArea) throws SystemException {
+                try {
+                    DataOutput out = abvs.getDataOutput();
+                    out.write(ValueTag.SEQUENCE_TAG);
+                    int size = slots.getSize();
+                    out.writeInt(size);
+                    if (size > 0) {
+                        int[] slotArray = slots.getArray();
+                        for (int i = 0; i < size; ++i) {
+                            out.writeInt(slotArray[i]);
+                        }
+                        out.write(dataArea.getByteArray(), dataArea.getStartOffset(), dataArea.getLength());
+                    }
+                } catch (IOException e) {
+                    throw new SystemException(ErrorCode.SYSE0001, e);
+                }
+            }
+
+            private void addItem(final IPointable p) throws SystemException {
+                try {
+                    dataArea.getDataOutput().write(p.getByteArray(), p.getStartOffset(), p.getLength());
+                    slots.append(dataArea.getLength());
+                } catch (IOException e) {
+                    throw new SystemException(ErrorCode.SYSE0001, e);
+                }
+            }
+        };
+    }
+}
\ No newline at end of file

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/TreatScalarEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/TreatScalarEvaluatorFactory.java?rev=1352067&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/TreatScalarEvaluatorFactory.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/TreatScalarEvaluatorFactory.java Wed Jun 20 12:36:51 2012
@@ -0,0 +1,26 @@
+package org.apache.vxquery.runtime.functions.type;
+
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+
+public class TreatScalarEvaluatorFactory extends AbstractTypeScalarEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public TreatScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IScalarEvaluator[] args) throws AlgebricksException {
+        return new AbstractTypeScalarEvaluator(args) {
+            @Override
+            protected void evaluate(TaggedValuePointable tvp, IPointable result) {
+                result.set(tvp);
+            }
+        };
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java Wed Jun 20 12:36:51 2012
@@ -980,7 +980,7 @@ public class XMLQueryTranslator {
                         ILogicalExpression seq = vre(translateExpression(fvdNode.getSequence(), tCtx));
                         tCtx.pushVariableScope();
                         LogicalVariable forLVar = newLogicalVariable();
-                        LogicalVariable posLVar = newLogicalVariable();
+                        LogicalVariable posLVar = fvdNode.getPosVar() != null ? newLogicalVariable() : null;
                         UnnestOperator unnest = new UnnestOperator(forLVar,
                                 mutable(ufce(BuiltinOperators.ITERATE, seq)), posLVar, null);
                         SequenceType forVarType = SequenceType.create(AnyItemType.INSTANCE, Quantifier.QUANT_ONE);

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-fn-defns.xsl
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-fn-defns.xsl?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-fn-defns.xsl (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-fn-defns.xsl Wed Jun 20 12:36:51 2012
@@ -50,6 +50,16 @@
                         return new <xsl:value-of select="@class"/>(args);
                     }
                     </xsl:if>
+                    <xsl:if test="@type = 'aggregate'">
+                    public edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory createAggregateEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[] args) {
+                        return new <xsl:value-of select="@class"/>(args);
+                    }
+                    </xsl:if>
+                    <xsl:if test="@type = 'unnesting'">
+                    public edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[] args) {
+                        return new <xsl:value-of select="@class"/>(args);
+                    }
+                    </xsl:if>
                 </xsl:for-each>
                 };
         </xsl:for-each>

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-op-defns.xsl
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-op-defns.xsl?rev=1352067&r1=1352066&r2=1352067&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-op-defns.xsl (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/xslt/generate-op-defns.xsl Wed Jun 20 12:36:51 2012
@@ -42,6 +42,16 @@
                         return new <xsl:value-of select="@class"/>(args);
                     }
                     </xsl:if>
+                    <xsl:if test="@type = 'aggregate'">
+                    public edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory createAggregateEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[] args) {
+                        return new <xsl:value-of select="@class"/>(args);
+                    }
+                    </xsl:if>
+                    <xsl:if test="@type = 'unnesting'">
+                    public edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[] args) {
+                        return new <xsl:value-of select="@class"/>(args);
+                    }
+                    </xsl:if>
                 </xsl:for-each>
                 };
         </xsl:for-each>