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 2014/03/05 02:57:39 UTC

[13/14] git commit: Finished the new SAXContentHandler with the option to hold copies when they are sequential.

Finished the new SAXContentHandler with the option to hold copies when they are sequential.


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

Branch: refs/heads/prestonc/parser
Commit: 11664ee82020d48612f4505360b064895cc9cdcb
Parents: 278c0db
Author: Preston Carman <pr...@apache.org>
Authored: Wed Feb 19 17:10:45 2014 -0800
Committer: Preston Carman <pr...@apache.org>
Committed: Thu Feb 27 14:24:55 2014 -0800

----------------------------------------------------------------------
 .../vxquery/xmlparser/SAXContentHandler.java    | 302 +++----------------
 1 file changed, 43 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/11664ee8/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java
index 2b3d613..e6ebee8 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/SAXContentHandler.java
@@ -16,11 +16,8 @@ package org.apache.vxquery.xmlparser;
 
 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.AbstractNodeBuilder;
 import org.apache.vxquery.datamodel.builders.nodes.AttributeNodeBuilder;
 import org.apache.vxquery.datamodel.builders.nodes.CommentNodeBuilder;
 import org.apache.vxquery.datamodel.builders.nodes.DictionaryBuilder;
@@ -42,7 +39,7 @@ import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public class SAXContentHandler implements ContentHandler, LexicalHandler {
-    private final ArrayBackedValueStorage docABVS;
+    private static final int DEFAULT_INT_SIZE = 600;
 
     private final boolean createNodeIds;
 
@@ -50,8 +47,6 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
 
     private final ITreeNodeIdProvider nodeIdProvider;
 
-    private final ArrayBackedValueStorage tempABVS;
-
     private final DocumentNodeBuilder docb;
 
     private final TextNodeBuilder tnb;
@@ -66,13 +61,9 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
 
     private final StringBuilder buffer;
 
-    private final List<ElementNodeBuilder> enbStack;
-
-    private final List<ElementNodeBuilder> freeENBList;
-
     private int nodeIdCounter;
-    private int copyOldCounter = 0;
-    private int copyNewCounter = 0;
+    
+    private int copyNewCounter;
 
     private boolean pendingText;
 
@@ -82,10 +73,7 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
 
     // Structure and data.
     private final GrowableIntArray leavesKind;
-    //    private final GrowableIntArray leavesStart;
     private final GrowableIntArray leavesEnd;
-    //    private final GrowableIntArray leavesDepth;
-    //    private final GrowableIntArray leavesParent;
     private final GrowableIntArray leavesAttributeCount;
     private final GrowableIntArray leavesChildrenCount;
 
@@ -93,6 +81,7 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
     private int textCount = 0;
     private int textCurrentDepth = 0;
     private int childSlotCounter = 0;
+    private int currentOffset = 0;
     private final GrowableIntArray previousLeaf;
     private final GrowableIntArray childStartOffset;
     private final GrowableIntArray childSlotOffset;
@@ -108,11 +97,9 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
     private final int LEAF_POST_NODE = 7;
 
     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();
         cnb = new CommentNodeBuilder();
@@ -121,25 +108,19 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
         anb = new AttributeNodeBuilder();
         db = new DictionaryBuilder();
         buffer = new StringBuilder();
-        enbStack = new ArrayList<ElementNodeBuilder>();
-        freeENBList = new ArrayList<ElementNodeBuilder>();
         pendingText = false;
 
-        leavesKind = new GrowableIntArray(600);
-        //leavesStart = new GrowableIntArray(600);
+        leavesKind = new GrowableIntArray(DEFAULT_INT_SIZE);
         leavesABVS = new ArrayBackedValueStorage();
-        leavesEnd = new GrowableIntArray(600);
-        //        leavesDepth = new GrowableIntArray(600);
-        //        leavesParent = new GrowableIntArray(600);
-        leavesAttributeCount = new GrowableIntArray(600);
-        leavesChildrenCount = new GrowableIntArray(600);
-        previousLeaf = new GrowableIntArray(600);
-        childStartOffset = new GrowableIntArray(600);
-        childSlotOffset = new GrowableIntArray(600);
+        leavesEnd = new GrowableIntArray(DEFAULT_INT_SIZE);
+        leavesAttributeCount = new GrowableIntArray(DEFAULT_INT_SIZE);
+        leavesChildrenCount = new GrowableIntArray(DEFAULT_INT_SIZE);
+        previousLeaf = new GrowableIntArray(DEFAULT_INT_SIZE);
+        childStartOffset = new GrowableIntArray(DEFAULT_INT_SIZE);
+        childSlotOffset = new GrowableIntArray(DEFAULT_INT_SIZE);
         textCount = 0;
         textCurrentDepth = 0;
         childSlotCounter = 0;
-
     }
 
     @Override
@@ -152,10 +133,6 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
     public void endDocument() throws SAXException {
         try {
             flushText();
-            docb.endChildrenChunk();
-            copyOldCounter++;
-            docb.finish();
-
             leafNodeStart(LEAF_POST_DOCUMENT);
             leafNodeEnd();
             textCurrentDepth--;
@@ -163,30 +140,12 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
             e.printStackTrace();
             throw new SAXException(e);
         }
-
-        //        int[] k = leavesKind.getArray();
-        //        int[] s = leavesStart.getArray();
-        //        int[] e = leavesEnd.getArray();
-        //        int[] d = leavesDepth.getArray();
-        //        int[] p = leavesParent.getArray();
-        //        int[] c = leavesChildrenCount.getArray();
-        //        int[] a = leavesAttributeCount.getArray();
-        //        for (int i = 0; i < s.length; ++i) {
-        //            System.err.println(i + " " + k[i] + " - " + d[i] + ":" + s[i] + ":" + e[i] + " p=" + p[i] + " a=" + a[i]
-        //                    + " c=" + c[i]);
-        //        }
     }
 
     @Override
     public void endElement(String uri, String localName, String name) throws SAXException {
         try {
             flushText();
-            ElementNodeBuilder enb = enbStack.remove(enbStack.size() - 1);
-            enb.endChildrenChunk();
-            copyOldCounter++;
-            endChildInParent(enb);
-            freeENB(enb);
-
             leafNodeStart(LEAF_POST_NODE);
             leafNodeEnd();
             textCurrentDepth--;
@@ -208,21 +167,6 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
     public void processingInstruction(String target, String data) throws SAXException {
         try {
             flushText();
-            startChildInParent(pinb);
-            tempABVS.reset();
-            tempABVS.getDataOutput().writeUTF(target);
-            if (createNodeIds) {
-                pinb.setLocalNodeId(nodeIdCounter);
-            }
-            pinb.setTarget(tempABVS);
-            copyOldCounter++;
-            tempABVS.reset();
-            tempABVS.getDataOutput().writeUTF(data);
-            pinb.setContent(tempABVS);
-            copyOldCounter++;
-            endChildInParent(pinb);
-
-            // Save to leavesABVS
             leafNodeStart(LEAF_PI);
             pinb.reset(leavesABVS);
             if (createNodeIds) {
@@ -247,14 +191,10 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
 
     @Override
     public void startDocument() throws SAXException {
-        copyOldCounter = 0;
         copyNewCounter = 0;
         leavesKind.clear();
-        //leavesStart.clear();
         leavesABVS.reset();
         leavesEnd.clear();
-        //        leavesDepth.clear();
-        //        leavesParent.clear();
         leavesAttributeCount.clear();
         leavesChildrenCount.clear();
         previousLeaf.clear();
@@ -263,23 +203,17 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
         textCount = 0;
         textCurrentDepth = 0;
         childSlotCounter = 0;
+        currentOffset = 0;
 
         textCurrentDepth++;
         try {
             nodeIdCounter = 0;
             db.reset();
-            docABVS.reset();
-            docb.reset(docABVS);
-            if (createNodeIds) {
-                docb.setLocalNodeId(nodeIdCounter++);
-            }
-            docb.startChildrenChunk();
 
             leafNodeStart(LEAF_PRE_DOCUMENT);
-            DocumentNodeBuilder docb2 = new DocumentNodeBuilder();
-            docb2.reset(leavesABVS);
+            docb.reset(leavesABVS);
             if (createNodeIds) {
-                docb2.setLocalNodeId(nodeIdCounter);
+                docb.setLocalNodeId(nodeIdCounter++);
             }
             leafNodeEnd();
 
@@ -297,11 +231,12 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
             flushText();
             int idx = name.indexOf(':');
             String prefix = idx < 0 ? "" : name.substring(0, idx);
-            ElementNodeBuilder enb = createENB();
-            startChildInParent(enb);
             int uriCode = db.lookup(uri);
             int localNameCode = db.lookup(localName);
             int prefixCode = db.lookup(prefix);
+
+            leafNodeStart(LEAF_PRE_NODE);
+            enb.setMvs(leavesABVS);
             enb.setName(uriCode, localNameCode, prefixCode);
             if (attachTypes) {
                 int typeUriCode = db.lookup(XQueryConstants.XS_NSURI);
@@ -312,21 +247,6 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
             if (createNodeIds) {
                 enb.setLocalNodeId(nodeIdCounter++);
             }
-
-            // Save to leavesABVS
-            leafNodeStart(LEAF_PRE_NODE);
-            ElementNodeBuilder enb2 = createENB();
-            enb2.setMvs(leavesABVS);
-            enb2.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);
-                enb2.setType(typeUriCode, typeLocalNameCode, typePrefixCode);
-            }
-            if (createNodeIds) {
-                enb2.setLocalNodeId(nodeIdCounter);
-            }
             leafNodeEnd();
             textCurrentDepth++;
 
@@ -338,27 +258,7 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
                 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);
-                tempABVS.reset();
-                DataOutput tempOut = tempABVS.getDataOutput();
-                tempOut.write(ValueTag.XS_UNTYPED_ATOMIC_TAG);
-                tempOut.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);
-                copyOldCounter++;
-                enb.endAttribute(anb);
 
-                // Save to leavesABVS
                 leafNodeStart(LEAF_ATTRIBUTE);
                 anb.reset(leavesABVS);
                 anb.setName(aUriCode, aLocalNameCode, aPrefixCode);
@@ -373,12 +273,7 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
                 }
                 anb.setValue(atts.getValue(i));
                 leafNodeEnd();
-
             }
-            enb.endAttributeChunk();
-            copyOldCounter++;
-            enb.startChildrenChunk();
-            enbStack.add(enb);
         } catch (IOException e) {
             e.printStackTrace();
             throw new SAXException(e);
@@ -393,16 +288,7 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
     public void comment(char[] ch, int start, int length) throws SAXException {
         try {
             flushText();
-            startChildInParent(cnb);
             buffer.append(ch, start, length);
-            tempABVS.reset();
-            tempABVS.getDataOutput().writeUTF(buffer.toString());
-            if (createNodeIds) {
-                cnb.setLocalNodeId(nodeIdCounter);
-            }
-            cnb.setValue(tempABVS);
-            copyOldCounter++;
-            endChildInParent(cnb);
 
             // Save to leavesABVS
             leafNodeStart(LEAF_COMMENT);
@@ -422,17 +308,6 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
 
     private void flushText() throws IOException {
         if (pendingText) {
-            peekENBStackTop().startChild(tnb);
-            tempABVS.reset();
-            tempABVS.getDataOutput().writeUTF(buffer.toString());
-            if (createNodeIds) {
-                tnb.setLocalNodeId(nodeIdCounter);
-            }
-            tnb.setValue(tempABVS);
-            copyOldCounter++;
-            peekENBStackTop().endChild(tnb);
-
-            // Save to leavesABVS
             leafNodeStart(LEAF_TEXT);
             tnb.reset(leavesABVS);
             if (createNodeIds) {
@@ -470,29 +345,31 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
     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;
+    private void leafNodeStart(int kind) {
+        leavesKind.append(kind);
+        leavesAttributeCount.append(0);
+        leavesChildrenCount.append(0);
+
+        int parent = previousLeaf.getArray()[textCurrentDepth - 1];
+        if (kind == LEAF_POST_NODE || kind == LEAF_POST_DOCUMENT) {
+            // Skip Count
+        } else if (kind == LEAF_ATTRIBUTE) {
+            leavesAttributeCount.getArray()[parent] += 1;
+        } else if (textCount != parent) {
+            leavesChildrenCount.getArray()[parent] += 1;
         }
-        out.write(header);
-        if (createNodeIds) {
-            out.writeInt(nodeIdProvider.getId());
+        // If it can have children.
+        if (kind == LEAF_PRE_NODE || kind == LEAF_PRE_DOCUMENT) {
+            previousLeaf.getArray()[textCurrentDepth] = textCount;
         }
-        db.write(abvs);
-        out.write(docABVS.getByteArray(), docABVS.getStartOffset(), docABVS.getLength());
-        copyOldCounter++;
-        System.err.println("copyCounter: " + copyOldCounter);
     }
 
-    int currentOffset = 0;
+    private void leafNodeEnd() {
+        leavesEnd.append(leavesABVS.getLength());
+        textCount++;
+    }
 
-    public void writeOnce(ArrayBackedValueStorage abvs) throws IOException {
+    public void write(ArrayBackedValueStorage abvs) throws IOException {
         DataOutput out = abvs.getDataOutput();
         out.write(ValueTag.NODE_TREE_TAG);
         byte header = NodeTreePointable.HEADER_DICTIONARY_EXISTS_MASK;
@@ -507,25 +384,17 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
             out.writeInt(nodeIdProvider.getId());
         }
         db.write(abvs);
-
         copyNewCounter++;
 
-        // leavesStart.getArray()[i]
         for (int i = 0; i < leavesKind.getSize(); ++i) {
             if (leavesKind.getArray()[i] == LEAF_PRE_DOCUMENT) {
                 flushLeaveNodesUpTo(out, i);
 
-                //                for (int x = currentOffset; x < leavesEnd.getArray()[i]; ++x) {
-                //                    System.err.println(i + "\t" + leavesKind.getArray()[i] + "\t" + leavesABVS.getByteArray()[x]);
-                //                }
-
                 int children = leavesChildrenCount.getArray()[i];
-                System.err.println("children " + children);
                 if (children > 0) {
                     sequenceSlotStub(abvs, children);
                 }
                 // Continue with nodes.
-                childrenLength = 0;
 
             } else if (leavesKind.getArray()[i] == LEAF_PRE_NODE) {
                 flushLeaveNodesUpTo(out, i - 1);
@@ -541,11 +410,9 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
 
                 if (attrCount > 0) {
                     sequenceSlotStub(abvs, attrCount);
-                    int attributeLength = 0;
                     for (int s = 0; s < attrCount; ++s) {
                         ++i;
-                        attributeLength = leavesEnd.getArray()[i] - currentOffset;
-                        updateSequenceSlot(abvs, attributeLength);
+                        updateSequenceSlot(abvs, leavesEnd.getArray()[i] - currentOffset);
                     }
                     flushLeaveNodesUpTo(out, i);
                 }
@@ -554,22 +421,19 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
                     sequenceSlotStub(abvs, childrenCount);
                 }
                 // Continue with nodes.
-                startChildHunk(currentOffset);
 
             } else if (leavesKind.getArray()[i] == LEAF_POST_DOCUMENT) {
                 flushLeaveNodesUpTo(out, i - 1);
-
                 // no action
             } else if (leavesKind.getArray()[i] == LEAF_POST_NODE) {
                 flushLeaveNodesUpTo(out, i - 1);
 
-                updateSequenceSlot(abvs);
+                updateSequenceSlot(abvs, 0);
             } else {
-                childrenLength = leavesEnd.getArray()[i] - currentOffset;
-                updateSequenceSlot(abvs, childrenLength);
+                updateSequenceSlot(abvs, leavesEnd.getArray()[i] - currentOffset);
             }
         }
-        System.err.println("copyNewCounter: " + copyNewCounter);
+//        System.err.println("copyNewCounter: " + copyNewCounter);
     }
 
     private void flushLeaveNodesUpTo(DataOutput out, int i) throws IOException {
@@ -580,28 +444,14 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
         }
     }
 
-    int childrenLength = 0;
-    int childrenOffset = 0;
-
-    private void startChildHunk(int currentOffset) {
-        childrenLength = 0;
-        childrenOffset = currentOffset;
-    }
-
     private void sequenceSlotStub(ArrayBackedValueStorage abvs, int count) throws IOException {
         DataOutput out = abvs.getDataOutput();
         out.writeInt(count);
-        //        System.err.println("Slot count " + count);
         int offset = abvs.getLength();
         for (int s = 0; s < count; ++s) {
             out.writeInt(-1);
             addSequenceSlot(offset, s, count);
         }
-        //
-        //        for (int i = 0; i < childSlotCounter; ++i) {
-        //            System.err.println("\t\t" + i + " " + childStartOffset.getArray()[i] + " - "
-        //                    + childSlotOffset.getArray()[i]);
-        //        }
     }
 
     private void addSequenceSlot(int offset, int count, int total) {
@@ -610,76 +460,10 @@ public class SAXContentHandler implements ContentHandler, LexicalHandler {
         childSlotCounter++;
     }
 
-    private void updateSequenceSlot(ArrayBackedValueStorage abvs, int length) {
-        //        for (int i = 0; i < childSlotCounter; ++i) {
-        //            System.err.println("\t" + i + " " + childStartOffset.getArray()[i] + " - " + childSlotOffset.getArray()[i]);
-        //        }
+    private void updateSequenceSlot(ArrayBackedValueStorage abvs, int additional) {
         childSlotCounter--;
-        //        int length = abvs.getLength() - childStartOffset.getArray()[childSlotCounter];
+        int length = abvs.getLength() - childStartOffset.getArray()[childSlotCounter] + additional;
         IntegerPointable.setInteger(abvs.getByteArray(), childSlotOffset.getArray()[childSlotCounter], length);
     }
 
-    private void updateSequenceSlot(ArrayBackedValueStorage abvs) {
-        int length = abvs.getLength() - childStartOffset.getArray()[childSlotCounter];
-        updateSequenceSlot(abvs, length);
-    }
-
-    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(AbstractNodeBuilder anb) throws IOException {
-        if (enbStack.isEmpty()) {
-            docb.startChild(anb);
-        } else {
-            peekENBStackTop().startChild(anb);
-        }
-    }
-
-    private void endChildInParent(AbstractNodeBuilder anb) throws IOException {
-        if (enbStack.isEmpty()) {
-            docb.endChild(anb);
-        } else {
-            peekENBStackTop().endChild(anb);
-        }
-    }
-
-    private void leafNodeStart(int kind) {
-        leavesKind.append(kind);
-        //leavesStart.append(leavesABVS.getLength());
-        //        leavesDepth.append(textCurrentDepth);
-        leavesAttributeCount.append(0);
-        leavesChildrenCount.append(0);
-
-        int parent = previousLeaf.getArray()[textCurrentDepth - 1];
-        //        leavesParent.append(parent);
-        if (kind == LEAF_POST_NODE || kind == LEAF_POST_DOCUMENT) {
-            // Skip Count
-        } else if (kind == LEAF_ATTRIBUTE) {
-            leavesAttributeCount.getArray()[parent] += 1;
-        } else if (textCount != parent) {
-            leavesChildrenCount.getArray()[parent] += 1;
-        }
-        // If it can have children.
-        if (kind == LEAF_PRE_NODE || kind == LEAF_PRE_DOCUMENT) {
-            previousLeaf.getArray()[textCurrentDepth] = textCount;
-        }
-    }
-
-    private void leafNodeEnd() {
-        leavesEnd.append(leavesABVS.getLength());
-        textCount++;
-    }
-
 }
\ No newline at end of file