You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by pr...@apache.org on 2015/06/29 03:00:42 UTC

[3/3] vxquery git commit: doc rewrite rule test

doc rewrite rule test


Project: http://git-wip-us.apache.org/repos/asf/vxquery/repo
Commit: http://git-wip-us.apache.org/repos/asf/vxquery/commit/654f93f3
Tree: http://git-wip-us.apache.org/repos/asf/vxquery/tree/654f93f3
Diff: http://git-wip-us.apache.org/repos/asf/vxquery/diff/654f93f3

Branch: refs/heads/master
Commit: 654f93f3bd6c071cc2988c7ea853f67014299a15
Parents: 1f922a7
Author: Shivani Mall <sm...@ucr.edu>
Authored: Mon Jun 22 12:20:54 2015 -0700
Committer: Preston Carman <pr...@apache.org>
Committed: Sun Jun 28 18:00:22 2015 -0700

----------------------------------------------------------------------
 .../compiler/rewriter/RewriteRuleset.java       |   4 +-
 .../rules/ConvertDocExpressionToFile.java       | 180 +++++++++++++++++++
 .../EliminateUnnestAggregateSubplanRule.java    |   3 +-
 .../rewriter/rules/util/OperatorToolbox.java    |   3 +
 .../metadata/VXQueryMetadataProvider.java       |   2 +-
 vxquery-xtest/src/test/resources/DocTest.xml    |  47 +++++
 .../ExpectedTestResults/Simple/US000000001.xml  |  17 ++
 .../ExpectedTestResults/Simple/test.xml         |   1 +
 .../resources/Queries/XQuery/Simple/test.xq     |  19 ++
 .../src/test/resources/VXQueryCatalog.xml       |  10 +-
 .../test/resources/cat/SingleAlternateQuery.xml |   2 -
 .../src/test/resources/cat/TestRewriteRules.xml |  28 +++
 12 files changed, 305 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
index 687164e..672ddf2 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
@@ -21,7 +21,7 @@ import java.util.List;
 
 import org.apache.vxquery.compiler.rewriter.rules.ConsolidateAssignAggregateRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToUnnestRule;
-import org.apache.vxquery.compiler.rewriter.rules.ReplaceSourceMapInDocExpression;
+import org.apache.vxquery.compiler.rewriter.rules.ConvertDocExpressionToFile;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertFromAlgebricksExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertToAlgebricksExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.EliminateSubplanForSingleItemsRule;
@@ -118,7 +118,7 @@ public class RewriteRuleset {
         normalization.add(new IntroduceCollectionRule());
         normalization.add(new RemoveUnusedAssignAndAggregateRule());
 
-        normalization.add(new ReplaceSourceMapInDocExpression());
+        normalization.add(new ConvertDocExpressionToFile());
         // Adds child steps to the data source scan.
         // TODO Replace consolidate with a new child function that takes multiple paths.
         //        normalization.add(new ConsolidateUnnestsRule());

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertDocExpressionToFile.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertDocExpressionToFile.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertDocExpressionToFile.java
new file mode 100644
index 0000000..c64157f
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertDocExpressionToFile.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 at
+ *
+ *     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 org.apache.vxquery.compiler.rewriter.rules;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.vxquery.compiler.algebricks.VXQueryConstantValue;
+import org.apache.vxquery.compiler.rewriter.rules.util.ExpressionToolbox;
+import org.apache.vxquery.compiler.rewriter.rules.util.OperatorToolbox;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.builders.atomic.StringValueBuilder;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.functions.BuiltinFunctions;
+import org.apache.vxquery.metadata.VXQueryMetadataProvider;
+import org.apache.vxquery.types.BuiltinTypeRegistry;
+import org.apache.vxquery.types.Quantifier;
+import org.apache.vxquery.types.SequenceType;
+
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+/**
+ * The rule searches for where the function_doc1 function is in the plan in place of XQuery function.
+ * It replaces the string contained in the function with its absolute file path.
+ * 
+ * <pre>
+ * Before
+ * 
+ *   plan__parent
+ *   %OPERATOR( $v1 : function_doc1( \@string ) )
+ *   plan__child
+ *   
+ *   Where xquery_function creates an atomic value.
+ *   
+ * After 
+ * 
+ *   plan__parent
+ *   %OPERATOR( $v1 : function_doc1( \@absolute_file_path ) ) )
+ *   plan__child
+ * </pre>
+ * 
+ * @author shivanim
+ */
+
+public class ConvertDocExpressionToFile implements IAlgebraicRewriteRule {
+
+    final ByteBufferInputStream bbis = new ByteBufferInputStream();
+    final DataInputStream di = new DataInputStream(bbis);
+    final UTF8StringPointable stringp = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
+    final TaggedValuePointable tvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+    final DataOutput dOut = abvs.getDataOutput();
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        boolean modified = false;
+        //returns the list of expressions inside the operator.
+        List<Mutable<ILogicalExpression>> expressions = OperatorToolbox.getExpressions(opRef);
+        //for each expression we go in
+        for (Mutable<ILogicalExpression> expression : expressions) {
+            //checks if the expression is a function call
+            //checks if the function call is fn_doc1
+            //returns the first expression contained in it only!
+            //what is a function has multiple arguments that is multiple expressions
+            Mutable<ILogicalExpression> docExpression = ExpressionToolbox.findFirstFunctionExpression(expression,
+                    BuiltinFunctions.FN_DOC_1.getFunctionIdentifier());
+            if (docExpression != null) {
+                AbstractFunctionCallExpression absFnCall = (AbstractFunctionCallExpression) docExpression.getValue();
+                if (docExpression != null && ifDocExpressionFound(opRef, absFnCall.getArguments().get(0), context)) {
+                    modified = true;
+                }
+            }
+        }
+        return modified;
+    }
+
+    //side note: I only see nested arguments, not multiple expressions in most cases.//
+    //Expressions == arguments ??
+
+    protected boolean ifDocExpressionFound(Mutable<ILogicalOperator> opRef, Mutable<ILogicalExpression> funcExpression,
+            IOptimizationContext context) {
+        VXQueryConstantValue constantValue = null;
+        ConstantExpression constantExpression = null;
+        ILogicalExpression logicalExpression = (ILogicalExpression) funcExpression.getValue();
+
+        if (logicalExpression.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+
+            constantExpression = (ConstantExpression) logicalExpression;
+            constantValue = (VXQueryConstantValue) constantExpression.getValue();
+        } else if (logicalExpression.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+
+            VariableReferenceExpression varLogicalExpression = (VariableReferenceExpression) logicalExpression;
+            Mutable<ILogicalOperator> lop = OperatorToolbox.findProducerOf(opRef,
+                    varLogicalExpression.getVariableReference());
+            ILogicalExpression variableLogicalExpression = (ILogicalExpression) OperatorToolbox.getExpressionOf(lop,
+                    varLogicalExpression.getVariableReference()).getValue();
+            if (variableLogicalExpression.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                return false;
+            }
+            constantExpression = (ConstantExpression) variableLogicalExpression;
+            constantValue = (VXQueryConstantValue) constantExpression.getValue();
+        } else {
+            return false;
+        }
+
+        if (constantValue.getType() != SequenceType.create(BuiltinTypeRegistry.XS_STRING, Quantifier.QUANT_ONE)) {
+            return false;
+        }
+        tvp.set(constantValue.getValue(), 0, constantValue.getValue().length);
+        String collectionName = null;
+        tvp.getValue(stringp);
+        if (tvp.getTag() != ValueTag.XS_STRING_TAG) {
+            return false;
+        }
+        try {
+            bbis.setByteBuffer(
+                    ByteBuffer.wrap(Arrays.copyOfRange(stringp.getByteArray(), stringp.getStartOffset(),
+                            stringp.getLength() + stringp.getStartOffset())), 0);
+            collectionName = di.readUTF();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+        VXQueryMetadataProvider mdp = (VXQueryMetadataProvider) context.getMetadataProvider();
+        if (!mdp.sourceFileMap.containsKey(collectionName)) {
+            return false;
+        }
+        File file = mdp.sourceFileMap.get(collectionName);
+        StringValueBuilder svb = new StringValueBuilder();
+        try {
+            abvs.reset();
+            dOut.write(ValueTag.XS_STRING_TAG);
+            svb.write(file.getAbsolutePath(), dOut);
+
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        VXQueryConstantValue vxqcv = new VXQueryConstantValue(SequenceType.create(BuiltinTypeRegistry.XS_STRING,
+                Quantifier.QUANT_ONE), abvs.getByteArray());
+        constantExpression.setValue(vxqcv);
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/EliminateUnnestAggregateSubplanRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/EliminateUnnestAggregateSubplanRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/EliminateUnnestAggregateSubplanRule.java
index 193b53d..3504d7d 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/EliminateUnnestAggregateSubplanRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/EliminateUnnestAggregateSubplanRule.java
@@ -78,9 +78,10 @@ public class EliminateUnnestAggregateSubplanRule implements IAlgebraicRewriteRul
             return false;
         }
         AbstractFunctionCallExpression functionCall = (AbstractFunctionCallExpression) logicalExpression;
+        /*
         if (!functionCall.getFunctionIdentifier().equals(BuiltinOperators.ITERATE.getFunctionIdentifier())) {
             return false;
-        }
+        }*/
 
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) unnest.getInputs().get(0).getValue();
         if (op2.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
index 78cd80f..d0384a8 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
@@ -67,6 +67,9 @@ public class OperatorToolbox {
         switch (op.getOperatorTag()) {
             case AGGREGATE:
             case ASSIGN:
+                AbstractAssignOperator aap = (AbstractAssignOperator) op;
+                result.addAll(aap.getExpressions());
+                break;
             case RUNNINGAGGREGATE:
                 AbstractAssignOperator aao = (AbstractAssignOperator) op;
                 result.addAll(aao.getExpressions());

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java b/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java
index b2ab17c..e9cd424 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/metadata/VXQueryMetadataProvider.java
@@ -53,7 +53,7 @@ import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 
 public class VXQueryMetadataProvider implements IMetadataProvider<String, String> {
     String[] nodeList;
-    Map<String, File> sourceFileMap;
+    public Map<String, File> sourceFileMap;
 
     public VXQueryMetadataProvider(String[] nodeList, Map<String, File> sourceFileMap) {
         this.nodeList = nodeList;

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-xtest/src/test/resources/DocTest.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/DocTest.xml b/vxquery-xtest/src/test/resources/DocTest.xml
new file mode 100644
index 0000000..d38f112
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/DocTest.xml
@@ -0,0 +1,47 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You 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 at
+
+      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.
+-->
+<!DOCTYPE test-suite [
+
+<!ENTITY SingleQuery SYSTEM "cat/SingleQuery.xml">
+
+<!ENTITY XMarkQueries SYSTEM "cat/XMarkOriginalQueries.xml">
+
+<!ENTITY TestRewriteRules SYSTEM "cat/TestRewriteRules.xml">
+
+]>
+<test-suite xmlns="http://www.w3.org/2005/02/query-test-XQTSCatalog"
+            xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+            CatalogDesignDate="2014-04-01"
+            version="0.0.1"
+            SourceOffsetPath="./"
+            ResultOffsetPath="ExpectedTestResults/"
+            XQueryQueryOffsetPath="Queries/XQuery/"
+            XQueryXQueryOffsetPath="Queries/XQueryX/"
+            XQueryFileExtension=".xq"
+            XQueryXFileExtension=".xqx"
+            xsi:schemaLocation="http://www.w3.org/2005/02/query-test-XQTSCatalog XQTSCatalog.xsd">
+
+  
+   <source ID="station_xml_file" FileName="TestSources/ghcnd/half_1/quarter_1/stations/US000000001.xml" Creator="Shivani Mall">
+       <description last-mod="2015-03-25">Collection of files</description>
+   </source>
+
+       <test-group name="XMarkQueriesExecutionTests" featureOwner="Shivani Mall">
+         &TestRewriteRules;
+      </test-group>
+</test-suite>

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/US000000001.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/US000000001.xml b/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/US000000001.xml
new file mode 100644
index 0000000..e934f2c
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/US000000001.xml
@@ -0,0 +1,17 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You 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 at
+
+      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.
+-->
+<stationCollection pageSize="100" pageCount="1" totalCount="1"><station><id>GHCND:US000000001</id><displayName>Station 1</displayName><latitude>10.000</latitude><longitude>-10.000</longitude><elevation>1000.0</elevation><locationLabels><type>ST</type><id>FIPS:1</id><displayName>State 1</displayName></locationLabels><locationLabels><type>CNTY</type><id>FIPS:-9999</id><displayName>County 1</displayName></locationLabels><locationLabels><type>CNTRY</type><id>FIPS:US</id><displayName>UNITED STATES</displayName></locationLabels></station></stationCollection>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/test.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/test.xml b/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/test.xml
new file mode 100644
index 0000000..169a175
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Simple/test.xml
@@ -0,0 +1 @@
+<stationCollection pageSize="100"pageCount="1"totalCount="1"><station><id>GHCND:US000000001</id><displayName>Station 1</displayName><latitude>10.000</latitude><longitude>-10.000</longitude><elevation>1000.0</elevation><locationLabels><type>ST</type><id>FIPS:1</id><displayName>State 1</displayName></locationLabels><locationLabels><type>CNTY</type><id>FIPS:-9999</id><displayName>County 1</displayName></locationLabels><locationLabels><type>CNTRY</type><id>FIPS:US</id><displayName>UNITED STATES</displayName></locationLabels></station></stationCollection>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-xtest/src/test/resources/Queries/XQuery/Simple/test.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Simple/test.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Simple/test.xq
new file mode 100644
index 0000000..bce856c
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Simple/test.xq
@@ -0,0 +1,19 @@
+(: Licensed to the Apache Software Foundation (ASF) under one
+   or more contributor license agreements.  See the NOTICE file
+   distributed with this work for additional information
+   regarding copyright ownership.  The ASF licenses this file
+   to you 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 at
+   
+     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. :)
+   
+   doc("station_xml_file")/stationCollection
+   
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-xtest/src/test/resources/VXQueryCatalog.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/VXQueryCatalog.xml b/vxquery-xtest/src/test/resources/VXQueryCatalog.xml
index 6f7acee..30e0dd2 100644
--- a/vxquery-xtest/src/test/resources/VXQueryCatalog.xml
+++ b/vxquery-xtest/src/test/resources/VXQueryCatalog.xml
@@ -84,12 +84,12 @@
        <description last-mod="2014-04-02">Collection of files</description>
      </source>
      <source ID="station_xml_file" FileName="TestSources/ghcnd/half_1/quarter_1/stations/US000000001.xml" Creator="Shivani Mall">
-       <description last-mod="2015-06-26">File</description>
+       <description last-mod="2015-06-19">File</description>
      </source>
    </sources>
-   <test-group name="SingleQuery" featureOwner="Preston Carman">
+   <test-group name="BasicQueries" featureOwner="Preston Carman">
       <GroupInfo>
-         <title>Single Query</title>
+         <title>Basic Queries</title>
          <description/>
       </GroupInfo>
       <test-group name="SingleTestAdd" featureOwner="Preston Carman">
@@ -103,7 +103,7 @@
          <GroupInfo>
             <title>Single Test List</title>
             <description/>
-         </GroupInfo>
+          </GroupInfo>
          &SingleAlternateQuery;
       </test-group>
    </test-group>
@@ -193,4 +193,4 @@
          &GhcndRecordsPartition4Queries;
       </test-group>
    </test-group>
-</test-suite>
\ No newline at end of file
+</test-suite>

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-xtest/src/test/resources/cat/SingleAlternateQuery.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/cat/SingleAlternateQuery.xml b/vxquery-xtest/src/test/resources/cat/SingleAlternateQuery.xml
index bba230c..4009c0a 100644
--- a/vxquery-xtest/src/test/resources/cat/SingleAlternateQuery.xml
+++ b/vxquery-xtest/src/test/resources/cat/SingleAlternateQuery.xml
@@ -5,9 +5,7 @@
   The ASF licenses this file to You 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 at
-
       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.

http://git-wip-us.apache.org/repos/asf/vxquery/blob/654f93f3/vxquery-xtest/src/test/resources/cat/TestRewriteRules.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/cat/TestRewriteRules.xml b/vxquery-xtest/src/test/resources/cat/TestRewriteRules.xml
new file mode 100644
index 0000000..28677b3
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/cat/TestRewriteRules.xml
@@ -0,0 +1,28 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You 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 at
+
+      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.
+-->
+
+<test-group xmlns="http://www.w3.org/2005/02/query-test-XQTSCatalog" name="TestRewriteRuleQuery" featureOwner="VXQuery">
+   <GroupInfo>
+      <title>Test Rewrite Rule Query</title>
+   </GroupInfo>
+   <test-case name="TestConverDocExpRR" FilePath="Simple/" Creator="Shivani Mall">
+      <description>Test Rewrite Rule</description>
+      <query name="test" date="2015-06-19"/>
+      <output-file compare="Text">test.xml</output-file>
+   </test-case>
+</test-group>
+ 
\ No newline at end of file