You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@usergrid.apache.org by sf...@apache.org on 2014/11/21 22:35:53 UTC

[15/19] incubator-usergrid git commit: Refactored tests.

Refactored tests.

Made configuration options easier to override in tests

Added checks for entity too large on write.

Changed default max load size to 100.


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

Branch: refs/heads/two-dot-o
Commit: fb90ae705c29ee2cf61063d57f1e4e97d05ee5aa
Parents: e32d521
Author: Todd Nine <tn...@apigee.com>
Authored: Thu Nov 20 17:49:13 2014 -0700
Committer: Todd Nine <tn...@apigee.com>
Committed: Thu Nov 20 17:49:13 2014 -0700

----------------------------------------------------------------------
 .../usergrid/corepersistence/GuiceModule.java   |   2 +
 .../org/apache/usergrid/CoreITSetupImpl.java    |  13 ++
 .../usergrid/persistence/LargeEntityIT.java     |  30 +++-
 .../exception/EntityTooLargeException.java      |  67 ++++++++
 .../serialization/SerializationFig.java         |  14 +-
 .../MvccEntitySerializationStrategyImpl.java    |  41 ++---
 .../MvccEntitySerializationStrategyV2Impl.java  |  82 ++++++----
 .../collection/EntityCollectionManagerIT.java   |  16 +-
 ...MvccEntitySerializationStrategyImplTest.java |  52 +-----
 ...cEntitySerializationStrategyProxyV1Test.java |  17 --
 ...cEntitySerializationStrategyProxyV2Test.java |  19 +--
 ...ccEntitySerializationStrategyV1ImplTest.java |  18 --
 ...ccEntitySerializationStrategyV2ImplTest.java |  19 +--
 .../MvccEntitySerializationStrategyV2Test.java  | 164 +++++++++++++++++++
 .../impl/MvccLESSTransientTest.java             |  53 +-----
 .../collection/util/EntityHelper.java           |  35 ++--
 .../persistence/core/astyanax/CassandraFig.java |  12 ++
 .../core/guicyfig/SetConfigTestBypass.java      | 108 ++++++++++++
 .../persistence/queue/QueueManagerTest.java     |  26 +--
 19 files changed, 522 insertions(+), 266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/core/src/main/java/org/apache/usergrid/corepersistence/GuiceModule.java
----------------------------------------------------------------------
diff --git a/stack/core/src/main/java/org/apache/usergrid/corepersistence/GuiceModule.java b/stack/core/src/main/java/org/apache/usergrid/corepersistence/GuiceModule.java
index 2d80672..018a9b7 100644
--- a/stack/core/src/main/java/org/apache/usergrid/corepersistence/GuiceModule.java
+++ b/stack/core/src/main/java/org/apache/usergrid/corepersistence/GuiceModule.java
@@ -20,6 +20,7 @@ package org.apache.usergrid.corepersistence;
 import com.google.inject.AbstractModule;
 import com.google.inject.multibindings.Multibinder;
 
+import org.apache.usergrid.corepersistence.migration.EntityDataMigration;
 import org.apache.usergrid.corepersistence.migration.EntityTypeMappingMigration;
 import org.apache.usergrid.corepersistence.migration.GraphShardVersionMigration;
 import org.apache.usergrid.persistence.collection.guice.CollectionModule;
@@ -56,6 +57,7 @@ public class GuiceModule  extends AbstractModule {
         Multibinder<DataMigration> dataMigrationMultibinder = Multibinder.newSetBinder( binder(), DataMigration.class );
         dataMigrationMultibinder.addBinding().to( EntityTypeMappingMigration.class );
         dataMigrationMultibinder.addBinding().to( GraphShardVersionMigration.class );
+        dataMigrationMultibinder.addBinding().to( EntityDataMigration.class );
 
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/core/src/test/java/org/apache/usergrid/CoreITSetupImpl.java
----------------------------------------------------------------------
diff --git a/stack/core/src/test/java/org/apache/usergrid/CoreITSetupImpl.java b/stack/core/src/test/java/org/apache/usergrid/CoreITSetupImpl.java
index b3cc046..a5f59ed 100644
--- a/stack/core/src/test/java/org/apache/usergrid/CoreITSetupImpl.java
+++ b/stack/core/src/test/java/org/apache/usergrid/CoreITSetupImpl.java
@@ -24,10 +24,13 @@ import org.junit.runners.model.Statement;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.usergrid.cassandra.CassandraResource;
+import org.apache.usergrid.corepersistence.CpSetup;
 import org.apache.usergrid.mq.QueueManagerFactory;
 import org.apache.usergrid.persistence.EntityManagerFactory;
 import org.apache.usergrid.persistence.IndexBucketLocator;
 import org.apache.usergrid.persistence.cassandra.CassandraService;
+import org.apache.usergrid.persistence.core.migration.data.DataMigrationManager;
+import org.apache.usergrid.persistence.core.migration.schema.MigrationException;
 import org.apache.usergrid.persistence.index.impl.ElasticSearchResource;
 import org.apache.usergrid.utils.JsonUtils;
 
@@ -90,7 +93,17 @@ public class CoreITSetupImpl implements CoreITSetup {
             emf = cassandraResource.getBean( EntityManagerFactory.class );
             qmf = cassandraResource.getBean( QueueManagerFactory.class );
             indexBucketLocator = cassandraResource.getBean( IndexBucketLocator.class );
+
+            //run the migration
+            try {
+                CpSetup.getInjector().getInstance( DataMigrationManager.class ).migrate();
+            }
+            catch ( MigrationException e ) {
+                throw new RuntimeException( "Unable to run migration", e );
+            }
+
             enabled = true;
+
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/core/src/test/java/org/apache/usergrid/persistence/LargeEntityIT.java
----------------------------------------------------------------------
diff --git a/stack/core/src/test/java/org/apache/usergrid/persistence/LargeEntityIT.java b/stack/core/src/test/java/org/apache/usergrid/persistence/LargeEntityIT.java
index eef2820..c95f7d6 100644
--- a/stack/core/src/test/java/org/apache/usergrid/persistence/LargeEntityIT.java
+++ b/stack/core/src/test/java/org/apache/usergrid/persistence/LargeEntityIT.java
@@ -25,7 +25,7 @@ import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Map;
 
-import org.aspectj.lang.annotation.Before;
+import org.junit.After;
 import org.junit.Rule;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -35,6 +35,9 @@ import org.apache.usergrid.AbstractCoreIT;
 import org.apache.usergrid.Application;
 import org.apache.usergrid.CoreApplication;
 import org.apache.usergrid.cassandra.Concurrent;
+import org.apache.usergrid.corepersistence.CpSetup;
+import org.apache.usergrid.persistence.collection.serialization.SerializationFig;
+import org.apache.usergrid.persistence.core.guicyfig.SetConfigTestBypass;
 import org.apache.usergrid.utils.JsonUtils;
 
 import static org.junit.Assert.assertEquals;
@@ -48,6 +51,28 @@ public class LargeEntityIT extends AbstractCoreIT {
     public Application app = new CoreApplication( setup );
 
 
+    /**
+     * Set our max size before and after so we can override it in our runtime tests
+     */
+    private int setMaxEntitySize;
+
+    private SerializationFig serializationFig;
+
+
+    @org.junit.Before
+    public void setUp() {
+
+        serializationFig = CpSetup.getInjector().getInstance( SerializationFig.class );
+
+        setMaxEntitySize = serializationFig.getMaxEntitySize();
+    }
+
+
+    @After
+    public void tearDown() {
+        SetConfigTestBypass.setValueByPass( serializationFig, "getMaxEntitySize", setMaxEntitySize + "" );
+    }
+
 
     /**
      * Tests creating a large entity, then loading it, modifying it, saving it, then loading it again
@@ -57,6 +82,9 @@ public class LargeEntityIT extends AbstractCoreIT {
 
         LOG.debug( "testLargeEntityCrud" );
 
+        SetConfigTestBypass.setValueByPass( serializationFig, "getMaxEntitySize", 641814 + "" );
+
+
         final URL resource = this.getClass().getClassLoader().getResource( TEST_DATA_FILE );
 
         final byte[] fileData = Files.readAllBytes( Paths.get( resource.toURI() ) );

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/exception/EntityTooLargeException.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/exception/EntityTooLargeException.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/exception/EntityTooLargeException.java
new file mode 100644
index 0000000..11224db
--- /dev/null
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/exception/EntityTooLargeException.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.usergrid.persistence.collection.exception;
+
+import org.apache.usergrid.persistence.collection.CollectionScope;
+import org.apache.usergrid.persistence.collection.MvccEntity;
+import org.apache.usergrid.persistence.model.entity.Entity;
+
+
+public class EntityTooLargeException extends RuntimeException {
+
+    private final Entity entity;
+    private final int maxSize;
+    private final int attemptedSize;
+
+
+    public EntityTooLargeException( Entity entity, int maxSize,final int attemptedSize,  final String message ) {
+        super( message );
+        this.entity = entity;
+        this.maxSize = maxSize;
+
+        this.attemptedSize = attemptedSize;
+    }
+
+
+    /**
+     * Get the max size allowed
+     * @return
+     */
+    public int getMaxSize() {
+        return maxSize;
+    }
+
+
+    /**
+     * Get the size of the entity we attempted to save
+     * @return
+     */
+    public int getAttemptedSize() {
+        return attemptedSize;
+    }
+
+
+    /**
+     * Entity involved in operation.
+     * @return Entity or null if entity not instantiated yet in operation. 
+     */
+    public Entity getEntity() {
+        return entity;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/SerializationFig.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/SerializationFig.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/SerializationFig.java
index 294c7ab..0687312 100644
--- a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/SerializationFig.java
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/SerializationFig.java
@@ -58,17 +58,21 @@ public interface SerializationFig extends GuicyFig {
     int getTaskPoolQueueSize();
 
     /**
-     * The maximum amount of entites we can load at any one time
+     * The maximum amount of entities we can load at any one time
      */
     @Key( "collection.max.load.size" )
-    @Default( "1000" )
+    @Default( "100" )
     int getMaxLoadSize();
 
 
     /**
      * The maximum number of bytes a serialized entity can be.  Any thing beyond this is rejected
+     * This default is based on the following equation
+     *
+     * (15mb thrift buffer * .9) / 100 (default max load size)
      */
-    @Key( "collection.max.entry.size" )
-    @Default( "512" )
-    int getMaxEntrySize();
+    @Key( "collection.max.entity.size" )
+    @Default( "141557" )
+    int getMaxEntitySize();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImpl.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImpl.java
index 4ac5d24..f3f4c13 100644
--- a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImpl.java
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImpl.java
@@ -77,7 +77,6 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
     protected final Keyspace keyspace;
     protected final SerializationFig serializationFig;
     protected final EntityRepair repair;
-    private final AbstractSerializer<EntityWrapper> entityJsonSerializer;
     private final MultiTennantColumnFamily<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID>  columnFamily;
 
 
@@ -86,7 +85,6 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
         this.keyspace = keyspace;
         this.serializationFig = serializationFig;
         this.repair = new EntityRepairImpl( this, serializationFig );
-        this.entityJsonSerializer = getEntitySerializer();
         this.columnFamily = getColumnFamily();
     }
 
@@ -102,17 +100,17 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
         return doWrite( collectionScope, entityId, new RowOp() {
             @Override
             public void doOp( final ColumnListMutation<UUID> colMutation ) {
-                try {
-                    colMutation.putColumn( colName, entityJsonSerializer
+//                try {
+                    colMutation.putColumn( colName, getEntitySerializer()
                             .toByteBuffer( new EntityWrapper( entity.getStatus(), entity.getEntity() ) ) );
-                }
-                catch ( Exception e ) {
-                    // throw better exception if we can
-                    if ( entity != null || entity.getEntity().get() != null ) {
-                        throw new CollectionRuntimeException( entity, collectionScope, e );
-                    }
-                    throw e;
-                }
+//                }
+//                catch ( Exception e ) {
+//                    // throw better exception if we can
+//                    if ( entity != null || entity.getEntity().get() != null ) {
+//                        throw new CollectionRuntimeException( entity, collectionScope, e );
+//                    }
+//                    throw e;
+//                }
             }
         } );
     }
@@ -129,9 +127,9 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
         Preconditions.checkNotNull( maxVersion, "version is required" );
 
 
-        //didn't put the max in the error message, I don't want to take the string construction hit every time
-        Preconditions.checkArgument( entityIds.size() <= serializationFig.getMaxLoadSize(),
-                "requested size cannot be over configured maximum" );
+        if( entityIds.size() > serializationFig.getMaxLoadSize()){
+            throw new IllegalArgumentException(  "requested load size cannot be over configured maximum of " + serializationFig.getMaxLoadSize() );
+        }
 
 
         final Id applicationId = collectionScope.getApplication();
@@ -185,7 +183,7 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
             final Column<UUID> column = columns.getColumnByIndex( 0 );
 
             final MvccEntity parsedEntity =
-                    new MvccColumnParser( entityId, entityJsonSerializer ).parseColumn( column );
+                    new MvccColumnParser( entityId, getEntitySerializer() ).parseColumn( column );
 
             //we *might* need to repair, it's not clear so check before loading into result sets
             final MvccEntity maybeRepaired = repair.maybeRepair( collectionScope, parsedEntity );
@@ -223,7 +221,7 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
                 keyspace.prepareQuery( columnFamily ).getKey( rowKey )
                         .withColumnRange( version, null, false, fetchSize );
 
-        return new ColumnNameIterator( query, new MvccColumnParser( entityId, entityJsonSerializer ), false );
+        return new ColumnNameIterator( query, new MvccColumnParser( entityId, getEntitySerializer() ), false );
     }
 
 
@@ -253,7 +251,7 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
                 keyspace.prepareQuery( columnFamily ).getKey( rowKey )
                         .withColumnRange( null, version, true, fetchSize );
 
-        return new ColumnNameIterator( query, new MvccColumnParser( entityId, entityJsonSerializer ), false );
+        return new ColumnNameIterator( query, new MvccColumnParser( entityId, getEntitySerializer() ), false );
     }
 
 
@@ -268,7 +266,7 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
         return doWrite( collectionScope, entityId, new RowOp() {
             @Override
             public void doOp( final ColumnListMutation<UUID> colMutation ) {
-                colMutation.putColumn( version, entityJsonSerializer
+                colMutation.putColumn( version, getEntitySerializer()
                         .toByteBuffer( new EntityWrapper( MvccEntity.Status.COMPLETE, Optional.<Entity>absent() ) ) );
             }
         } );
@@ -410,10 +408,5 @@ public abstract class MvccEntitySerializationStrategyImpl implements MvccEntityS
      * @return
      */
     protected abstract MultiTennantColumnFamily<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> getColumnFamily();
-    /**
-     *
-     private static final MultiTennantColumnFamily<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> CF_ENTITY_DATA =
-             new MultiTennantColumnFamily<>( "Entity_Version_Data", ROW_KEY_SER, UUIDSerializer.get() );
 
-     */
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Impl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Impl.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Impl.java
index 723bb38..923c399 100644
--- a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Impl.java
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Impl.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 
 import org.apache.usergrid.persistence.collection.MvccEntity;
 import org.apache.usergrid.persistence.collection.exception.DataCorruptionException;
+import org.apache.usergrid.persistence.collection.exception.EntityTooLargeException;
 import org.apache.usergrid.persistence.collection.serialization.SerializationFig;
 import org.apache.usergrid.persistence.core.astyanax.FieldBuffer;
 import org.apache.usergrid.persistence.core.astyanax.FieldBufferBuilder;
@@ -36,6 +37,7 @@ import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.model.entity.Entity;
 import org.apache.usergrid.persistence.model.entity.Id;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.dataformat.smile.SmileFactory;
 import com.google.common.base.Optional;
@@ -52,8 +54,6 @@ import com.netflix.astyanax.serializers.UUIDSerializer;
 @Singleton
 public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializationStrategyImpl {
 
-    private static final EntitySerializer ENTITY_JSON_SER = new EntitySerializer();
-
 
     private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
 
@@ -63,21 +63,24 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
 
 
     private static final MultiTennantColumnFamily<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> CF_ENTITY_DATA =
-                new MultiTennantColumnFamily<>( "Entity_Version_Data_V2", ROW_KEY_SER, UUIDSerializer.get() );
+            new MultiTennantColumnFamily<>( "Entity_Version_Data_V2", ROW_KEY_SER, UUIDSerializer.get() );
 
     private static final FieldBufferSerializer FIELD_BUFFER_SERIALIZER = FieldBufferSerializer.get();
 
 
+    private final EntitySerializer entitySerializer;
+
 
     @Inject
     public MvccEntitySerializationStrategyV2Impl( final Keyspace keyspace, final SerializationFig serializationFig ) {
         super( keyspace, serializationFig );
+        entitySerializer = new EntitySerializer( serializationFig );
     }
 
 
     @Override
     protected AbstractSerializer<EntityWrapper> getEntitySerializer() {
-        return ENTITY_JSON_SER;
+        return entitySerializer;
     }
 
 
@@ -87,35 +90,33 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
     }
 
 
-    public static class EntitySerializer extends AbstractSerializer<EntityWrapper> {
+    /**
+     * We should only ever create this once, since this impl is a singleton
+     */
+    public final class EntitySerializer extends AbstractSerializer<EntityWrapper> {
 
 
-//        private static final ByteBufferSerializer BUFFER_SERIALIZER = ByteBufferSerializer.get();
-//
-//        private static final BytesArraySerializer BYTES_ARRAY_SERIALIZER = BytesArraySerializer.get();
+        private final SmileFactory SMILE_FACTORY = new SmileFactory();
 
+        private final ObjectMapper MAPPER = new ObjectMapper( SMILE_FACTORY );
 
-        public static final SmileFactory f = new SmileFactory();
 
-        public static ObjectMapper mapper;
+        private SerializationFig serializationFig;
 
-        private static byte STATE_COMPLETE =  0 ;
-        private static byte STATE_DELETED = 1 ;
-        private static byte STATE_PARTIAL = 2 ;
 
-        private static byte VERSION = 1;
+        private byte STATE_COMPLETE = 0;
+        private byte STATE_DELETED = 1;
+        private byte STATE_PARTIAL = 2;
 
+        private byte VERSION = 1;
 
-        public EntitySerializer() {
-            try {
-                mapper = new ObjectMapper( f );
-                //                mapper.enable(SerializationFeature.INDENT_OUTPUT); don't indent output,
-                // causes slowness
-                mapper.enableDefaultTypingAsProperty( ObjectMapper.DefaultTyping.JAVA_LANG_OBJECT, "@class" );
-            }
-            catch ( Exception e ) {
-                throw new RuntimeException( "Error setting up mapper", e );
-            }
+
+        public EntitySerializer( final SerializationFig serializationFig ) {
+            this.serializationFig = serializationFig;
+
+            //                mapper.enable(SerializationFeature.INDENT_OUTPUT); don't indent output,
+            // causes slowness
+            MAPPER.enableDefaultTypingAsProperty( ObjectMapper.DefaultTyping.JAVA_LANG_OBJECT, "@class" );
         }
 
 
@@ -126,7 +127,7 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
             }
 
             //we always have a max of 3 fields
-            FieldBufferBuilder builder = new FieldBufferBuilder( 3  );
+            FieldBufferBuilder builder = new FieldBufferBuilder( 3 );
 
             builder.addByte( VERSION );
 
@@ -149,14 +150,28 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
                 builder.addByte( STATE_PARTIAL );
             }
 
+
+            final Entity entity = wrapper.entity.get();
+            final byte[] entityBytes;
+
             try {
-                final byte[] entityBytes = mapper.writeValueAsBytes( wrapper.entity.get() );
-                builder.addBytes( entityBytes );
+                entityBytes = MAPPER.writeValueAsBytes( entity );
             }
-            catch ( Exception e ) {
+            catch ( JsonProcessingException e ) {
                 throw new RuntimeException( "Unable to serialize entity", e );
             }
 
+
+            final int maxEntrySize = serializationFig.getMaxEntitySize();
+
+            if ( entityBytes.length > maxEntrySize ) {
+                throw new EntityTooLargeException( entity, maxEntrySize, entityBytes.length,
+                        "Your entity cannot exceed " + maxEntrySize + " bytes. The entity you tried to save was "
+                                + entityBytes.length + " bytes" );
+            }
+
+            builder.addBytes( entityBytes );
+
             return FIELD_BUFFER_SERIALIZER.toByteBuffer( builder.build() );
         }
 
@@ -181,13 +196,12 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
                 throw new DataCorruptionException( "Unable to de-serialze entity", e );
             }
 
-            FieldBufferParser  parser = new FieldBufferParser( fieldBuffer );
-
+            FieldBufferParser parser = new FieldBufferParser( fieldBuffer );
 
 
             byte version = parser.readByte();
 
-            if (  VERSION !=  version ) {
+            if ( VERSION != version ) {
                 throw new UnsupportedOperationException( "A version of type " + version + " is unsupported" );
             }
 
@@ -195,7 +209,7 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
 
             // it's been deleted, remove it
 
-            if (  STATE_DELETED ==  state ) {
+            if ( STATE_DELETED == state ) {
                 return new EntityWrapper( MvccEntity.Status.COMPLETE, Optional.<Entity>absent() );
             }
 
@@ -204,7 +218,7 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
             byte[] array = parser.readBytes();
 
             try {
-                storedEntity = mapper.readValue( array, Entity.class );
+                storedEntity = MAPPER.readValue( array, Entity.class );
             }
             catch ( Exception e ) {
                 throw new DataCorruptionException( "Unable to read entity data", e );
@@ -212,7 +226,7 @@ public class MvccEntitySerializationStrategyV2Impl extends MvccEntitySerializati
 
             final Optional<Entity> entity = Optional.of( storedEntity );
 
-            if (  STATE_COMPLETE ==  state ) {
+            if ( STATE_COMPLETE == state ) {
                 return new EntityWrapper( MvccEntity.Status.COMPLETE, entity );
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/EntityCollectionManagerIT.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/EntityCollectionManagerIT.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/EntityCollectionManagerIT.java
index 21e3260..5eba979 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/EntityCollectionManagerIT.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/EntityCollectionManagerIT.java
@@ -28,12 +28,13 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import org.apache.usergrid.persistence.collection.exception.WriteUniqueVerifyException;
-import org.apache.usergrid.persistence.collection.util.EntityHelper;
-import org.apache.usergrid.persistence.core.guice.MigrationManagerRule;
 import org.apache.usergrid.persistence.collection.guice.TestCollectionModule;
 import org.apache.usergrid.persistence.collection.impl.CollectionScopeImpl;
 import org.apache.usergrid.persistence.collection.mvcc.entity.Stage;
 import org.apache.usergrid.persistence.collection.serialization.SerializationFig;
+import org.apache.usergrid.persistence.collection.util.EntityHelper;
+import org.apache.usergrid.persistence.core.guice.MigrationManagerRule;
+import org.apache.usergrid.persistence.core.guicyfig.SetConfigTestBypass;
 import org.apache.usergrid.persistence.core.test.ITRunner;
 import org.apache.usergrid.persistence.core.test.UseModules;
 import org.apache.usergrid.persistence.core.util.Health;
@@ -713,8 +714,13 @@ public class EntityCollectionManagerIT {
     public void largeEntityWriteRead() {
         final int setSize = 65535 * 2;
 
+        final int currentMaxSize = serializationFig.getMaxEntitySize();
 
-        final Entity entity =  EntityHelper.generateEntity( setSize );
+        //override our default
+        SetConfigTestBypass.setValueByPass( serializationFig, "getMaxEntitySize", 65535 * 10 + "" );
+
+
+        final Entity entity = EntityHelper.generateEntity( setSize );
 
         //now we have one massive, entity, save it and retrieve it.
         CollectionScope context =
@@ -732,5 +738,9 @@ public class EntityCollectionManagerIT {
 
 
         EntityHelper.verifyDeepEquals( entity, loaded );
+
+
+        //override our default
+        SetConfigTestBypass.setValueByPass( serializationFig, "getMaxEntitySize", currentMaxSize + "" );
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImplTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImplTest.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImplTest.java
index 0cdc320..6ce88cf 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImplTest.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyImplTest.java
@@ -75,7 +75,7 @@ import static org.mockito.Mockito.mock;
 public abstract class MvccEntitySerializationStrategyImplTest {
 
 
-    private MvccEntitySerializationStrategy serializationStrategy;
+    protected MvccEntitySerializationStrategy serializationStrategy;
 
 
     @Inject
@@ -582,56 +582,6 @@ public abstract class MvccEntitySerializationStrategyImplTest {
 
 
 
-    /**
-     * Tests an entity with more than  65535 bytes worth of data is successfully stored and retrieved
-     */
-    @Test
-    public void largeEntityWriteRead() throws ConnectionException {
-        final int setSize = 65535 * 2;
-
-        final Entity entity = EntityHelper.generateEntity( setSize );
-
-        //now we have one massive, entity, save it and retrieve it.
-        CollectionScope context =
-                new CollectionScopeImpl( new SimpleId( "organization" ), new SimpleId( "parent" ), "tests" );
-
-
-        final Id id = entity.getId();
-        ValidationUtils.verifyIdentity( id );
-        final UUID version = UUIDGenerator.newTimeUUID();
-        final MvccEntity.Status status = MvccEntity.Status.COMPLETE;
-
-        final MvccEntity mvccEntity = new MvccEntityImpl( id, version, status, entity );
-
-
-        getMvccEntitySerializationStrategy().write( context, mvccEntity ).execute();
-
-        //now load it
-        final Iterator<MvccEntity> loaded =
-                getMvccEntitySerializationStrategy().loadDescendingHistory( context, id, version, 100 );
-
-
-        assertLargeEntity( mvccEntity, loaded );
-
-        MvccEntity returned = serializationStrategy.load( context, Collections.singleton( id ) , version ).getEntity( id );
-
-        assertLargeEntity( mvccEntity, returned );
-    }
-
-
-    /**
-     * Assertion for the loaded values of a large entity.  Note the returned may be nullable
-     * @param expected
-     * @param returned
-     */
-    protected abstract void assertLargeEntity(final MvccEntity expected, final Iterator<MvccEntity> returned);
-
-    /**
-     * Tests large entity that is directly returned
-     * @param expected
-     * @param returned
-     */
-    protected abstract void assertLargeEntity(final MvccEntity expected, final MvccEntity returned);
 
     @Test(expected = NullPointerException.class)
     public void writeParamsContext() throws ConnectionException {

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV1Test.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV1Test.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV1Test.java
index df5f7d1..a936d2a 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV1Test.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV1Test.java
@@ -76,23 +76,6 @@ public class MvccEntitySerializationStrategyProxyV1Test extends MvccEntitySerial
 
 
 
-    @Override
-    protected void assertLargeEntity( final MvccEntity expected, final Iterator<MvccEntity> returned ) {
-        //known bug in v1, it should return null.  Fixed in v2
-        assertTrue( "V1 should be broken, but still have a next", returned.hasNext() );
-
-        final MvccEntity entity = returned.next();
-
-
-        assertLargeEntity( expected, entity );
-    }
-
-
-    protected void assertLargeEntity( final MvccEntity expected, final MvccEntity returned ) {
-        assertEquals( "Marked as deleted since we can't parse", MvccEntity.Status.DELETED, returned.getStatus() );
-
-        assertFalse( "V1 returns empty b/c parse fails", returned.getEntity().isPresent() );
-    }
 
     @After
     public void reSetMigrationVersion() {

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV2Test.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV2Test.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV2Test.java
index dc31ff7..1709cc1 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV2Test.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyProxyV2Test.java
@@ -42,7 +42,7 @@ import static org.junit.Assert.assertTrue;
 
 @RunWith( ITRunner.class )
 @UseModules( TestCollectionModule.class )
-public class MvccEntitySerializationStrategyProxyV2Test extends MvccEntitySerializationStrategyImplTest {
+public class MvccEntitySerializationStrategyProxyV2Test extends MvccEntitySerializationStrategyV2Test {
 
     @Inject
     @ProxyImpl
@@ -74,23 +74,6 @@ public class MvccEntitySerializationStrategyProxyV2Test extends MvccEntitySerial
     }
 
 
-    @Override
-    protected void assertLargeEntity( final MvccEntity expected, final Iterator<MvccEntity> returned ) {
-        assertTrue( returned.hasNext() );
-
-        final MvccEntity loadedEntity = returned.next();
-
-        assertLargeEntity( expected, loadedEntity );
-    }
-
-
-    @Override
-    protected void assertLargeEntity( final MvccEntity expected, final MvccEntity returned ) {
-
-        org.junit.Assert.assertEquals( "The loaded entity should match the stored entity", expected, returned );
-
-        EntityHelper.verifyDeepEquals( expected.getEntity().get(), returned.getEntity().get() );
-    }
 
 
     @After

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV1ImplTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV1ImplTest.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV1ImplTest.java
index 9b0ccfb..e59426d 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV1ImplTest.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV1ImplTest.java
@@ -47,24 +47,6 @@ public class MvccEntitySerializationStrategyV1ImplTest extends MvccEntitySeriali
     private MvccEntitySerializationStrategy serializationStrategy;
 
 
-    @Override
-    protected void assertLargeEntity( final MvccEntity expected, final Iterator<MvccEntity> returned ) {
-        //known bug in v1, it should return null.  Fixed in v2
-        assertTrue( "V1 should be broken, but still have a next", returned.hasNext() );
-
-        final MvccEntity entity = returned.next();
-
-
-        assertLargeEntity( expected, entity );
-    }
-
-
-    protected void assertLargeEntity( final MvccEntity expected, final MvccEntity returned ) {
-        assertEquals( "Marked as deleted since we can't parse", MvccEntity.Status.DELETED, returned.getStatus() );
-
-        assertFalse( "V1 returns empty b/c parse fails", returned.getEntity().isPresent() );
-    }
-
 
     @Override
     protected MvccEntitySerializationStrategy getMvccEntitySerializationStrategy() {

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2ImplTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2ImplTest.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2ImplTest.java
index 53952d0..c931904 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2ImplTest.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2ImplTest.java
@@ -39,7 +39,7 @@ import static org.junit.Assert.assertTrue;
 
 @RunWith( ITRunner.class )
 @UseModules( TestCollectionModule.class )
-public class MvccEntitySerializationStrategyV2ImplTest extends MvccEntitySerializationStrategyImplTest {
+public class MvccEntitySerializationStrategyV2ImplTest extends MvccEntitySerializationStrategyV2Test {
 
     @Inject
     @CurrentImpl
@@ -52,21 +52,4 @@ public class MvccEntitySerializationStrategyV2ImplTest extends MvccEntitySeriali
     }
 
 
-    @Override
-    protected void assertLargeEntity( final MvccEntity expected, final Iterator<MvccEntity> returned ) {
-        assertTrue( returned.hasNext() );
-
-        final MvccEntity loadedEntity = returned.next();
-
-        assertLargeEntity( expected, loadedEntity );
-    }
-
-
-    @Override
-    protected void assertLargeEntity( final MvccEntity expected, final MvccEntity returned ) {
-
-        org.junit.Assert.assertEquals( "The loaded entity should match the stored entity", expected, returned );
-
-        EntityHelper.verifyDeepEquals( expected.getEntity().get(), returned.getEntity().get() );
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Test.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Test.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Test.java
new file mode 100644
index 0000000..a496519
--- /dev/null
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccEntitySerializationStrategyV2Test.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.usergrid.persistence.collection.serialization.impl;
+
+
+import java.lang.annotation.Annotation;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.UUID;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.safehaus.guicyfig.Bypass;
+import org.safehaus.guicyfig.Env;
+import org.safehaus.guicyfig.Option;
+
+import org.apache.usergrid.persistence.collection.CollectionScope;
+import org.apache.usergrid.persistence.collection.MvccEntity;
+import org.apache.usergrid.persistence.collection.exception.EntityTooLargeException;
+import org.apache.usergrid.persistence.collection.impl.CollectionScopeImpl;
+import org.apache.usergrid.persistence.collection.mvcc.entity.impl.MvccEntityImpl;
+import org.apache.usergrid.persistence.collection.serialization.SerializationFig;
+import org.apache.usergrid.persistence.collection.util.EntityHelper;
+import org.apache.usergrid.persistence.core.guicyfig.SetConfigTestBypass;
+import org.apache.usergrid.persistence.core.util.ValidationUtils;
+import org.apache.usergrid.persistence.model.entity.Entity;
+import org.apache.usergrid.persistence.model.entity.Id;
+import org.apache.usergrid.persistence.model.entity.SimpleId;
+import org.apache.usergrid.persistence.model.util.UUIDGenerator;
+
+import com.google.inject.Inject;
+import com.netflix.astyanax.connectionpool.exceptions.ConnectionException;
+
+import static org.junit.Assert.assertTrue;
+
+
+public abstract class MvccEntitySerializationStrategyV2Test extends MvccEntitySerializationStrategyImplTest {
+
+
+    @Inject
+    protected SerializationFig serializationFig;
+
+    private int setMaxEntitySize;
+
+
+    @Before
+    public void setUp(){
+
+
+      setMaxEntitySize =  serializationFig.getMaxEntitySize();
+    }
+
+    @After
+    public void tearDown(){
+        SetConfigTestBypass.setValueByPass( serializationFig, "getMaxEntitySize", setMaxEntitySize + "" );
+    }
+
+    /**
+     * Tests an entity with more than  65535 bytes worth of data is successfully stored and retrieved
+     */
+    @Test
+    public void largeEntityWriteRead() throws ConnectionException {
+        final int setSize = 65535 * 2;
+
+
+        //this is the size it works out to be when serialized, we want to allow this size
+
+        SetConfigTestBypass.setValueByPass( serializationFig, "getMaxEntitySize", 65535*10+"");
+        final Entity entity = EntityHelper.generateEntity( setSize );
+
+        //now we have one massive, entity, save it and retrieve it.
+        CollectionScope context =
+                new CollectionScopeImpl( new SimpleId( "organization" ), new SimpleId( "parent" ), "tests" );
+
+
+        final Id id = entity.getId();
+        ValidationUtils.verifyIdentity( id );
+        final UUID version = UUIDGenerator.newTimeUUID();
+        final MvccEntity.Status status = MvccEntity.Status.COMPLETE;
+
+        final MvccEntity mvccEntity = new MvccEntityImpl( id, version, status, entity );
+
+
+        getMvccEntitySerializationStrategy().write( context, mvccEntity ).execute();
+
+        //now load it
+        final Iterator<MvccEntity> loaded =
+                getMvccEntitySerializationStrategy().loadDescendingHistory( context, id, version, 100 );
+
+
+        assertLargeEntity( mvccEntity, loaded );
+
+        MvccEntity returned =
+                serializationStrategy.load( context, Collections.singleton( id ), version ).getEntity( id );
+
+        assertLargeEntity( mvccEntity, returned );
+    }
+
+
+
+    /**
+     * Tests an entity with more than  65535 bytes worth of data is successfully stored and retrieved
+     */
+    @Test( expected = EntityTooLargeException.class )
+    public void entityLargerThanAllowedWrite() throws ConnectionException {
+        final int setSize = serializationFig.getMaxEntitySize() + 1;
+
+        final Entity entity = EntityHelper.generateEntity( setSize );
+
+        //now we have one massive, entity, save it and retrieve it.
+        CollectionScope context =
+                new CollectionScopeImpl( new SimpleId( "organization" ), new SimpleId( "parent" ), "tests" );
+
+
+        final Id id = entity.getId();
+        ValidationUtils.verifyIdentity( id );
+        final UUID version = UUIDGenerator.newTimeUUID();
+        final MvccEntity.Status status = MvccEntity.Status.COMPLETE;
+
+        final MvccEntity mvccEntity = new MvccEntityImpl( id, version, status, entity );
+
+
+        getMvccEntitySerializationStrategy().write( context, mvccEntity ).execute();
+    }
+
+
+    protected void assertLargeEntity( final MvccEntity expected, final Iterator<MvccEntity> returned ) {
+        assertTrue( returned.hasNext() );
+
+        final MvccEntity loadedEntity = returned.next();
+
+        assertLargeEntity( expected, loadedEntity );
+    }
+
+
+    protected void assertLargeEntity( final MvccEntity expected, final MvccEntity returned ) {
+
+        org.junit.Assert.assertEquals( "The loaded entity should match the stored entity", expected, returned );
+
+        EntityHelper.verifyDeepEquals( expected.getEntity().get(), returned.getEntity().get() );
+    }
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLESSTransientTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLESSTransientTest.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLESSTransientTest.java
index 4f96594..09b350c 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLESSTransientTest.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLESSTransientTest.java
@@ -24,6 +24,7 @@ import java.lang.annotation.Annotation;
 import java.util.Collections;
 import java.util.UUID;
 
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -41,6 +42,7 @@ import org.apache.usergrid.persistence.collection.mvcc.MvccLogEntrySerialization
 import org.apache.usergrid.persistence.collection.mvcc.entity.Stage;
 import org.apache.usergrid.persistence.collection.mvcc.entity.impl.MvccLogEntryImpl;
 import org.apache.usergrid.persistence.collection.serialization.SerializationFig;
+import org.apache.usergrid.persistence.core.guicyfig.SetConfigTestBypass;
 import org.apache.usergrid.persistence.core.test.ITRunner;
 import org.apache.usergrid.persistence.core.test.UseModules;
 import org.apache.usergrid.persistence.model.entity.Id;
@@ -80,58 +82,15 @@ public class MvccLESSTransientTest {
     public void setTimeout() {
         originalTimeout = serializationFig.getTimeout();
         //set the bypass options
-        serializationFig.setBypass( new TestByPass() );
+        SetConfigTestBypass.setValueByPass( serializationFig,"getTimeout", "1"  );
     }
 
-
-    /**
-     * Test bypass that sets all environments to use the timeout of 1 second
-     */
-    public class TestByPass implements Bypass {
-
-
-        @Override
-        public Option[] options() {
-            return new Option[] { new TestOption() };
-        }
-
-
-        @Override
-        public Env[] environments() {
-            return new Env[] { Env.ALL, Env.UNIT };
-        }
-
-
-        @Override
-        public Class<? extends Annotation> annotationType() {
-            return Bypass.class;
-        }
+    @After
+    public void resetTimeout(){
+        SetConfigTestBypass.setValueByPass( serializationFig,"getTimeout", originalTimeout + ""  );
     }
 
 
-    /**
-     * TestOption
-     */
-    public class TestOption implements Option {
-
-
-        @Override
-        public Class<? extends Annotation> annotationType() {
-            return Bypass.class;
-        }
-
-
-        @Override
-        public String method() {
-            return "getTimeout";
-        }
-
-
-        @Override
-        public String override() {
-            return "1";
-        }
-    }
 
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/util/EntityHelper.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/util/EntityHelper.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/util/EntityHelper.java
index 5c99121..4dafbaf 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/util/EntityHelper.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/util/EntityHelper.java
@@ -40,34 +40,33 @@ import static org.junit.Assert.assertNotNull;
 public class EntityHelper {
 
     /**
-     * Generates an entity with all string fields to have at least the minSize
-     * number of characters in the field names + field values
-     * @param minSize
-     * @return
+     * Generates an entity with all string fields to have at least the minSize number of characters in the field names +
+     * field values
      */
-    public static Entity generateEntity(final int minSize){
+    public static Entity generateEntity( final int minSize ) {
         int currentLength = 0;
 
-                final Entity entity = new Entity( new SimpleId( "test" ) );
+        final Entity entity = new Entity( new SimpleId( "test" ) );
 
-                //generate a really large string value
-                StringBuilder builder = new StringBuilder();
+        //generate a really large string value
 
-                for ( int i = 0; i < 100; i++ ) {
-                    builder.append( UUIDGenerator.newTimeUUID().toString() );
-                }
 
-                final String value = builder.toString();
+        //loop until our size is beyond the set size
+        for ( int i = 0; currentLength < minSize; i++ ) {
+            final String key = "newStringField" + i;
 
+            currentLength += key.length();
 
-                //loop until our size is beyond the set size
-                for ( int i = 0; currentLength < minSize; i++ ) {
-                    final String key = "newStringField" + i;
+            StringBuilder builder = new StringBuilder();
 
-                    entity.setField( new StringField( key, value ) );
+            for ( int j = 0; j < 1000 && currentLength < minSize; j++ ) {
+                builder.append( "a" );
+                currentLength ++;
+            }
 
-                    currentLength += key.length() + value.length();
-                }
+
+            entity.setField( new StringField( key, builder.toString() ) );
+        }
 
         return entity;
     }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraFig.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraFig.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraFig.java
index 4cf76f4..df94adc 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraFig.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraFig.java
@@ -39,6 +39,9 @@ public interface CassandraFig extends GuicyFig {
 
     public static final String SHARD_VALUES = "cassandra.shardvalues";
 
+    public static final String THRIFT_TRANSPORT_SIZE = "cassandra.thrift.transport.frame";
+
+
     @Key( "cassandra.hosts" )
     String getHosts();
 
@@ -94,5 +97,14 @@ public interface CassandraFig extends GuicyFig {
     @Key(SHARD_VALUES)
     String getShardValues();
 
+    /**
+     * Get the thrift transport size.  Should be set to what is on the cassandra servers.  As we move to CQL, this will become obsolete
+     * @return
+     */
+    @Key( THRIFT_TRANSPORT_SIZE)
+    @Default( "15728640" )
+    int getThriftBufferSize();
+
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guicyfig/SetConfigTestBypass.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guicyfig/SetConfigTestBypass.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guicyfig/SetConfigTestBypass.java
new file mode 100644
index 0000000..cb28449
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guicyfig/SetConfigTestBypass.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.usergrid.persistence.core.guicyfig;
+
+
+import java.lang.annotation.Annotation;
+
+import org.safehaus.guicyfig.Bypass;
+import org.safehaus.guicyfig.Env;
+import org.safehaus.guicyfig.GuicyFig;
+import org.safehaus.guicyfig.Option;
+
+
+public class SetConfigTestBypass {
+
+
+    /**
+     * Set the value bypass on the guicyfig
+     * @param guicyFig
+     * @param methodName
+     * @param valueToSet
+     */
+    public static void setValueByPass(final GuicyFig guicyFig, final String methodName, final String valueToSet){
+        guicyFig.setBypass( new TestByPass( methodName, valueToSet ) );
+    }
+
+    /**
+     * Test bypass that sets all environments to use the timeout of 1 second
+     */
+    public static final class TestByPass implements Bypass {
+
+        private Option[] options;
+
+
+        public TestByPass( final String methodName, final String value ) {
+            options = new Option[] { new TestOption( methodName, value ) };
+        }
+
+
+        @Override
+        public Option[] options() {
+            return options;
+        }
+
+
+        @Override
+        public Env[] environments() {
+            return new Env[] { Env.ALL, Env.UNIT };
+        }
+
+
+        @Override
+        public Class<? extends Annotation> annotationType() {
+            return Bypass.class;
+        }
+    }
+
+
+    /**
+     * TestOption
+     */
+    public static final class TestOption implements Option {
+
+        private final String methodName;
+        private final String valueToReturn;
+
+
+        public TestOption( final String methodName, final String valueToReturn ) {
+            this.methodName = methodName;
+            this.valueToReturn = valueToReturn;
+        }
+
+
+        @Override
+        public Class<? extends Annotation> annotationType() {
+            return Bypass.class;
+        }
+
+
+        @Override
+        public String method() {
+            return methodName;
+        }
+
+
+        @Override
+        public String override() {
+            return valueToReturn;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/fb90ae70/stack/corepersistence/queue/src/test/java/org/apache/usergrid/persistence/queue/QueueManagerTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queue/src/test/java/org/apache/usergrid/persistence/queue/QueueManagerTest.java b/stack/corepersistence/queue/src/test/java/org/apache/usergrid/persistence/queue/QueueManagerTest.java
index 8eb5625..935fd16 100644
--- a/stack/corepersistence/queue/src/test/java/org/apache/usergrid/persistence/queue/QueueManagerTest.java
+++ b/stack/corepersistence/queue/src/test/java/org/apache/usergrid/persistence/queue/QueueManagerTest.java
@@ -19,28 +19,30 @@
 
 package org.apache.usergrid.persistence.queue;
 
-import org.apache.usergrid.persistence.queue.guice.TestQueueModule;
-import org.apache.usergrid.persistence.queue.impl.QueueScopeFactoryImpl;
-import org.apache.usergrid.persistence.queue.impl.QueueScopeImpl;
-import org.apache.usergrid.persistence.core.test.UseModules;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import org.apache.usergrid.persistence.core.test.ITRunner;
+import org.apache.usergrid.persistence.core.test.UseModules;
 import org.apache.usergrid.persistence.model.entity.SimpleId;
+import org.apache.usergrid.persistence.queue.guice.TestQueueModule;
+import org.apache.usergrid.persistence.queue.impl.QueueScopeFactoryImpl;
+import org.apache.usergrid.persistence.queue.impl.QueueScopeImpl;
 
 import com.google.inject.Inject;
-import org.safehaus.guicyfig.Bypass;
-import org.safehaus.guicyfig.OptionState;
-import org.safehaus.guicyfig.Overrides;
-
-import java.beans.PropertyChangeListener;
-import java.io.IOException;
-import java.util.*;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 @RunWith( ITRunner.class )
 @UseModules( { TestQueueModule.class } )