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/29 08:14:10 UTC

svn commit: r1355244 - in /incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery: datamodel/accessors/nodes/ datamodel/builders/nodes/ datamodel/values/ functions/ runtime/functions/node/ types/ util/ xmlparser/

Author: vinayakb
Date: Fri Jun 29 06:14:08 2012
New Revision: 1355244

URL: http://svn.apache.org/viewvc?rev=1355244&view=rev
Log:
Added support for fn:doc()

Added:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AbstractNodeBuilder.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DocumentNodeBuilder.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/node/
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/node/FnDocScalarEvaluatorFactory.java
Removed:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/IEventAcceptor.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/NodeTreeBuildingEventAcceptor.java
Modified:
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/ElementNodePointable.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/NodeTreePointable.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AttributeNodeBuilder.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DictionaryBuilder.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/ElementNodeBuilder.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/TextNodeBuilder.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/values/ValueTag.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/types/BuiltinTypeQNames.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/util/GrowableIntArray.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/ITreeNodeIdProvider.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java
    incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/ElementNodePointable.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/ElementNodePointable.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/ElementNodePointable.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/ElementNodePointable.java Fri Jun 29 06:14:08 2012
@@ -38,9 +38,9 @@ import edu.uci.ics.hyracks.data.std.prim
  * }
  */
 public class ElementNodePointable extends AbstractPointable {
-    private static final byte NS_CHUNK_EXISTS_MASK = (0x1 << 0);
-    private static final byte ATTRIBUTES_CHUNK_EXISTS_MASK = (0x1 << 1);
-    private static final byte CHILDREN_CHUNK_EXISTS_MASK = (0x1 << 2);
+    public static final byte NS_CHUNK_EXISTS_MASK = (0x1 << 0);
+    public static final byte ATTRIBUTES_CHUNK_EXISTS_MASK = (0x1 << 1);
+    public static final byte CHILDREN_CHUNK_EXISTS_MASK = (0x1 << 2);
 
     private static final int HEADER_SIZE = 1;
     private static final int LOCAL_NODE_ID_SIZE = 4;

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/NodeTreePointable.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/NodeTreePointable.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/NodeTreePointable.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/NodeTreePointable.java Fri Jun 29 06:14:08 2012
@@ -11,9 +11,9 @@ import edu.uci.ics.hyracks.data.std.prim
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class NodeTreePointable extends AbstractPointable {
-    private static final int HEADER_NODEID_EXISTS_MASK = 0x01;
-    private static final int HEADER_DICTIONARY_EXISTS_MASK = 0x02;
-    private static final int HEADER_TYPE_EXISTS_MASK = 0x03;
+    public static final int HEADER_NODEID_EXISTS_MASK = 0x01;
+    public static final int HEADER_DICTIONARY_EXISTS_MASK = 0x02;
+    public static final int HEADER_TYPE_EXISTS_MASK = 0x03;
 
     private static final int HEADER_OFFSET = 0;
     private static final int HEADER_SIZE = 1;

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AbstractNodeBuilder.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AbstractNodeBuilder.java?rev=1355244&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AbstractNodeBuilder.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AbstractNodeBuilder.java Fri Jun 29 06:14:08 2012
@@ -0,0 +1,11 @@
+package org.apache.vxquery.datamodel.builders.nodes;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public abstract class AbstractNodeBuilder {
+    public abstract void reset(ArrayBackedValueStorage abvs) throws IOException;
+
+    public abstract void finish() throws IOException;
+}
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AttributeNodeBuilder.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AttributeNodeBuilder.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AttributeNodeBuilder.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/AttributeNodeBuilder.java Fri Jun 29 06:14:08 2012
@@ -1,5 +1,43 @@
 package org.apache.vxquery.datamodel.builders.nodes;
 
-public class AttributeNodeBuilder {
+import java.io.DataOutput;
+import java.io.IOException;
 
-}
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+import edu.uci.ics.hyracks.data.std.api.IValueReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class AttributeNodeBuilder extends AbstractNodeBuilder {
+    private DataOutput out;
+
+    @Override
+    public void reset(ArrayBackedValueStorage abvs) throws IOException {
+        out = abvs.getDataOutput();
+        out.write(ValueTag.ATTRIBUTE_NODE_TAG);
+    }
+
+    @Override
+    public void finish() throws IOException {
+    }
+
+    public void setName(int uriCode, int localNameCode, int prefixCode) throws IOException {
+        out.writeInt(prefixCode);
+        out.writeInt(uriCode);
+        out.writeInt(localNameCode);
+    }
+
+    public void setType(int uriCode, int localNameCode, int prefixCode) throws IOException {
+        out.writeInt(prefixCode);
+        out.writeInt(uriCode);
+        out.writeInt(localNameCode);
+    }
+
+    public void setLocalNodeId(int localNodeId) throws IOException {
+        out.writeInt(localNodeId);
+    }
+
+    public void setValue(IValueReference value) throws IOException {
+        out.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DictionaryBuilder.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DictionaryBuilder.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DictionaryBuilder.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DictionaryBuilder.java Fri Jun 29 06:14:08 2012
@@ -8,8 +8,10 @@ import org.apache.vxquery.util.GrowableI
 
 import edu.uci.ics.hyracks.data.std.algorithms.BinarySearchAlgorithm;
 import edu.uci.ics.hyracks.data.std.collections.api.IValueReferenceVector;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
 
 public class DictionaryBuilder {
     private final GrowableIntArray stringEndOffsets;
@@ -61,6 +63,24 @@ public class DictionaryBuilder {
         tempStringPointable = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
     }
 
+    public void write(ArrayBackedValueStorage abvs) throws IOException {
+        DataOutput out = abvs.getDataOutput();
+        int sizeOffset = abvs.getLength();
+        out.writeInt(0);
+        int entryCount = stringEndOffsets.getSize();
+        out.writeInt(entryCount);
+        int[] entryOffsets = stringEndOffsets.getArray();
+        for (int i = 0; i < entryCount; ++i) {
+            out.writeInt(entryOffsets[i]);
+        }
+        int[] sortedOffsets = sortedSlotIndexes.getArray();
+        for (int i = 0; i < entryCount; ++i) {
+            out.writeInt(sortedOffsets[i]);
+        }
+        out.write(dataBuffer.getByteArray(), 0, dataBuffer.size());
+        IntegerPointable.setInteger(abvs.getByteArray(), sizeOffset, abvs.getLength() - sizeOffset);
+    }
+
     public int lookup(String str) {
         tempStringData.reset();
         try {

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DocumentNodeBuilder.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DocumentNodeBuilder.java?rev=1355244&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DocumentNodeBuilder.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/DocumentNodeBuilder.java Fri Jun 29 06:14:08 2012
@@ -0,0 +1,65 @@
+package org.apache.vxquery.datamodel.builders.nodes;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.util.GrowableIntArray;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class DocumentNodeBuilder extends AbstractNodeBuilder {
+    private final GrowableIntArray childrenSlots;
+
+    private final ArrayBackedValueStorage childrenDataArea;
+
+    private DataOutput out;
+
+    private int childrenCount;
+
+    public DocumentNodeBuilder() {
+        childrenSlots = new GrowableIntArray();
+        childrenDataArea = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    public void reset(ArrayBackedValueStorage abvs) throws IOException {
+        out = abvs.getDataOutput();
+        out.write(ValueTag.DOCUMENT_NODE_TAG);
+    }
+
+    @Override
+    public void finish() throws IOException {
+    }
+
+    public void setLocalNodeId(int localNodeId) throws IOException {
+        out.writeInt(localNodeId);
+    }
+
+    public void startChildrenChunk() {
+        childrenSlots.clear();
+        childrenDataArea.reset();
+    }
+
+    public void startChild(AbstractNodeBuilder nb) throws IOException {
+        nb.reset(childrenDataArea);
+    }
+
+    public void endChild(AbstractNodeBuilder nb) throws IOException {
+        nb.finish();
+        childrenSlots.append(childrenDataArea.getLength());
+    }
+
+    public void endChildrenChunk() throws IOException {
+        childrenCount = childrenSlots.getSize();
+        if (childrenCount > 0) {
+            out.writeInt(childrenCount);
+            int[] slotArray = childrenSlots.getArray();
+            for (int i = 0; i < childrenCount; ++i) {
+                int slot = slotArray[i];
+                out.writeInt(slot);
+            }
+            out.write(childrenDataArea.getByteArray(), childrenDataArea.getStartOffset(), childrenDataArea.getLength());
+        }
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/ElementNodeBuilder.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/ElementNodeBuilder.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/ElementNodeBuilder.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/ElementNodeBuilder.java Fri Jun 29 06:14:08 2012
@@ -1,37 +1,156 @@
 package org.apache.vxquery.datamodel.builders.nodes;
 
-public class ElementNodeBuilder {
-    private DictionaryBuilder db;
+import java.io.DataOutput;
+import java.io.IOException;
 
-    private int nameUriCode;
+import org.apache.vxquery.datamodel.accessors.nodes.ElementNodePointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.util.GrowableIntArray;
 
-    private int nameLocalNameCode;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
 
-    private int namePrefixCode;
+public class ElementNodeBuilder extends AbstractNodeBuilder {
+    private final GrowableIntArray attrSlots;
 
-    private int typeUriCode;
+    private final ArrayBackedValueStorage attrDataArea;
 
-    private int typeLocalNameCode;
+    private final GrowableIntArray childrenSlots;
 
-    private int typePrefixCode;
+    private final ArrayBackedValueStorage childrenDataArea;
 
-    public void reset() {
-        db = null;
+    private ArrayBackedValueStorage abvs;
+
+    private DataOutput out;
+
+    private int headerOffset;
+
+    private int nsChunkStart;
+
+    private int nsCount;
+
+    private int attrCount;
+
+    private int childrenCount;
+
+    public ElementNodeBuilder() {
+        attrSlots = new GrowableIntArray();
+        attrDataArea = new ArrayBackedValueStorage();
+        childrenSlots = new GrowableIntArray();
+        childrenDataArea = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    public void reset(ArrayBackedValueStorage abvs) throws IOException {
+        this.abvs = abvs;
+        out = abvs.getDataOutput();
+        headerOffset = abvs.getLength();
+        out.write(ValueTag.ELEMENT_NODE_TAG);
+        out.write(0);
+    }
+
+    @Override
+    public void finish() throws IOException {
+        byte header = 0;
+        if (nsCount > 0) {
+            header |= ElementNodePointable.NS_CHUNK_EXISTS_MASK;
+        }
+        if (attrCount > 0) {
+            header |= ElementNodePointable.ATTRIBUTES_CHUNK_EXISTS_MASK;
+        }
+        if (childrenCount > 0) {
+            header |= ElementNodePointable.CHILDREN_CHUNK_EXISTS_MASK;
+        }
+        BytePointable.setByte(abvs.getByteArray(), headerOffset, header);
+    }
+
+    public void setName(int uriCode, int localNameCode, int prefixCode) throws IOException {
+        out.writeInt(prefixCode);
+        out.writeInt(uriCode);
+        out.writeInt(localNameCode);
+    }
+
+    public void setType(int uriCode, int localNameCode, int prefixCode) throws IOException {
+        out.writeInt(prefixCode);
+        out.writeInt(uriCode);
+        out.writeInt(localNameCode);
+    }
+
+    public void setLocalNodeId(int localNodeId) throws IOException {
+        out.writeInt(localNodeId);
+    }
+
+    public void startNamespaceChunk() {
+        nsChunkStart = abvs.getLength();
+        nsCount = 0;
+    }
+
+    public void addNamespace(int prefixCode, int uriCode) throws IOException {
+        if (nsCount == 0) {
+            out.writeInt(0);
+        }
+        out.writeInt(prefixCode);
+        out.writeInt(uriCode);
+        ++nsCount;
+    }
+
+    public void endNamespaceChunk() {
+        byte[] bytes = abvs.getByteArray();
+        IntegerPointable.setInteger(bytes, nsChunkStart, nsCount);
+    }
+
+    public void startAttributeChunk() {
+        attrSlots.clear();
+        attrDataArea.reset();
+    }
+
+    public void startAttribute(AttributeNodeBuilder attrb) throws IOException {
+        attrb.reset(attrDataArea);
+    }
+
+    public void endAttribute(AttributeNodeBuilder attrb) throws IOException {
+        attrb.finish();
+        attrSlots.append(attrDataArea.getLength());
+    }
+
+    public void endAttributeChunk() throws IOException {
+        attrCount = attrSlots.getSize();
+        if (attrCount > 0) {
+            out.writeInt(attrCount);
+            int[] slotArray = attrSlots.getArray();
+            for (int i = 0; i < attrCount; ++i) {
+                int slot = slotArray[i];
+                out.writeInt(slot);
+            }
+            out.write(attrDataArea.getByteArray(), attrDataArea.getStartOffset(), attrDataArea.getLength());
+        }
+    }
+
+    public void startChildrenChunk() {
+        childrenSlots.clear();
+        childrenDataArea.reset();
     }
 
-    public void setDictionaryBuilder(DictionaryBuilder db) {
-        this.db = db;
+    public void startChild(AbstractNodeBuilder nb) throws IOException {
+        nb.reset(childrenDataArea);
     }
 
-    public void setName(int uriCode, int localNameCode, int prefixCode) {
-        nameUriCode = uriCode;
-        nameLocalNameCode = localNameCode;
-        namePrefixCode = prefixCode;
+    public void endChild(AbstractNodeBuilder nb) throws IOException {
+        nb.finish();
+        childrenSlots.append(childrenDataArea.getLength());
     }
 
-    public void setType(int uriCode, int localNameCode, int prefixCode) {
-        typeUriCode = uriCode;
-        typeLocalNameCode = localNameCode;
-        typePrefixCode = prefixCode;
+    public void endChildrenChunk() throws IOException {
+        childrenCount = childrenSlots.getSize();
+        if (childrenCount > 0) {
+            out.writeInt(childrenCount);
+            int[] slotArray = childrenSlots.getArray();
+            for (int i = 0; i < childrenCount; ++i) {
+                int slot = slotArray[i];
+                out.writeInt(slot);
+            }
+            out.write(childrenDataArea.getByteArray(), childrenDataArea.getStartOffset(), childrenDataArea.getLength());
+        }
     }
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/TextNodeBuilder.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/TextNodeBuilder.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/TextNodeBuilder.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/builders/nodes/TextNodeBuilder.java Fri Jun 29 06:14:08 2012
@@ -1,5 +1,31 @@
 package org.apache.vxquery.datamodel.builders.nodes;
 
-public class TextNodeBuilder {
+import java.io.DataOutput;
+import java.io.IOException;
 
-}
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+import edu.uci.ics.hyracks.data.std.api.IValueReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class TextNodeBuilder extends AbstractNodeBuilder {
+    private DataOutput out;
+
+    @Override
+    public void reset(ArrayBackedValueStorage abvs) throws IOException {
+        out = abvs.getDataOutput();
+        out.write(ValueTag.TEXT_NODE_TAG);
+    }
+
+    @Override
+    public void finish() throws IOException {
+    }
+
+    public void setLocalNodeId(int localNodeId) throws IOException {
+        out.writeInt(localNodeId);
+    }
+
+    public void setValue(IValueReference value) throws IOException {
+        out.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/values/ValueTag.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/values/ValueTag.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/values/ValueTag.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/datamodel/values/ValueTag.java Fri Jun 29 06:14:08 2012
@@ -65,4 +65,5 @@ public class ValueTag {
     public static final int TEXT_NODE_TAG = 104;
     public static final int COMMENT_NODE_TAG = 105;
     public static final int PI_NODE_TAG = 106;
+    public static final int NODE_TREE_TAG = 107;
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml Fri Jun 29 06:14:08 2012
@@ -246,6 +246,7 @@
     <function name="fn:doc">
         <param name="uri" type="xs:string?"/>
         <return type="document-node()?"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.node.FnDocScalarEvaluatorFactory"/>
     </function>
 
     <!-- fn:doc-available($uri  as xs:string?) as  xs:boolean -->

Added: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/node/FnDocScalarEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/node/FnDocScalarEvaluatorFactory.java?rev=1355244&view=auto
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/node/FnDocScalarEvaluatorFactory.java (added)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/node/FnDocScalarEvaluatorFactory.java Fri Jun 29 06:14:08 2012
@@ -0,0 +1,63 @@
+package org.apache.vxquery.runtime.functions.node;
+
+import java.io.DataInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+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.AbstractTaggedValueArgumentScalarEvaluator;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+import org.apache.vxquery.xmlparser.XMLParser;
+import org.xml.sax.InputSource;
+
+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;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class FnDocScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public FnDocScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IScalarEvaluator[] args) throws AlgebricksException {
+        final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+        final InputSource in = new InputSource();
+        final UTF8StringPointable sp = new UTF8StringPointable();
+        final ByteBufferInputStream bbis = new ByteBufferInputStream();
+        final DataInputStream di = new DataInputStream(bbis);
+        return new AbstractTaggedValueArgumentScalarEvaluator(args) {
+            @Override
+            protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+                try {
+                    TaggedValuePointable tvp = args[0];
+                    if (tvp.getTag() != ValueTag.XS_STRING_TAG) {
+                        throw new SystemException(ErrorCode.FORG0006);
+                    }
+                    tvp.getValue(sp);
+                    bbis.setByteBuffer(
+                            ByteBuffer.wrap(Arrays.copyOfRange(sp.getByteArray(), sp.getStartOffset(), sp.getLength()
+                                    + sp.getStartOffset())), 0);
+                    String fName = di.readUTF();
+                    in.setCharacterStream(new InputStreamReader(new FileInputStream(fName)));
+                    XMLParser.parseInputSource(in, abvs, false, null);
+                    result.set(abvs);
+                } catch (IOException e) {
+                    throw new SystemException(ErrorCode.SYSE0001, e);
+                }
+            }
+        };
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/types/BuiltinTypeQNames.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/types/BuiltinTypeQNames.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/types/BuiltinTypeQNames.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/types/BuiltinTypeQNames.java Fri Jun 29 06:14:08 2012
@@ -23,6 +23,8 @@ import org.apache.vxquery.xmlquery.query
 public class BuiltinTypeQNames {
     public static final String UNTYPED_STR = "untyped";
 
+    public static final String UNTYPED_ATOMIC_STR = "untyped";
+
     public static final String ANY_TYPE_STR = "anyType";
 
     public static final QName XS_ANY_TYPE_QNAME = new QName(XQueryConstants.XS_NSURI, ANY_TYPE_STR,

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/util/GrowableIntArray.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/util/GrowableIntArray.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/util/GrowableIntArray.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/util/GrowableIntArray.java Fri Jun 29 06:14:08 2012
@@ -49,10 +49,19 @@ public class GrowableIntArray {
     }
 
     public void insert(int index, int value) {
-        if (array.length <= size) {
-            grow(DEFAULT_GROWTH);
+        if (index >= size) {
+            if (index >= array.length) {
+                int increment = (((index + 1 - array.length) - 1) / DEFAULT_GROWTH + 1) * DEFAULT_GROWTH;
+                grow(increment);
+            }
+            size = index + 1;
+        } else {
+            if (size >= array.length) {
+                grow(DEFAULT_GROWTH);
+            }
+            System.arraycopy(array, index, array, index + 1, size - index);
+            ++size;
         }
-        System.arraycopy(array, index, array, index + 1, size - index);
         array[index] = value;
     }
 

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/ITreeNodeIdProvider.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/ITreeNodeIdProvider.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/ITreeNodeIdProvider.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/ITreeNodeIdProvider.java Fri Jun 29 06:14:08 2012
@@ -1,5 +1,5 @@
 package org.apache.vxquery.xmlparser;
 
 public interface ITreeNodeIdProvider {
-    public int getId();
+    public long getId();
 }
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java Fri Jun 29 06:14:08 2012
@@ -14,22 +14,70 @@
  */
 package org.apache.vxquery.xmlparser;
 
-import org.apache.vxquery.exceptions.SystemException;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.vxquery.datamodel.accessors.nodes.NodeTreePointable;
+import org.apache.vxquery.datamodel.builders.nodes.AttributeNodeBuilder;
+import org.apache.vxquery.datamodel.builders.nodes.DictionaryBuilder;
+import org.apache.vxquery.datamodel.builders.nodes.DocumentNodeBuilder;
+import org.apache.vxquery.datamodel.builders.nodes.ElementNodeBuilder;
+import org.apache.vxquery.datamodel.builders.nodes.TextNodeBuilder;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.types.BuiltinTypeQNames;
+import org.apache.vxquery.xmlquery.query.XQueryConstants;
 import org.xml.sax.Attributes;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.Locator;
 import org.xml.sax.SAXException;
 import org.xml.sax.ext.LexicalHandler;
 
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
 public class SAXContentHandler implements ContentHandler, LexicalHandler {
-    private final IEventAcceptor acceptor;
+    private final ArrayBackedValueStorage docABVS;
+
+    private final boolean createNodeIds;
+
+    private final boolean attachTypes;
+
+    private final ITreeNodeIdProvider nodeIdProvider;
+
+    private final ArrayBackedValueStorage tempABVS;
+
+    private final DocumentNodeBuilder docb;
+
+    private final TextNodeBuilder tnb;
+
+    private final AttributeNodeBuilder anb;
+
+    private final DictionaryBuilder db;
+
     private final StringBuilder buffer;
-    private char[] chars;
+
+    private final List<ElementNodeBuilder> enbStack;
+
+    private final List<ElementNodeBuilder> freeENBList;
+
+    private int nodeIdCounter;
+
     private boolean pendingText;
 
-    public SAXContentHandler(IEventAcceptor acceptor) {
-        this.acceptor = acceptor;
+    public SAXContentHandler(boolean attachTypes, ITreeNodeIdProvider nodeIdProvider) {
+        docABVS = new ArrayBackedValueStorage();
+        this.createNodeIds = nodeIdProvider != null;
+        this.attachTypes = attachTypes;
+        this.nodeIdProvider = nodeIdProvider;
+        this.tempABVS = new ArrayBackedValueStorage();
+        docb = new DocumentNodeBuilder();
+        tnb = new TextNodeBuilder();
+        anb = new AttributeNodeBuilder();
+        db = new DictionaryBuilder();
         buffer = new StringBuilder();
+        enbStack = new ArrayList<ElementNodeBuilder>();
+        freeENBList = new ArrayList<ElementNodeBuilder>();
         pendingText = false;
     }
 
@@ -43,8 +91,9 @@ public class SAXContentHandler implement
     public void endDocument() throws SAXException {
         try {
             flushText();
-            acceptor.endDocument();
-        } catch (SystemException e) {
+            docb.endChildrenChunk();
+            docb.finish();
+        } catch (IOException e) {
             e.printStackTrace();
             throw new SAXException(e);
         }
@@ -54,8 +103,11 @@ public class SAXContentHandler implement
     public void endElement(String uri, String localName, String name) throws SAXException {
         try {
             flushText();
-            acceptor.endElement();
-        } catch (SystemException e) {
+            ElementNodeBuilder enb = enbStack.remove(enbStack.size() - 1);
+            enb.endChildrenChunk();
+            endChildInParent(enb);
+            freeENB(enb);
+        } catch (IOException e) {
             e.printStackTrace();
             throw new SAXException(e);
         }
@@ -73,8 +125,8 @@ public class SAXContentHandler implement
     public void processingInstruction(String target, String data) throws SAXException {
         try {
             flushText();
-            acceptor.pi(target, data);
-        } catch (SystemException e) {
+            throw new UnsupportedOperationException();
+        } catch (IOException e) {
             e.printStackTrace();
             throw new SAXException(e);
         }
@@ -91,9 +143,13 @@ public class SAXContentHandler implement
     @Override
     public void startDocument() throws SAXException {
         try {
+            docb.reset(docABVS);
+            if (createNodeIds) {
+                docb.setLocalNodeId(nodeIdCounter++);
+            }
+            docb.startChildrenChunk();
             flushText();
-            acceptor.startDocument();
-        } catch (SystemException e) {
+        } catch (IOException e) {
             e.printStackTrace();
             throw new SAXException(e);
         }
@@ -105,18 +161,50 @@ public class SAXContentHandler implement
             flushText();
             int idx = name.indexOf(':');
             String prefix = idx < 0 ? "" : name.substring(0, idx);
-            acceptor.startElement(uri, localName, prefix);
+            ElementNodeBuilder enb = createENB();
+            startChildInParent(enb);
+            int uriCode = db.lookup(uri);
+            int localNameCode = db.lookup(localName);
+            int prefixCode = db.lookup(prefix);
+            enb.setName(uriCode, localNameCode, prefixCode);
+            if (attachTypes) {
+                int typeUriCode = db.lookup(XQueryConstants.XS_NSURI);
+                int typeLocalNameCode = db.lookup(BuiltinTypeQNames.UNTYPED_STR);
+                int typePrefixCode = db.lookup(XQueryConstants.XS_PREFIX);
+                enb.setType(typeUriCode, typeLocalNameCode, typePrefixCode);
+            }
+            if (createNodeIds) {
+                enb.setLocalNodeId(nodeIdCounter++);
+            }
+            enb.startAttributeChunk();
             final int nAttrs = atts.getLength();
             for (int i = 0; i < nAttrs; ++i) {
                 String aName = atts.getQName(i);
                 int aIdx = aName.indexOf(':');
-                String aPrefix = aIdx < 0 ? "" : aName.substring(0, aIdx);
-                String aLocalName = atts.getLocalName(i);
-                String aUri = atts.getURI(i);
+                int aPrefixCode = db.lookup(aIdx < 0 ? "" : aName.substring(0, aIdx));
+                int aLocalNameCode = db.lookup(atts.getLocalName(i));
+                int aUriCode = db.lookup(atts.getURI(i));
                 String aValue = atts.getValue(i);
-                acceptor.attribute(aUri, aLocalName, aPrefix, aValue);
+                tempABVS.reset();
+                tempABVS.getDataOutput().writeUTF(aValue);
+                enb.startAttribute(anb);
+                anb.setName(aUriCode, aLocalNameCode, aPrefixCode);
+                if (attachTypes) {
+                    int typeUriCode = db.lookup(XQueryConstants.XS_NSURI);
+                    int typeLocalNameCode = db.lookup(BuiltinTypeQNames.UNTYPED_ATOMIC_STR);
+                    int typePrefixCode = db.lookup(XQueryConstants.XS_PREFIX);
+                    anb.setType(typeUriCode, typeLocalNameCode, typePrefixCode);
+                }
+                if (createNodeIds) {
+                    anb.setLocalNodeId(nodeIdCounter++);
+                }
+                anb.setValue(tempABVS);
+                enb.endAttribute(anb);
             }
-        } catch (SystemException e) {
+            enb.endAttributeChunk();
+            enb.startChildrenChunk();
+            enbStack.add(enb);
+        } catch (IOException e) {
             e.printStackTrace();
             throw new SAXException(e);
         }
@@ -130,22 +218,21 @@ public class SAXContentHandler implement
     public void comment(char[] ch, int start, int length) throws SAXException {
         try {
             flushText();
-            acceptor.comment(String.valueOf(ch, start, length));
-        } catch (SystemException e) {
+            throw new UnsupportedOperationException();
+        } catch (IOException e) {
             e.printStackTrace();
             throw new SAXException(e);
         }
     }
 
-    private void flushText() throws SystemException {
+    private void flushText() throws IOException {
         if (pendingText) {
-            int length = buffer.length();
-            if (chars == null || chars.length < length) {
-                chars = new char[length];
-            }
-            buffer.getChars(0, length, chars, 0);
-            acceptor.text(chars, 0, length);
-            buffer.delete(0, length);
+            peekENBStackTop().startChild(tnb);
+            tempABVS.reset();
+            tempABVS.getDataOutput().writeUTF(buffer.toString());
+            tnb.setValue(tempABVS);
+            peekENBStackTop().endChild(tnb);
+            buffer.delete(0, buffer.length());
             pendingText = false;
         }
     }
@@ -174,4 +261,52 @@ public class SAXContentHandler implement
     public void startEntity(String name) throws SAXException {
     }
 
-}
+    public void write(ArrayBackedValueStorage abvs) throws IOException {
+        DataOutput out = abvs.getDataOutput();
+        out.write(ValueTag.NODE_TREE_TAG);
+        byte header = NodeTreePointable.HEADER_DICTIONARY_EXISTS_MASK;
+        if (attachTypes) {
+            header |= NodeTreePointable.HEADER_TYPE_EXISTS_MASK;
+        }
+        if (createNodeIds) {
+            header |= NodeTreePointable.HEADER_NODEID_EXISTS_MASK;
+        }
+        out.write(header);
+        if (createNodeIds) {
+            out.writeLong(nodeIdProvider.getId());
+        }
+        db.write(abvs);
+        out.write(docABVS.getByteArray(), docABVS.getStartOffset(), docABVS.getLength());
+    }
+
+    private ElementNodeBuilder createENB() {
+        if (freeENBList.isEmpty()) {
+            return new ElementNodeBuilder();
+        }
+        return freeENBList.remove(freeENBList.size() - 1);
+    }
+
+    private void freeENB(ElementNodeBuilder enb) {
+        freeENBList.add(enb);
+    }
+
+    private ElementNodeBuilder peekENBStackTop() {
+        return enbStack.get(enbStack.size() - 1);
+    }
+
+    private void startChildInParent(ElementNodeBuilder enb) throws IOException {
+        if (enbStack.isEmpty()) {
+            docb.startChild(enb);
+        } else {
+            peekENBStackTop().startChild(enb);
+        }
+    }
+
+    private void endChildInParent(ElementNodeBuilder enb) throws IOException {
+        if (enbStack.isEmpty()) {
+            docb.endChild(enb);
+        } else {
+            peekENBStackTop().endChild(enb);
+        }
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java?rev=1355244&r1=1355243&r2=1355244&view=diff
==============================================================================
--- incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java (original)
+++ incubator/vxquery/branches/vxquery_algebricks/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java Fri Jun 29 06:14:08 2012
@@ -20,17 +20,19 @@ import org.xml.sax.InputSource;
 import org.xml.sax.XMLReader;
 import org.xml.sax.helpers.XMLReaderFactory;
 
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
 public class XMLParser {
-    public static void parseInputSource(InputSource in, IEventAcceptor acceptor) throws SystemException {
+    public static void parseInputSource(InputSource in, ArrayBackedValueStorage abvs, boolean attachTypes,
+            ITreeNodeIdProvider idProvider) throws SystemException {
         XMLReader parser;
         try {
             parser = XMLReaderFactory.createXMLReader();
-            SAXContentHandler handler = new SAXContentHandler(acceptor);
+            SAXContentHandler handler = new SAXContentHandler(attachTypes, idProvider);
             parser.setContentHandler(handler);
             parser.setProperty("http://xml.org/sax/properties/lexical-handler", handler);
-            acceptor.open();
             parser.parse(in);
-            acceptor.close();
+            handler.write(abvs);
         } catch (Exception e) {
             throw new SystemException(ErrorCode.FODC0002, e, in.getSystemId());
         }