You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@uima.apache.org by sc...@apache.org on 2015/12/28 21:55:16 UTC

svn commit: r1722002 [2/2] - in /uima/uimaj/branches/experiment-v3-jcas/uimaj-core/src/main/java/org/apache/uima/cas: function/DeserBinaryIndexes.java impl/BinaryCasSerDes6.java

Modified: uima/uimaj/branches/experiment-v3-jcas/uimaj-core/src/main/java/org/apache/uima/cas/impl/BinaryCasSerDes6.java
URL: http://svn.apache.org/viewvc/uima/uimaj/branches/experiment-v3-jcas/uimaj-core/src/main/java/org/apache/uima/cas/impl/BinaryCasSerDes6.java?rev=1722002&r1=1722001&r2=1722002&view=diff
==============================================================================
--- uima/uimaj/branches/experiment-v3-jcas/uimaj-core/src/main/java/org/apache/uima/cas/impl/BinaryCasSerDes6.java (original)
+++ uima/uimaj/branches/experiment-v3-jcas/uimaj-core/src/main/java/org/apache/uima/cas/impl/BinaryCasSerDes6.java Mon Dec 28 20:55:15 2015
@@ -21,9 +21,7 @@ package org.apache.uima.cas.impl;
 
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.NBR_SLOT_KIND_ZIP_STREAMS;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_ArrayLength;
-import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_BooleanRef;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_Byte;
-import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_ByteRef;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_Control;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_DoubleRef;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_Double_Exponent;
@@ -36,13 +34,10 @@ import static org.apache.uima.cas.impl.S
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_LongRef;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_Long_High;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_Long_Low;
-import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_MainHeap;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_Short;
-import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_ShortRef;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_StrChars;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_StrLength;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_StrOffset;
-import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_StrRef;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_StrSeg;
 import static org.apache.uima.cas.impl.SlotKinds.SlotKind.Slot_TypeCode;
 
@@ -62,7 +57,13 @@ import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
+import java.util.Collections;
+import java.util.IdentityHashMap;
 import java.util.List;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.IntPredicate;
+import java.util.stream.Stream;
 import java.util.zip.Deflater;
 import java.util.zip.DeflaterOutputStream;
 import java.util.zip.Inflater;
@@ -70,13 +71,29 @@ import java.util.zip.InflaterInputStream
 
 import org.apache.uima.cas.AbstractCas;
 import org.apache.uima.cas.CASRuntimeException;
+import org.apache.uima.cas.FSIterator;
+import org.apache.uima.cas.Feature;
+import org.apache.uima.cas.impl.CASImpl.FsChange;
 import org.apache.uima.cas.impl.FSsTobeAddedback.FSsTobeAddedbackSingle;
 import org.apache.uima.cas.impl.SlotKinds.SlotKind;
-import org.apache.uima.cas.impl.TypeSystemImpl.TypeInfo;
+import org.apache.uima.internal.util.Int2ObjHashMap;
+import org.apache.uima.internal.util.IntListIterator;
 import org.apache.uima.internal.util.IntVector;
-import org.apache.uima.internal.util.rb_trees.Int2IntRBT;
 import org.apache.uima.jcas.JCas;
+import org.apache.uima.jcas.cas.BooleanArray;
+import org.apache.uima.jcas.cas.ByteArray;
+import org.apache.uima.jcas.cas.CommonArray;
+import org.apache.uima.jcas.cas.DoubleArray;
+import org.apache.uima.jcas.cas.FSArray;
+import org.apache.uima.jcas.cas.FloatArray;
+import org.apache.uima.jcas.cas.IntegerArray;
+import org.apache.uima.jcas.cas.LongArray;
+import org.apache.uima.jcas.cas.ShortArray;
+import org.apache.uima.jcas.cas.Sofa;
+import org.apache.uima.jcas.cas.StringArray;
+import org.apache.uima.jcas.cas.TOP;
 import org.apache.uima.resource.ResourceInitializationException;
+import org.apache.uima.util.Misc;
 import org.apache.uima.util.impl.DataIO;
 import org.apache.uima.util.impl.OptimizeStrings;
 import org.apache.uima.util.impl.SerializationMeasures;
@@ -194,11 +211,14 @@ import org.apache.uima.util.impl.Seriali
  */
 public class BinaryCasSerDes6 {
 
-  private static final int[] INT0 = new int[0];
+  private static final String EMPTY_STRING = "";
   
   private static final boolean TRACE_SER = false;
   private static final boolean TRACE_DES = false;
   
+  private static final boolean TRACE_MOD_SER = false;
+  private static final boolean TRACE_MOD_DES = false;
+  
   private static final boolean TRACE_STR_ARRAY = false;
   /**
    * Version of the serializer/deserializer, used to allow deserialization of 
@@ -209,8 +229,6 @@ public class BinaryCasSerDes6 {
    */
   private static final int VERSION = 1;  
   
-  private static final long DBL_1 = Double.doubleToLongBits(1D);
-
   /**
    * Compression alternatives
    */
@@ -255,8 +273,8 @@ public class BinaryCasSerDes6 {
      *   if Null, recomputed when needed
      * BitSet used to test if fsRef needs to be serialized   
      */
-    final private BitSet foundFSs;
-    final private int[] foundFSsArray; // ordered set of FSs found in indexes or linked from other found FSs
+    final private BitSet foundFSsBitset;
+    final private List<TOP> fssToSerialize; // ordered list of FSs found in indexes or linked from other found FSs
     
     /**
      * Multiple uses:
@@ -266,17 +284,17 @@ public class BinaryCasSerDes6 {
     final private CasSeqAddrMaps fsStartIndexes;
     
     private ReuseInfo(
-        BitSet foundFSs,
-        int[] foundFSsArray, 
+        BitSet foundFSsBitset,
+        List<TOP> fssToSerialize, 
         CasSeqAddrMaps fsStartIndexes) {
-      this.foundFSs = foundFSs;
-      this.foundFSsArray = foundFSsArray;
+      this.foundFSsBitset = foundFSsBitset;
+      this.fssToSerialize = fssToSerialize;
       this.fsStartIndexes = fsStartIndexes;
     }
   }
   
   public ReuseInfo getReuseInfo() {
-    return new ReuseInfo(foundFSs, foundFSsArray, fsStartIndexes);
+    return new ReuseInfo(foundFSsBitset, fssToSerialize, fsStartIndexes);
   }
     
   // speedups - ints for SlotKind ordinals
@@ -302,36 +320,35 @@ public class BinaryCasSerDes6 {
   /**
    * Things set up for one instance of this class
    */
-  private TypeSystemImpl ts;
+  private TypeSystemImpl srcTs;
+  final private TypeSystemImpl tgtTs;
   final private CompressLevel compressLevel;
   final private CompressStrat compressStrategy;  
   
-  /**
-   * Things that are used by common routines among serialization and deserialization
-   */
-  private boolean isTypeMappingCmn;
-  private CasTypeSystemMapper typeMapperCmn;
-
   /*****************************************************
    *  Things for both serialization and Deserialization
    *****************************************************/
   final private CASImpl cas;  // cas being serialized or deserialized into
-  private int[] heap;           // main heap, can't be final because grow replaces it
-  final private StringHeap stringHeapObj;
-  final private LongHeap longHeapObj;
-  final private ShortHeap shortHeapObj;
-  final private ByteHeap byteHeapObj;
-
-  private int heapStart;
-  private int heapEnd;                 // set when deserializing   
-  private int totalMappedHeapSize = 0; // heapEnd - heapStart, but with FS that don't exist in the target type system deleted    
+  final private BinaryCasSerDes bcsd; // common binary ser/des code
+//  private int[] heap;           // main heap, can't be final because grow replaces it
+  final private StringHeap stringHeapObj; // needed for compression encoding/decoding
+//  final private LongHeap longHeapObj;
+//  final private ShortHeap shortHeapObj;
+//  final private ByteHeap byteHeapObj;
+//
+  private int nextFsId;
+//  private int heapEnd;                 // set when deserializing   
+//  private int totalMappedHeapSize = 0; // heapEnd - heapStart, but with FS that don't exist in the target type system deleted    
 
   final private boolean isSerializingDelta;        // if true, there is a marker indicating the start spot(s)
         private boolean isDelta;
         private boolean isReadingDelta;
   final private MarkerImpl mark;  // the mark to serialize from
 
-  
+  /**
+   * maps from src id &lt;-&gt; tgt id 
+   *   For deserialization: if src type not exist, tgt to src is 0
+   */
   final private CasSeqAddrMaps fsStartIndexes;
   final private boolean reuseInfoProvided;
   final private boolean doMeasurements;  // if true, doing measurements
@@ -339,30 +356,73 @@ public class BinaryCasSerDes6 {
   private OptimizeStrings os;
   private boolean only1CommonString;  // true if only one common string
 
-  final private TypeSystemImpl tgtTs;
-
-  private TypeInfo typeInfo; // type info for the current type being serialized/deserialized
-                             // always the "src" typeInfo I think, except for compareCas use
+//  private TypeInfo typeInfo; // type info for the current type being serialized/deserialized
+//                             // always the "src" typeInfo I think, except for compareCas use
   final private CasTypeSystemMapper typeMapper;
   private boolean isTypeMapping;
 
-  final private int[] iPrevHeapArray; // index of previous instance of this typecode in heap, by typecode
-  private int iPrevHeap;        // 0 or heap addr of previous instance of current type
+//  /**
+//   * hold previous instance of FS by typecode, for compression calculation
+//   */
+//  final private TOP[] prevFsByType;
+//  /**
+//   * previous FS serialized or deserialized, of the current type, may be null
+//   */
+//  private TOP prevFs; 
+
   /**
    * Hold prev instance of FS which have non-array FSRef slots, to allow 
    * computing these to match case where a 0 value is used because of type filtering
-   *   for each typecode, only set if the type has 1 or more non-array fsref
+   *   for each target typecode, only set if the type has 1 or more non-array fsref
    *   set only for non-filtered domain types
    *     set only for non-0 values
    *       if fsRef is to filtered type, value serialized will be 0, but this slot not set
    *       On deserialization: if value is 0, skip setting 
+   * first index: key is type code
+   * 2nd index: key is slot-offset number (0-based)
+   * 
+   * Also used for array refs sometimes, for the 1st entry in the array
+   *   - feature slot 0 is used for this when reading (not when writing - could be made more uniform)
    */
   final private int[] [] prevHeapInstanceWithIntValues;
   
-  private BitSet foundFSs; // ordered set of FSs found in indexes or linked from other found FSs
-  private BitSet foundFSsBelowMark; // for delta serialization use only
-  private int[] foundFSsArray;  // sorted fss's being serialized.  For delta, just the deltas
-  final private IntVector toBeScanned = new IntVector();
+  /**
+   * Hold prev values of slots, by type, for instances of FS which are non-arrays containing 
+   *   slots which have long values, used for differencing 
+   *   - not using the actual FS instance, because during deserialization, these may not be 
+   *     deserialized due to type filtering
+   *   set only for non-filtered domain types
+   *     set only for non-0 values
+   *       if fsRef is to filtered type, value serialized will be 0, but this slot not set
+   *       On deserialization: if value is 0, skip setting 
+   * first index: key is type code
+   * 2nd index: key is slot-offset number (0-based) 
+   */
+  final private Int2ObjHashMap<long[]> prevFsWithLongValues;
+  
+  /**
+   * ordered set of FSs found in indexes or linked from other found FSs.
+   * used to control loops/recursion when locating things
+   */
+  private BitSet foundFSsBitset;
+  
+  /**
+   * ordered set of FSs found in indexes or linked from other found FSs, which are below the mark.
+   * used to control loops/recursion when locating things
+   */
+  private BitSet foundFSsBelowMarkBitset;
+  
+  /**
+   * FSs being serialized. For delta, just the deltas above the delta line.
+   * Constructed from indexed plus reachable, above the delta line.
+   * Built from foundFSsBitset.
+   */
+  private List<TOP> fssToSerialize; 
+  
+  /**
+   * FSs being processed, including below-the-line deltas.
+   */
+  final private List<TOP> toBeScanned = new ArrayList<TOP>();
 //  private HashSetInt ffssBelowMark;  // sorted fss's found below the mark
 //  final private int[] typeCodeHisto = new int[ts.getTypeArraySize()]; 
 
@@ -374,9 +434,9 @@ public class BinaryCasSerDes6 {
   
   final private SerializationMeasures sm;  // null or serialization measurements
   final private ByteArrayOutputStream[] baosZipSources = new ByteArrayOutputStream[NBR_SLOT_KIND_ZIP_STREAMS];  // lazily created, indexed by SlotKind.i
-  final private DataOutputStream[] dosZipSources = new DataOutputStream[NBR_SLOT_KIND_ZIP_STREAMS];      // lazily created, indexed by SlotKind.i
-  private int[] savedAllIndexesFSs;  // speedup - avoid computing this twice
+  final private DataOutputStream[] dosZipSources = new DataOutputStream           [NBR_SLOT_KIND_ZIP_STREAMS];      // lazily created, indexed by SlotKind.i
   
+  /** used to compute the size of the byte array output stream */
   final private int[] estimatedZipSize = new int[NBR_SLOT_KIND_ZIP_STREAMS]; // one entry for each output stream kind
   // speedups
   
@@ -411,19 +471,28 @@ public class BinaryCasSerDes6 {
   private int version;
 
   final private DataInputStream[] dataInputs = new DataInputStream[NBR_SLOT_KIND_ZIP_STREAMS];
-  final private Inflater[] inflaters = new Inflater[NBR_SLOT_KIND_ZIP_STREAMS];
+  final private Inflater[] inflaters = new Inflater               [NBR_SLOT_KIND_ZIP_STREAMS];
 
-  private IntVector fixupsNeeded;  // for deserialization, the "fixups" for relative heap refs needed  
-  private int stringTableOffset;
-  
-  /**
-   * These indexes remember sharable common values in aux heaps
-   * Values must be in aux heap, but not part of arrays there
-   *   so that rules out boolean, byte, and shorts
-   */
-  private int longZeroIndex = -1; // also used for double 0 index
-  private int double1Index = -1;
+  /** the "fixups" for relative heap refs */
+  final private List<Runnable> fixupsNeeded = new ArrayList<>();
+  /** hold on to FS prior to getting them indexed to prevent them from being GC'd */
+  final private List<TOP> preventFsGc = new ArrayList<>();
+
+  /** 
+   * the deferrals needed when deserializing a subtype of AnnotationBase before the sofa is known
+   * Also for Sofa creation where some fields are final
+   * */
+  final private List<Runnable> singleFsDefer = new ArrayList<>(); 
+  
+  /** used for deferred creation */
+  private int sofaNum;
+  private String sofaName;
+  private String sofaMimeType;
+  private Sofa sofaRef;
   
+  /** the FS being deserialized */
+  private TOP currentFs;
+    
   private boolean isUpdatePrevOK; // false if shouldn't update prev value because written value was 0
 
   private String[] readCommonString;
@@ -474,43 +543,47 @@ public class BinaryCasSerDes6 {
       CompressLevel compressLevel, 
       CompressStrat compressStrategy) throws ResourceInitializationException {
     cas = ((CASImpl) ((aCas instanceof JCas) ? ((JCas)aCas).getCas(): aCas)).getBaseCAS();
+    bcsd = cas.getBinaryCasSerDes();
     
-    this.ts = cas.getTypeSystemImpl();
+    this.srcTs = cas.getTypeSystemImpl();
     this.mark = mark;
+    
     if (null != mark && !mark.isValid() ) {
-      throw new CASRuntimeException(
-                CASRuntimeException.INVALID_MARKER, new String[] { "Invalid Marker." });
+      throw new CASRuntimeException(CASRuntimeException.INVALID_MARKER, "Invalid Marker.");
     }
 
     this.doMeasurements = doMeasurements;
     this.sm = doMeasurements ? new SerializationMeasures() : null;
     
     isDelta = isSerializingDelta = (mark != null);
-    typeMapperCmn = typeMapper = ts.getTypeSystemMapper(tgtTs);
-    isTypeMappingCmn = isTypeMapping = (null != typeMapper);
+    typeMapper = srcTs.getTypeSystemMapper(tgtTs);
+    isTypeMapping = (null != typeMapper);
     
-    heap = cas.getHeap().heap;
-    heapEnd = cas.getHeap().getCellsUsed();
-    heapStart = isSerializingDelta ? mark.getNextFSId() : 0;
-    
-    stringHeapObj = cas.getStringHeap();
-    longHeapObj   = cas.getLongHeap();
-    shortHeapObj  = cas.getShortHeap();
-    byteHeapObj   = cas.getByteHeap();
+//    heap = cas.getHeap().heap;
+//    heapEnd = cas.getHeap().getCellsUsed();
+    nextFsId = isSerializingDelta ? mark.getNextFSId() : 0;
+//    
+    stringHeapObj = new StringHeap();
+//    longHeapObj   = cas.getLongHeap();
+//    shortHeapObj  = cas.getShortHeap();
+//    byteHeapObj   = cas.getByteHeap();
        
-    iPrevHeapArray = new int[ts.getTypeArraySize()];
-    prevHeapInstanceWithIntValues = new int[ts.getTypeArraySize()] [];
+//    prevFsByType = new TOP[srcTs.getTypeArraySize()];
+    int sz = Math.max(srcTs.getTypeArraySize(), (tgtTs == null) ? 0 : tgtTs.getTypeArraySize());
+    prevHeapInstanceWithIntValues = new int[sz] [];
+    prevFsWithLongValues = new Int2ObjHashMap<>(long[].class);
 
     this.compressLevel = compressLevel;
     this.compressStrategy = compressStrategy;
     reuseInfoProvided = (rfs != null);
     if (reuseInfoProvided) {
-      foundFSs = rfs.foundFSs;
-      foundFSsArray = rfs.foundFSsArray;
+      foundFSsBitset = rfs.foundFSsBitset;
+      fssToSerialize = rfs.fssToSerialize;
+      // TODO figure out why there's a copy for next
       fsStartIndexes = rfs.fsStartIndexes.copy();
     } else {
-      foundFSs = null;
-      foundFSsArray = null;
+      foundFSsBitset = null;
+      fssToSerialize = null;
       fsStartIndexes = new CasSeqAddrMaps();
     }
     this.tgtTs = tgtTs;
@@ -587,24 +660,14 @@ public class BinaryCasSerDes6 {
     if (isSerializingDelta && (tgtTs != null)) {
       throw new UnsupportedOperationException("Can't do Delta Serialization with different target TS");
     }
-
-    if (fsStartIndexes == null) {
-      if (isSerializingDelta) {
-        throw new UnsupportedOperationException("Serializing a delta requires valid ReuseInfo for Cas being serialized," +
-        		" captured right after it was deserialized");
-      }
-      if (isReadingDelta) {
-        throw new UnsupportedOperationException("Deserializing a delta requires valid ReuseInfo for Cas being deserialized into");
-      }
-    }
     
     setupOutputStreams(out);
     
     if (doMeasurements) {
-      System.out.println(printCasInfo(cas));
-      sm.origAuxBytes = cas.getByteHeap().getSize();
-      sm.origAuxShorts = cas.getShortHeap().getSize() * 2;
-      sm.origAuxLongs = cas.getLongHeap().getSize() * 8;
+//      System.out.println(printCasInfo(cas));
+//      sm.origAuxBytes = cas.getByteHeap().getSize();
+//      sm.origAuxShorts = cas.getShortHeap().getSize() * 2;
+//      sm.origAuxLongs = cas.getLongHeap().getSize() * 8;
       sm.totalTime = System.currentTimeMillis();
     }
 
@@ -634,18 +697,12 @@ public class BinaryCasSerDes6 {
      *      for strings above the mark
      ***************************/
    
-      // scan thru all fs and save their offsets in the heap
-      // to allow conversion from addr to sequential fs numbers
-      // Also, compute sequential maps for non-equal type systems
-      // As a side effect, also add all strings that are included
-      // in the target type system to the set to be optimized.
+      // scan thru all fs (above the line if delta) and create a map from
+      //   the src id to the tgt id (some types may be missing, so is not identity map).
+      // Add all strings to string optimizer. 
       //   Note: for delta cas, this only picks up strings 
       //   referenced by FSs above the line
-    totalMappedHeapSize = initFsStartIndexes();
-    if (heapStart == 0) {
-      totalMappedHeapSize++;  // include the null at the start
-      heapStart = 1;  // slot 0 not serialized, it's null / 0
-    }
+   initSrcTgtIdMapsAndStrings();
     
     // add remaining strings for this case:
     //   deltaCas, FS below the line modified, modification is new string.
@@ -666,88 +723,57 @@ public class BinaryCasSerDes6 {
     /***************************
      * Prepare to walk main heap
      ***************************/
-    writeVnumber(control_dos, totalMappedHeapSize);  
-    if (doMeasurements) {
-      sm.statDetails[Slot_MainHeap.ordinal()].original = (1 + heapEnd - heapStart) * 4;      
-    }
+    writeVnumber(control_dos, fssToSerialize.size());  // was totalMappedHeapSize  
     
-    Arrays.fill(iPrevHeapArray, 0);
+//    Arrays.fill(prevFsByType, null);
     Arrays.fill(prevHeapInstanceWithIntValues, null);
+    prevFsWithLongValues.clear();
     
     /***************************
      * walk main heap
      ***************************/
+   
+    for (TOP fs : fssToSerialize) {
 
-    int iHeap;
-    
-//    { // debug
-//      IntListIterator dit = foundFSs.iterator();
-//      int column = 0;
-//      int[] va = new int[100];
-//      while (dit.hasNext()) {
-//        va[column++] = dit.next();
-//        if (column == 100) {
-//          column = 0;
-//          for (int i = 0; i < 100; i++) {
-//            System.err.format("%,8d ", va[i]);
-//          }
-//          System.err.println("");
-//        }
-//      }
-//      for (int i = 0; i < column; i++) {
-//        System.err.format("%9d ", va[i]);
-//      }
-//      System.err.println("");
-//    }
-    for (int fssi = 0; fssi < foundFSsArray.length; fssi++) {
-      iHeap = foundFSsArray[fssi];
-      if (isDelta && iHeap < mark.nextFSId) {
-        continue;
-      }
-      final int tCode = heap[iHeap];  // get type code
-      final int mappedTypeCode = isTypeMapping ? typeMapper.mapTypeCodeSrc2Tgt(tCode) : tCode;
+      final TypeImpl srcType = fs._typeImpl;
+      final int tCode = srcType.getCode();
+      final TypeImpl tgtType = isTypeMapping ? typeMapper.mapTypeSrc2Tgt(srcType) : srcType;
+      assert(null != tgtType); // because those are not put on queue for serialization
+     
+//      prevFs = prevFsByType[tCode];
+      
       if (TRACE_SER) {
-        System.out.format("Ser: adr: %,d tCode: %s %,d tgtTypeCode: %,d %n", iHeap, tCode, ts.getTypeInfo(tCode), mappedTypeCode);
-      }
-      if (mappedTypeCode == 0) { // means no corresponding type in target system
-        continue;
+        System.out.format("Ser: %,d adr: %,8d tCode: %,3d %13s tgtTypeCode: %,3d %n", 
+            fs._id, fs._id, srcType.getCode(), srcType.getShortName(), tgtType.getCode());
       }
 
-      typeInfo = ts.getTypeInfo(tCode);
-      iPrevHeap = iPrevHeapArray[tCode];
-      
-      writeVnumber(typeCode_dos, mappedTypeCode);
+      writeVnumber(typeCode_dos, tgtType.getCode());
 
-      if (typeInfo.isHeapStoredArray) {
-        serializeHeapStoredArray(iHeap);
-      } else if (typeInfo.isArray) {
-        serializeNonHeapStoredArray(iHeap);
+      if (fs instanceof CommonArray) {
+        serializeArray(fs);
       } else {
         if (isTypeMapping) {
           // Serialize out in the order the features are in the target
-          final int[] tgtFeatOffsets2Src = typeMapper.getTgtFeatOffsets2Src(tCode);
-          for (int i = 0; i < tgtFeatOffsets2Src.length; i++) {
-            final int featOffsetInSrc = tgtFeatOffsets2Src[i] + 1;  // add one for origin 1
-            if (featOffsetInSrc == 0) {
-              throw new RuntimeException(); // never happen because for serialization, target is never a superset of features of src
-            }
-            serializeByKind(iHeap, featOffsetInSrc);
+          for (FeatureImpl tgtFeat : tgtType.getFeatureImpls()) {
+            FeatureImpl srcFeat = typeMapper.getSrcFeature(tgtType, tgtFeat);
+            assert(srcFeat != null); //for serialization, target is never a superset of features of src
+            serializeByKind(fs, srcFeat);
           }
-        } else {
-          final int nbrSlots_p_1 = typeInfo.slotKinds.length + 1;
-          for (int i = 1; i < nbrSlots_p_1; i++) {
-            serializeByKind(iHeap, i);
+        } else { // not type mapping
+          for (FeatureImpl srcFeat : srcType.getFeatureImpls()) {
+            serializeByKind(fs, srcFeat);
           }
         }
       }
-    
-      iPrevHeapArray[tCode] = iHeap;  // make this one the "prev" one for subsequent testing
+      
+//      prevFsByType[tCode] = fs;
       if (doMeasurements) {
         sm.statDetails[typeCode_i].incr(DataIO.lengthVnumber(tCode));
         sm.mainHeapFSs ++;
-      }
-    }  // end of heap walk
-    
+      }    
+    } // end of FSs above the line walk
+        
+    // write out views, sofas, indexes
     processIndexedFeatureStructures(cas, true /* pass 2 */);
 
     if (isSerializingDelta) {
@@ -762,171 +788,239 @@ public class BinaryCasSerDes6 {
     return sm;
   }
           
-  private void serializeHeapStoredArray(int iHeap) throws IOException {
-    final int length = serializeArrayLength(iHeap);
+  private void serializeArray(TOP fs) throws IOException {
+    final TypeImpl_array arrayType = (TypeImpl_array) fs._typeImpl;
+    CommonArray a = (CommonArray) fs;
+    final SlotKind arrayElementKind = arrayType.getComponentSlotKind();
+
+    final int length = serializeArrayLength(a);
     // output values
     // special case 0 and 1st value
     if (length == 0) {
+      if (arrayElementKind == SlotKind.Slot_HeapRef ||
+          arrayElementKind == SlotKind.Slot_Int) {
+        updatePrevArray0IntValue(arrayType, 0);
+      }
       return;
     }
-    SlotKind arrayElementKind = typeInfo.slotKinds[1];
-    final int endi = iHeap + length + 2;
+   
+    final int io = arrayElementKind.ordinal();
+    
+    int prev = 0;
+    long longPrev;
+    boolean isFirstElement = true;
+    
     switch (arrayElementKind) {
-    //  NOTE: short, byte, boolean, long, double arrays not stored on the heap
-    case Slot_HeapRef: case Slot_Int:
-      {
-        int prev = (iPrevHeap == 0) ? 0 : 
-                   (heap[iPrevHeap + 1] == 0) ? 0 : // prev length is 0
-                   getPrevIntValue(iHeap, 2);
-//                    heap[iPrevHeap + 2];  // use prev array 1st element
-        final int startIheap = iHeap + 2;
-        for (int i = startIheap; i < endi; i++) {
-          final int maybeConverted = writeIntOrHeapRef(arrayElementKind.ordinal(), i, prev);
-          if (isUpdatePrevOK && (i == startIheap)) {
-            updatePrevIntValue(iHeap, 2, maybeConverted);
-          } 
-          prev = maybeConverted;
+
+    case Slot_HeapRef:
+      prev = getPrevIntValue(arrayType.getCode(), 0);
+//      FSArray prevFsArray = (FSArray) prevFs;
+//      if (prevFsArray != null && prevFsArray.size() != 0) {
+//        prev = getTgtSeqFromSrcFS(prevFsArray.get(0));
+//      } // else use the preset 0 value
+      
+      for (TOP element : ((FSArray)fs)._getTheArray()) {
+        final int v = getTgtSeqFromSrcFS(element);
+        writeDiff(io, v, prev);
+        if (isUpdatePrevOK && isFirstElement) {
+          updatePrevArray0IntValue(arrayType, v);        
         }
+        prev = v;
+        isFirstElement = false;
       }
       break;
-    case Slot_Float: 
-      for (int i = iHeap + 2; i < endi; i++) {
-        writeFloat(heap[i]);
+      
+    case Slot_Int:
+      prev = getPrevIntValue(arrayType.getCode(), 0);
+//      IntegerArray prevIntArray = (IntegerArray) prevFs;
+//      if (prevIntArray != null && prevIntArray.size() != 0) {
+//        prev = prevIntArray.get(0);
+//      }
+       
+      for (int element : ((IntegerArray)fs)._getTheArray()) {
+        writeDiff(io, element, prev);
+        if (isUpdatePrevOK && isFirstElement) {
+          updatePrevArray0IntValue(arrayType, element);
+        } 
+        isFirstElement = false;
+        prev = element;
+      }
+      break;
+      
+    case Slot_Float:
+      for (float item : ((FloatArray)fs)._getTheArray()) {
+        writeFloat(Float.floatToRawIntBits(item));
       }
       break;
+
     case Slot_StrRef:
-      for (int i = iHeap + 2; i < endi; i++) {
-        if (TRACE_STR_ARRAY) {
-          System.out.format("Trace Str Array Ser: addr: %,d string=%s%n", i, stringHeapObj.getStringForCode(heap[i]));
-        }
-        writeString(stringHeapObj.getStringForCode(heap[i]));
+      for (String item : ((StringArray)fs)._getTheArray()) {
+        writeString(item);
       }
       break;
       
-    default: throw new RuntimeException("internal error");
-    } // end of switch    
-  }
-  
-  private int writeIntOrHeapRef(int kind, int index, int prev) throws IOException {
-    final int v = heap[index];
-    return writeDiff(kind, v, prev);
-  }
-  
-  private long writeLongFromHeapIndex(int index, long prev) throws IOException {
-    final long v = longHeapObj.getHeapValue(heap[index]);      
-    writeLong(v, prev); 
-    return v;
-  }
-  
-  private void serializeNonHeapStoredArray(int iHeap) throws IOException {
-    final int length = serializeArrayLength(iHeap);
-    if (length == 0) {
-      return;
-    }
-    SlotKind refKind = typeInfo.getSlotKind(2);
-    switch (refKind) {
-    case Slot_BooleanRef: case Slot_ByteRef:
-      writeFromByteArray(refKind, heap[iHeap + 2], length);
-      if (doMeasurements) {
-        sm.statDetails[byte_i].incr(1);
-        sm.origAuxByteArrayRefs += 4;
+    case Slot_BooleanRef:
+      for (boolean b : ((BooleanArray)fs)._getTheArray()) {
+        byte_dos.write(b ? 1 : 0);
       }
-      break; 
-    case Slot_ShortRef:
-      writeFromShortArray(heap[iHeap + 2], length);
-      if (doMeasurements) {
-        sm.origAuxShortArrayRefs += 4;
+      break;
+      
+    case Slot_ByteRef:
+      byte_dos.write(((ByteArray)fs)._getTheArray(), 0, length);
+      break;
+      
+    case Slot_ShortRef: 
+      for (short v : ((ShortArray)fs)._getTheArray()) {
+        writeDiff(short_i, v, prev);
+        prev = v;
       }
-      break; 
-    case Slot_LongRef: case Slot_DoubleRef:
-      writeFromLongArray(refKind, heap[iHeap + 2], length);
-      if (doMeasurements) {
-        sm.origAuxLongArrayRefs += 4;
+      break;
+
+    case Slot_LongRef: 
+      longPrev = 0L;
+      for (long v : ((LongArray)fs)._getTheArray()) {
+        writeLong(v, longPrev);
+        longPrev = v;
       }
-      break; 
-    default:
-      throw new RuntimeException();
-    }
+      break;
+    
+    case Slot_DoubleRef:
+      for (double v : ((DoubleArray)fs)._getTheArray()) {
+        writeDouble(Double.doubleToRawLongBits(v));
+      }
+      break;
+
+    default: assert(false); 
+    } // end of switch    
   }
-  
-  private void serializeByKind(int iHeap, int offset) throws IOException {
-    SlotKind kind = typeInfo.getSlotKind(offset);      
+          
+  /**
+   * serialize one feature structure, which is
+   *   guaranteed not to be null
+   *   guaranteed to exist in target if there is type mapping
+   * Caller iterates over target slots, but the feat arg is for the corresponding src feature
+   * @param fs the FS whose slot "feat" is to be serialize
+   * @param feat the corresponding source feature slot to serialize
+   * @throws IOException
+   */
+  private void serializeByKind(TOP fs, FeatureImpl feat) throws IOException {
+    SlotKind kind = feat.getSlotKind();
     switch (kind) {
     //Slot_Int, Slot_Float, Slot_Boolean, Slot_Byte, Slot_Short
-    case Slot_Int: case Slot_Short: case Slot_HeapRef:
-      serializeDiffWithPrevTypeSlot(kind, iHeap, offset);
-      break;
-    case Slot_Float:
-      writeFloat(heap[iHeap + offset]);
-      break;
-    case Slot_Boolean: case Slot_Byte:
-      byte_dos.write(heap[iHeap + offset]);
-      break;
-    case Slot_StrRef: 
-      writeString(stringHeapObj.getStringForCode(heap[iHeap + offset]));
-      break;
-    case Slot_LongRef: 
-      writeLongFromHeapIndex(iHeap + offset, 
-                (iPrevHeap == 0) ? 
-                  0L : 
-                  longHeapObj.getHeapValue(heap[iPrevHeap + offset]));
+    case Slot_Short:     serializeDiffWithPrevTypeSlot(kind, fs, feat, fs.getShortValue(feat));       break;
+    case Slot_Int:       serializeDiffWithPrevTypeSlot(kind, fs, feat, fs.getIntValue(feat));         break;
+    case Slot_HeapRef:
+//      if ()
+      serializeDiffWithPrevTypeSlot(kind, fs, feat, getTgtSeqFromSrcFS(fs.getFeatureValue(feat))); 
       break;
-    case Slot_DoubleRef: 
-      writeDouble(longHeapObj.getHeapValue(heap[iHeap + offset]));
+    case Slot_Float:     writeFloat(Float.floatToRawIntBits(fs.getFloatValue(feat)));                 break;
+    case Slot_Boolean:   byte_dos.write(fs.getBooleanValue(feat) ? 1 : 0);                         break;
+    case Slot_Byte:      byte_dos.write(fs.getByteValue(feat));                                       break;
+    case Slot_StrRef:    writeString(fs.getStringValue(feat));                                        break;
+    case Slot_LongRef:
+      final TypeImpl ti = fs._typeImpl;
+      final int offset = feat.getOffset();
+      final long prevLong = getPrevLongValue(ti.getCode(), offset);
+      final long vLong = fs.getLongValue(feat);
+      writeLong(vLong, prevLong);
+      updatePrevLongValue(ti, offset, vLong);
       break;
+      
+    case Slot_DoubleRef: writeDouble(Double.doubleToRawLongBits(fs.getDoubleValue(feat)));            break;
     default: 
       throw new RuntimeException("internal error");
     } // end of switch
   }
-  
-  private int serializeArrayLength(int iHeap) throws IOException {
-    final int length = heap[iHeap + 1];
+    
+  private int serializeArrayLength(CommonArray array) throws IOException {
+    final int length = array.size();
     writeVnumber(arrayLength_i, length);
     return length;
   }
   
-  private void serializeDiffWithPrevTypeSlot(SlotKind kind, int iHeap, int offset) throws IOException {
-    final int prev = (iPrevHeap == 0) ? 0 : 
-//      heap[iPrevHeap + offset];
-      getPrevIntValue(iHeap, offset);
-    final int newValue = heap[iHeap + offset];
-    final int maybeConverted = writeDiff(kind.ordinal(), newValue, prev);
+  private void serializeDiffWithPrevTypeSlot(SlotKind kind, TOP fs, FeatureImpl feat, int newValue) throws IOException {
+    final int prev = getPrevIntValue(fs._getTypeCode(), feat.getOffset());
+    writeDiff(kind.ordinal(), newValue, prev);
     if (isUpdatePrevOK) {
-      updatePrevIntValue(iHeap, offset, maybeConverted);
+      updatePrevIntValue(fs._typeImpl, feat.getOffset(), newValue);
     }
   }
   
   /**
-   * 
-   * @param iHeap index in the heap
-   * @param offset offset to the slot
+   * Called for non-arrays 
+   * @param fs used to get the type
+   * @param featOffset offset to the slot
    * @param newValue for heap refs, is the converted-from-addr-to-seq-number value
    */
-  private void updatePrevIntValue(final int iHeap, final int offset, final int newValue) {
-    final int[] featCache = initPrevIntValue(iHeap); 
-    featCache[offset -1] = newValue;
+  private void updatePrevIntValue(TypeImpl ti, final int featOffset, final int newValue) {
+    final int[] featCache = initPrevIntValue(ti);    
+    featCache[featOffset] = newValue;
+  } 
+  
+  private void updatePrevLongValue(TypeImpl ti, final int featOffset, final long newValue) {
+    final long[] featCache = initPrevLongValue(ti); 
+    featCache[featOffset] = newValue;
+  } 
+    
+  /** 
+   * version called for arrays, captures the 0th value
+   * @param ti
+   * @param newValue
+   */
+  private void updatePrevArray0IntValue(TypeImpl ti, int newValue) {
+    final int[] featCache = initPrevIntValue(ti);
+    featCache[0] = newValue;     
   }
   
-  private int[] initPrevIntValue(final int iHeap) {
-    final int[] featCache = prevHeapInstanceWithIntValues[heap[iHeap]];
+  /**
+   * Get and lazily initialize if needed the feature cache values for a type
+   * For Serializing, the type belongs to the srcTs
+   * For Deserializing, the type belongs to the tgtTs
+   * @param ti the type
+   * @return the int feature cache
+   */
+  private int[] initPrevIntValue(TypeImpl ti) {
+    int tcode = ti.getCode();
+    final int[] featCache = prevHeapInstanceWithIntValues[tcode];
     if (null == featCache) {
-      return prevHeapInstanceWithIntValues[heap[iHeap]] = new int[typeInfo.slotKinds.length];
+      return prevHeapInstanceWithIntValues[tcode] = new int[ti.isArray() ? 1 : ti.getNumberOfFeatures()];
     }
     return featCache;
   }
-  
+
   /**
-   * 
-   * @param iHeap index in the heap
-   * @param offset true offset, 1 = first feature...
+   * Get and lazily initialize if needed the long values for a type
+   * For Serializing and Deserializing, the type belongs to the tgtTs
+   * @param ti the type
+   * @return the int feature cache
+   */
+  private long[] initPrevLongValue(TypeImpl ti) {
+    int tcode = ti.getCode();
+    long[] featCache = prevFsWithLongValues.get(tcode);
+    if (null == featCache) {
+      featCache = new long[ti.getNumberOfFeatures()];
+      prevFsWithLongValues.put(tcode,  featCache);
+    }
+    return featCache;
+  }
+
+  /**
+   * For heaprefs this gets the previously serialized int value
+   * @param typeCode the type code
+   * @param featOffset true offset, 1 = first feature...
    * @return the previous int value for use in difference calculations
    */
-  private int getPrevIntValue(final int iHeap, final int offset) {
-    final int[] featCache = prevHeapInstanceWithIntValues[heap[iHeap]];
+  private int getPrevIntValue(int typeCode, int featOffset) {
+    final int[] featCache = prevHeapInstanceWithIntValues[typeCode];
     if (null == featCache) {
       return 0;
     }
-    return featCache[offset -1];
+    return featCache[featOffset];  // for arrays, the offset is 0 to allow diffng from previous 0th element
+  }
+  
+  private long getPrevLongValue(int typeCode, int featOffset) {
+    final long[] featCache = prevFsWithLongValues.get(typeCode);
+    return (featCache == null) ? 0L : featCache[featOffset];
   }
   
   /**
@@ -975,7 +1069,17 @@ public class BinaryCasSerDes6 {
           idxAndLen.add((int)deflater.getBytesRead());
         }
       } 
-    }
+    } // end of for loop
+    
+    /** 
+     * format of serialized data, as DataOutputStream:
+     *   - number of written kinds of sources (may be less than baosZipSources.length if some are not used)
+     *   - Triples, for each non-null baosZipSources:
+     *     - the index of the baosZipSource
+     *     - the number of bytes in the deflated stream for this source
+     *     - the number of uncompressed bytes for this stream
+     *   - the compressed bytes for all the non-null baosZipSources streams, in order   
+     */
     serializedOut.writeInt(nbrEntries);                     // write number of entries
     for (int i = 0; i < idxAndLen.size();) {
       serializedOut.write(idxAndLen.get(i++));
@@ -1080,7 +1184,7 @@ public class BinaryCasSerDes6 {
       }
       return;
     }
-   
+    
     final int exponent = ((raw >>> 23) & 0xff) + 1;   // because we reserve 0, see above
     final int revMants = Integer.reverse((raw & 0x007fffff) << 9);  
     final int mants = (revMants << 1) + ((raw < 0) ? 1 : 0);
@@ -1156,8 +1260,9 @@ public class BinaryCasSerDes6 {
    * Encoding:
    *    bit 6 = sign:   1 = negative
    *    bit 7 = delta:  1 = delta
-   * @param kind
-   * @param i  runs from iHeap + 3 to end of array
+   * @param kind selects the stream to write to
+   * @param v  runs from iHeap + 3 to end of array
+   * @param prev for difference encoding
    * sets isUpdatePrevOK true if ok to update prev, false if writing 0 for any reason, or max neg nbr
    * @returns possibly converted input value (converted if was heap ref to seq heap ref)
    * @throws IOException passthru 
@@ -1179,26 +1284,6 @@ public class BinaryCasSerDes6 {
       return 0;
     }
   
-    // fsIndexes_i is for writing out modified FSs
-    if ((kind == heapRef_i) || (kind == fsIndexes_i)) {
-      if (!isInstanceInTgtTs(v)) {
-        write0(kind);
-        isUpdatePrevOK = false;
-        return 0;
-      }
-      // for heap refs, we write out the seq # instead
-      v = fsStartIndexes.getTgtSeqFromSrcAddr(v);
-      if (v == -1) { // this ref goes to some fs not in target, substitute null
-        if (kind == fsIndexes_i) {
-          // can't happen - delta ser never done with a tgtTs different from srcTs
-          throw new RuntimeException();
-        }
-        write0(kind); 
-        isUpdatePrevOK = false;
-        return 0;
-      }
-    }
-
     final int absV = Math.abs(v);
     if (((v > 0) && (prev > 0)) ||
         ((v < 0) && (prev < 0))) {
@@ -1212,24 +1297,27 @@ public class BinaryCasSerDes6 {
 //      final int absDiff = Math.abs(diff);
       // this seems to work around
       final int absDiff = (diff < 0) ? -diff : diff; 
-      // debug failure in Math.abs
-      if (absDiff < 0) {
-        System.err.format("********* caught absdiff v = %s, prev = %s diff = %s absDiff = %s%n", 
-            Integer.toHexString(v),
-            Integer.toHexString(prev),
-            Integer.toHexString(diff),
-            Integer.toHexString(absDiff));
-      }
-      if (absV < 0) {
-        System.err.format("********* caught absv v = %s, absV = %s%n", 
-            Integer.toHexString(v),
-            Integer.toHexString(absV));
-      }
-
-      writeVnumber(kind, 
-          (absV <= absDiff) ? 
-              ((long)absV << 2)    + ((v < 0) ? 2L : 0L) :
-              ((long)absDiff << 2) + ((diff < 0) ? 3L : 1L));
+//      // debug failure in Math.abs
+//      if (absDiff < 0) {
+//        System.err.format("********* caught absdiff v = %s, prev = %s diff = %s absDiff = %s%n", 
+//            Integer.toHexString(v),
+//            Integer.toHexString(prev),
+//            Integer.toHexString(diff),
+//            Integer.toHexString(absDiff));
+//      }
+//      if (absV < 0) {
+//        System.err.format("********* caught absv v = %s, absV = %s%n", 
+//            Integer.toHexString(v),
+//            Integer.toHexString(absV));
+//      }
+      assert(absDiff >= 0);
+      assert(absV >= 0);
+      
+      final long v2write = (absV <= absDiff) ? 
+          ((long)absV << 2)    + ((v < 0) ? 2L : 0L) :
+          ((long)absDiff << 2) + ((diff < 0) ? 3L : 1L);
+
+      writeVnumber(kind, v2write);
       if (doMeasurements) {
         sm.statDetails[kind].diffEncoded ++;
         sm.statDetails[kind].valueLeDiff += (absV <= absDiff) ? 1 : 0;
@@ -1254,36 +1342,7 @@ public class BinaryCasSerDes6 {
       sm.statDetails[kind].valueLeDiff ++;
     }    
   }
-  private void writeFromByteArray(SlotKind kind, int startPos, int length) throws IOException {
-    byte_dos.write(byteHeapObj.heap, startPos, length);
-  }
-
-  private void writeFromLongArray(SlotKind kind, int startPos, int length) throws IOException {
-    final long[] h = longHeapObj.heap;
-    final int endPos = startPos + length;
-    long prev = 0;
-    for (int i = startPos; i < endPos; i++) {
-      final long e = h[i];
-      if (kind == Slot_DoubleRef) {
-        writeDouble(e);
-      } else {
-        writeLong(e, prev);
-        prev = e;
-      }
-    }
-  }
   
-  private void writeFromShortArray(int startPos, int length) throws IOException {
-    final short[] h = shortHeapObj.heap;
-    final int endPos = startPos + length;
-    int prev = 0;
-    for (int i = startPos; i < endPos; i++) {
-      final short e = h[i];
-      writeDiff(short_i, e, prev);
-      prev = e;
-    }
-  }
-
   /******************************************************************************
    * Modified Values
    * Output:
@@ -1296,282 +1355,252 @@ public class BinaryCasSerDes6 {
    ******************************************************************************/
   private class SerializeModifiedFSs {
 
-    final int[] modifiedMainHeapAddrs = toArrayOrINT0(cas.getModifiedFSHeapAddrs());
-    final int[] modifiedFSs = toArrayOrINT0(cas.getModifiedFSList());
-    final int[] modifiedByteHeapAddrs = toArrayOrINT0(cas.getModifiedByteHeapAddrs());
-    final int[] modifiedShortHeapAddrs = toArrayOrINT0(cas.getModifiedShortHeapAddrs());
-    final int[] modifiedLongHeapAddrs = toArrayOrINT0(cas.getModifiedLongHeapAddrs());
-
-    {sortModifications();}  // a non-static initialization block
-    
-    final int modMainHeapAddrsLength = eliminateDuplicatesInMods(modifiedMainHeapAddrs);
-    final int modFSsLength = eliminateDuplicatesInMods(modifiedFSs);
-    final int modByteHeapAddrsLength = eliminateDuplicatesInMods(modifiedByteHeapAddrs);
-    final int modShortHeapAddrsLength = eliminateDuplicatesInMods(modifiedShortHeapAddrs);
-    final int modLongHeapAddrsLength = eliminateDuplicatesInMods(modifiedLongHeapAddrs);
+//    final int[] modifiedMainHeapAddrs = toArrayOrINT0(cas.getModifiedFSHeapAddrs());
+    final List<FsChange> modifiedFSs = cas.getModifiedFSList();  // does sorting and other cleanup
+//    final int[] modifiedByteHeapAddrs = toArrayOrINT0(cas.getModifiedByteHeapAddrs());
+//    final int[] modifiedShortHeapAddrs = toArrayOrINT0(cas.getModifiedShortHeapAddrs());
+//    final int[] modifiedLongHeapAddrs = toArrayOrINT0(cas.getModifiedLongHeapAddrs());
+
+//    {sortModifications();}  // a non-static initialization block
+    
+//    final int modMainHeapAddrsLength = cas.cleanupFsChanges(modifiedMainHeapAddrs);
+//    final int modFSsLength = modifiedFSs.size();
+//    final int modByteHeapAddrsLength = eliminateDuplicatesInMods(modifiedByteHeapAddrs);
+//    final int modShortHeapAddrsLength = eliminateDuplicatesInMods(modifiedShortHeapAddrs);
+//    final int modLongHeapAddrsLength = eliminateDuplicatesInMods(modifiedLongHeapAddrs);
 
     // ima           - index into modified arrays
     // ixx, iPrevxxx - index in heap being changed
     //                 value comes via the main heap or aux heaps
       
-      int imaModMainHeap = 0;
-      int imaModByteRef = 0;
-      int imaModShortRef = 0;
-      int imaModLongRef = 0;
+//      int imaModMainHeap = 0;
+//      int imaModByteRef = 0;
+//      int imaModShortRef = 0;
+//      int imaModLongRef = 0;
  
       // previous value - for things diff encoded
-    int vPrevModInt = 0;
-    int vPrevModHeapRef = 0;
-    short vPrevModShort = 0;
-    long vPrevModLong = 0;
-    
-    int iHeap;
-    TypeInfo typeInfo;   
-    
+    private int vPrevModInt = 0;
+    private int vPrevModHeapRef = 0;
+    private short vPrevModShort = 0;
+    private long vPrevModLong = 0;
+
+
     /**
      * For Delta Serialization:
      * Add any strings below the line 
      * Assume: no TS mapping (because it's delta serialization)
+     * Skips a modified item if in FS that isn't reachable
      */
     private void addModifiedStrings() {
 //      System.out.println("Enter addModifiedStrings");
-      for (int i = 0; i < modFSsLength; i++) {
-        iHeap = modifiedFSs[i];
-        // skip if no longer indexed-reachable change
-        if (!foundFSsBelowMark.get(iHeap)) {
-//          System.out.format("  skipping heap addr %,d%n", iHeap);
+      for (FsChange changedFs : modifiedFSs) {
+        final TOP fs = (TOP) changedFs.fs;
+        final TypeImpl srcType = fs._typeImpl;
+        if (isTypeMapping && null == typeMapper.mapTypeSrc2Tgt(srcType)) {
+          continue; // skip this fs - it's not in target type system
+        }
+        if (!foundFSsBelowMarkBitset.get(fs._id)) {
+//          System.out.format("  skipping heap addr %,d%n", currentFsId);
           continue;        
         }
-        final int tCode = heap[iHeap];
-        final TypeInfo typeInfo = ts.getTypeInfo(tCode);
-//        System.out.format("  maybe adding string ");
-        addStringFromFS(typeInfo, iHeap, tCode);
+        if (changedFs.arrayUpdates != null) {
+          if (fs instanceof StringArray) {
+            String[] strings = ((StringArray)fs)._getTheArray();
+            IntListIterator it = changedFs.arrayUpdates.iterator();
+            while (it.hasNext()) {
+              os.add(strings[it.next()]);
+            }
+          }
+        } else {
+          final BitSet featuresModified = changedFs.featuresModified;
+          int next = featuresModified.nextSetBit(0);
+          List<FeatureImpl> feats = fs._typeImpl.getFeatureImpls();
+          while (next >= 0) {
+            FeatureImpl srcFeat = feats.get(next);
+            // add only those strings in slots that are in target type system
+            if (isTypeMapping && typeMapper.getTgtFeature(srcType, srcFeat) == null) {
+                continue;  // skip - feature not in target type
+            }
+            if (srcFeat.getRangeImpl().isStringOrStringSubtype()) {
+              os.add(fs.getStringValue(feats.get(next)));
+            }
+            next = featuresModified.nextSetBit(next + 1);
+          }
+        }
       }
-//      System.out.println("Exit addModifiedStrings");
     }
     
     private void serializeModifiedFSs() throws IOException {
-      int skipped = 0;
+      int nbrModifiedFSWritten = 0;
       // iterate over all modified feature structures
-      /**
-       * Theorems about these data
-       *   1) Assumption: if an AuxHeap array is modified, its heap FS is in the list of modFSs
-       *   2) FSs with AuxHeap values have increasing ref values into the Aux heap as FS addr increases
-       *      (because the ref is not updateable).
-       *   3) Assumption: String array element modifications are main heap slot changes
-       *      and recorded as such
-       */
       int prevHeapSeq = 0;
       final int splitPoint = mark.nextFSId;
-      for (int i = 0; i < modFSsLength; i++) {
-        iHeap = modifiedFSs[i];
-        final boolean skipping = ((iHeap >= splitPoint) && !foundFSs.get(iHeap)) ||
-                                 ((iHeap < splitPoint) && !foundFSsBelowMark.get(iHeap));
-        final int tCode = heap[iHeap];
-        typeInfo = ts.getTypeInfo(tCode);
-        
-        // write out the address of the modified FS
-        // will convert to seq# internally
-        if (!skipping) {
-          prevHeapSeq = writeDiff(fsIndexes_i, iHeap, prevHeapSeq);
-        }
-        // delay updating prevHeapSeq until end of "for" loop - no longer done
-        
-        /**************************************************
-         * handle aux byte, short, long array modifications
-         **************************************************/
-        if (typeInfo.isArray && (!typeInfo.isHeapStoredArray)) {
-          writeAuxHeapMods(skipping);           
-        } else { 
-          writeMainHeapMods(skipping); 
-        }  // end of processing 1 modified FS
-        if (skipping) {
-          skipped ++;
-        } 
-      }  // end of for loop over all modified FSs
-      // write out number of modified Feature Structures
-      writeVnumber(control_dos, modFSsLength - skipped);
-
-    }  // end of method
-    
-    // sort and remove duplicates
-    private void sortModifications() {
-      Arrays.sort(modifiedMainHeapAddrs);
-      Arrays.sort(modifiedFSs);
-      Arrays.sort(modifiedByteHeapAddrs);
-      Arrays.sort(modifiedShortHeapAddrs);
-      Arrays.sort(modifiedLongHeapAddrs);
-    }
-    
-    private int eliminateDuplicatesInMods(final int[] sorted) {
-      int length = sorted.length;
-      if (length < 2) {
-        return length;
-      }
-      
-      int prev = sorted[0];
-      int to = 1;
-      for(int from = 1; from < length; from++) {
-        int s = sorted[from];
-        if (s == prev) {
+      for (FsChange fsChange : modifiedFSs) {
+        final TOP fs = fsChange.fs;
+        final TypeImpl srcType = fs._typeImpl;
+        if (isTypeMapping && typeMapper.mapTypeSrc2Tgt(srcType) == null) {
+          continue;  // skip - type is not in target
+        }
+        final int id= fs._id;
+        if ((id >= splitPoint && !foundFSsBitset.get(id)) ||
+            (id < splitPoint && !foundFSsBelowMarkBitset.get(id))) {
+          // although it was modified, it isn't going to be serialized because
+          //   it isn't indexed or referenced
           continue;
         }
-        prev = s;
-        sorted[to] = s;
-        to++;
-      }    
-      return to;  // to is length
-    }
+        
+        int v = fsStartIndexes.getTgtSeqFromSrcAddr(id);
+        assert (v != -1);
+//        System.out.format("debug ser mod, fsid: %,d after map %,d%n", id, v);
+        // no isUpdatePrevOK here, to match what was serialized
+        prevHeapSeq = writeDiff(fsIndexes_i, v, prevHeapSeq);
+        writeModsForOneFs(fsChange);
+      
+        nbrModifiedFSWritten ++;
+      }  // end of for loop over all modified FSs
 
-    private int countModifiedSlotsInFs(int fsLength) {
-      return countModifiedSlots(iHeap, fsLength, modifiedMainHeapAddrs, imaModMainHeap, modMainHeapAddrsLength);
-    }
-    
-    private int countModifiedSlotsInAuxHeap(int[] modifiedAddrs, int indexInModAddrs, int length) {
-      return countModifiedSlots(heap[iHeap + 2], heap[iHeap + 1], modifiedAddrs, indexInModAddrs, length);
-    }
-    
-    private int countModifiedSlots(int firstAddr, int length, int[] modifiedAddrs, int indexInModAddrs, int modAddrsLength) {
-      if (0 == length) {
-        throw new RuntimeException();  // can't happen
+      if (TRACE_MOD_SER) {
+        System.out.format("trace writing mods, length mod list: %,d nbr written: %,d%n",
+          modifiedFSs.size(), nbrModifiedFSWritten);
       }
-      final int nextAddr = firstAddr + length;
-      int nextModAddr = modifiedAddrs[indexInModAddrs]; 
-      if ((firstAddr > nextModAddr) ||
-          (nextModAddr >= nextAddr)) {
-        throw new RuntimeException(); // never happen - must have one slot at least modified in this fs          
-      }
-      int i = 1;
-      for (;; i++) {
-        if ((indexInModAddrs + i) == modAddrsLength) {
-          break;
-        }
-        nextModAddr = modifiedAddrs[indexInModAddrs + i];
-        if (nextModAddr >= nextAddr) {
-          break;
-        }
-      }
-      return i;
-    }
-    
-    private void writeMainHeapMods(final boolean skipping) throws IOException {
-      final int fsLength = incrToNextFs(heap, iHeap, typeInfo);
-      final int numberOfModsInFs = countModifiedSlotsInFs(fsLength);
-      if (!skipping) {
-        writeVnumber(fsIndexes_dos, numberOfModsInFs);
-      }
-      int iPrevOffsetInFs = 0;
 
-      for (int i = 0; i < numberOfModsInFs; i++) {
-        final int nextMainHeapIndex = modifiedMainHeapAddrs[imaModMainHeap++];
-        if (skipping) {
-          continue;
-        }
-        final int offsetInFs = nextMainHeapIndex - iHeap;
-        
-        writeVnumber(fsIndexes_dos, offsetInFs - iPrevOffsetInFs);
-        iPrevOffsetInFs = offsetInFs;
-        
-//        if (typeInfo.isArray && (typeInfo.getSlotKind(2) == Slot_StrRef)) {
-//          System.out.println("writing string array mod");
-//        }
-        final SlotKind kind = typeInfo.getSlotKind(typeInfo.isArray ? 2 : offsetInFs);
-//        System.out.format("mainHeapModWrite type: %s slot: %s%n", typeInfo, kind);
+      // write out number of modified Feature Structures
+      writeVnumber(control_dos, nbrModifiedFSWritten);
+    }  // end of method
+            
+    /**
+     * Write the modifications for one feature structure, based on the data in the fsChange
+     *   - this is either an array or non-array (meaning changed Features)
+     *     - array changes are written out in index order.
+     *     - feature changes are written out in offset order.
+     *     - sorting and elimination of duplicates happens when extracting info from fsChange
+     * Not called if skipping writing because obj not reachable 
+     * 
+     * NOTE: the serialized values for the index are 0-based,
+     *       vs. V2, which are base on the original offset in 
+     *       various "heaps".
+     *       - Because of this, 
+     *          -- v2 deserialization can't read v3 serializations 
+     *          -- v3 deserialization can   read v2 serializatoins, though.
+     *         
+     * @param fsChange
+     * @throws IOException 
+     */
+    private void writeModsForOneFs(FsChange fsChange) throws IOException {
+      TOP fs = fsChange.fs;
+      TypeImpl ti = fs._typeImpl;
+
+      if (fsChange.arrayUpdates != null) {
+        int prevIndex = 0;
+        writeVnumber(fsIndexes_dos, fsChange.arrayUpdates.size());
+        IntListIterator it = fsChange.arrayUpdates.iterator();
+        final SlotKind slotKind = ti.getComponentSlotKind();
         
-        switch (kind) {
-        case Slot_HeapRef:
-          vPrevModHeapRef = writeIntOrHeapRef(heapRef_i, nextMainHeapIndex, vPrevModHeapRef);
-          break;
-        case Slot_Int:
-          vPrevModInt = writeIntOrHeapRef(int_i, nextMainHeapIndex, vPrevModInt);
-          break;
-        case Slot_Short:
-          vPrevModShort = (short)writeIntOrHeapRef(int_i, nextMainHeapIndex, vPrevModShort);
-          break;
-        case Slot_LongRef:
-          vPrevModLong = writeLongFromHeapIndex(nextMainHeapIndex, vPrevModLong); 
-          break;
-        case Slot_Byte: case Slot_Boolean:
-          byte_dos.write(heap[nextMainHeapIndex]);
-          break;
-        case Slot_Float:
-          writeFloat(heap[nextMainHeapIndex]);
-          break;
-        case Slot_StrRef:
-          writeString(stringHeapObj.getStringForCode(heap[nextMainHeapIndex]));
-          break;
-        case Slot_DoubleRef:
-          writeDouble(longHeapObj.getHeapValue(heap[nextMainHeapIndex]));
-          break;
-        default:
-          throw new RuntimeException();
+        if (TRACE_MOD_SER) {
+          System.out.format("trace ser mod array fsId: %,d nbrMods: %,d type: %s%n", 
+              fs._id, fsChange.arrayUpdates.size(), ti.getShortName());
         }
+        
+        while (it.hasNext()) {
+          int index = it.next();
+          writeVnumber(fsIndexes_dos, index - prevIndex);
+          prevIndex = index;
+
+          if (TRACE_MOD_SER) {
+            System.out.format("  tr se mod fsId: %,d offset: %,d%n", 
+                fs._id, index);
+          }
+
+          switch (slotKind) {
+          case Slot_BooleanRef: writeUnsignedByte(byte_dos, ((BooleanArray)fs).get(index) ? 1 : 0); break;
+          case Slot_ByteRef: writeUnsignedByte(byte_dos, ((ByteArray   )fs).get(index));         break;
+          case Slot_ShortRef: 
+            final short vs = ((ShortArray)fs).get(index);
+            writeDiff(int_i, vs, vPrevModShort);
+            vPrevModShort = vs;
+            break;
+          
+          case Slot_LongRef: {
+            final long v = ((LongArray)fs).get(index);
+            writeLong(v, vPrevModLong);
+            vPrevModLong = v;
+            break;
+          }
 
-      }  // end of looping for all modified slots in this FS
-    }
-    
-    private void writeAuxHeapMods(final boolean skipping) throws IOException {
-      final int auxHeapIndex = heap[iHeap + 2];
-      int iPrevOffsetInAuxArray = 0;
-      
-      final SlotKind kind = typeInfo.getSlotKind(2);  // get kind of element
-      final boolean isAuxByte = ((kind == Slot_BooleanRef) || (kind == Slot_ByteRef));
-      final boolean isAuxShort = (kind == Slot_ShortRef);
-      final boolean isAuxLong = ((kind == Slot_LongRef) || (kind == Slot_DoubleRef));
+          case Slot_DoubleRef: {
+            final long v = Double.doubleToRawLongBits(((DoubleArray)fs).get(index));
+            writeDouble(v);
+            break;
+          }
+
+          case Slot_Int: vPrevModInt = writeDiff(int_i, ((IntegerArray)fs).get(index), vPrevModInt); break;
+          case Slot_Float: writeFloat(Float.floatToRawIntBits(((FloatArray)fs).get(index))); break;
+          case Slot_HeapRef: {
+            final int v = getTgtSeqFromSrcFS(((FSArray)fs).get(index)); 
+            vPrevModHeapRef = writeDiff(heapRef_i, v, vPrevModHeapRef); break;
+          }
+          case Slot_StrRef: writeString(((StringArray)fs).get(index)); break;
+
+          case Slot_JavaObjectRef: throw new UnsupportedOperationException();
+          
+          default: 
+            assert(false);
+          } // end of switch for array types
+        } // end of loop for elements in array
+        return;
+      } // end of if array type
       
-      if (!(isAuxByte || isAuxShort || isAuxLong)) {
-        throw new RuntimeException();  // never happen
-      }
+      // normal Feature mods, not array
       
-      final int[] modXxxHeapAddrs = isAuxByte  ? modifiedByteHeapAddrs :
-                                    isAuxShort ? modifiedShortHeapAddrs :
-                                                 modifiedLongHeapAddrs;
-      final int modXxxHeapAddrsLength = isAuxByte  ? modByteHeapAddrsLength :
-                                        isAuxShort ? modShortHeapAddrsLength :
-                                                     modLongHeapAddrsLength;
-      int imaModXxxRef = isAuxByte  ? imaModByteRef :
-                         isAuxShort ? imaModShortRef : 
-                                      imaModLongRef;
+      writeVnumber(fsIndexes_dos, fsChange.featuresModified.cardinality());
+      int iPrevOffsetInFs = 0;
       
-      final int numberOfModsInAuxHeap = countModifiedSlotsInAuxHeap(modXxxHeapAddrs, imaModXxxRef, modXxxHeapAddrsLength);
-      if (!skipping) {
-        writeVnumber(fsIndexes_dos, numberOfModsInAuxHeap);
+      if (TRACE_MOD_SER) {
+        System.out.format("trace ser mod feats fsId: %,d nbrMods: %,d type: %s%n", 
+            fs._id, fsChange.featuresModified.cardinality(), ti.getShortName());
       }
-      
-      for (int i = 0; i < numberOfModsInAuxHeap; i++) {
-        final int nextModAuxIndex = modXxxHeapAddrs[imaModXxxRef++];
-        final int offsetInAuxArray = nextModAuxIndex - auxHeapIndex;
-        if (!skipping) {
-          writeVnumber(fsIndexes_dos, offsetInAuxArray - iPrevOffsetInAuxArray);
-          iPrevOffsetInAuxArray = offsetInAuxArray;
-          
-          if (isAuxByte) {
-            writeUnsignedByte(byte_dos, byteHeapObj.getHeapValue(nextModAuxIndex));
-          } else if (isAuxShort) {
-            final short v = shortHeapObj.getHeapValue(nextModAuxIndex);
-            writeDiff(int_i, v, vPrevModShort);
-            vPrevModShort = v;
-          } else {
-            long v = longHeapObj.getHeapValue(nextModAuxIndex);
-            if (kind == Slot_LongRef) {
-              writeLong(v, vPrevModLong);
-              vPrevModLong = v;    
-            } else {
-              writeDouble(v);
-            }
-          }
-        }
+
+      BitSet bs = fsChange.featuresModified;
+      int offset = bs.nextSetBit(0);
+            
+      while (offset >= 0) {
+       
+        writeVnumber(fsIndexes_dos, offset - iPrevOffsetInFs);
+        iPrevOffsetInFs = offset;
         
-        if (isAuxByte) {
-          imaModByteRef++;
-        } else if (isAuxShort) {
-          imaModShortRef++;
-        } else {
-          imaModLongRef++;
+        final FeatureImpl fi = ti.getFeatureImpls().get(offset);
+        
+        if (TRACE_MOD_SER) {
+          System.out.format("  tr se mod fsId: %,d offset: %,d type: %s%n", 
+              fs._id, offset, fi.getShortName());
         }
         
-      }
-    }
+        final SlotKind slotKind = fi.getSlotKind();
+        switch (slotKind) {
+        case Slot_Boolean: byte_dos.write(fs.getBooleanValue(fi) ? 1 : 0); break;
+        case Slot_Byte: byte_dos.write(fs.getByteValue(fi)); break;
+        case Slot_Short: vPrevModShort = (short) writeDiff(int_i, fs.getShortValue(fi), vPrevModShort); break;
+        case Slot_Int: vPrevModInt = writeDiff(int_i, fs.getIntValue(fi), vPrevModInt); break;
+        case Slot_LongRef: {
+          long v = fs.getLongValue(fi);  
+          writeLong(v, vPrevModLong);
+          vPrevModLong = v;
+          break;
+        }
+        case Slot_Float: writeFloat(Float.floatToRawIntBits(fs.getFloatValue(fi))); break;
+        case Slot_DoubleRef: writeDouble(Double.doubleToRawLongBits(fs.getDoubleValue(fi))); break;
+        case Slot_HeapRef: {
+            final int v = getTgtSeqFromSrcFS(fs.getFeatureValue(fi));
+            vPrevModHeapRef = writeDiff(heapRef_i, v, vPrevModHeapRef);
+          }
+          break;
+        case Slot_StrRef: writeString(fs.getStringValue(fi)); break;
+        case Slot_JavaObjectRef: throw new UnsupportedOperationException(); 
+        default: assert(false);
+        } // end of Switch
+        offset = bs.nextSetBit(offset + 1);
+      } // end of iterator over all features
+    }        
   } // end of class definition for SerializeModifiedFSs
           
   /*************************************************************************************
@@ -1640,22 +1669,20 @@ public class BinaryCasSerDes6 {
     /***************************
      * Prepare to walk main heap
      ***************************/
-    int heapUsedInTarget = readVnumber(control_dis);         
-    final Heap heapObj = cas.getHeap();
+    int nbrNewFSsInTarget = readVnumber(control_dis);  // is nbr of FSs serialized (excluding mods) in v3         
     
-    heapStart = isReadingDelta ? heapObj.getNextId() : 0;
-    stringTableOffset = isReadingDelta ? (stringHeapObj.getSize() - 1) : 0;
+    nextFsId = isReadingDelta ? (cas.getLastUsedFsId() + 1) : 0;
     
-    if (!isReadingDelta) {
-      heapObj.reinitSizeOnly(1);
-      heap = heapObj.heap;
-    }
+//    if (!isReadingDelta) {
+//      heapObj.reinitSizeOnly(1);
+//      heap = heapObj.heap;
+//    }
     
-    Arrays.fill(iPrevHeapArray, 0);
     Arrays.fill(prevHeapInstanceWithIntValues, null);
+    prevFsWithLongValues.clear();
 
-    if (heapStart == 0) {
-      heapStart = 1;  // slot 0 not serialized, it's null / 0
+    if (nextFsId == 0) {
+      nextFsId = 1;  // slot 0 not serialized, it's null / 0
     }
 
     // For Delta CAS,
@@ -1674,19 +1701,18 @@ public class BinaryCasSerDes6 {
       }
     }
 
-    fixupsNeeded = new IntVector(Math.max(16, heapObj.getCellsUsed() / 10));
-
+    fixupsNeeded.clear();
+    preventFsGc.clear();
     /**********************************************************
      * Read in new FSs being deserialized and add them to heap
      **********************************************************/
-    for (int iHeap = heapStart, targetHeapUsed = isReadingDelta ? 0 : 1; targetHeapUsed < heapUsedInTarget;) {
-      if (iHeap != heapObj.getNextId()) {
-        throw new RuntimeException();
-      }
+    for (int currentFsId = nextFsId, nbrFSs = 0; nbrFSs < nbrNewFSsInTarget; nbrFSs++) {
+
       final int tgtTypeCode = readVnumber(typeCode_dis); // get type code
-      final int srcTypeCode = isTypeMapping ? typeMapper.mapTypeCodeTgt2Src(tgtTypeCode) : tgtTypeCode;
+      final TypeImpl tgtType = (isTypeMapping ? tgtTs : srcTs).getTypeForCode(tgtTypeCode); 
+      final TypeImpl srcType = isTypeMapping ? typeMapper.mapTypeCodeTgt2Src(tgtTypeCode) : tgtType;
       
-      final boolean storeIt = (srcTypeCode != 0);
+      final boolean storeIt = (srcType != null);
       // A receiving client from a service always
       // has a superset of the service's types due to type merging so this
       // won't happen for that use case. But 
@@ -1697,269 +1723,372 @@ public class BinaryCasSerDes6 {
       // the prev value for that type.  This must be done in the context 
       // of the sending CAS's type system
     
-      // typeInfo is Target Type Info
-      final TypeInfo tgtTypeInfo = isTypeMapping ? tgtTs.getTypeInfo(tgtTypeCode) :
-                                 ts.getTypeInfo(srcTypeCode);
-      final TypeInfo srcTypeInfo = 
-        (!isTypeMapping) ? tgtTypeInfo : 
-        storeIt ?       ts.getTypeInfo(srcTypeCode) : 
-                        null;
-      if (storeIt) { 
-        typeInfo = tgtTypeInfo;
-        initPrevIntValue(iHeap);  // note "typeInfo" a hidden parameter - ugly...
+                                                      
+//      SlotKind slotKind = srcType.slotKind;
+      
+      if (storeIt) {
+        // we can skip the cache for prev values if the value will not be stored. 
+//        typeImpl = tgtType;
+        initPrevIntValue(tgtType);  
       }
+
+//      typeInfo = storeIt ? srcTypeInfo : tgtTypeInfo; // if !storeIt, then srcTypeInfo is null.
+      
+      fsStartIndexes.addSrcAddrForTgt(currentFsId, storeIt);
+ 
       if (TRACE_DES) {
-        System.out.format("Des: addr %,d tgtTypeCode: %,d %s srcTypeCode: %,d%n", iHeap, tgtTypeCode, tgtTypeInfo,  srcTypeCode);
+        System.out.format("Des: fsnbr %,4d fsid %,4d tgtTypeCode: %,3d %13s srcTypeCode: %s%n", 
+            nbrFSs, cas.getLastUsedFsId() + 1, tgtTypeCode, tgtType.getShortName(),  (null == srcType) ? "<null>" : Integer.toString(srcType.getCode()));
       }
 
-//      if (srcTypeInfo == null) {
-//        typeInfo = null;  // debugging
-//      }
-      typeInfo = storeIt ? srcTypeInfo : tgtTypeInfo; // if !storeIt, then srcTypeInfo is null.
+      if (tgtType.isArray()) {
+        readArray(storeIt, srcType, tgtType);
       
-      fsStartIndexes.addSrcAddrForTgt(iHeap, storeIt);
-      if (storeIt) {
-        iPrevHeap = iPrevHeapArray[srcTypeCode];  // will be ignored for non-existant type
-      }
-      if (typeInfo.isHeapStoredArray) {
-        readHeapStoredArray(iHeap, storeIt, heapObj, srcTypeCode);
-      } else if (typeInfo.isArray) {
-        if (storeIt) {
-          heapObj.add(3, srcTypeCode);
-          heap = heapObj.heap;
-        }
-        readNonHeapStoredArray(iHeap, storeIt);
       } else {
+        /**
+         * is not array, handle features
+         * If storing the value, create the FS unless it's a Sofa or a subtype of AnnotationBase
+         *   Those are deferred until the slots are known, because they're needed
+         *   as part of the creation of the FS due to final values. 
+         */
         if (storeIt) {
-          cas.ll_createFS(srcTypeCode);
-          heap = heapObj.heap;
+          if (!srcTs.annotBaseType.subsumes(srcType) &&  // defer subtypes of AnnotationBase
+              !(srcTs.sofaType == srcType)) {            // defer sofa types
+            preventFsGc.add(currentFs = cas.createFS(srcType));
+          } else {
+            currentFs = null;
+            singleFsDefer.clear();
+            sofaRef = null;
+            sofaNum = -1;
+            sofaName = null;
+            sofaMimeType = null;
+          }
         }
-        // is normal type with slots
+        
+        // is normal type with slots, not an array
         if (isTypeMapping && storeIt) {
-          final int[] tgtFeatOffsets2Src = typeMapper.getTgtFeatOffsets2Src(srcTypeCode);
-          for (int i = 0; i < tgtFeatOffsets2Src.length; i++) {
-            final int featOffsetInSrc = tgtFeatOffsets2Src[i] + 1;
-            SlotKind kind = tgtTypeInfo.slotKinds[i];  // target kind , may not exist in src
-            readByKind(iHeap, featOffsetInSrc, kind, storeIt);
+          for (FeatureImpl tgtFeat : tgtType.getFeatureImpls()) {
+            final FeatureImpl srcFeat = typeMapper.getSrcFeature(tgtType, tgtFeat);
+            readByKind(currentFs, tgtFeat, srcFeat, storeIt, tgtType);
           }
         } else {
-          for (int i = 1; i < typeInfo.slotKinds.length + 1; i++) {
-            SlotKind kind = typeInfo.getSlotKind(i);
-            readByKind(iHeap, i, kind, storeIt);
+          for (FeatureImpl tgtFeat : tgtType.getFeatureImpls()) {
+            readByKind(currentFs, tgtFeat, tgtFeat, storeIt, tgtType);
+          }
+        }
+        
+        if (currentFs == null) {
+          
+          /**
+           * Create single deferred FS
+           *   Either: Sofa (has final fields) or
+           *           Subtype of AnnotationBase - needs to be in the right view
+           *   
+           *   For the latter, handle document annotation specially
+           */
+
+          if (srcTs.sofaType == srcType) {
+            currentFs = cas.createSofa(sofaNum, sofaName, sofaMimeType);  
+          } else {
+            CASImpl view = (CASImpl) cas.getView(sofaRef);
+            if (srcType.getCode() == TypeSystemImpl.docTypeCode) {
+              currentFs = view.getDocumentAnnotation();  // creates the document annotation if it doesn't exist
+              // we could remove this from the indexes until deserialization is over, but then, other calls to getDocumentAnnotation
+              // would end up creating additional instances
+            } else {
+              preventFsGc.add(currentFs = view.createFS(srcType));
+            }
+          }
+          for (Runnable r : singleFsDefer) {
+            r.run();
           }
         }
       }
-      if (storeIt) {
-        iPrevHeapArray[srcTypeCode] = iHeap;  // make this one the "prev" one for subsequent testing
-      }
+//      if (storeIt) {
+//        prevFsByType[srcType.getCode()] = currentFs;  // make this one the "prev" one for subsequent testing
+//        //debug
+//        assert(currentFs._id == currentFsId);
+//      }
 //       todo need to incr src heap by amt filtered (in case some slots missing, 
 //                 need to incr tgt (for checking end) by unfiltered amount
 //                 need to fixup final heap to account for skipped slots
 //                 need to have read skip slots not present in src
-      targetHeapUsed += incrToNextFs(heap, iHeap, tgtTypeInfo);  // typeInfo is target type info
-      iHeap += storeIt ? incrToNextFs(heap, iHeap, srcTypeInfo) : 0;
+//      targetHeapUsed += incrToNextFs(heap, currentFsId, tgtTypeInfo);  // typeInfo is target type info
+      currentFsId += storeIt ? 1 : 0;
+    }
+
+    for (Runnable r : fixupsNeeded) {
+      r.run();
     }
     
-    final int end = fixupsNeeded.size();
-    for (int i = 0; i < end; i++) {
-      final int heapAddrToFix = fixupsNeeded.get(i);
-      heap[heapAddrToFix] = fsStartIndexes.getSrcAddrFromTgtSeq(heap[heapAddrToFix]);
-    }        
-    
+    // process the index information
     readIndexedFeatureStructures();
-
+    // for delta, process below-the-line updates
     if (isReadingDelta) {
       (new ReadModifiedFSs()).readModifiedFSs();
     }
+    
+    preventFsGc.clear();
 
     closeDataInputs();
 //      System.out.format("Deserialize took %,d ms%n", System.currentTimeMillis() - startTime1);
   }
   
-  private void readNonHeapStoredArray(int iHeap, boolean storeIt) throws IOException {
-
+  /**
+   * 
+   * @param storeIt
+   * @param srcType may be null if there's no source type for target when deserializing
+   * @param tgtType the type being deserialized
+   * @throws IOException
+   */
+  private void readArray(boolean storeIt, TypeImpl srcType, TypeImpl tgtType) throws IOException {
     final int length = readArrayLength();
-    if (storeIt) {
-      heap[iHeap + 1] = length;
-    }
-    if (length == 0) {
-      return;
-    }
-    SlotKind refKind = typeInfo.getSlotKind(2);
-    switch (refKind) {
-    case Slot_BooleanRef: case Slot_ByteRef:
-      final int byteRef =  readIntoByteArray(length, storeIt);
-      if (storeIt) {
-        heap[iHeap + 2] = byteRef;
-      }
-      break; 
-    case Slot_ShortRef:
-      final int shortRef = readIntoShortArray(length, storeIt);
-      if (storeIt) {
-        heap[iHeap + 2] = shortRef;
-      }
-      break; 
-    case Slot_LongRef: case Slot_DoubleRef:
-      final int longDblRef = readIntoLongArray(refKind, length, storeIt);
-      if (storeIt) {
-        heap[iHeap + 2] = longDblRef;
-      }
-      break; 
-    default:
-      throw new RuntimeException();
-    }
-  }
-  
-  private int readArrayLength() throws IOException {
-    return readVnumber(arrayLength_dis);
-  }
+    final SlotKind slotKind = tgtType.getComponentSlotKind();
 
-  private void readHeapStoredArray(int iHeap, final boolean storeIt, final Heap heapObj, final int srcTypeCode) throws IOException {
-    final int length = readArrayLength();
-    if (storeIt) {
-      heapObj.add(2 + length, srcTypeCode);
-      heap = heapObj.heap;
-      heap[iHeap + 1] = length;
-    }
-    // output values
-    // special case 0 and 1st value
-    if (length == 0) {
-      return;
-    }
-    SlotKind arrayElementKind = typeInfo.slotKinds[1];
-    final int endi = iHeap + length + 2;
-    switch (arrayElementKind) {
-    case Slot_HeapRef: case Slot_Int:
-      {
-        int prev = (iPrevHeap == 0) ? 0 : 
-                   (heap[iPrevHeap + 1] == 0) ? 0 : // prev array length = 0
-//                    heap[iPrevHeap + 2]; // prev array 0th element
-                    getPrevIntValue(iHeap, 2);
-        final int startIheap = iHeap + 2;
-        for (int i = startIheap; i < endi; i++) {
-          final int v = readDiff(arrayElementKind, prev);
-          prev = v;
-          if (startIheap == i && isUpdatePrevOK && storeIt) {
-            updatePrevIntValue(iHeap, 2, v);
-          }
-          if (storeIt) {
-            heap[i] = v;
-            if (arrayElementKind == Slot_HeapRef) {
-              fixupsNeeded.add(i);
-//              System.out.format("debug adding to fixup, slot = %,d heapValue = %,d array%n", i, v);
-            }
-          }
+    final TOP fs = storeIt ? cas.createArray(srcType, length) : null;
+    preventFsGc.add(currentFs = fs); 
+        
+    switch (slotKind) {
+    case Slot_BooleanRef: 
+      if (storeIt) {  
+        for (int i = 0; i < length; i++) {
+          ((BooleanArray)fs).set(i,  byte_dis.readByte() == 1);
         }
+      } else {
+        skipBytes(byte_dis, length);
       }
       break;
-    case Slot_Float: 
-      for (int i = iHeap + 2; i < endi; i++) {
-        final int floatRef = readFloat();
+    
+    case Slot_ByteRef: readIntoByteArray(((ByteArray)fs)._getTheArray(), length, storeIt); break;
+    
+    
+    case Slot_ShortRef: {
+      ShortArray sa = (ShortArray)fs;
+      readIntoShortArray(sa._getTheArray(), length, storeIt);
+      break;
+    }
+    
+    case Slot_Int: {
+      IntegerArray ia = (IntegerArray)fs;
+      int prev = getPrevIntValue(TypeSystemImpl.intArrayTypeCode, 0);
+      for (int i = 0; i < length; i++) {
+        int v = readDiff(Slot_Int, prev);
+        prev = v;
+        if (0 == i && isUpdatePrevOK && storeIt) {
+          updatePrevArray0IntValue(ia._typeImpl, v);
+        }
         if (storeIt) {
-          heap[i] = floatRef;
+          ia.set(i, v);
         }
       }
       break;
-    case Slot_StrRef:
-      for (int i = iHeap + 2; i < endi; i++) {
-        final int strRef = readString(storeIt);
-        if (TRACE_STR_ARRAY) {
-          System.out.format("Trace String Array Des addr: %,d storeIt=%s, string=%s%n", i, storeIt ? "Y" : "N", stringHeapObj.getStringForCode(strRef));
-        }
+    }
+    
+    case Slot_LongRef: readIntoLongArray(((LongArray)fs)._getTheArray(), Slot_LongRef, length, storeIt); break;
+    
+    case Slot_Float: 
+      for (int i = 0; i < length; i++) {
+        final int floatRef = readFloat();
         if (storeIt) {
-          heap[i] = strRef; 
+          ((FloatArray)fs).set(i,  Float.intBitsToFloat(floatRef));
         }
       }
       break;
       
-    default: throw new RuntimeException("internal error");
-    } // end of switch    
+      case Slot_DoubleRef: readIntoDoubleArray(((DoubleArray)fs)._getTheArray(), Slot_DoubleRef, length, storeIt); break;
+      
+      case Slot_HeapRef: {
+        FSArray fsa = (FSArray)fs;
+        int prev = getPrevIntValue(TypeSystemImpl.fsArrayTypeCode, 0);
+        for (int i = 0; i < length; i++) {
+          final int v = readDiff(Slot_HeapRef, prev);
+          prev = v;
+          if (0 == i && isUpdatePrevOK && storeIt) {
+            updatePrevArray0IntValue(fsa._typeImpl, v);
+          }
+          if (storeIt) {
+            final int locali = i;
+            maybeStoreOrDefer_slotFixups(v, refd_fs -> fsa.set(locali, refd_fs));
+          }
+        }
+        break;
+      }
+      case Slot_StrRef: {
+          StringArray sa = (StringArray)fs;
+          for (int i = 0; i < length; i++) {
+            String s = readString(storeIt);
+
+            if (storeIt) {
+              sa.set(i, s);
+            }
+          }
+        }
+        break;
+
+      case Slot_JavaObjectRef: throw new UnsupportedOperationException();
+      default: assert(false);
+    } // end of switch
+  }
+  
+  private TOP getRefVal(int tgtSeq) {
+    return (tgtSeq == 0) ? null : cas.getFsFromId(fsStartIndexes.getSrcAddrFromTgtSeq(tgtSeq));
+  }
+    
+  private int readArrayLength() throws IOException {
+    return readVnumber(arrayLength_dis);
   }
+
   
   /**
-   *       
-   * @param iHeap index in the heap
-   * @param offset can be -1 - in which case read, but don't store
+   * @param The feature structure to set feature value in, but may be null if it was deferred,
+   *          - happens for Sofas and subtypes of AnnotationBase
+   *            because those have "final" values
+   *        For Sofa: these are the sofaid (String) and sofanum (int)
+   *        For AnnotationBase : this is the sofaRef (and the view).  
+   *                    
+   * @param tgtFeat the Feature being read
+   * @param srcFeat the Feature being set (may be null if the feature doesn't exist)
+   * @param storeIt false causes storing of values to be skipped
    * @throws IOException passthru
    */
-  private void readByKind(int iHeap, int offset, SlotKind kind, boolean storeIt) throws IOException {
-    
-    if (offset == 0) {
-      storeIt = false;
+  private void readByKind(TOP fs, FeatureImpl tgtFeat, FeatureImpl srcFeat, boolean storeIt, TypeImpl tgtType) throws IOException {
+    final int tgtFeatOffset = tgtFeat.getOffset();
+    if (srcFeat == null) {  
+      storeIt = false; // because feature doesn't exist in the source type system
     }
+    
+    final SlotKind kind = tgtFeat.getSlotKind();
+    
     switch (kind) {
-    case Slot_Int: case Slot_Short:
-      readDiffWithPrevTypeSlot(kind, iHeap, offset, storeIt);
+    
+    case Slot_Int: 
+      int vi = readDiffIntSlot(storeIt, tgtFeatOffset, kind, tgtType);
+      if (srcFeat == srcTs.sofaNum) {
+        sofaNum = vi;
+      } else {
+        maybeStoreOrDefer(storeIt, fs, (lfs) -> lfs.setIntLikeValue(kind,  srcFeat, vi));
+      }
       break;
+      
+    case Slot_Short:
+      int vs = readDiffIntSlot(storeIt, tgtFeatOffset, kind, tgtType);
+      maybeStoreOrDefer(storeIt, fs, (lfs) -> lfs.setIntLikeValue(kind, srcFeat, vs));
+      break;
+      
+    case Slot_HeapRef:
+      final int vh = readDiffIntSlot(storeIt, tgtFeatOffset, kind, tgtType);
+      if (srcTs.annotBaseSofaFeat == srcFeat) {
+        sofaRef = (Sofa) getRefVal(vh);
+      } else {
+        maybeStoreOrDefer(storeIt, fs, (lfs) -> { 
+        
+          // outer defer done if fs is null; it is a one-feature-structure defer for sofa or subtypes of annotationbase
+          
+          // When the setting is done for this one feature structure (now or at the end of deserializing features for it)
+          //   two cases: the ref'd value is known, or not.
+          //     - if not known, a fixup is added to
+          if (tgtType.getCode() == TypeSystemImpl.sofaTypeCode) {
+            if (tgtFeat.getCode() == TypeSystemImpl.sofaArrayFeatCode) {
+              Sofa sofa = (Sofa) lfs;
+              maybeStoreOrDefer_slotFixups(vh, ref_fs -> sofa.setLocalSofaData(ref_fs));
+            }
+          } else {
+            maybeStoreOrDefer_slotFixups(vh, ref_fs -> lfs.setFeatureValue(srcFeat,  ref_fs));
+          }
+        });
+      }
+      break;
+      
     case Slot_Float:
       final int floatAsInt = readFloat();
-      if (storeIt) {
-        heap[iHeap + offset] = floatAsInt;
-      }
+      maybeStoreOrDefer(storeIt, fs, (lfs) -> lfs.setFloatValue(srcFeat, Float.intBitsToFloat(floatAsInt)));
       break;
+      
     case Slot_Boolean: case Slot_Byte:
       final byte vByte = byte_dis.readByte();
-      if (storeIt) {
-        heap[iHeap + offset] = vByte;
-      }
-      break;
-    case Slot_HeapRef:
-      readDiffWithPrevTypeSlot(kind, iHeap, offset, storeIt);
-      if (storeIt) {
-        fixupsNeeded.add(iHeap + offset);
-      }
-//      System.out.format("debug adding to fixup, slot = %,d heapValue = %,d readByKind%n", iHeap + offset, heap[iHeap + offset]);
+      maybeStoreOrDefer(storeIt, fs, (lfs) -> lfs.setIntLikeValue(kind, srcFeat, vByte));
       break;
+      
     case Slot_StrRef: 
-      final int vStrRef = readString(storeIt);
+      final String vString = readString(storeIt);
       if (storeIt) {
-        heap[iHeap + offset] = vStrRef;
-      }
-      break;
-    case Slot_LongRef: {
-      long v = readLongOrDouble(kind, (!storeIt || (iPrevHeap == 0)) ? 0L : longHeapObj.getHeapValue(heap[iPrevHeap + offset]));
-      if (v == 0L) {
-        if (longZeroIndex == -1) {
-          longZeroIndex = longHeapObj.addLong(0L);
-        }
-        if (storeIt) {
-          heap[iHeap + offset] = longZeroIndex;
-        }
-      } else {
-        if (storeIt) {
-          heap[iHeap + offset] = longHeapObj.addLong(v);
+        if (tgtType.getCode() == TypeSystemImpl.sofaTypeCode) {
+          if (srcFeat == srcTs.sofaId) {
+            sofaName = vString; 
+            break;
+          } else if (srcFeat == srcTs.sofaMime) {
+            maybeStoreOrDefer(storeIt, fs, lfs -> ((Sofa)lfs).setMimeType(vString));
+            break;
+          } else if (srcFeat == srcTs.sofaUri && vString != null) {
+            maybeStoreOrDefer(storeIt, fs, lfs -> ((Sofa)lfs).setRemoteSofaURI(vString));
+            break;
+          } else if (srcFeat == srcTs.sofaString && vString != null) {
+            maybeStoreOrDefer(storeIt, fs, lfs -> ((Sofa)lfs).setLocalSofaData(vString));
+            break;
+          }
+        } else { // is not a sofa
+          maybeStoreOrDefer(storeIt, fs, (lfs) -> lfs.setStringValue(srcFeat, vString));
         }
       }
       break;
-    }
-    case Slot_DoubleRef: {
-      long v = readDouble();
-      if (v == 0L) {
-        if (longZeroIndex == -1) {
-          longZeroIndex = longHeapObj.addLong(0L);
-        }
-        if (storeIt) {
-          heap[iHeap + offset] = longZeroIndex;
-        }
-      } else if (v == DBL_1) {
-        if (double1Index == -1) {
-          double1Index = longHeapObj.addLong(DBL_1);
-        }
-        if (storeIt) {
-          heap[iHeap + offset] = double1Index;
-        }
+      
+    case Slot_LongRef:
+      long prevLong = getPrevLongValue(tgtType.getCode(), tgtFeatOffset);
+      long vl = readLongOrDouble(kind, prevLong);
+      updatePrevLongValue(tgtType, tgtFeatOffset, vl);
+      maybeStoreOrDefer(storeIt, fs, (lfs) -> lfs.setLongValue(srcFeat, vl));
+      break;
+
+    case Slot_DoubleRef: 
+      long vd = readDouble();
+      maybeStoreOrDefer(storeIt, fs, (lfs) -> lfs.setDoubleValue(srcFeat, Double.longBitsToDouble(vd)));
+      break;
+    
+    default: assert(false);                
+    } // end of switch
+  }
+  
+  private void maybeStoreOrDefer(boolean storeIt, TOP fs, Consumer<TOP> doStore) {
+    if (storeIt) {
+      if (null == fs) {
+        singleFsDefer.add( () -> doStore.accept(currentFs));
       } else {
-        if (storeIt) {
-          heap[iHeap + offset] = longHeapObj.addLong(v);
-        }
+        doStore.accept(fs);
       }
-      break;
     }
-    default: 
-      throw new RuntimeException("internal error");                
-    } // end of switch
+  }
+  
+  /**
+   * FS Ref slots fixups
+   */
+  /**
+   * FS Ref slots fixups
+   * @param tgtSeq the int value of the target seq number
+   * @param r is sofa-or-lfs.setFeatureValue-or-setLocalSofaData(TOP ref-d-fs)
+   */
+  private void maybeStoreOrDefer_slotFixups(final int tgtSeq, Consumer<TOP> r) {
+    if (tgtSeq == 0) {
+      r.accept(null);
+      return;
+    }
+    TOP src = getRefVal(tgtSeq);
+    if (src == null) {
+      //  need to do the getRefVal later when it's known
+      // () -> sofa.setLocalSofaData(getRefVal(vh))
+      // () -> lfs.setFeatureValue(srcFeat, getRefVal(vh))
+      // debug
+      fixupsNeeded.add(() -> r.accept(getRefVal(tgtSeq)));
+    } else {
+      // sofa.setLocalSofaData(tgt);
+      // lfs.setFeatureValue(srcFeat, src)
+      r.accept(src);
+    }
   }
 
+  /**
+   * process index information to re-index things
+   * @throws IOException
+   */
   private void readIndexedFeatureStructures() throws IOException {
     final int nbrViews = readVnumber(control_dis);
     final int nbrSofas = readVnumber(control_dis);
@@ -1983,9 +2112,9 @@ public class BinaryCasSerDes6 {
     if (isDelta) {
       // getArray avoids copying.
       // length is too long, but extra is never accessed
-      cas.reinitDeltaIndexedFSs(fsIndexes.getArray());
+      bcsd.reinitDeltaIndexedFSs(fsIndexes.getArray());
     } else {
-      cas.reinitIndexedFSs(fsIndexes.getArray());
+      bcsd.reinitIndexedFSs(fsIndexes.getArray());
     }
   }
 
@@ -2024,69 +2153,53 @@ public class BinaryCasSerDes6 {
     return DataIO.readVlong(dis);
   }
 
-  private int readIntoByteArray(int length, boolean storeIt) throws IOException { 
+  private void readIntoByteArray(byte[] array, int length, boolean storeIt) throws IOException { 
     if (storeIt) {
-      final int startPos = byteHeapObj.reserve(length);
-      byte_dis.readFully(byteHeapObj.heap, startPos, length);
-      return startPos;
+      byte_dis.readFully(array, 0, length);
     } else {
       skipBytes(byte_dis, length);
-      return 0;
     }
   }
 
-  private int readIntoShortArray(int length, boolean storeIt) throws IOException {

[... 2081 lines stripped ...]