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/08 13:37:29 UTC

svn commit: r1348029 - in /incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery: metadata/QueryResultDataSink.java xmlquery/translator/XMLQueryTranslator.java

Author: vinayakb
Date: Fri Jun  8 11:37:29 2012
New Revision: 1348029

URL: http://svn.apache.org/viewvc?rev=1348029&view=rev
Log:
Fixed translated plan to have an output operator as the top-most operator to write the result

Added:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java
Modified:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java?rev=1348029&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java Fri Jun  8 11:37:29 2012
@@ -0,0 +1,31 @@
+package org.apache.vxquery.metadata;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FileSplitDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
+public class QueryResultDataSink implements IDataSink {
+    private final IPartitioningProperty pProperty;
+
+    public QueryResultDataSink() {
+        pProperty = new RandomPartitioningProperty(new FileSplitDomain(new FileSplit[] { new FileSplit("FOOnode",
+                "/tmp/junk") }));
+    }
+
+    @Override
+    public Object getId() {
+        return null;
+    }
+
+    @Override
+    public Object[] getSchemaTypes() {
+        return null;
+    }
+
+    @Override
+    public IPartitioningProperty getPartitioningProperty() {
+        return pProperty;
+    }
+}
\ 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=1348029&r1=1348028&r2=1348029&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 Fri Jun  8 11:37:29 2012
@@ -27,6 +27,7 @@ import org.apache.vxquery.functions.Exte
 import org.apache.vxquery.functions.Function;
 import org.apache.vxquery.functions.Signature;
 import org.apache.vxquery.functions.UserDefinedXQueryFunction;
+import org.apache.vxquery.metadata.QueryResultDataSink;
 import org.apache.vxquery.types.AnyItemType;
 import org.apache.vxquery.types.AnyNodeType;
 import org.apache.vxquery.types.AnyType;
@@ -149,6 +150,7 @@ import edu.uci.ics.hyracks.algebricks.co
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 
 public class XMLQueryTranslator {
@@ -225,16 +227,16 @@ public class XMLQueryTranslator {
         parsePrologPass1(prologNode);
         parsePrologPass2(prologNode);
 
-        Pair<ILogicalPlan, LogicalVariable> planAndVar = null;
+        ILogicalPlan plan = null;
         switch (moduleNode.getTag()) {
             case LIBRARY_MODULE:
                 throw new SystemException(ErrorCode.TODO);
 
             case MAIN_MODULE:
-                planAndVar = translateMainModule((MainModuleNode) moduleNode);
+                plan = translateMainModule((MainModuleNode) moduleNode);
         }
 
-        module.setBody(planAndVar.getLeft());
+        module.setBody(plan);
         return module;
     }
 
@@ -599,16 +601,18 @@ public class XMLQueryTranslator {
         }
     }
 
-    private Pair<ILogicalPlan, LogicalVariable> translateMainModule(MainModuleNode moduleNode) throws SystemException {
+    private ILogicalPlan translateMainModule(MainModuleNode moduleNode) throws SystemException {
         QueryBodyNode qbn = moduleNode.getQueryBody();
         ASTNode queryBody = qbn.getExpression();
         TranslationContext tCtx = new TranslationContext(null, new EmptyTupleSourceOperator());
         LogicalVariable lVar = translateExpression(queryBody, tCtx);
         List<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
         exprs.add(mutable(vre(lVar)));
-        ALogicalPlanImpl lp = new ALogicalPlanImpl(mutable(tCtx.op));
+        WriteOperator op = new WriteOperator(exprs, new QueryResultDataSink());
+        op.getInputs().add(mutable(tCtx.op));
+        ALogicalPlanImpl lp = new ALogicalPlanImpl(mutable(op));
 
-        return Pair.<ILogicalPlan, LogicalVariable> of(lp, lVar);
+        return lp;
     }
 
     private LogicalVariable translateExpression(ASTNode value, TranslationContext tCtx) throws SystemException {