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 16:04:59 UTC

svn commit: r1348078 - in /incubator/vxquery/branches/vxquery_algebricks: vxquery-core/src/main/java/org/apache/vxquery/api/ vxquery-core/src/main/java/org/apache/vxquery/compiler/ vxquery-core/src/main/java/org/apache/vxquery/drivers/ vxquery-core/src...

Author: vinayakb
Date: Fri Jun  8 14:04:59 2012
New Revision: 1348078

URL: http://svn.apache.org/viewvc?rev=1348078&view=rev
Log:
Added code for Hyracks job generation

Removed:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/api/
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCodeGenerator.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryOptimizer.java
Modified:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/CompilerControlBlock.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/drivers/VXQuery.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/Module.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/NoopXQueryCompilationListener.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XQueryCompilationListener.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/test/java/org/apache/vxquery/xmlquery/query/SimpleXQueryTest.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/TestRunnerFactory.java

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/CompilerControlBlock.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/CompilerControlBlock.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/CompilerControlBlock.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/compiler/CompilerControlBlock.java Fri Jun  8 14:04:59 2012
@@ -17,22 +17,15 @@
 package org.apache.vxquery.compiler;
 
 import org.apache.vxquery.context.StaticContext;
-import org.apache.vxquery.v0datamodel.DatamodelStaticInterface;
 
 public class CompilerControlBlock {
     private final StaticContext ctx;
-    private final DatamodelStaticInterface dmStaticInterface;
 
-    public CompilerControlBlock(StaticContext ctx, DatamodelStaticInterface dmStaticInterface) {
+    public CompilerControlBlock(StaticContext ctx) {
         this.ctx = ctx;
-        this.dmStaticInterface = dmStaticInterface;
     }
 
     public StaticContext getStaticContext() {
         return ctx;
     }
-
-    public DatamodelStaticInterface getDatamodelStaticInterface() {
-        return dmStaticInterface;
-    }
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/drivers/VXQuery.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/drivers/VXQuery.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/drivers/VXQuery.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/drivers/VXQuery.java Fri Jun  8 14:04:59 2012
@@ -16,7 +16,6 @@ package org.apache.vxquery.drivers;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.PrintWriter;
 import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -24,14 +23,12 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.vxquery.api.InternalAPI;
-import org.apache.vxquery.v0datamodel.XDMItem;
-import org.apache.vxquery.v0datamodel.dtm.DTMDatamodelStaticInterfaceImpl;
-import org.apache.vxquery.v0datamodel.serialization.XMLSerializer;
-import org.apache.vxquery.v0runtime.base.OpenableCloseableIterator;
+import org.apache.vxquery.compiler.CompilerControlBlock;
+import org.apache.vxquery.context.RootStaticContextImpl;
+import org.apache.vxquery.context.StaticContextImpl;
 import org.apache.vxquery.xmlquery.ast.ModuleNode;
 import org.apache.vxquery.xmlquery.query.Module;
-import org.apache.vxquery.xmlquery.query.PrologVariable;
+import org.apache.vxquery.xmlquery.query.XMLQueryCompiler;
 import org.apache.vxquery.xmlquery.query.XQueryCompilationListener;
 import org.kohsuke.args4j.Argument;
 import org.kohsuke.args4j.CmdLineParser;
@@ -43,6 +40,7 @@ import com.thoughtworks.xstream.io.xml.D
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
 import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class VXQuery {
     public static void main(String[] args) throws Exception {
@@ -58,25 +56,18 @@ public class VXQuery {
             parser.printUsage(System.err);
             return;
         }
-        InternalAPI iapi = new InternalAPI(new DTMDatamodelStaticInterfaceImpl());
         for (String query : opts.arguments) {
             String qStr = slurp(query);
             if (opts.showQuery) {
                 System.err.println(qStr);
             }
-            ModuleNode ast = iapi.parse(query, new StringReader(qStr));
-            if (opts.showAST) {
-                System.err.println(new XStream(new DomDriver()).toXML(ast));
-            }
             XQueryCompilationListener listener = new XQueryCompilationListener() {
                 @Override
                 public void notifyCodegenResult(Module module) {
-                    /*
-                    if (opts.showRI) {
-                        System.err.println(new XStream(new DomDriver()).toXML(module.getBodyRuntimePlan()
-                                .getRuntimeIterator()));
+                    if (opts.showRP) {
+                        JobSpecification jobSpec = module.getHyracksJobSpecification();
+                        System.err.println(jobSpec.toString());
                     }
-                    */
                 }
 
                 @Override
@@ -110,46 +101,20 @@ public class VXQuery {
                         }
                     }
                 }
+
+                @Override
+                public void notifyParseResult(ModuleNode moduleNode) {
+                    if (opts.showAST) {
+                        System.err.println(new XStream(new DomDriver()).toXML(moduleNode));
+                    }
+                }
             };
-            Module module = iapi.compile(listener, ast, opts.optimizationLevel);
+            XMLQueryCompiler compiler = new XMLQueryCompiler(listener);
+            CompilerControlBlock ccb = new CompilerControlBlock(new StaticContextImpl(RootStaticContextImpl.INSTANCE));
+            compiler.compile(query, new StringReader(qStr), ccb, opts.optimizationLevel);
             if (opts.compileOnly) {
                 continue;
             }
-            PrologVariable[] prologVariables = module.getPrologVariables();
-            if (prologVariables != null) {
-                for (PrologVariable pVar : prologVariables) {
-                    String fName = opts.bindings.get(pVar.getVariable().getName().getLocalPart());
-                    if (fName != null) {
-                        File f = new File(fName);
-                        System.err.println("Binding: " + pVar.getVariable().getName() + " to " + f.getAbsolutePath());
-                        iapi.bindExternalVariable(pVar.getVariable(), f);
-                    }
-                }
-            }
-            OpenableCloseableIterator ri = iapi.execute(module);
-            for (int i = 0; i < opts.repeatExec; ++i) {
-                long start = System.currentTimeMillis();
-                ri.open();
-                System.err.println("--- Results begin");
-                XDMItem o;
-                PrintWriter out = new PrintWriter(System.out, true);
-                XMLSerializer s = new XMLSerializer(out, false);
-                s.open();
-                try {
-                    while ((o = (XDMItem) ri.next()) != null) {
-                        s.item(o);
-                    }
-                } finally {
-                    s.close();
-                    out.flush();
-                    ri.close();
-                }
-                System.err.println("--- Results end");
-                long end = System.currentTimeMillis();
-                if (opts.timing) {
-                    System.err.println("Run time: " + (end - start) + " ms");
-                }
-            }
         }
     }
 
@@ -173,8 +138,8 @@ public class VXQuery {
         @Option(name = "-showoet", usage = "Show optimized expression tree")
         private boolean showOET;
 
-        @Option(name = "-showri", usage = "Show Runtime plan")
-        private boolean showRI;
+        @Option(name = "-showrp", usage = "Show Runtime plan")
+        private boolean showRP;
 
         @Option(name = "-compileonly", usage = "Compile the query and stop")
         private boolean compileOnly;

Modified: 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=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/QueryResultDataSink.java Fri Jun  8 14:04:59 2012
@@ -7,11 +7,12 @@ import edu.uci.ics.hyracks.algebricks.co
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 
 public class QueryResultDataSink implements IDataSink {
+    private final FileSplit[] fileSplits;
     private final IPartitioningProperty pProperty;
 
     public QueryResultDataSink() {
-        pProperty = new RandomPartitioningProperty(new FileSplitDomain(new FileSplit[] { new FileSplit("FOOnode",
-                "/tmp/junk") }));
+        fileSplits = new FileSplit[] { new FileSplit("FOOnode", "/tmp/junk") };
+        pProperty = new RandomPartitioningProperty(new FileSplitDomain(fileSplits));
     }
 
     @Override
@@ -28,4 +29,8 @@ public class QueryResultDataSink impleme
     public IPartitioningProperty getPartitioningProperty() {
         return pProperty;
     }
+
+    public FileSplit[] getFileSplits() {
+        return fileSplits;
+    }
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java Fri Jun  8 14:04:59 2012
@@ -2,6 +2,7 @@ package org.apache.vxquery.metadata;
 
 import java.util.List;
 
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
@@ -18,9 +19,12 @@ import edu.uci.ics.hyracks.algebricks.co
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 
 public class VXQueryMetadataProvider implements IMetadataProvider<String, String> {
     @Override
@@ -44,7 +48,16 @@ public class VXQueryMetadataProvider imp
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
             int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
             throws AlgebricksException {
-        return null;
+        QueryResultDataSink ds = (QueryResultDataSink) sink;
+        FileSplit[] fileSplits = ds.getFileSplits();
+        String[] locations = new String[fileSplits.length];
+        for (int i = 0; i < fileSplits.length; ++i) {
+            locations[i] = fileSplits[i].getNodeName();
+        }
+        IPushRuntimeFactory prf = new SinkWriterRuntimeFactory(printColumns, printerFactories, fileSplits[0]
+                .getLocalFile().getFile(), PrinterBasedWriterFactory.INSTANCE, inputDesc);
+        AlgebricksAbsolutePartitionConstraint constraint = new AlgebricksAbsolutePartitionConstraint(locations);
+        return new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(prf, constraint);
     }
 
     @Override

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/Module.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/Module.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/Module.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/Module.java Fri Jun  8 14:04:59 2012
@@ -19,6 +19,7 @@ import org.apache.vxquery.context.Static
 import org.apache.vxquery.v0runtime.RegisterSet;
 
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class Module {
     private ModuleType moduleType;
@@ -27,6 +28,7 @@ public class Module {
     private String namespaceUri;
     private PrologVariable[] gVariables;
     private ILogicalPlan body;
+    private JobSpecification jobSpec;
 
     public Module() {
     }
@@ -79,6 +81,14 @@ public class Module {
         this.body = body;
     }
 
+    public JobSpecification getHyracksJobSpecification() {
+        return jobSpec;
+    }
+
+    public void setHyracksJobSpecification(JobSpecification jobSpec) {
+        this.jobSpec = jobSpec;
+    }
+
     public RegisterSet createGlobalRegisterSet() {
         return new RegisterSet(new Object[gVariables.length]);
     }

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/NoopXQueryCompilationListener.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/NoopXQueryCompilationListener.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/NoopXQueryCompilationListener.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/NoopXQueryCompilationListener.java Fri Jun  8 14:04:59 2012
@@ -14,6 +14,8 @@
  */
 package org.apache.vxquery.xmlquery.query;
 
+import org.apache.vxquery.xmlquery.ast.ModuleNode;
+
 public class NoopXQueryCompilationListener implements XQueryCompilationListener {
     public static final XQueryCompilationListener INSTANCE = new NoopXQueryCompilationListener();
 
@@ -35,4 +37,8 @@ public class NoopXQueryCompilationListen
     @Override
     public void notifyOptimizedResult(Module module) {
     }
+
+    @Override
+    public void notifyParseResult(ModuleNode moduleNode) {
+    }
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java Fri Jun  8 14:04:59 2012
@@ -14,34 +14,133 @@
  */
 package org.apache.vxquery.xmlquery.query;
 
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.vxquery.compiler.CompilerControlBlock;
+import org.apache.vxquery.compiler.algebricks.PrinterFactoryProvider;
+import org.apache.vxquery.compiler.jobgen.ExpressionJobGen;
+import org.apache.vxquery.compiler.rewriter.RewriteRuleset;
+import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.metadata.VXQueryMetadataProvider;
 import org.apache.vxquery.xmlquery.ast.ModuleNode;
 import org.apache.vxquery.xmlquery.translator.XMLQueryTranslator;
 
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
 public class XMLQueryCompiler {
-    public static Module compile(ModuleNode moduleNode, CompilerControlBlock ccb, int optimizationLevel,
-            boolean debugOptimizer) throws SystemException {
-        return compile(NoopXQueryCompilationListener.INSTANCE, moduleNode, ccb, optimizationLevel);
+    private final XQueryCompilationListener listener;
+
+    private final ICompilerFactory cFactory;
+
+    private final VXQueryMetadataProvider mdProvider;
+
+    private ModuleNode moduleNode;
+
+    private Module module;
+
+    private ICompiler compiler;
+
+    public XMLQueryCompiler(XQueryCompilationListener listener) {
+        this.listener = listener == null ? NoopXQueryCompilationListener.INSTANCE : listener;
+        HeuristicCompilerFactoryBuilder builder = new HeuristicCompilerFactoryBuilder();
+        builder.setLogicalRewrites(buildDefaultLogicalRewrites());
+        builder.setPhysicalRewrites(buildDefaultPhysicalRewrites());
+        builder.setSerializerDeserializerProvider(new ISerializerDeserializerProvider() {
+            @SuppressWarnings("unchecked")
+            @Override
+            public ISerializerDeserializer getSerializerDeserializer(Object type) throws AlgebricksException {
+                return null;
+            }
+        });
+        builder.setPrinterProvider(PrinterFactoryProvider.INSTANCE);
+        builder.setExprJobGen(new ExpressionJobGen());
+        builder.setExpressionTypeComputer(new IExpressionTypeComputer() {
+            @Override
+            public Object getType(ILogicalExpression expr, IMetadataProvider<?, ?> metadataProvider,
+                    IVariableTypeEnvironment env) throws AlgebricksException {
+                return null;
+            }
+        });
+        cFactory = builder.create();
+        mdProvider = new VXQueryMetadataProvider();
     }
 
-    public static Module compile(XQueryCompilationListener listener, ModuleNode moduleNode, CompilerControlBlock ccb,
-            int optimizationLevel) throws SystemException {
-        if (listener == null) {
-            listener = NoopXQueryCompilationListener.INSTANCE;
-        }
-        Module module = new XMLQueryTranslator(ccb).translateModule(moduleNode);
+    public void compile(String name, Reader query, CompilerControlBlock ccb, int optimizationLevel)
+            throws SystemException {
+        moduleNode = XMLQueryParser.parse(name, query);
+        listener.notifyParseResult(moduleNode);
+        module = new XMLQueryTranslator(ccb).translateModule(moduleNode);
+        compiler = cFactory.createCompiler(module.getBody(), mdProvider, 0);
         listener.notifyTranslationResult(module);
         XMLQueryTypeChecker.typeCheckModule(module);
         listener.notifyTypecheckResult(module);
-        XMLQueryOptimizer optimizer = new XMLQueryOptimizer();
-        optimizer.optimize(module, optimizationLevel);
+        try {
+            compiler.optimize();
+        } catch (AlgebricksException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
+        }
         listener.notifyOptimizedResult(module);
-        XMLQueryCodeGenerator.codegenModule(module);
+        JobSpecification jobSpec;
+        try {
+            jobSpec = compiler.createJob(null);
+        } catch (AlgebricksException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
+        }
+        module.setHyracksJobSpecification(jobSpec);
         listener.notifyCodegenResult(module);
-        return module;
     }
 
-    private XMLQueryCompiler() {
+    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildDefaultLogicalRewrites() {
+        List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultLogicalRewrites = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+        SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+        SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
+        SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+                RewriteRuleset.buildTypeInferenceRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+                RewriteRuleset.buildNormalizationRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                RewriteRuleset.buildCondPushDownRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                RewriteRuleset.buildJoinInferenceRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                RewriteRuleset.buildOpPushDownRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+                RewriteRuleset.buildDataExchangeRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                RewriteRuleset.buildConsolidationRuleCollection()));
+        return defaultLogicalRewrites;
     }
+
+    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildDefaultPhysicalRewrites() {
+        List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultPhysicalRewrites = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+        SequentialOnceRuleController seqOnceCtrlAllLevels = new SequentialOnceRuleController(true);
+        SequentialOnceRuleController seqOnceCtrlTopLevel = new SequentialOnceRuleController(false);
+        defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrlAllLevels,
+                RewriteRuleset.buildPhysicalRewritesAllLevelsRuleCollection()));
+        defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrlTopLevel,
+                RewriteRuleset.buildPhysicalRewritesTopLevelRuleCollection()));
+        defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrlAllLevels,
+                RewriteRuleset.prepareForJobGenRuleCollection()));
+        return defaultPhysicalRewrites;
+    }
+
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XQueryCompilationListener.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XQueryCompilationListener.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XQueryCompilationListener.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XQueryCompilationListener.java Fri Jun  8 14:04:59 2012
@@ -14,7 +14,11 @@
  */
 package org.apache.vxquery.xmlquery.query;
 
+import org.apache.vxquery.xmlquery.ast.ModuleNode;
+
 public interface XQueryCompilationListener {
+    public void notifyParseResult(ModuleNode moduleNode);
+
     public void notifyTranslationResult(Module module);
 
     public void notifyTypecheckResult(Module module);

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/test/java/org/apache/vxquery/xmlquery/query/SimpleXQueryTest.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/test/java/org/apache/vxquery/xmlquery/query/SimpleXQueryTest.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/test/java/org/apache/vxquery/xmlquery/query/SimpleXQueryTest.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/test/java/org/apache/vxquery/xmlquery/query/SimpleXQueryTest.java Fri Jun  8 14:04:59 2012
@@ -24,15 +24,10 @@ import java.util.zip.GZIPInputStream;
 
 import junit.framework.Assert;
 
-import org.apache.vxquery.api.InternalAPI;
+import org.apache.vxquery.compiler.CompilerControlBlock;
+import org.apache.vxquery.context.RootStaticContextImpl;
+import org.apache.vxquery.context.StaticContextImpl;
 import org.apache.vxquery.exceptions.SystemException;
-import org.apache.vxquery.v0datamodel.DMOKind;
-import org.apache.vxquery.v0datamodel.XDMItem;
-import org.apache.vxquery.v0datamodel.XDMSequence;
-import org.apache.vxquery.v0datamodel.XDMValue;
-import org.apache.vxquery.v0datamodel.dtm.DTMDatamodelStaticInterfaceImpl;
-import org.apache.vxquery.v0runtime.base.CloseableIterator;
-import org.apache.vxquery.v0runtime.base.OpenableCloseableIterator;
 import org.junit.Test;
 
 public class SimpleXQueryTest {
@@ -157,27 +152,8 @@ public class SimpleXQueryTest {
     }
 
     private static void runTestInternal(String testName, String query) throws SystemException {
-        InternalAPI iapi = new InternalAPI(new DTMDatamodelStaticInterfaceImpl());
-        OpenableCloseableIterator ri = iapi.execute(iapi.compile(null, iapi.parse(testName, new StringReader(query)),
-                Integer.MAX_VALUE));
-        ri.open();
-        System.err.println("--- Results begin");
-        XDMValue o;
-        try {
-            while ((o = (XDMValue) ri.next()) != null) {
-                if (o.getDMOKind() == DMOKind.SEQUENCE) {
-                    CloseableIterator si = ((XDMSequence) o).createItemIterator();
-                    XDMItem item;
-                    while ((item = (XDMItem) si.next()) != null) {
-                        System.err.println(item.getStringValue());
-                    }
-                } else {
-                    System.err.println(((XDMItem) o).getStringValue());
-                }
-            }
-        } finally {
-            ri.close();
-        }
-        System.err.println("--- Results end");
+        XMLQueryCompiler compiler = new XMLQueryCompiler(null);
+        CompilerControlBlock ccb = new CompilerControlBlock(new StaticContextImpl(RootStaticContextImpl.INSTANCE));
+        compiler.compile(testName, new StringReader(query), ccb, Integer.MAX_VALUE);
     }
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/TestRunnerFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/TestRunnerFactory.java?rev=1348078&r1=1348077&r2=1348078&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/TestRunnerFactory.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/TestRunnerFactory.java Fri Jun  8 14:04:59 2012
@@ -14,27 +14,15 @@
  */
 package org.apache.vxquery.xtest;
 
-import java.io.BufferedWriter;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
 import java.io.FileReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import javax.xml.namespace.QName;
-
-import org.apache.vxquery.api.InternalAPI;
-import org.apache.vxquery.context.XQueryVariable;
-import org.apache.vxquery.v0datamodel.XDMItem;
-import org.apache.vxquery.v0datamodel.dtm.DTMDatamodelStaticInterfaceImpl;
-import org.apache.vxquery.v0datamodel.serialization.XMLSerializer;
-import org.apache.vxquery.v0runtime.base.OpenableCloseableIterator;
-import org.apache.vxquery.xmlquery.ast.ModuleNode;
-import org.apache.vxquery.xmlquery.query.Module;
-import org.apache.vxquery.xmlquery.query.PrologVariable;
+import org.apache.vxquery.compiler.CompilerControlBlock;
+import org.apache.vxquery.context.RootStaticContextImpl;
+import org.apache.vxquery.context.StaticContextImpl;
+import org.apache.vxquery.xmlquery.query.XMLQueryCompiler;
 
 public class TestRunnerFactory {
     private List<ResultReporter> reporters;
@@ -59,43 +47,11 @@ public class TestRunnerFactory {
                 }
                 long start = System.currentTimeMillis();
                 try {
-                    InternalAPI iapi = new InternalAPI(new DTMDatamodelStaticInterfaceImpl());
+                    XMLQueryCompiler compiler = new XMLQueryCompiler(null);
                     FileReader in = new FileReader(testCase.getXQueryFile());
-                    ModuleNode ast;
-                    try {
-                        ast = iapi.parse(testCase.getXQueryDisplayName(), in);
-                    } finally {
-                        in.close();
-                    }
-                    Module module = iapi.compile(null, ast, opts.optimizationLevel);
-                    for (PrologVariable pVar : module.getPrologVariables()) {
-                        XQueryVariable var = pVar.getVariable();
-                        QName varName = var.getName();
-                        File binding = testCase.getExternalVariableBinding(varName);
-                        if (binding != null) {
-                            iapi.bindExternalVariable(var, testCase.getExternalVariableBinding(varName));
-                        }
-                    }
-                    OpenableCloseableIterator ri = iapi.execute(module);
-                    ri.open();
-                    XDMItem o;
-                    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                    PrintWriter out = new PrintWriter(new BufferedWriter(new OutputStreamWriter(baos)), true);
-                    XMLSerializer s = new XMLSerializer(out, false);
-                    try {
-                        while ((o = (XDMItem) ri.next()) != null) {
-                            s.item(o);
-                        }
-                    } finally {
-                        out.flush();
-                        ri.close();
-                    }
-                    try {
-                        res.result = baos.toString("UTF-8");
-                    } catch (Exception e) {
-                        System.err.println("Framework error");
-                        e.printStackTrace();
-                    }
+                    CompilerControlBlock ccb = new CompilerControlBlock(new StaticContextImpl(
+                            RootStaticContextImpl.INSTANCE));
+                    compiler.compile(testCase.getXQueryDisplayName(), in, ccb, opts.optimizationLevel);
                 } catch (Throwable e) {
                     res.error = e;
                 } finally {