ss: History trees can define their own node types
[deliverable/tracecompass.git] / statesystem / org.eclipse.tracecompass.statesystem.core / src / org / eclipse / tracecompass / internal / statesystem / core / backend / historytree / HTNode.java
index 688c7212b8d89b8946906affed9868112b22c32a..9cada2d4f4e989f8ea981490187fd168b34ba40e 100644 (file)
@@ -1,5 +1,5 @@
 /*******************************************************************************
- * Copyright (c) 2010, 2015 Ericsson, École Polytechnique de Montréal, and others
+ * Copyright (c) 2010, 2016 Ericsson, École Polytechnique de Montréal, and others
  *
  * All rights reserved. This program and the accompanying materials are
  * made available under the terms of the Eclipse Public License v1.0 which
@@ -24,10 +24,13 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import org.eclipse.jdt.annotation.NonNull;
 import org.eclipse.tracecompass.statesystem.core.exceptions.TimeRangeException;
 import org.eclipse.tracecompass.statesystem.core.interval.ITmfStateInterval;
 import org.eclipse.tracecompass.statesystem.core.statevalue.TmfStateValue;
 
+import com.google.common.collect.Iterables;
+
 /**
  * The base class for all the types of nodes that go in the History Tree.
  *
@@ -92,35 +95,46 @@ public abstract class HTNode {
         }
     }
 
+    /**
+     * <pre>
+     *  1 - byte (type)
+     * 16 - 2x long (start time, end time)
+     * 16 - 4x int (seq number, parent seq number, intervalcount,
+     *              strings section pos.)
+     *  1 - byte (done or not)
+     * </pre>
+     */
+    private static final int COMMON_HEADER_SIZE = Byte.BYTES
+            + 2 * Long.BYTES
+            + 4 * Integer.BYTES
+            + Byte.BYTES;
+
     // ------------------------------------------------------------------------
     // Attributes
     // ------------------------------------------------------------------------
 
     /* Configuration of the History Tree to which belongs this node */
-    private final HTConfig config;
+    private final HTConfig fConfig;
 
     /* Time range of this node */
-    private final long nodeStart;
-    private long nodeEnd;
+    private final long fNodeStart;
+    private long fNodeEnd;
 
     /* Sequence number = position in the node section of the file */
-    private final int sequenceNumber;
-    private int parentSequenceNumber; /* = -1 if this node is the root node */
-
-    /* Where the Strings section begins (from the start of the node */
-    private int stringSectionOffset;
+    private final int fSequenceNumber;
+    private int fParentSequenceNumber; /* = -1 if this node is the root node */
 
     /* Sum of bytes of all intervals in the node */
-    private int sizeOfIntervalSection;
+    private int fSizeOfIntervalSection;
 
     /* True if this node was read from disk (meaning its end time is now fixed) */
-    private volatile boolean isOnDisk;
+    private volatile boolean fIsOnDisk;
 
     /* Vector containing all the intervals contained in this node */
-    private final List<HTInterval> intervals;
+    private final List<HTInterval> fIntervals;
 
     /* Lock used to protect the accesses to intervals, nodeEnd and such */
-    private final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(false);
+    private final ReentrantReadWriteLock fRwl = new ReentrantReadWriteLock(false);
 
     /**
      * Constructor
@@ -135,15 +149,14 @@ public abstract class HTNode {
      *            The earliest timestamp stored in this node
      */
     protected HTNode(HTConfig config, int seqNumber, int parentSeqNumber, long start) {
-        this.config = config;
-        this.nodeStart = start;
-        this.sequenceNumber = seqNumber;
-        this.parentSequenceNumber = parentSeqNumber;
-
-        this.stringSectionOffset = config.getBlockSize();
-        this.sizeOfIntervalSection = 0;
-        this.isOnDisk = false;
-        this.intervals = new ArrayList<>();
+        fConfig = config;
+        fNodeStart = start;
+        fSequenceNumber = seqNumber;
+        fParentSequenceNumber = parentSeqNumber;
+
+        fSizeOfIntervalSection = 0;
+        fIsOnDisk = false;
+        fIntervals = new ArrayList<>();
     }
 
     /**
@@ -155,11 +168,13 @@ public abstract class HTNode {
      * @param fc
      *            FileChannel to the history file, ALREADY SEEKED at the start
      *            of the node.
+     * @param nodeFactory
+     *            The factory to create the nodes for this tree
      * @return The node object
      * @throws IOException
      *             If there was an error reading from the file channel
      */
-    public static final HTNode readNode(HTConfig config, FileChannel fc)
+    public static final @NonNull HTNode readNode(HTConfig config, FileChannel fc, IHistoryTree.IHTNodeFactory nodeFactory)
             throws IOException {
         HTNode newNode = null;
         int res, i;
@@ -179,20 +194,19 @@ public abstract class HTNode {
         int seqNb = buffer.getInt();
         int parentSeqNb = buffer.getInt();
         int intervalCount = buffer.getInt();
-        int stringSectionOffset = buffer.getInt();
         buffer.get(); // TODO Used to be "isDone", to be removed from the header
 
         /* Now the rest of the header depends on the node type */
         switch (type) {
         case CORE:
             /* Core nodes */
-            newNode = new CoreNode(config, seqNb, parentSeqNb, start);
+            newNode = nodeFactory.createCoreNode(config, seqNb, parentSeqNb, start);
             newNode.readSpecificHeader(buffer);
             break;
 
         case LEAF:
             /* Leaf nodes */
-            newNode = new LeafNode(config, seqNb, parentSeqNb, start);
+            newNode = nodeFactory.createLeafNode(config, seqNb, parentSeqNb, start);
             newNode.readSpecificHeader(buffer);
             break;
 
@@ -206,13 +220,14 @@ public abstract class HTNode {
          * should only have the intervals left
          */
         for (i = 0; i < intervalCount; i++) {
-            newNode.intervals.add(HTInterval.readFrom(buffer));
+            HTInterval interval = HTInterval.readFrom(buffer);
+            newNode.fIntervals.add(interval);
+            newNode.fSizeOfIntervalSection += interval.getSizeOnDisk();
         }
 
         /* Assign the node's other information we have read previously */
-        newNode.nodeEnd = end;
-        newNode.stringSectionOffset = stringSectionOffset;
-        newNode.isOnDisk = true;
+        newNode.fNodeEnd = end;
+        newNode.fIsOnDisk = true;
 
         return newNode;
     }
@@ -231,62 +246,47 @@ public abstract class HTNode {
          * Yes, we are taking the *read* lock here, because we are reading the
          * information in the node to write it to disk.
          */
-        rwl.readLock().lock();
+        fRwl.readLock().lock();
         try {
-            final int blockSize = config.getBlockSize();
-            int curStringsEntryEndPos = blockSize;
+            final int blockSize = fConfig.getBlockSize();
 
             ByteBuffer buffer = ByteBuffer.allocate(blockSize);
             buffer.order(ByteOrder.LITTLE_ENDIAN);
             buffer.clear();
 
             /* Write the common header part */
-            buffer.put(this.getNodeType().toByte());
-            buffer.putLong(nodeStart);
-            buffer.putLong(nodeEnd);
-            buffer.putInt(sequenceNumber);
-            buffer.putInt(parentSequenceNumber);
-            buffer.putInt(intervals.size());
-            buffer.putInt(stringSectionOffset);
+            buffer.put(getNodeType().toByte());
+            buffer.putLong(fNodeStart);
+            buffer.putLong(fNodeEnd);
+            buffer.putInt(fSequenceNumber);
+            buffer.putInt(fParentSequenceNumber);
+            buffer.putInt(fIntervals.size());
             buffer.put((byte) 1); // TODO Used to be "isDone", to be removed from header
 
             /* Now call the inner method to write the specific header part */
-            this.writeSpecificHeader(buffer);
+            writeSpecificHeader(buffer);
 
             /* Back to us, we write the intervals */
-            for (HTInterval interval : intervals) {
-                int size = interval.writeInterval(buffer, curStringsEntryEndPos);
-                curStringsEntryEndPos -= size;
-            }
+            fIntervals.forEach(i -> i.writeInterval(buffer));
 
             /*
-             * Write padding between the end of the Data section and the start
-             * of the Strings section (needed to fill the node in case there is
-             * no Strings section)
+             * Fill the rest with zeros
              */
-            while (buffer.position() < stringSectionOffset) {
+            while (buffer.position() < blockSize) {
                 buffer.put((byte) 0);
             }
 
-            /*
-             * If the offsets were right, the size of the Strings section should
-             * be == to the expected size
-             */
-            assert (curStringsEntryEndPos == stringSectionOffset);
-
             /* Finally, write everything in the Buffer to disk */
-
-            // if we don't do this, flip() will lose what's after.
-            buffer.position(blockSize);
-
             buffer.flip();
             int res = fc.write(buffer);
-            assert (res == blockSize);
+            if (res != blockSize) {
+                throw new IllegalStateException("Wrong size of block written: Actual: " + res + ", Expected: " + blockSize); //$NON-NLS-1$ //$NON-NLS-2$
+            }
 
         } finally {
-            rwl.readLock().unlock();
+            fRwl.readLock().unlock();
         }
-        isOnDisk = true;
+        fIsOnDisk = true;
     }
 
     // ------------------------------------------------------------------------
@@ -299,7 +299,7 @@ public abstract class HTNode {
      * @return The history tree config
      */
     protected HTConfig getConfig() {
-        return config;
+        return fConfig;
     }
 
     /**
@@ -308,7 +308,7 @@ public abstract class HTNode {
      * @return The start time of this node
      */
     public long getNodeStart() {
-        return nodeStart;
+        return fNodeStart;
     }
 
     /**
@@ -317,8 +317,8 @@ public abstract class HTNode {
      * @return The end time of this node
      */
     public long getNodeEnd() {
-        if (this.isOnDisk) {
-            return nodeEnd;
+        if (fIsOnDisk) {
+            return fNodeEnd;
         }
         return 0;
     }
@@ -329,7 +329,7 @@ public abstract class HTNode {
      * @return The sequence number of this node
      */
     public int getSequenceNumber() {
-        return sequenceNumber;
+        return fSequenceNumber;
     }
 
     /**
@@ -338,7 +338,7 @@ public abstract class HTNode {
      * @return The parent sequence number
      */
     public int getParentSequenceNumber() {
-        return parentSequenceNumber;
+        return fParentSequenceNumber;
     }
 
     /**
@@ -349,7 +349,7 @@ public abstract class HTNode {
      *            The sequence number of the node that is the new parent
      */
     public void setParentSequenceNumber(int newParent) {
-        parentSequenceNumber = newParent;
+        fParentSequenceNumber = newParent;
     }
 
     /**
@@ -358,7 +358,7 @@ public abstract class HTNode {
      * @return If this node is done or not
      */
     public boolean isOnDisk() {
-        return isOnDisk;
+        return fIsOnDisk;
     }
 
     /**
@@ -368,24 +368,22 @@ public abstract class HTNode {
      *            Interval to add to this node
      */
     public void addInterval(HTInterval newInterval) {
-        rwl.writeLock().lock();
+        fRwl.writeLock().lock();
         try {
             /* Just in case, should be checked before even calling this function */
-            assert (newInterval.getIntervalSize() <= this.getNodeFreeSpace());
+            assert (newInterval.getSizeOnDisk() <= getNodeFreeSpace());
 
             /* Find the insert position to keep the list sorted */
-            int index = intervals.size();
-            while (index > 0 && newInterval.compareTo(intervals.get(index - 1)) < 0) {
+            int index = fIntervals.size();
+            while (index > 0 && newInterval.compareTo(fIntervals.get(index - 1)) < 0) {
                 index--;
             }
 
-            intervals.add(index, newInterval);
-            sizeOfIntervalSection += newInterval.getIntervalSize();
+            fIntervals.add(index, newInterval);
+            fSizeOfIntervalSection += newInterval.getSizeOnDisk();
 
-            /* Update the in-node offset "pointer" */
-            stringSectionOffset -= (newInterval.getStringsEntrySize());
         } finally {
-            rwl.writeLock().unlock();
+            fRwl.writeLock().unlock();
         }
     }
 
@@ -397,22 +395,30 @@ public abstract class HTNode {
      *            The nodeEnd time that the node will have
      */
     public void closeThisNode(long endtime) {
-        rwl.writeLock().lock();
+        fRwl.writeLock().lock();
         try {
-            assert (endtime >= this.nodeStart);
+            /**
+             * FIXME: was assert (endtime >= fNodeStart); but that exception
+             * is reached with an empty node that has start time endtime + 1
+             */
+//            if (endtime < fNodeStart) {
+//                throw new IllegalArgumentException("Endtime " + endtime + " cannot be lower than start time " + fNodeStart);
+//            }
 
-            if (!intervals.isEmpty()) {
+            if (!fIntervals.isEmpty()) {
                 /*
                  * Make sure there are no intervals in this node with their
                  * EndTime > the one requested. Only need to check the last one
                  * since they are sorted
                  */
-                assert (endtime >= intervals.get(intervals.size() - 1).getEndTime());
+                if (endtime < Iterables.getLast(fIntervals).getEndTime()) {
+                    throw new IllegalArgumentException("Closing end time should be greater than or equal to the end time of the intervals of this node"); //$NON-NLS-1$
+                }
             }
 
-            this.nodeEnd = endtime;
+            fNodeEnd = endtime;
         } finally {
-            rwl.writeLock().unlock();
+            fRwl.writeLock().unlock();
         }
     }
 
@@ -432,9 +438,9 @@ public abstract class HTNode {
     public void writeInfoFromNode(List<ITmfStateInterval> stateInfo, long t)
             throws TimeRangeException {
         /* This is from a state system query, we are "reading" this node */
-        rwl.readLock().lock();
+        fRwl.readLock().lock();
         try {
-            for (int i = getStartIndexFor(t); i < intervals.size(); i++) {
+            for (int i = getStartIndexFor(t); i < fIntervals.size(); i++) {
                 /*
                  * Now we only have to compare the Start times, since we now the
                  * End times necessarily fit.
@@ -443,14 +449,14 @@ public abstract class HTNode {
                  * been created after stateInfo was instantiated (they would be
                  * null anyway).
                  */
-                ITmfStateInterval interval = intervals.get(i);
-                if (interval.getStartTime() <= t &&
+                ITmfStateInterval interval = fIntervals.get(i);
+                if (t >= interval.getStartTime() &&
                         interval.getAttribute() < stateInfo.size()) {
                     stateInfo.set(interval.getAttribute(), interval);
                 }
             }
         } finally {
-            rwl.readLock().unlock();
+            fRwl.readLock().unlock();
         }
     }
 
@@ -468,10 +474,10 @@ public abstract class HTNode {
      *             If 't' is invalid
      */
     public HTInterval getRelevantInterval(int key, long t) throws TimeRangeException {
-        rwl.readLock().lock();
+        fRwl.readLock().lock();
         try {
-            for (int i = getStartIndexFor(t); i < intervals.size(); i++) {
-                HTInterval curInterval = intervals.get(i);
+            for (int i = getStartIndexFor(t); i < fIntervals.size(); i++) {
+                HTInterval curInterval = fIntervals.get(i);
                 if (curInterval.getAttribute() == key
                         && curInterval.getStartTime() <= t
                         && curInterval.getEndTime() >= t) {
@@ -483,14 +489,14 @@ public abstract class HTNode {
             return null;
 
         } finally {
-            rwl.readLock().unlock();
+            fRwl.readLock().unlock();
         }
     }
 
     private int getStartIndexFor(long t) throws TimeRangeException {
         /* Should only be called by methods with the readLock taken */
 
-        if (intervals.isEmpty()) {
+        if (fIntervals.isEmpty()) {
             return 0;
         }
         /*
@@ -499,7 +505,7 @@ public abstract class HTNode {
          * provides a .binarySearch method, but its API is quite weird...
          */
         HTInterval dummy = new HTInterval(0, t, 0, TmfStateValue.nullValue());
-        int index = Collections.binarySearch(intervals, dummy);
+        int index = Collections.binarySearch(fIntervals, dummy);
 
         if (index < 0) {
             /*
@@ -509,41 +515,22 @@ public abstract class HTNode {
              */
             index = -index - 1;
 
-        }
-
-        /* Sometimes binarySearch yields weird stuff... */
-        if (index < 0) {
-            index = 0;
-        }
-        if (index >= intervals.size()) {
-            index = intervals.size() - 1;
-        }
-
-        /*
-         * Another API quirkiness, the returned index is the one of the *last*
-         * element of a series of equal endtimes, which happens sometimes. We
-         * want the *first* element of such a series, to read through them
-         * again.
-         */
-        while (index > 0
-                && intervals.get(index - 1).compareTo(intervals.get(index)) == 0) {
-            index--;
+        } else {
+            /*
+             * Another API quirkiness, the returned index is the one of the *last*
+             * element of a series of equal endtimes, which happens sometimes. We
+             * want the *first* element of such a series, to read through them
+             * again.
+             */
+            while (index > 0
+                    && fIntervals.get(index - 1).compareTo(fIntervals.get(index)) == 0) {
+                index--;
+            }
         }
 
         return index;
     }
 
-    /**
-     * <pre>
-     *  1 - byte (type)
-     * 16 - 2x long (start time, end time)
-     * 16 - 4x int (seq number, parent seq number, intervalcount,
-     *              strings section pos.)
-     *  1 - byte (done or not)
-     * </pre>
-     */
-    private static final int COMMON_HEADER_SIZE = 34;
-
     /**
      * Return the total header size of this node (will depend on the node type).
      *
@@ -557,7 +544,7 @@ public abstract class HTNode {
      * @return The offset, within the node, where the Data section ends
      */
     private int getDataSectionEndOffset() {
-        return this.getTotalHeaderSize() + sizeOfIntervalSection;
+        return getTotalHeaderSize() + fSizeOfIntervalSection;
     }
 
     /**
@@ -567,9 +554,9 @@ public abstract class HTNode {
      * @return The amount of free space in the node (in bytes)
      */
     public int getNodeFreeSpace() {
-        rwl.readLock().lock();
-        int ret = stringSectionOffset - this.getDataSectionEndOffset();
-        rwl.readLock().unlock();
+        fRwl.readLock().lock();
+        int ret = fConfig.getBlockSize() - getDataSectionEndOffset();
+        fRwl.readLock().unlock();
 
         return ret;
     }
@@ -582,16 +569,16 @@ public abstract class HTNode {
      *         in this node.
      */
     public long getNodeUsagePercent() {
-        rwl.readLock().lock();
+        fRwl.readLock().lock();
         try {
-            final int blockSize = config.getBlockSize();
-            float freePercent = (float) this.getNodeFreeSpace()
-                    / (float) (blockSize - this.getTotalHeaderSize())
+            final int blockSize = fConfig.getBlockSize();
+            float freePercent = (float) getNodeFreeSpace()
+                    / (float) (blockSize - getTotalHeaderSize())
                     * 100F;
             return (long) (100L - freePercent);
 
         } finally {
-            rwl.readLock().unlock();
+            fRwl.readLock().unlock();
         }
     }
 
@@ -603,18 +590,14 @@ public abstract class HTNode {
     @Override
     public String toString() {
         /* Only used for debugging, shouldn't be externalized */
-        StringBuffer buf = new StringBuffer("Node #" + sequenceNumber + ", ");
-        buf.append(this.toStringSpecific());
-        buf.append(intervals.size() + " intervals (" + this.getNodeUsagePercent()
-                + "% used), ");
-
-        buf.append("[" + this.nodeStart + " - ");
-        if (this.isOnDisk) {
-            buf = buf.append("" + this.nodeEnd + "]");
-        } else {
-            buf = buf.append("...]");
-        }
-        return buf.toString();
+        return String.format("Node #%d, %s, %s, %d intervals (%d%% used), [%d - %s]",
+                fSequenceNumber,
+                (fParentSequenceNumber == -1) ? "Root" : "Parent #" + fParentSequenceNumber,
+                toStringSpecific(),
+                fIntervals.size(),
+                getNodeUsagePercent(),
+                fNodeStart,
+                (fIsOnDisk || fNodeEnd != 0) ? fNodeEnd : "...");
     }
 
     /**
@@ -626,11 +609,11 @@ public abstract class HTNode {
     @SuppressWarnings("nls")
     public void debugPrintIntervals(PrintWriter writer) {
         /* Only used for debugging, shouldn't be externalized */
-        writer.println("Node #" + sequenceNumber + ":");
+        writer.println("Intervals for node #" + fSequenceNumber + ":");
 
         /* Array of children */
-        if (this.getNodeType() == NodeType.CORE) { /* Only Core Nodes can have children */
-            CoreNode thisNode = (CoreNode) this;
+        if (getNodeType() != NodeType.LEAF) { /* Only Core Nodes can have children */
+            ParentNode thisNode = (ParentNode) this;
             writer.print("  " + thisNode.getNbChildren() + " children");
             if (thisNode.getNbChildren() >= 1) {
                 writer.print(": [ " + thisNode.getChild(0));
@@ -644,8 +627,8 @@ public abstract class HTNode {
 
         /* List of intervals in the node */
         writer.println("  Intervals contained:");
-        for (int i = 0; i < intervals.size(); i++) {
-            writer.println(intervals.get(i).toString());
+        for (int i = 0; i < fIntervals.size(); i++) {
+            writer.println(fIntervals.get(i).toString());
         }
         writer.println('\n');
     }
This page took 0.044438 seconds and 5 git commands to generate.