You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@usergrid.apache.org by to...@apache.org on 2014/10/28 00:04:40 UTC

[1/7] git commit: Added new classes for locating buckets using consistent hashing

Repository: incubator-usergrid
Updated Branches:
  refs/heads/key-row-sharding a3e3b1f2f -> 963fe57c4


Added new classes for locating buckets using consistent hashing


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

Branch: refs/heads/key-row-sharding
Commit: 2df4770321d0fae921302bd3a2a945fa48522598
Parents: a3e3b1f
Author: Todd Nine <tn...@apigee.com>
Authored: Thu Oct 23 16:10:39 2014 -0600
Committer: Todd Nine <tn...@apigee.com>
Committed: Thu Oct 23 16:10:39 2014 -0600

----------------------------------------------------------------------
 .../core/astyanax/BucketLocator.java            |  67 ++++++++++++
 .../core/astyanax/BucketScopedRowKey.java       | 104 +++++++++++++++++++
 2 files changed, 171 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/2df47703/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java
new file mode 100644
index 0000000..49041f8
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java
@@ -0,0 +1,67 @@
+package org.apache.usergrid.persistence.core.astyanax;/*
+ * 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.
+ */
+
+
+import com.google.common.hash.Funnel;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+
+
+/**
+ * Simple utility to locate which bucket an element should be in
+ */
+public class BucketLocator<T> {
+
+    /**
+     * Use the murmur 3 hash
+     */
+    private static final HashFunction HASHER = Hashing.murmur3_128();
+
+    private final int totalBuckets;
+    private final Funnel<T> funnel;
+
+
+    public BucketLocator( final int totalBuckets, final Funnel<T> funnel ) {
+        this.totalBuckets = totalBuckets;
+        this.funnel = funnel;
+    }
+
+
+    /**
+     * Locate the bucket number given the value, the funnel and the total buckets.
+     *
+     * Assigns to {@code hashCode} a "bucket" in the range {@code [0, buckets)}, in a uniform manner that minimizes the
+     * need for remapping as {@code buckets} grows. That is, {@code consistentHash(h, n)} equals:
+     *
+     * <ul> <li>{@code n - 1}, with approximate probability {@code 1/n} <li>{@code consistentHash(h, n - 1)}, otherwise
+     * (probability {@code 1 - 1/n}) </ul>
+     *
+     * <p>See the <a href="http://en.wikipedia.org/wiki/Consistent_hashing">wikipedia article on consistent hashing</a>
+     * for more information.
+     */
+    public int getBucket( T value ) {
+
+        final HashCode hashCode = HASHER.hashObject( value, funnel );
+
+        int owningIndex = Hashing.consistentHash( hashCode, totalBuckets );
+
+        return owningIndex;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/2df47703/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
new file mode 100644
index 0000000..4ecddcf
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
@@ -0,0 +1,104 @@
+package org.apache.usergrid.persistence.core.astyanax;/*
+ * 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.
+ */
+
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+
+
+public class BucketScopedRowKey<S extends ApplicationScope, K> extends ScopedRowKey<S, K> {
+
+    private final int bucketNumber;
+
+
+    /**
+     * Create a scoped row key, with the funnel for determining the bucket
+     *
+     * @param bucketNumber The bucket number for this row key
+     */
+    public BucketScopedRowKey( final S scope, final K key, int bucketNumber ) {
+        super( scope, key );
+        this.bucketNumber = bucketNumber;
+    }
+
+
+    @Override
+    public boolean equals( final Object o ) {
+        if ( this == o ) {
+            return true;
+        }
+        if ( !( o instanceof BucketScopedRowKey ) ) {
+            return false;
+        }
+        if ( !super.equals( o ) ) {
+            return false;
+        }
+
+        final BucketScopedRowKey that = ( BucketScopedRowKey ) o;
+
+        if ( bucketNumber != that.bucketNumber ) {
+            return false;
+        }
+
+        return true;
+    }
+
+
+    @Override
+    public int hashCode() {
+        int result = super.hashCode();
+        result = 31 * result + bucketNumber;
+        return result;
+    }
+
+
+    @Override
+    public String toString() {
+        return "BucketScopedRowKey{" +
+                "bucketNumber=" + bucketNumber +
+                "} " + super.toString();
+    }
+
+
+    /**
+     * Utility function to generate a new key from the scope
+     */
+    public static <S extends ApplicationScope, K> BucketScopedRowKey<S, K> fromKey( final S scope, final K key,
+                                                                                    final int bucketNumber ) {
+        return new BucketScopedRowKey<>( scope, key, bucketNumber );
+    }
+
+
+    /**
+     * Create a list of all buckets from [0,  totalBuckets}.  Note that this is an n-1 0 based system
+     */
+    public static <S extends ApplicationScope, K> List<BucketScopedRowKey<S, K>> fromRange( final S scope, final K key, final int totalBuckets ) {
+
+        final List<BucketScopedRowKey<S, K>> results = new ArrayList<>( totalBuckets );
+
+
+        for ( int i = 0; i < totalBuckets; i++ ) {
+            results.add( new BucketScopedRowKey<>( scope, key, i ) );
+        }
+
+        return results;
+    }
+}


[3/7] git commit: Finished low level serialization refactor

Posted by to...@apache.org.
Finished low level serialization refactor


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

Branch: refs/heads/key-row-sharding
Commit: 5a56dd450b509d07abd9e363736a762093f864af
Parents: 2df4770
Author: Todd Nine <tn...@apigee.com>
Authored: Fri Oct 24 09:46:53 2014 -0600
Committer: Todd Nine <tn...@apigee.com>
Committed: Fri Oct 24 10:26:58 2014 -0600

----------------------------------------------------------------------
 .../impl/CollectionPrefixedKey.java             | 100 +++++++++++++++++++
 .../impl/CollectionScopedRowKeySerializer.java  |  25 +++--
 .../MvccEntitySerializationStrategyImpl.java    |  86 ++++++++++++----
 .../MvccLogEntrySerializationStrategyImpl.java  |  64 +++++++++---
 .../UniqueValueSerializationStrategyImpl.java   |  29 ++++--
 .../impl/ScopedRowKeySerializerTest.java        |  18 ++--
 .../core/astyanax/BucketScopedRowKey.java       |  20 +++-
 .../core/astyanax/MultiTennantColumnFamily.java |   8 +-
 ...rganizationScopedBucketRowKeySerializer.java |  93 +++++++++++++++++
 .../OrganizationScopedRowKeySerializer.java     |  12 ++-
 .../persistence/core/astyanax/ScopedRowKey.java |  13 +--
 .../impl/EdgeMetadataSerializationImpl.java     |  55 +++++-----
 .../impl/NodeSerializationImpl.java             |  28 +++---
 .../impl/shard/EdgeColumnFamilies.java          |  11 +-
 .../NodeShardCounterSerializationImpl.java      |   6 +-
 .../impl/shard/impl/EdgeSearcher.java           |   8 +-
 .../shard/impl/EdgeShardSerializationImpl.java  |  10 +-
 .../impl/ShardedEdgeSerializationImpl.java      |  76 +++++++-------
 .../impl/shard/impl/ShardsColumnIterator.java   |   8 +-
 .../shard/impl/SizebasedEdgeColumnFamilies.java |  21 ++--
 .../map/impl/MapSerializationImpl.java          |  81 ++++++++++-----
 21 files changed, 559 insertions(+), 213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionPrefixedKey.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionPrefixedKey.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionPrefixedKey.java
new file mode 100644
index 0000000..529e06c
--- /dev/null
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionPrefixedKey.java
@@ -0,0 +1,100 @@
+package org.apache.usergrid.persistence.collection.serialization.impl;/*
+ * 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.
+ */
+
+
+import org.apache.usergrid.persistence.model.entity.Id;
+
+
+/**
+ * Wrapper object to create collections in prefixes
+ */
+public class CollectionPrefixedKey<K> {
+
+    private final String collectionName;
+    private final Id owner;
+    private final K subKey;
+
+
+    public CollectionPrefixedKey( final String collectionName, final Id owner, final K subKey ) {
+        this.collectionName = collectionName;
+        this.owner = owner;
+        this.subKey = subKey;
+    }
+
+
+    /**
+     * Get the name of the
+     * @return
+     */
+    public String getCollectionName() {
+        return collectionName;
+    }
+
+
+    /**
+     * Get the owner of the collection
+     * @return
+     */
+    public Id getOwner() {
+        return owner;
+    }
+
+
+    /**
+     * Get the object to be used in the key
+     * @return
+     */
+    public K getSubKey() {
+        return subKey;
+    }
+
+
+    @Override
+    public boolean equals( final Object o ) {
+        if ( this == o ) {
+            return true;
+        }
+        if ( !( o instanceof CollectionPrefixedKey ) ) {
+            return false;
+        }
+
+        final CollectionPrefixedKey that = ( CollectionPrefixedKey ) o;
+
+        if ( !collectionName.equals( that.collectionName ) ) {
+            return false;
+        }
+        if ( !subKey.equals( that.subKey ) ) {
+            return false;
+        }
+        if ( !owner.equals( that.owner ) ) {
+            return false;
+        }
+
+        return true;
+    }
+
+
+    @Override
+    public int hashCode() {
+        int result = collectionName.hashCode();
+        result = 31 * result + owner.hashCode();
+        result = 31 * result + subKey.hashCode();
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionScopedRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionScopedRowKeySerializer.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionScopedRowKeySerializer.java
index 556a662..d560145 100644
--- a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionScopedRowKeySerializer.java
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/CollectionScopedRowKeySerializer.java
@@ -20,8 +20,6 @@ package org.apache.usergrid.persistence.collection.serialization.impl;
 
 import java.nio.ByteBuffer;
 
-import org.apache.usergrid.persistence.collection.CollectionScope;
-import org.apache.usergrid.persistence.collection.impl.CollectionScopeImpl;
 import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
 import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
@@ -37,7 +35,7 @@ import com.netflix.astyanax.serializers.AbstractSerializer;
  * Serializer for serializing CollectionScope + any type into row keys
  */
 public class CollectionScopedRowKeySerializer<K> 
-    extends AbstractSerializer<ScopedRowKey<CollectionScope, K>> {
+    extends AbstractSerializer<ScopedRowKey<CollectionPrefixedKey<K>>> {
 
     private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
 
@@ -52,21 +50,23 @@ public class CollectionScopedRowKeySerializer<K>
     }
 
     @Override
-    public ByteBuffer toByteBuffer( final ScopedRowKey<CollectionScope, K> scopedRowKey ) {
+    public ByteBuffer toByteBuffer( final ScopedRowKey<CollectionPrefixedKey<K>> scopedRowKey ) {
 
         final CompositeBuilder builder = Composites.newCompositeBuilder();
 
         //add the organization's id
-        ID_SER.toComposite( builder, scopedRowKey.getScope().getApplication() );
+        ID_SER.toComposite( builder, scopedRowKey.getScope() );
+
+        final CollectionPrefixedKey<K> key = scopedRowKey.getKey();
 
         //add the scope's owner id to the composite
-        ID_SER.toComposite( builder, scopedRowKey.getScope().getOwner() );
+        ID_SER.toComposite( builder, key.getOwner() );
 
         //add the scope's name
-        builder.addString( scopedRowKey.getScope().getName() );
+        builder.addString( key.getCollectionName() );
 
         //add the key type
-        keySerializer.toComposite( builder, scopedRowKey.getKey() );
+        keySerializer.toComposite( builder, key.getSubKey() );
 
         //addOtherComponents( builder, scopedRowKey );
 
@@ -74,7 +74,7 @@ public class CollectionScopedRowKeySerializer<K>
     }
 
     @Override
-    public ScopedRowKey<CollectionScope, K> fromByteBuffer( final ByteBuffer byteBuffer ) {
+    public ScopedRowKey<CollectionPrefixedKey<K>> fromByteBuffer( final ByteBuffer byteBuffer ) {
         final CompositeParser parser = Composites.newCompositeParser( byteBuffer );
 
         //read back the id
@@ -83,8 +83,11 @@ public class CollectionScopedRowKeySerializer<K>
         final String scopeName = parser.readString();
         final K value = keySerializer.fromComposite( parser );
 
-        return new ScopedRowKey<CollectionScope, K>( 
-                new CollectionScopeImpl( orgId, scopeId, scopeName ), value ); 
+
+        final CollectionPrefixedKey<K> collectionPrefixedKey = new CollectionPrefixedKey<>( scopeName,  scopeId, value );
+
+
+        return new ScopedRowKey<>( orgId, collectionPrefixedKey );
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/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 31c26a9..dfa41ea 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
@@ -36,9 +36,9 @@ import org.apache.cassandra.db.marshal.UUIDType;
 
 import org.apache.usergrid.persistence.collection.CollectionScope;
 import org.apache.usergrid.persistence.collection.EntitySet;
+import org.apache.usergrid.persistence.collection.MvccEntity;
 import org.apache.usergrid.persistence.collection.exception.CollectionRuntimeException;
 import org.apache.usergrid.persistence.collection.mvcc.MvccEntitySerializationStrategy;
-import org.apache.usergrid.persistence.collection.MvccEntity;
 import org.apache.usergrid.persistence.collection.mvcc.entity.impl.MvccEntityImpl;
 import org.apache.usergrid.persistence.collection.serialization.EntityRepair;
 import org.apache.usergrid.persistence.collection.serialization.SerializationFig;
@@ -96,9 +96,8 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
     private static final CollectionScopedRowKeySerializer<Id> ROW_KEY_SER =
             new CollectionScopedRowKeySerializer<Id>( ID_SER );
 
-    private static final MultiTennantColumnFamily<CollectionScope, Id, UUID> CF_ENTITY_DATA =
-            new MultiTennantColumnFamily<CollectionScope, Id, UUID>( "Entity_Version_Data", ROW_KEY_SER,
-                    UUIDSerializer.get() );
+    private static final MultiTennantColumnFamily<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> CF_ENTITY_DATA =
+            new MultiTennantColumnFamily<>( "Entity_Version_Data", ROW_KEY_SER, UUIDSerializer.get() );
 
 
     protected final Keyspace keyspace;
@@ -141,7 +140,6 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
     }
 
 
-
     @Override
     public EntitySet load( final CollectionScope collectionScope, final Collection<Id> entityIds,
                            final UUID maxVersion ) {
@@ -154,37 +152,49 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
 
 
         //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");
+        Preconditions.checkArgument( entityIds.size() <= serializationFig.getMaxLoadSize(),
+                "requested size cannot be over configured maximum" );
+
 
+        final Id applicationId = collectionScope.getApplication();
+        final Id ownerId = collectionScope.getOwner();
+        final String collectionName = collectionScope.getName();
 
 
-        final List<ScopedRowKey<CollectionScope, Id>> rowKeys = new ArrayList<>( entityIds.size() );
+        final List<ScopedRowKey<CollectionPrefixedKey<Id>>> rowKeys = new ArrayList<>( entityIds.size() );
 
 
         for ( final Id entityId : entityIds ) {
-            rowKeys.add( ScopedRowKey.fromKey( collectionScope, entityId ) );
-        }
+            final CollectionPrefixedKey<Id> collectionPrefixedKey =
+                    new CollectionPrefixedKey<>( collectionName, ownerId, entityId );
+
+
+            final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                    ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
 
 
-        final Iterator<Row<ScopedRowKey<CollectionScope, Id>, UUID>> latestEntityColumns;
+            rowKeys.add( rowKey );
+        }
+
 
+        final Iterator<Row<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID>> latestEntityColumns;
 
 
         try {
             latestEntityColumns = keyspace.prepareQuery( CF_ENTITY_DATA ).getKeySlice( rowKeys )
                                           .withColumnRange( maxVersion, null, false, 1 ).execute().getResult()
                                           .iterator();
-        } catch ( ConnectionException e ) {
+        }
+        catch ( ConnectionException e ) {
             throw new CollectionRuntimeException( null, collectionScope, "An error occurred connecting to cassandra",
                     e );
         }
 
 
-
         final EntitySetImpl entitySetResults = new EntitySetImpl( entityIds.size() );
 
         while ( latestEntityColumns.hasNext() ) {
-            final Row<ScopedRowKey<CollectionScope, Id>, UUID> row = latestEntityColumns.next();
+            final Row<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> row = latestEntityColumns.next();
 
             final ColumnList<UUID> columns = row.getColumns();
 
@@ -192,7 +202,7 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
                 continue;
             }
 
-            final Id entityId = row.getKey().getKey();
+            final Id entityId = row.getKey().getKey().getSubKey();
 
             final Column<UUID> column = columns.getColumnByIndex( 0 );
 
@@ -202,7 +212,6 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
             final MvccEntity maybeRepaired = repair.maybeRepair( collectionScope, parsedEntity );
 
             entitySetResults.addEntity( maybeRepaired );
-
         }
 
         return entitySetResults;
@@ -219,8 +228,20 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
         Preconditions.checkArgument( fetchSize > 0, "max Size must be greater than 0" );
 
 
-        RowQuery<ScopedRowKey<CollectionScope, Id>, UUID> query =
-                keyspace.prepareQuery( CF_ENTITY_DATA ).getKey( ScopedRowKey.fromKey( collectionScope, entityId ) )
+        final Id applicationId = collectionScope.getApplication();
+        final Id ownerId = collectionScope.getOwner();
+        final String collectionName = collectionScope.getName();
+
+        final CollectionPrefixedKey<Id> collectionPrefixedKey =
+                new CollectionPrefixedKey<>( collectionName, ownerId, entityId );
+
+
+        final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
+
+
+        RowQuery<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> query =
+                keyspace.prepareQuery( CF_ENTITY_DATA ).getKey( rowKey )
                         .withColumnRange( version, null, false, fetchSize );
 
         return new ColumnNameIterator( query, new MvccColumnParser( entityId ), false );
@@ -236,8 +257,21 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
         Preconditions.checkNotNull( version, "version is required" );
         Preconditions.checkArgument( fetchSize > 0, "max Size must be greater than 0" );
 
-        RowQuery<ScopedRowKey<CollectionScope, Id>, UUID> query =
-                keyspace.prepareQuery( CF_ENTITY_DATA ).getKey( ScopedRowKey.fromKey( collectionScope, entityId ) )
+
+        final Id applicationId = collectionScope.getApplication();
+        final Id ownerId = collectionScope.getOwner();
+        final String collectionName = collectionScope.getName();
+
+        final CollectionPrefixedKey<Id> collectionPrefixedKey =
+                new CollectionPrefixedKey<>( collectionName, ownerId, entityId );
+
+
+        final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
+
+
+        RowQuery<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> query =
+                keyspace.prepareQuery( CF_ENTITY_DATA ).getKey( rowKey )
                         .withColumnRange( null, version, true, fetchSize );
 
         return new ColumnNameIterator( query, new MvccColumnParser( entityId ), false );
@@ -299,7 +333,19 @@ public class MvccEntitySerializationStrategyImpl implements MvccEntitySerializat
     private MutationBatch doWrite( final CollectionScope collectionScope, final Id entityId, final RowOp op ) {
         final MutationBatch batch = keyspace.prepareMutationBatch();
 
-        op.doOp( batch.withRow( CF_ENTITY_DATA, ScopedRowKey.fromKey( collectionScope, entityId ) ) );
+        final Id applicationId = collectionScope.getApplication();
+        final Id ownerId = collectionScope.getOwner();
+        final String collectionName = collectionScope.getName();
+
+        final CollectionPrefixedKey<Id> collectionPrefixedKey =
+                new CollectionPrefixedKey<>( collectionName, ownerId, entityId );
+
+
+        final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
+
+
+        op.doOp( batch.withRow( CF_ENTITY_DATA, rowKey ) );
 
         return batch;
     }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLogEntrySerializationStrategyImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLogEntrySerializationStrategyImpl.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLogEntrySerializationStrategyImpl.java
index e4aeb0b..8577fbf 100644
--- a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLogEntrySerializationStrategyImpl.java
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/MvccLogEntrySerializationStrategyImpl.java
@@ -82,8 +82,8 @@ public class MvccLogEntrySerializationStrategyImpl implements MvccLogEntrySerial
     private static final CollectionScopedRowKeySerializer<Id> ROW_KEY_SER =
             new CollectionScopedRowKeySerializer<Id>( ID_SER );
 
-    private static final MultiTennantColumnFamily<CollectionScope, Id, UUID> CF_ENTITY_LOG =
-            new MultiTennantColumnFamily<CollectionScope, Id, UUID>( "Entity_Log", ROW_KEY_SER, UUIDSerializer.get() );
+    private static final MultiTennantColumnFamily<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> CF_ENTITY_LOG =
+            new MultiTennantColumnFamily<>( "Entity_Log", ROW_KEY_SER, UUIDSerializer.get() );
 
 
     protected final Keyspace keyspace;
@@ -139,15 +139,28 @@ public class MvccLogEntrySerializationStrategyImpl implements MvccLogEntrySerial
                 "requested size cannot be over configured maximum" );
 
 
-        final List<ScopedRowKey<CollectionScope, Id>> rowKeys = new ArrayList<>( entityIds.size() );
+        final Id applicationId = collectionScope.getApplication();
+        final Id ownerId = collectionScope.getOwner();
+        final String collectionName = collectionScope.getName();
+
+
+        final List<ScopedRowKey<CollectionPrefixedKey<Id>>> rowKeys = new ArrayList<>( entityIds.size() );
 
 
         for ( final Id entityId : entityIds ) {
-            rowKeys.add( ScopedRowKey.fromKey( collectionScope, entityId ) );
+            final CollectionPrefixedKey<Id> collectionPrefixedKey =
+                    new CollectionPrefixedKey<>( collectionName, ownerId, entityId );
+
+
+            final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                    ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
+
+
+            rowKeys.add( rowKey );
         }
 
 
-        final Iterator<Row<ScopedRowKey<CollectionScope, Id>, UUID>> latestEntityColumns;
+        final Iterator<Row<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID>> latestEntityColumns;
 
 
         try {
@@ -164,7 +177,7 @@ public class MvccLogEntrySerializationStrategyImpl implements MvccLogEntrySerial
         final VersionSetImpl versionResults = new VersionSetImpl( entityIds.size() );
 
         while ( latestEntityColumns.hasNext() ) {
-            final Row<ScopedRowKey<CollectionScope, Id>, UUID> row = latestEntityColumns.next();
+            final Row<ScopedRowKey<CollectionPrefixedKey<Id>>, UUID> row = latestEntityColumns.next();
 
             final ColumnList<UUID> columns = row.getColumns();
 
@@ -173,7 +186,7 @@ public class MvccLogEntrySerializationStrategyImpl implements MvccLogEntrySerial
             }
 
 
-            final Id entityId = row.getKey().getKey();
+            final Id entityId = row.getKey().getKey().getSubKey();
 
             final Column<UUID> column = columns.getColumnByIndex( 0 );
 
@@ -202,9 +215,23 @@ public class MvccLogEntrySerializationStrategyImpl implements MvccLogEntrySerial
         Preconditions.checkArgument( maxSize > 0, "max Size must be greater than 0" );
 
 
-        ColumnList<UUID> columns = null;
+        ColumnList<UUID> columns;
         try {
-            columns = keyspace.prepareQuery( CF_ENTITY_LOG ).getKey( ScopedRowKey.fromKey( collectionScope, entityId ) )
+
+            final Id applicationId = collectionScope.getApplication();
+            final Id ownerId = collectionScope.getOwner();
+            final String collectionName = collectionScope.getName();
+
+
+            final CollectionPrefixedKey<Id> collectionPrefixedKey =
+                    new CollectionPrefixedKey<>( collectionName, ownerId, entityId );
+
+
+            final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                    ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
+
+
+            columns = keyspace.prepareQuery( CF_ENTITY_LOG ).getKey( rowKey )
                               .withColumnRange( version, null, false, maxSize ).execute().getResult();
         }
         catch ( ConnectionException e ) {
@@ -270,9 +297,9 @@ public class MvccLogEntrySerializationStrategyImpl implements MvccLogEntrySerial
     /**
      * Do the column update or delete for the given column and row key
      *
-     * @param context We need to use this when getting the keyspace
+     * @param collectionScope We need to use this when getting the keyspace
      */
-    private MutationBatch doWrite( CollectionScope context, Id entityId, UUID version, RowOp op ) {
+    private MutationBatch doWrite( CollectionScope collectionScope, Id entityId, UUID version, RowOp op ) {
 
         final MutationBatch batch = keyspace.prepareMutationBatch();
 
@@ -280,7 +307,20 @@ public class MvccLogEntrySerializationStrategyImpl implements MvccLogEntrySerial
 
         LOG.debug( "Writing version with timestamp '{}'", timestamp );
 
-        op.doOp( batch.withRow( CF_ENTITY_LOG, ScopedRowKey.fromKey( context, entityId ) ) );
+        final Id applicationId = collectionScope.getApplication();
+        final Id ownerId = collectionScope.getOwner();
+        final String collectionName = collectionScope.getName();
+
+
+        final CollectionPrefixedKey<Id> collectionPrefixedKey =
+                new CollectionPrefixedKey<>( collectionName, ownerId, entityId );
+
+
+        final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
+
+
+        op.doOp( batch.withRow( CF_ENTITY_LOG, rowKey ) );
 
         return batch;
     }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/UniqueValueSerializationStrategyImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/UniqueValueSerializationStrategyImpl.java b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/UniqueValueSerializationStrategyImpl.java
index a3e4adc..954cd1c 100644
--- a/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/UniqueValueSerializationStrategyImpl.java
+++ b/stack/corepersistence/collection/src/main/java/org/apache/usergrid/persistence/collection/serialization/impl/UniqueValueSerializationStrategyImpl.java
@@ -50,7 +50,6 @@ import com.netflix.astyanax.Keyspace;
 import com.netflix.astyanax.MutationBatch;
 import com.netflix.astyanax.connectionpool.exceptions.ConnectionException;
 import com.netflix.astyanax.model.Column;
-import com.netflix.astyanax.model.ColumnList;
 import com.netflix.astyanax.model.Row;
 import com.netflix.astyanax.util.RangeBuilder;
 
@@ -68,8 +67,8 @@ public class UniqueValueSerializationStrategyImpl implements UniqueValueSerializ
 
     private static final EntityVersionSerializer ENTITY_VERSION_SER = new EntityVersionSerializer();
 
-    private static final MultiTennantColumnFamily<CollectionScope, Field, EntityVersion> CF_UNIQUE_VALUES =
-            new MultiTennantColumnFamily<CollectionScope, Field, EntityVersion>( "Unique_Values", ROW_KEY_SER,
+    private static final MultiTennantColumnFamily<ScopedRowKey<CollectionPrefixedKey<Field>>, EntityVersion> CF_UNIQUE_VALUES =
+            new MultiTennantColumnFamily<>( "Unique_Values", ROW_KEY_SER,
                     ENTITY_VERSION_SER );
 
     protected final Keyspace keyspace;
@@ -164,7 +163,10 @@ public class UniqueValueSerializationStrategyImpl implements UniqueValueSerializ
      */
     private MutationBatch doWrite( CollectionScope context, Field field, RowOp op ) {
         final MutationBatch batch = keyspace.prepareMutationBatch();
-        op.doOp( batch.withRow( CF_UNIQUE_VALUES, ScopedRowKey.fromKey( context, field ) ) );
+        final CollectionPrefixedKey<Field> collectionPrefixedKey = new CollectionPrefixedKey<>( context.getName(), context.getOwner(), field );
+
+
+        op.doOp( batch.withRow( CF_UNIQUE_VALUES, ScopedRowKey.fromKey( context.getApplication(), collectionPrefixedKey ) ) );
         return batch;
     }
 
@@ -176,17 +178,26 @@ public class UniqueValueSerializationStrategyImpl implements UniqueValueSerializ
         Preconditions.checkNotNull( fields, "fields are required" );
         Preconditions.checkArgument( fields.size() > 0, "More than 1 field msut be specified" );
 
-        final List<ScopedRowKey<CollectionScope, Field>> keys = new ArrayList<>( fields.size() );
+
+        final List<ScopedRowKey<CollectionPrefixedKey<Field>>> keys = new ArrayList<>( fields.size() );
+
+        final Id applicationId = colScope.getApplication();
+        final Id ownerId = colScope.getOwner();
+        final String collectionName = colScope.getName();
 
         for ( Field field : fields ) {
-            final ScopedRowKey<CollectionScope, Field> rowKey = ScopedRowKey.fromKey( colScope, field );
+
+            final CollectionPrefixedKey<Field> collectionPrefixedKey = new CollectionPrefixedKey<>( collectionName, ownerId, field );
+
+
+            final ScopedRowKey<CollectionPrefixedKey<Field>> rowKey = ScopedRowKey.fromKey(applicationId, collectionPrefixedKey );
 
             keys.add( rowKey );
         }
 
         final UniqueValueSetImpl uniqueValueSet = new UniqueValueSetImpl( fields.size() );
 
-        Iterator<Row<ScopedRowKey<CollectionScope, Field>, EntityVersion>> results =
+        Iterator<Row<ScopedRowKey<CollectionPrefixedKey<Field>>, EntityVersion>> results =
                 keyspace.prepareQuery( CF_UNIQUE_VALUES ).getKeySlice( keys )
                         .withColumnRange( new RangeBuilder().setLimit( 1 ).build() ).execute().getResult().iterator();
 
@@ -195,10 +206,10 @@ public class UniqueValueSerializationStrategyImpl implements UniqueValueSerializ
 
         {
 
-            final Row<ScopedRowKey<CollectionScope, Field>, EntityVersion> unique = results.next();
+            final Row<ScopedRowKey<CollectionPrefixedKey<Field>>, EntityVersion> unique = results.next();
 
 
-            final Field field = unique.getKey().getKey();
+            final Field field = unique.getKey().getKey().getSubKey();
 
             final Iterator<Column<EntityVersion>> columnList = unique.getColumns().iterator();
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/ScopedRowKeySerializerTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/ScopedRowKeySerializerTest.java b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/ScopedRowKeySerializerTest.java
index ef2df0e..a09b623 100644
--- a/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/ScopedRowKeySerializerTest.java
+++ b/stack/corepersistence/collection/src/test/java/org/apache/usergrid/persistence/collection/serialization/impl/ScopedRowKeySerializerTest.java
@@ -24,8 +24,6 @@ import java.nio.ByteBuffer;
 
 import org.junit.Test;
 
-import org.apache.usergrid.persistence.collection.CollectionScope;
-import org.apache.usergrid.persistence.collection.impl.CollectionScopeImpl;
 import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.model.entity.Id;
@@ -43,20 +41,22 @@ public class ScopedRowKeySerializerTest {
         final Id testId = new SimpleId( "scopeType" );
         final String name = "scopeName";
         final Id testKey = new SimpleId( "testKey" );
+        final Id applicationId = new SimpleId( "application" );
 
-        final CollectionScope collectionScope = new CollectionScopeImpl(new SimpleId( "organization" ), testId, name );
-        final ScopedRowKey<CollectionScope, Id>
-                rowKey = new ScopedRowKey<CollectionScope, Id>( collectionScope, testKey );
+        final CollectionPrefixedKey<Id> collectionPrefixedKey = new CollectionPrefixedKey<>( name, testId, testKey );
 
 
-        CollectionScopedRowKeySerializer<Id> collectionScopedRowKeySerializer = 
+        final ScopedRowKey<CollectionPrefixedKey<Id>> rowKey =
+                ScopedRowKey.fromKey( applicationId, collectionPrefixedKey );
+
+
+        CollectionScopedRowKeySerializer<Id> collectionScopedRowKeySerializer =
                 new CollectionScopedRowKeySerializer<Id>( IdRowCompositeSerializer.get() );
 
         ByteBuffer buff = collectionScopedRowKeySerializer.toByteBuffer( rowKey );
 
-        ScopedRowKey<CollectionScope, Id> parsedRowKey = collectionScopedRowKeySerializer.fromByteBuffer( buff );
-
-        assertEquals("Row key serialized correctly", rowKey, parsedRowKey);
+        ScopedRowKey<CollectionPrefixedKey<Id>> parsedRowKey = collectionScopedRowKeySerializer.fromByteBuffer( buff );
 
+        assertEquals( "Row key serialized correctly", rowKey, parsedRowKey );
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
index 4ecddcf..0687a7a 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
@@ -22,9 +22,10 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.model.entity.Id;
 
 
-public class BucketScopedRowKey<S extends ApplicationScope, K> extends ScopedRowKey<S, K> {
+public class BucketScopedRowKey< K> extends ScopedRowKey< K> {
 
     private final int bucketNumber;
 
@@ -34,12 +35,21 @@ public class BucketScopedRowKey<S extends ApplicationScope, K> extends ScopedRow
      *
      * @param bucketNumber The bucket number for this row key
      */
-    public BucketScopedRowKey( final S scope, final K key, int bucketNumber ) {
+    public BucketScopedRowKey( final Id scope, final K key, int bucketNumber ) {
         super( scope, key );
         this.bucketNumber = bucketNumber;
     }
 
 
+    /**
+     * Get the bucket number
+     * @return
+     */
+    public int getBucketNumber() {
+        return bucketNumber;
+    }
+
+
     @Override
     public boolean equals( final Object o ) {
         if ( this == o ) {
@@ -81,7 +91,7 @@ public class BucketScopedRowKey<S extends ApplicationScope, K> extends ScopedRow
     /**
      * Utility function to generate a new key from the scope
      */
-    public static <S extends ApplicationScope, K> BucketScopedRowKey<S, K> fromKey( final S scope, final K key,
+    public static < K> BucketScopedRowKey< K> fromKey( final Id scope, final K key,
                                                                                     final int bucketNumber ) {
         return new BucketScopedRowKey<>( scope, key, bucketNumber );
     }
@@ -90,9 +100,9 @@ public class BucketScopedRowKey<S extends ApplicationScope, K> extends ScopedRow
     /**
      * Create a list of all buckets from [0,  totalBuckets}.  Note that this is an n-1 0 based system
      */
-    public static <S extends ApplicationScope, K> List<BucketScopedRowKey<S, K>> fromRange( final S scope, final K key, final int totalBuckets ) {
+    public static < K> List<BucketScopedRowKey< K>> fromRange( final Id scope, final K key, final int totalBuckets ) {
 
-        final List<BucketScopedRowKey<S, K>> results = new ArrayList<>( totalBuckets );
+        final List<BucketScopedRowKey< K>> results = new ArrayList<>( totalBuckets );
 
 
         for ( int i = 0; i < totalBuckets; i++ ) {

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiTennantColumnFamily.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiTennantColumnFamily.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiTennantColumnFamily.java
index 6234184..faaa76d 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiTennantColumnFamily.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiTennantColumnFamily.java
@@ -30,17 +30,17 @@ import com.netflix.astyanax.model.ColumnFamily;
  *
  * @author tnine
  */
-public class MultiTennantColumnFamily<S extends ApplicationScope, K, V>
-    extends ColumnFamily<ScopedRowKey<S, K>, V> {
+public class MultiTennantColumnFamily<R extends ScopedRowKey<?>, V >
+    extends ColumnFamily<R, V> {
 
-    public MultiTennantColumnFamily( final String columnFamilyName, final Serializer<ScopedRowKey<S, K>> keySerializer,
+    public MultiTennantColumnFamily( final String columnFamilyName, final Serializer<R> keySerializer,
                                      final Serializer<V> columnSerializer ) {
 
         super( columnFamilyName, keySerializer, columnSerializer );
     }
 
 
-    public MultiTennantColumnFamily( final String columnFamilyName, final Serializer<ScopedRowKey<S, K>> keySerializer,
+    public MultiTennantColumnFamily( final String columnFamilyName, final Serializer<R> keySerializer,
                                      final Serializer<V> columnSerializer, final Serializer<?> defaultValueSerializer ) {
 
         super( columnFamilyName, keySerializer, columnSerializer, defaultValueSerializer );

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java
new file mode 100644
index 0000000..8fddedd
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java
@@ -0,0 +1,93 @@
+/*
+ * 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.astyanax;
+
+
+import java.nio.ByteBuffer;
+
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.core.scope.ApplicationScopeImpl;
+import org.apache.usergrid.persistence.model.entity.Id;
+
+import com.netflix.astyanax.model.CompositeBuilder;
+import com.netflix.astyanax.model.CompositeParser;
+import com.netflix.astyanax.model.Composites;
+import com.netflix.astyanax.serializers.AbstractSerializer;
+import com.netflix.astyanax.serializers.IntegerSerializer;
+
+
+/**
+ * Serializer for serializing CollectionScope + any type into row keys
+ *
+ * @author tnine
+ */
+public class OrganizationScopedBucketRowKeySerializer<K> extends AbstractSerializer<BucketScopedRowKey<K>> {
+
+
+    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
+
+
+    /**
+     * The delegate serializer for the key
+     */
+    private final CompositeFieldSerializer<K> keySerializer;
+
+
+
+
+    public OrganizationScopedBucketRowKeySerializer( final CompositeFieldSerializer<K> keySerializer ) {
+        this.keySerializer = keySerializer;
+    }
+
+
+    @Override
+    public ByteBuffer toByteBuffer( final BucketScopedRowKey<K> scopedRowKey ) {
+
+        final CompositeBuilder builder = Composites.newCompositeBuilder();
+
+        //add the organization's id
+        ID_SER.toComposite( builder, scopedRowKey.getScope() );
+
+        //add the bucket
+        builder.addInteger( scopedRowKey.getBucketNumber() );
+
+        //add the key type
+        keySerializer.toComposite( builder, scopedRowKey.getKey() );
+
+        return builder.build();
+    }
+
+
+    @Override
+    public BucketScopedRowKey<K> fromByteBuffer( final ByteBuffer byteBuffer ) {
+        final CompositeParser parser = Composites.newCompositeParser( byteBuffer );
+
+        //read back the id
+        final Id orgId = ID_SER.fromComposite( parser );
+
+        final int bucket = parser.readInteger();
+
+        final K value = keySerializer.fromComposite( parser );
+
+        return new BucketScopedRowKey<>( orgId, value, bucket );
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java
index bdc76dd..4289628 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java
@@ -37,7 +37,7 @@ import com.netflix.astyanax.serializers.AbstractSerializer;
  *
  * @author tnine
  */
-public class OrganizationScopedRowKeySerializer<K> extends AbstractSerializer<ScopedRowKey<ApplicationScope, K>> {
+public class OrganizationScopedRowKeySerializer<K> extends AbstractSerializer<ScopedRowKey<K>> {
 
 
     private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
@@ -49,18 +49,20 @@ public class OrganizationScopedRowKeySerializer<K> extends AbstractSerializer<Sc
     private final CompositeFieldSerializer<K> keySerializer;
 
 
+
+
     public OrganizationScopedRowKeySerializer( final CompositeFieldSerializer<K> keySerializer ) {
         this.keySerializer = keySerializer;
     }
 
 
     @Override
-    public ByteBuffer toByteBuffer( final ScopedRowKey<ApplicationScope, K> scopedRowKey ) {
+    public ByteBuffer toByteBuffer( final ScopedRowKey< K> scopedRowKey ) {
 
         final CompositeBuilder builder = Composites.newCompositeBuilder();
 
         //add the organization's id
-        ID_SER.toComposite( builder, scopedRowKey.getScope().getApplication() );
+        ID_SER.toComposite( builder, scopedRowKey.getScope() );
 
         //add the key type
         keySerializer.toComposite( builder, scopedRowKey.getKey() );
@@ -70,7 +72,7 @@ public class OrganizationScopedRowKeySerializer<K> extends AbstractSerializer<Sc
 
 
     @Override
-    public ScopedRowKey<ApplicationScope, K> fromByteBuffer( final ByteBuffer byteBuffer ) {
+    public ScopedRowKey<K> fromByteBuffer( final ByteBuffer byteBuffer ) {
         final CompositeParser parser = Composites.newCompositeParser( byteBuffer );
 
         //read back the id
@@ -78,7 +80,7 @@ public class OrganizationScopedRowKeySerializer<K> extends AbstractSerializer<Sc
 
         final K value = keySerializer.fromComposite( parser );
 
-        return new ScopedRowKey<ApplicationScope, K>( new ApplicationScopeImpl( orgId), value );
+        return new ScopedRowKey<K>(  orgId, value );
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKey.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKey.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKey.java
index 073abcd..2fe7259 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKey.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKey.java
@@ -20,6 +20,7 @@ package org.apache.usergrid.persistence.core.astyanax;
 
 
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.model.entity.Id;
 
 import com.google.common.base.Preconditions;
 
@@ -30,14 +31,14 @@ import com.google.common.base.Preconditions;
  *
  * @author tnine
  */
-public class ScopedRowKey<S extends ApplicationScope, K> {
+public class ScopedRowKey< K> {
 
-    private final S scope;
+    private final Id scope;
 
     private final K key;
 
 
-    public ScopedRowKey( final S scope, final K key ) {
+    public ScopedRowKey( final Id scope, final K key ) {
         Preconditions.checkNotNull( scope, "CollectionScope is required" );
         Preconditions.checkNotNull( key, "Key is required" );
 
@@ -49,7 +50,7 @@ public class ScopedRowKey<S extends ApplicationScope, K> {
     /**
      * Get the stored scope
      */
-    public S getScope() {
+    public Id getScope() {
         return scope;
     }
 
@@ -104,7 +105,7 @@ public class ScopedRowKey<S extends ApplicationScope, K> {
     /**
      * Utility function to generate a new key from the scope
      */
-    public static <S extends ApplicationScope, K> ScopedRowKey<S, K> fromKey( final S scope, K key ) {
-        return new ScopedRowKey<S, K>( scope, key );
+    public static <K> ScopedRowKey< K> fromKey( final Id scope, K key ) {
+        return new ScopedRowKey<>( scope, key );
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
index 924eddb..131e426 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
@@ -85,26 +85,26 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
     /**
      * CFs where the row key contains the source node id
      */
-    private static final MultiTennantColumnFamily<ApplicationScope, Id, String> CF_SOURCE_EDGE_TYPES =
-            new MultiTennantColumnFamily<ApplicationScope, Id, String>( "Graph_Source_Edge_Types", ROW_KEY_SER,
+    private static final MultiTennantColumnFamily<ScopedRowKey<Id>, String> CF_SOURCE_EDGE_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Types", ROW_KEY_SER,
                     STRING_SERIALIZER );
 
     //all target id types for source edge type
-    private static final MultiTennantColumnFamily<ApplicationScope, EdgeIdTypeKey, String> CF_SOURCE_EDGE_ID_TYPES =
-            new MultiTennantColumnFamily<ApplicationScope, EdgeIdTypeKey, String>( "Graph_Source_Edge_Id_Types",
+    private static final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> CF_SOURCE_EDGE_ID_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Id_Types",
                     EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
 
     /**
      * CFs where the row key is the target node id
      */
-    private static final MultiTennantColumnFamily<ApplicationScope, Id, String> CF_TARGET_EDGE_TYPES =
-            new MultiTennantColumnFamily<ApplicationScope, Id, String>( "Graph_Target_Edge_Types", ROW_KEY_SER,
+    private static final MultiTennantColumnFamily<ScopedRowKey<Id>, String> CF_TARGET_EDGE_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Types", ROW_KEY_SER,
                     STRING_SERIALIZER );
 
 
     //all source id types for target edge type
-    private static final MultiTennantColumnFamily<ApplicationScope, EdgeIdTypeKey, String> CF_TARGET_EDGE_ID_TYPES =
-            new MultiTennantColumnFamily<ApplicationScope, EdgeIdTypeKey, String>( "Graph_Target_Edge_Id_Types",
+    private static final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> CF_TARGET_EDGE_ID_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Id_Types",
                     EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
 
 
@@ -133,7 +133,7 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
         ValidationUtils.validateApplicationScope( scope );
         GraphValidation.validateEdge( edge );
 
-
+        final Id scopeId = scope.getApplication();
         final Id source = edge.getSourceNode();
         final Id target = edge.getTargetNode();
         final String edgeType = edge.getType();
@@ -143,31 +143,32 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
                                             .withTimestamp( timestamp );
 
 
+
         //add source->target edge type to meta data
-        final ScopedRowKey<ApplicationScope, Id> sourceKey = new ScopedRowKey<ApplicationScope, Id>( scope, source );
+        final ScopedRowKey< Id> sourceKey = new ScopedRowKey<>( scopeId, source );
 
         batch.withRow( CF_SOURCE_EDGE_TYPES, sourceKey ).putColumn( edgeType, HOLDER );
 
 
         //write source->target edge type and id type to meta data
         EdgeIdTypeKey tk = new EdgeIdTypeKey( source, edgeType );
-        final ScopedRowKey<ApplicationScope, EdgeIdTypeKey> sourceTypeKey =
-                new ScopedRowKey<ApplicationScope, EdgeIdTypeKey>( scope, tk );
+        final ScopedRowKey<EdgeIdTypeKey> sourceTypeKey =
+                new ScopedRowKey<>( scopeId, tk );
 
 
         batch.withRow( CF_SOURCE_EDGE_ID_TYPES, sourceTypeKey ).putColumn( target.getType(), HOLDER );
 
 
         //write target<--source edge type meta data
-        final ScopedRowKey<ApplicationScope, Id> targetKey = new ScopedRowKey<ApplicationScope, Id>( scope, target );
+        final ScopedRowKey< Id> targetKey = new ScopedRowKey<>( scopeId, target );
 
 
         batch.withRow( CF_TARGET_EDGE_TYPES, targetKey ).putColumn( edgeType, HOLDER );
 
 
         //write target<--source edge type and id type to meta data
-        final ScopedRowKey<ApplicationScope, EdgeIdTypeKey> targetTypeKey =
-                new ScopedRowKey<ApplicationScope, EdgeIdTypeKey>( scope, new EdgeIdTypeKey( target, edgeType ) );
+        final ScopedRowKey<EdgeIdTypeKey> targetTypeKey =
+                new ScopedRowKey<>( scopeId, new EdgeIdTypeKey( target, edgeType ) );
 
 
         batch.withRow( CF_TARGET_EDGE_ID_TYPES, targetTypeKey ).putColumn( source.getType(), HOLDER );
@@ -242,11 +243,11 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
      */
     private MutationBatch removeEdgeType( final ApplicationScope scope, final Id rowKeyId, final String edgeType,
                                           final long version,
-                                          final MultiTennantColumnFamily<ApplicationScope, Id, String> cf ) {
+                                          final MultiTennantColumnFamily<ScopedRowKey<Id>, String> cf ) {
 
 
         //write target<--source edge type meta data
-        final ScopedRowKey<ApplicationScope, Id> rowKey = new ScopedRowKey<ApplicationScope, Id>( scope, rowKeyId );
+        final ScopedRowKey< Id> rowKey = new ScopedRowKey< Id>( scope.getApplication(), rowKeyId );
 
         final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
 
@@ -270,15 +271,15 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
      */
     private MutationBatch removeIdType( final ApplicationScope scope, final Id rowId, final String idType,
                                         final String edgeType, final long version,
-                                        final MultiTennantColumnFamily<ApplicationScope, EdgeIdTypeKey, String> cf ) {
+                                        final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> cf ) {
 
 
         final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
 
 
         //write target<--source edge type and id type to meta data
-        final ScopedRowKey<ApplicationScope, EdgeIdTypeKey> rowKey =
-                new ScopedRowKey<ApplicationScope, EdgeIdTypeKey>( scope, new EdgeIdTypeKey( rowId, edgeType ) );
+        final ScopedRowKey< EdgeIdTypeKey> rowKey =
+                new ScopedRowKey<>( scope.getApplication(), new EdgeIdTypeKey( rowId, edgeType ) );
 
 
         batch.withRow( cf, rowKey ).deleteColumn( idType );
@@ -313,12 +314,12 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
      * @param cf The column family to execute on
      */
     private Iterator<String> getEdgeTypes( final ApplicationScope scope, final SearchEdgeType search,
-                                           final MultiTennantColumnFamily<ApplicationScope, Id, String> cf ) {
+                                           final MultiTennantColumnFamily<ScopedRowKey<Id>, String> cf ) {
         ValidationUtils.validateApplicationScope( scope );
         GraphValidation.validateSearchEdgeType( search );
 
 
-        final ScopedRowKey<ApplicationScope, Id> sourceKey = new ScopedRowKey<>( scope, search.getNode() );
+        final ScopedRowKey< Id> sourceKey = new ScopedRowKey<>( scope.getApplication(), search.getNode() );
 
 
         //resume from the last if specified.  Also set the range
@@ -326,7 +327,7 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
 
         final RangeBuilder rangeBuilder = createRange( search );
 
-        RowQuery<ScopedRowKey<ApplicationScope, Id>, String> query =
+        RowQuery<ScopedRowKey<Id>, String> query =
                 keyspace.prepareQuery( cf ).getKey( sourceKey ).autoPaginate( true )
                         .withColumnRange( rangeBuilder.build() );
 
@@ -348,19 +349,19 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
      * @param cf The column family to search
      */
     public Iterator<String> getIdTypes( final ApplicationScope scope, final SearchIdType search,
-                                        final MultiTennantColumnFamily<ApplicationScope, EdgeIdTypeKey, String> cf ) {
+                                        final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> cf ) {
         ValidationUtils.validateApplicationScope( scope );
         GraphValidation.validateSearchEdgeIdType( search );
 
 
-        final ScopedRowKey<ApplicationScope, EdgeIdTypeKey> sourceTypeKey =
-                new ScopedRowKey<>( scope, new EdgeIdTypeKey( search.getNode(), search.getEdgeType() ) );
+        final ScopedRowKey<EdgeIdTypeKey> sourceTypeKey =
+                new ScopedRowKey<>( scope.getApplication(), new EdgeIdTypeKey( search.getNode(), search.getEdgeType() ) );
 
 
         final RangeBuilder rangeBuilder = createRange( search );
 
 
-        RowQuery<ScopedRowKey<ApplicationScope, EdgeIdTypeKey>, String> query =
+        RowQuery<ScopedRowKey<EdgeIdTypeKey>, String> query =
                 keyspace.prepareQuery( cf ).getKey( sourceTypeKey ).autoPaginate( true )
                         .withColumnRange( rangeBuilder.build() );
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
index b14fad6..0febf96 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
@@ -88,8 +88,8 @@ public class NodeSerializationImpl implements NodeSerialization, Migration {
      * BloomFilter on read.  This means our performance will be no worse than checking a distributed cache in RAM for
      * the existence of a marked node.
      */
-    private static final MultiTennantColumnFamily<ApplicationScope, Id, Boolean> GRAPH_DELETE =
-            new MultiTennantColumnFamily<ApplicationScope, Id, Boolean>( "Graph_Marked_Nodes",
+    private static final MultiTennantColumnFamily<ScopedRowKey<Id>, Boolean> GRAPH_DELETE =
+            new MultiTennantColumnFamily<>( "Graph_Marked_Nodes",
                     new OrganizationScopedRowKeySerializer<Id>( ROW_SERIALIZER ), BOOLEAN_SERIALIZER );
 
 
@@ -121,7 +121,7 @@ public class NodeSerializationImpl implements NodeSerialization, Migration {
 
         MutationBatch batch = keyspace.prepareMutationBatch().withConsistencyLevel( fig.getWriteCL() );
 
-        batch.withRow( GRAPH_DELETE, ScopedRowKey.fromKey( scope, node ) ).setTimestamp( timestamp )
+        batch.withRow( GRAPH_DELETE, ScopedRowKey.fromKey( scope.getApplication(), node ) ).setTimestamp( timestamp )
              .putColumn( COLUMN_NAME, timestamp );
 
         return batch;
@@ -136,7 +136,7 @@ public class NodeSerializationImpl implements NodeSerialization, Migration {
 
         MutationBatch batch = keyspace.prepareMutationBatch().withConsistencyLevel( fig.getWriteCL() );
 
-        batch.withRow( GRAPH_DELETE, ScopedRowKey.fromKey( scope, node ) ).setTimestamp( timestamp )
+        batch.withRow( GRAPH_DELETE, ScopedRowKey.fromKey( scope.getApplication(), node ) ).setTimestamp( timestamp )
              .deleteColumn( COLUMN_NAME );
 
         return batch;
@@ -148,13 +148,13 @@ public class NodeSerializationImpl implements NodeSerialization, Migration {
         ValidationUtils.validateApplicationScope( scope );
         ValidationUtils.verifyIdentity( node );
 
-        ColumnFamilyQuery<ScopedRowKey<ApplicationScope, Id>, Boolean> query =
+        ColumnFamilyQuery<ScopedRowKey<Id>, Boolean> query =
                 keyspace.prepareQuery( GRAPH_DELETE ).setConsistencyLevel( fig.getReadCL() );
 
 
         try {
             Column<Boolean> result = HystrixCassandra
-                    .user( query.getKey( ScopedRowKey.fromKey( scope, node ) ).getColumn( COLUMN_NAME ) )
+                    .user( query.getKey( ScopedRowKey.fromKey( scope.getApplication(), node ) ).getColumn( COLUMN_NAME ) )
                     .getResult();
 
             return Optional.of( result.getLongValue() );
@@ -177,27 +177,29 @@ public class NodeSerializationImpl implements NodeSerialization, Migration {
         Preconditions.checkNotNull( edges, "edges cannot be null" );
 
 
-        final ColumnFamilyQuery<ScopedRowKey<ApplicationScope, Id>, Boolean> query =
+        final ColumnFamilyQuery<ScopedRowKey< Id>, Boolean> query =
                 keyspace.prepareQuery( GRAPH_DELETE ).setConsistencyLevel( fig.getReadCL() );
 
 
-        final List<ScopedRowKey<ApplicationScope, Id>> keys =
-                new ArrayList<ScopedRowKey<ApplicationScope, Id>>( edges.size() );
+        final List<ScopedRowKey< Id>> keys =
+                new ArrayList<>( edges.size() );
 
         //worst case all are marked
         final Map<Id, Long> versions = new HashMap<>( edges.size() );
 
+        final Id scopeId = scope.getApplication();
+
         for ( final Edge edge : edges ) {
-            keys.add( ScopedRowKey.fromKey( scope, edge.getSourceNode() ) );
-            keys.add( ScopedRowKey.fromKey( scope, edge.getTargetNode() ) );
+            keys.add( ScopedRowKey.fromKey( scopeId, edge.getSourceNode() ) );
+            keys.add( ScopedRowKey.fromKey( scopeId, edge.getTargetNode() ) );
         }
 
 
-        final Rows<ScopedRowKey<ApplicationScope, Id>, Boolean> results = HystrixCassandra
+        final Rows<ScopedRowKey<Id>, Boolean> results = HystrixCassandra
                 .user( query.getRowSlice( keys ).withColumnSlice( Collections.singletonList( COLUMN_NAME ) ) )
                 .getResult();
 
-        for ( Row<ScopedRowKey<ApplicationScope, Id>, Boolean> row : results ) {
+        for ( Row<ScopedRowKey<Id>, Boolean> row : results ) {
             Column<Boolean> column = row.getColumns().getColumnByName( COLUMN_NAME );
 
             if ( column != null ) {

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/EdgeColumnFamilies.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/EdgeColumnFamilies.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/EdgeColumnFamilies.java
index 6f6c72d..a7b5128 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/EdgeColumnFamilies.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/EdgeColumnFamilies.java
@@ -22,6 +22,7 @@ package org.apache.usergrid.persistence.graph.serialization.impl.shard;
 
 
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.core.migration.Migration;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 
@@ -34,27 +35,27 @@ public interface EdgeColumnFamilies extends Migration{
     /**
      * Get the name of the column family for getting source nodes
      */
-    public MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> getSourceNodeCfName();
+    public MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> getSourceNodeCfName();
 
     /**
      * Get the name of the column family for getting target nodes
      */
-    public MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> getTargetNodeCfName();
+    public MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> getTargetNodeCfName();
 
 
     /**
      * Get the name of the column family for getting source nodes  with a target type
      */
-    public MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> getSourceNodeTargetTypeCfName();
+    public MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> getSourceNodeTargetTypeCfName();
 
     /**
      * Get the name of the column family for getting target nodes with a source type
      */
-    public MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> getTargetNodeSourceTypeCfName();
+    public MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> getTargetNodeSourceTypeCfName();
 
     /**
      * Get the Graph edge versions cf
      * @return
      */
-    public MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> getGraphEdgeVersions();
+    public MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> getGraphEdgeVersions();
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
index 749416c..7c267a0 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
@@ -66,7 +66,7 @@ public class NodeShardCounterSerializationImpl implements NodeShardCounterSerial
     /**
      * Edge shards
      */
-    private static final MultiTennantColumnFamily<ApplicationScope, ShardKey, Boolean> EDGE_SHARD_COUNTS =
+    private static final MultiTennantColumnFamily<ScopedRowKey<ShardKey>, Boolean> EDGE_SHARD_COUNTS =
             new MultiTennantColumnFamily<>( "Edge_Shard_Counts",
                     new OrganizationScopedRowKeySerializer<>( SHARD_KEY_SERIALIZER ), BooleanSerializer.get() );
 
@@ -100,7 +100,7 @@ public class NodeShardCounterSerializationImpl implements NodeShardCounterSerial
             final long value = entry.getValue().get();
 
 
-            final ScopedRowKey rowKey = ScopedRowKey.fromKey( key.scope, key );
+            final ScopedRowKey rowKey = ScopedRowKey.fromKey( key.scope.getApplication(), key );
 
 
             batch.withRow( EDGE_SHARD_COUNTS, rowKey ).incrementCounterColumn(true , value );
@@ -114,7 +114,7 @@ public class NodeShardCounterSerializationImpl implements NodeShardCounterSerial
     @Override
     public long getCount( final ShardKey key ) {
 
-        final ScopedRowKey rowKey = ScopedRowKey.fromKey( key.scope, key );
+        final ScopedRowKey rowKey = ScopedRowKey.fromKey( key.scope.getApplication(), key );
 
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeSearcher.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeSearcher.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeSearcher.java
index 413c2a3..e1800c0 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeSearcher.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeSearcher.java
@@ -54,14 +54,14 @@ public abstract class EdgeSearcher<R, C, T> implements ColumnParser<C, T>, Colum
 
 
 
-    public List<ScopedRowKey<ApplicationScope, R>> getRowKeys() {
+    public List<ScopedRowKey<R>> getRowKeys() {
 
-        List<ScopedRowKey<ApplicationScope, R>> rowKeys = new ArrayList<>(shards.size());
+        List<ScopedRowKey<R>> rowKeys = new ArrayList<>(shards.size());
 
         for(Shard shard : shards){
 
-            final ScopedRowKey<ApplicationScope, R> rowKey = ScopedRowKey
-                    .fromKey( scope, generateRowKey(shard.getShardIndex() ) );
+            final ScopedRowKey< R> rowKey = ScopedRowKey
+                    .fromKey( scope.getApplication(), generateRowKey(shard.getShardIndex() ) );
 
             rowKeys.add( rowKey );
         }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
index 3b1f37a..4fd3155 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
@@ -61,7 +61,7 @@ public class EdgeShardSerializationImpl implements EdgeShardSerialization {
     /**
      * Edge shards
      */
-    private static final MultiTennantColumnFamily<ApplicationScope, DirectedEdgeMeta, Long> EDGE_SHARDS =
+    private static final MultiTennantColumnFamily<ScopedRowKey<DirectedEdgeMeta>, Long> EDGE_SHARDS =
             new MultiTennantColumnFamily<>( "Edge_Shards",
                     new OrganizationScopedRowKeySerializer<>( EdgeShardRowKeySerializer.INSTANCE ), LongSerializer.get() );
 
@@ -96,7 +96,7 @@ public class EdgeShardSerializationImpl implements EdgeShardSerialization {
         Preconditions.checkArgument( shard.getShardIndex() > -1, "shardid must be greater than -1" );
         Preconditions.checkArgument( shard.getCreatedTime() > -1, "createdTime must be greater than -1" );
 
-        final ScopedRowKey rowKey = ScopedRowKey.fromKey( scope, metaData );
+        final ScopedRowKey rowKey = ScopedRowKey.fromKey( scope.getApplication(), metaData );
 
         final MutationBatch batch = keyspace.prepareMutationBatch();
 
@@ -130,10 +130,10 @@ public class EdgeShardSerializationImpl implements EdgeShardSerialization {
         }
 
 
-        final ScopedRowKey rowKey = ScopedRowKey.fromKey( scope, metaData );
+        final ScopedRowKey rowKey = ScopedRowKey.fromKey( scope.getApplication(), metaData );
 
 
-        final RowQuery<ScopedRowKey<ApplicationScope, DirectedEdgeMeta>, Long> query =
+        final RowQuery<ScopedRowKey<DirectedEdgeMeta>, Long> query =
                 keyspace.prepareQuery( EDGE_SHARDS ).setConsistencyLevel( cassandraConfig.getReadCL() ).getKey( rowKey )
                         .autoPaginate( true ).withColumnRange( rangeBuilder.build() );
 
@@ -152,7 +152,7 @@ public class EdgeShardSerializationImpl implements EdgeShardSerialization {
 
 
 
-        final ScopedRowKey rowKey = ScopedRowKey.fromKey( scope, metaData );
+        final ScopedRowKey rowKey = ScopedRowKey.fromKey( scope.getApplication(), metaData );
 
         final MutationBatch batch = keyspace.prepareMutationBatch();
 


[5/7] git commit: Finished the migration of map keys to buckets

Posted by to...@apache.org.
Finished the migration of map keys to buckets


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

Branch: refs/heads/key-row-sharding
Commit: 6b203de04bf8bbe4279fd619e096d8d260e9851c
Parents: dd84d6b
Author: Todd Nine <tn...@apigee.com>
Authored: Fri Oct 24 12:25:55 2014 -0600
Committer: Todd Nine <tn...@apigee.com>
Committed: Fri Oct 24 12:25:55 2014 -0600

----------------------------------------------------------------------
 .../core/astyanax/BucketScopedRowKey.java       | 13 ++-
 .../astyanax/BucketScopedRowKeySerializer.java  | 94 ++++++++++++++++++++
 ...rganizationScopedBucketRowKeySerializer.java | 93 -------------------
 .../OrganizationScopedRowKeySerializer.java     | 87 ------------------
 .../core/astyanax/ScopedRowKeySerializer.java   | 87 ++++++++++++++++++
 .../core/hash/ExpandingBucketLocator.java       | 42 ++++-----
 .../core/hash/ExpandingBucketLocatorTest.java   |  9 +-
 .../impl/EdgeMetadataSerializationImpl.java     | 10 +--
 .../impl/NodeSerializationImpl.java             |  6 +-
 .../NodeShardCounterSerializationImpl.java      |  4 +-
 .../shard/impl/EdgeShardSerializationImpl.java  |  4 +-
 .../shard/impl/SizebasedEdgeColumnFamilies.java | 13 ++-
 .../map/impl/MapSerializationImpl.java          | 41 +++++----
 13 files changed, 256 insertions(+), 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
index 0687a7a..fe04875 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
@@ -25,6 +25,11 @@ import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.model.entity.Id;
 
 
+/**
+ * A scoped row key that also includes an index, which can be used for consistent hashing.
+ *
+ * @param <K>
+ */
 public class BucketScopedRowKey< K> extends ScopedRowKey< K> {
 
     private final int bucketNumber;
@@ -100,13 +105,13 @@ public class BucketScopedRowKey< K> extends ScopedRowKey< K> {
     /**
      * Create a list of all buckets from [0,  totalBuckets}.  Note that this is an n-1 0 based system
      */
-    public static < K> List<BucketScopedRowKey< K>> fromRange( final Id scope, final K key, final int totalBuckets ) {
+    public static < K> List<BucketScopedRowKey< K>> fromRange( final Id scope, final K key, final int... buckets ) {
 
-        final List<BucketScopedRowKey< K>> results = new ArrayList<>( totalBuckets );
+        final List<BucketScopedRowKey< K>> results = new ArrayList<>( buckets.length );
 
 
-        for ( int i = 0; i < totalBuckets; i++ ) {
-            results.add( new BucketScopedRowKey<>( scope, key, i ) );
+        for ( int i = 0; i < buckets.length; i++ ) {
+            results.add( new BucketScopedRowKey<>( scope, key, buckets[i] ) );
         }
 
         return results;

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java
new file mode 100644
index 0000000..401cc83
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java
@@ -0,0 +1,94 @@
+/*
+ * 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.astyanax;
+
+
+import java.nio.ByteBuffer;
+
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.core.scope.ApplicationScopeImpl;
+import org.apache.usergrid.persistence.model.entity.Id;
+
+import com.netflix.astyanax.model.CompositeBuilder;
+import com.netflix.astyanax.model.CompositeParser;
+import com.netflix.astyanax.model.Composites;
+import com.netflix.astyanax.serializers.AbstractSerializer;
+import com.netflix.astyanax.serializers.IntegerSerializer;
+
+
+/**
+ * Serializer for serializing CollectionScope + any type into row keys.
+ *
+ *
+ * @author tnine
+ */
+public class BucketScopedRowKeySerializer<K> extends AbstractSerializer<BucketScopedRowKey<K>> {
+
+
+    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
+
+
+    /**
+     * The delegate serializer for the key
+     */
+    private final CompositeFieldSerializer<K> keySerializer;
+
+
+
+
+    public BucketScopedRowKeySerializer( final CompositeFieldSerializer<K> keySerializer ) {
+        this.keySerializer = keySerializer;
+    }
+
+
+    @Override
+    public ByteBuffer toByteBuffer( final BucketScopedRowKey<K> scopedRowKey ) {
+
+        final CompositeBuilder builder = Composites.newCompositeBuilder();
+
+        //add the organization's id
+        ID_SER.toComposite( builder, scopedRowKey.getScope() );
+
+        //add the bucket
+        builder.addInteger( scopedRowKey.getBucketNumber() );
+
+        //add the key type
+        keySerializer.toComposite( builder, scopedRowKey.getKey() );
+
+        return builder.build();
+    }
+
+
+    @Override
+    public BucketScopedRowKey<K> fromByteBuffer( final ByteBuffer byteBuffer ) {
+        final CompositeParser parser = Composites.newCompositeParser( byteBuffer );
+
+        //read back the id
+        final Id orgId = ID_SER.fromComposite( parser );
+
+        final int bucket = parser.readInteger();
+
+        final K value = keySerializer.fromComposite( parser );
+
+        return new BucketScopedRowKey<>( orgId, value, bucket );
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java
deleted file mode 100644
index 8fddedd..0000000
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedBucketRowKeySerializer.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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.astyanax;
-
-
-import java.nio.ByteBuffer;
-
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
-import org.apache.usergrid.persistence.core.scope.ApplicationScopeImpl;
-import org.apache.usergrid.persistence.model.entity.Id;
-
-import com.netflix.astyanax.model.CompositeBuilder;
-import com.netflix.astyanax.model.CompositeParser;
-import com.netflix.astyanax.model.Composites;
-import com.netflix.astyanax.serializers.AbstractSerializer;
-import com.netflix.astyanax.serializers.IntegerSerializer;
-
-
-/**
- * Serializer for serializing CollectionScope + any type into row keys
- *
- * @author tnine
- */
-public class OrganizationScopedBucketRowKeySerializer<K> extends AbstractSerializer<BucketScopedRowKey<K>> {
-
-
-    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
-
-
-    /**
-     * The delegate serializer for the key
-     */
-    private final CompositeFieldSerializer<K> keySerializer;
-
-
-
-
-    public OrganizationScopedBucketRowKeySerializer( final CompositeFieldSerializer<K> keySerializer ) {
-        this.keySerializer = keySerializer;
-    }
-
-
-    @Override
-    public ByteBuffer toByteBuffer( final BucketScopedRowKey<K> scopedRowKey ) {
-
-        final CompositeBuilder builder = Composites.newCompositeBuilder();
-
-        //add the organization's id
-        ID_SER.toComposite( builder, scopedRowKey.getScope() );
-
-        //add the bucket
-        builder.addInteger( scopedRowKey.getBucketNumber() );
-
-        //add the key type
-        keySerializer.toComposite( builder, scopedRowKey.getKey() );
-
-        return builder.build();
-    }
-
-
-    @Override
-    public BucketScopedRowKey<K> fromByteBuffer( final ByteBuffer byteBuffer ) {
-        final CompositeParser parser = Composites.newCompositeParser( byteBuffer );
-
-        //read back the id
-        final Id orgId = ID_SER.fromComposite( parser );
-
-        final int bucket = parser.readInteger();
-
-        final K value = keySerializer.fromComposite( parser );
-
-        return new BucketScopedRowKey<>( orgId, value, bucket );
-    }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java
deleted file mode 100644
index 4289628..0000000
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/OrganizationScopedRowKeySerializer.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.astyanax;
-
-
-import java.nio.ByteBuffer;
-
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
-import org.apache.usergrid.persistence.core.scope.ApplicationScopeImpl;
-import org.apache.usergrid.persistence.model.entity.Id;
-
-import com.netflix.astyanax.model.CompositeBuilder;
-import com.netflix.astyanax.model.CompositeParser;
-import com.netflix.astyanax.model.Composites;
-import com.netflix.astyanax.serializers.AbstractSerializer;
-
-
-/**
- * Serializer for serializing CollectionScope + any type into row keys
- *
- * @author tnine
- */
-public class OrganizationScopedRowKeySerializer<K> extends AbstractSerializer<ScopedRowKey<K>> {
-
-
-    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
-
-
-    /**
-     * The delegate serializer for the key
-     */
-    private final CompositeFieldSerializer<K> keySerializer;
-
-
-
-
-    public OrganizationScopedRowKeySerializer( final CompositeFieldSerializer<K> keySerializer ) {
-        this.keySerializer = keySerializer;
-    }
-
-
-    @Override
-    public ByteBuffer toByteBuffer( final ScopedRowKey< K> scopedRowKey ) {
-
-        final CompositeBuilder builder = Composites.newCompositeBuilder();
-
-        //add the organization's id
-        ID_SER.toComposite( builder, scopedRowKey.getScope() );
-
-        //add the key type
-        keySerializer.toComposite( builder, scopedRowKey.getKey() );
-
-        return builder.build();
-    }
-
-
-    @Override
-    public ScopedRowKey<K> fromByteBuffer( final ByteBuffer byteBuffer ) {
-        final CompositeParser parser = Composites.newCompositeParser( byteBuffer );
-
-        //read back the id
-        final Id orgId = ID_SER.fromComposite( parser );
-
-        final K value = keySerializer.fromComposite( parser );
-
-        return new ScopedRowKey<K>(  orgId, value );
-    }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java
new file mode 100644
index 0000000..30e1792
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java
@@ -0,0 +1,87 @@
+/*
+ * 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.astyanax;
+
+
+import java.nio.ByteBuffer;
+
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.core.scope.ApplicationScopeImpl;
+import org.apache.usergrid.persistence.model.entity.Id;
+
+import com.netflix.astyanax.model.CompositeBuilder;
+import com.netflix.astyanax.model.CompositeParser;
+import com.netflix.astyanax.model.Composites;
+import com.netflix.astyanax.serializers.AbstractSerializer;
+
+
+/**
+ * Serializer for serializing CollectionScope + any type into row keys
+ *
+ * @author tnine
+ */
+public class ScopedRowKeySerializer<K> extends AbstractSerializer<ScopedRowKey<K>> {
+
+
+    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
+
+
+    /**
+     * The delegate serializer for the key
+     */
+    private final CompositeFieldSerializer<K> keySerializer;
+
+
+
+
+    public ScopedRowKeySerializer( final CompositeFieldSerializer<K> keySerializer ) {
+        this.keySerializer = keySerializer;
+    }
+
+
+    @Override
+    public ByteBuffer toByteBuffer( final ScopedRowKey< K> scopedRowKey ) {
+
+        final CompositeBuilder builder = Composites.newCompositeBuilder();
+
+        //add the organization's id
+        ID_SER.toComposite( builder, scopedRowKey.getScope() );
+
+        //add the key type
+        keySerializer.toComposite( builder, scopedRowKey.getKey() );
+
+        return builder.build();
+    }
+
+
+    @Override
+    public ScopedRowKey<K> fromByteBuffer( final ByteBuffer byteBuffer ) {
+        final CompositeParser parser = Composites.newCompositeParser( byteBuffer );
+
+        //read back the id
+        final Id orgId = ID_SER.fromComposite( parser );
+
+        final K value = keySerializer.fromComposite( parser );
+
+        return new ScopedRowKey<K>(  orgId, value );
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
index c9bba2f..3d81389 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
@@ -39,26 +39,19 @@ package org.apache.usergrid.persistence.core.hash;/*
  */
 
 
-import java.util.ArrayList;
-import java.util.List;
-
 import com.google.common.hash.Funnel;
 
 
 /**
- * An algorithm that will generate all possible keys for different "Levels" of sharding.  For instance, imagine this scheme.
+ * An algorithm that will generate all possible keys for different "Levels" of sharding.  For instance, imagine this
+ * scheme.
  *
- * 1 Shard
- * 2 Shards
- * 4 Shards
- * 8 Shards
+ * 1 Shard 2 Shards 4 Shards 8 Shards
  *
  * (note that we do not need to expand by 2x each time, this is merely an example).
  *
- * When seeking on a string key, for 4 levels of the key, we get 4 different keys due to different shard sizes.
- * This is faster than seeking ALL shards, since this would result in 15 shards, vs 4 in the example.
- *
- * @param <T>
+ * When seeking on a string key, for 4 levels of the key, we get 4 different keys due to different shard sizes. This is
+ * faster than seeking ALL shards, since this would result in 15 shards, vs 4 in the example.
  */
 public class ExpandingBucketLocator<T> {
 
@@ -66,34 +59,37 @@ public class ExpandingBucketLocator<T> {
 
 
     /**
-     * Create a new instance with the specified history. For instance, from the javadoc above, the constructor
-     * would contains {8, 4, 3, 2, 1}.  Shards are returned in the size order they are given in the constructor
-     * @param funnel
-     * @param bucketSizes
+     * Create a new instance with the specified history. For instance, from the javadoc above, the constructor would
+     * contains {8, 4, 3, 2, 1}.  Shards are returned in the size order they are given in the constructor
      */
     public ExpandingBucketLocator( final Funnel<T> funnel, final int... bucketSizes ) {
 
         bucketLocatorList = new BucketLocator[bucketSizes.length];
 
-        for(int i = 0; i < bucketSizes.length; i ++){
+        for ( int i = 0; i < bucketSizes.length; i++ ) {
             bucketLocatorList[i] = new BucketLocator<>( funnel, bucketSizes[i] );
         }
-
     }
 
 
     /**
      * Hash the results, and return them in the same order as specified in the constructor
-     * @param hash
-     * @return
      */
-    public int[] getBuckets(T hash){
+    public int[] getAllBuckets( T hash ) {
         int[] results = new int[bucketLocatorList.length];
 
-        for(int i = 0; i < bucketLocatorList.length; i ++){
-          results[i] = bucketLocatorList[i].getBucket( hash );
+        for ( int i = 0; i < bucketLocatorList.length; i++ ) {
+            results[i] = bucketLocatorList[i].getBucket( hash );
         }
 
         return results;
     }
+
+
+    /**
+     * Get the current bucket for the hash value.  Hashes from the first element in the list
+     */
+    public int getCurrentBucket( T hash ) {
+        return bucketLocatorList[0].getBucket( hash );
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
index 32deb07..ce691fc 100644
--- a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
@@ -44,8 +44,8 @@ public class ExpandingBucketLocatorTest extends TestCase {
         final String key = "mytestkey";
 
 
-        int[] results1 = expandingBucketLocator1.getBuckets(key  );
-        int[] results2 = expandingBucketLocator2.getBuckets(key  );
+        int[] results1 = expandingBucketLocator1.getAllBuckets(key  );
+        int[] results2 = expandingBucketLocator2.getAllBuckets(key  );
 
         assertTrue( "Same results returned", Arrays.equals(results1, results2));
 
@@ -53,6 +53,11 @@ public class ExpandingBucketLocatorTest extends TestCase {
         assertTrue("Within bounds", results1[1] <= 9);
         assertTrue("Within bounds", results1[2] <= 0);
 
+        //test the first hash
+        int newestBucket = expandingBucketLocator1.getCurrentBucket( key );
+
+        assertEquals("Same bucket returned", results1[0], newestBucket);
+
 
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
index 131e426..6c20cfe 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
@@ -33,7 +33,7 @@ import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
 import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
-import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.core.astyanax.StringColumnParser;
 import org.apache.usergrid.persistence.core.migration.Migration;
@@ -70,14 +70,14 @@ public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization,
 
     //row key serializers
     private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
-    private static final OrganizationScopedRowKeySerializer<Id> ROW_KEY_SER =
-            new OrganizationScopedRowKeySerializer<Id>( ID_SER );
+    private static final ScopedRowKeySerializer<Id> ROW_KEY_SER =
+            new ScopedRowKeySerializer<Id>( ID_SER );
 
     private static final StringSerializer STRING_SERIALIZER = StringSerializer.get();
 
     private static final EdgeTypeRowCompositeSerializer EDGE_SER = new EdgeTypeRowCompositeSerializer();
-    private static final OrganizationScopedRowKeySerializer<EdgeIdTypeKey> EDGE_TYPE_ROW_KEY =
-            new OrganizationScopedRowKeySerializer<EdgeIdTypeKey>( EDGE_SER );
+    private static final ScopedRowKeySerializer<EdgeIdTypeKey> EDGE_TYPE_ROW_KEY =
+            new ScopedRowKeySerializer<EdgeIdTypeKey>( EDGE_SER );
 
     private static final StringColumnParser PARSER = StringColumnParser.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
index 0febf96..eae7fa0 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/NodeSerializationImpl.java
@@ -36,14 +36,13 @@ import org.apache.usergrid.persistence.core.astyanax.CassandraConfig;
 import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
-import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.core.hystrix.HystrixCassandra;
 import org.apache.usergrid.persistence.core.migration.Migration;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.core.util.ValidationUtils;
 import org.apache.usergrid.persistence.graph.Edge;
-import org.apache.usergrid.persistence.graph.exception.GraphRuntimeException;
 import org.apache.usergrid.persistence.graph.serialization.NodeSerialization;
 import org.apache.usergrid.persistence.graph.serialization.util.GraphValidation;
 import org.apache.usergrid.persistence.model.entity.Id;
@@ -53,7 +52,6 @@ import com.google.common.base.Preconditions;
 import com.google.inject.Singleton;
 import com.netflix.astyanax.Keyspace;
 import com.netflix.astyanax.MutationBatch;
-import com.netflix.astyanax.connectionpool.exceptions.ConnectionException;
 import com.netflix.astyanax.connectionpool.exceptions.NotFoundException;
 import com.netflix.astyanax.model.Column;
 import com.netflix.astyanax.model.Row;
@@ -90,7 +88,7 @@ public class NodeSerializationImpl implements NodeSerialization, Migration {
      */
     private static final MultiTennantColumnFamily<ScopedRowKey<Id>, Boolean> GRAPH_DELETE =
             new MultiTennantColumnFamily<>( "Graph_Marked_Nodes",
-                    new OrganizationScopedRowKeySerializer<Id>( ROW_SERIALIZER ), BOOLEAN_SERIALIZER );
+                    new ScopedRowKeySerializer<Id>( ROW_SERIALIZER ), BOOLEAN_SERIALIZER );
 
 
     protected final Keyspace keyspace;

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
index 7c267a0..524a0cf 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/count/NodeShardCounterSerializationImpl.java
@@ -33,7 +33,7 @@ import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
 import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
-import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.core.hystrix.HystrixCassandra;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
@@ -68,7 +68,7 @@ public class NodeShardCounterSerializationImpl implements NodeShardCounterSerial
      */
     private static final MultiTennantColumnFamily<ScopedRowKey<ShardKey>, Boolean> EDGE_SHARD_COUNTS =
             new MultiTennantColumnFamily<>( "Edge_Shard_Counts",
-                    new OrganizationScopedRowKeySerializer<>( SHARD_KEY_SERIALIZER ), BooleanSerializer.get() );
+                    new ScopedRowKeySerializer<>( SHARD_KEY_SERIALIZER ), BooleanSerializer.get() );
 
 
     protected final Keyspace keyspace;

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
index 4fd3155..f107307 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/EdgeShardSerializationImpl.java
@@ -32,7 +32,7 @@ import org.apache.usergrid.persistence.core.astyanax.ColumnParser;
 import org.apache.usergrid.persistence.core.astyanax.ColumnTypes;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
-import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.core.util.ValidationUtils;
@@ -63,7 +63,7 @@ public class EdgeShardSerializationImpl implements EdgeShardSerialization {
      */
     private static final MultiTennantColumnFamily<ScopedRowKey<DirectedEdgeMeta>, Long> EDGE_SHARDS =
             new MultiTennantColumnFamily<>( "Edge_Shards",
-                    new OrganizationScopedRowKeySerializer<>( EdgeShardRowKeySerializer.INSTANCE ), LongSerializer.get() );
+                    new ScopedRowKeySerializer<>( EdgeShardRowKeySerializer.INSTANCE ), LongSerializer.get() );
 
 
     private static final ShardColumnParser COLUMN_PARSER = new ShardColumnParser();

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
index b7d5636..8f16448 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
@@ -30,9 +30,8 @@ import org.apache.cassandra.db.marshal.DynamicCompositeType;
 import org.apache.usergrid.persistence.core.astyanax.ColumnTypes;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
-import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.graph.serialization.impl.shard.DirectedEdge;
 import org.apache.usergrid.persistence.graph.serialization.impl.shard.EdgeColumnFamilies;
 import org.apache.usergrid.persistence.graph.serialization.impl.shard.EdgeRowKey;
@@ -78,17 +77,17 @@ public class SizebasedEdgeColumnFamilies implements EdgeColumnFamilies {
     //initialize the CF's from our implementation
     private static final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> SOURCE_NODE_EDGES =
             new MultiTennantColumnFamily<>( "Graph_Source_Node_Edges",
-                    new OrganizationScopedRowKeySerializer<>( ROW_SERIALIZER ), EDGE_SERIALIZER );
+                    new ScopedRowKeySerializer<>( ROW_SERIALIZER ), EDGE_SERIALIZER );
 
 
     private static final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> TARGET_NODE_EDGES =
             new MultiTennantColumnFamily<>( "Graph_Target_Node_Edges",
-                    new OrganizationScopedRowKeySerializer<>( ROW_SERIALIZER ), EDGE_SERIALIZER );
+                    new ScopedRowKeySerializer<>( ROW_SERIALIZER ), EDGE_SERIALIZER );
 
 
     private static final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> SOURCE_NODE_TARGET_TYPE =
             new MultiTennantColumnFamily<>( "Graph_Source_Node_Target_Type",
-                    new OrganizationScopedRowKeySerializer<>( ROW_TYPE_SERIALIZER ), EDGE_SERIALIZER );
+                    new ScopedRowKeySerializer<>( ROW_TYPE_SERIALIZER ), EDGE_SERIALIZER );
 
 
     /**
@@ -96,12 +95,12 @@ public class SizebasedEdgeColumnFamilies implements EdgeColumnFamilies {
      */
     private static final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> TARGET_NODE_SOURCE_TYPE =
             new MultiTennantColumnFamily<>( "Graph_Target_Node_Source_Type",
-                    new OrganizationScopedRowKeySerializer<>( ROW_TYPE_SERIALIZER ), EDGE_SERIALIZER );
+                    new ScopedRowKeySerializer<>( ROW_TYPE_SERIALIZER ), EDGE_SERIALIZER );
 
 
     private static final MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> EDGE_VERSIONS =
             new MultiTennantColumnFamily<>( "Graph_Edge_Versions",
-                    new OrganizationScopedRowKeySerializer<>( EDGE_ROW_KEY_SERIALIZER ), LONG_SERIALIZER );
+                    new ScopedRowKeySerializer<>( EDGE_ROW_KEY_SERIALIZER ), LONG_SERIALIZER );
 
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/6b203de0/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java b/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
index f43a500..7f2598b 100644
--- a/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
+++ b/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
@@ -28,15 +28,14 @@ import com.google.common.base.Preconditions;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 
-import org.apache.usergrid.persistence.core.astyanax.BucketLocator;
 import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
 import org.apache.usergrid.persistence.core.astyanax.BucketScopedRowKey;
-import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedBucketRowKeySerializer;
-import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.BucketScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.core.hash.ExpandingBucketLocator;
 import org.apache.usergrid.persistence.map.MapScope;
 
 import com.google.common.hash.Funnel;
@@ -59,13 +58,13 @@ public class MapSerializationImpl implements MapSerialization {
 
     private static final MapKeySerializer KEY_SERIALIZER = new MapKeySerializer();
 
-        private static final OrganizationScopedBucketRowKeySerializer<String> MAP_KEY_SERIALIZER =
-                new OrganizationScopedBucketRowKeySerializer<>( KEY_SERIALIZER );
+        private static final BucketScopedRowKeySerializer<String> MAP_KEY_SERIALIZER =
+                new BucketScopedRowKeySerializer<>( KEY_SERIALIZER );
 
 
         private static final MapEntrySerializer ENTRY_SERIALIZER = new MapEntrySerializer();
-        private static final OrganizationScopedRowKeySerializer<MapEntryKey> MAP_ENTRY_SERIALIZER =
-                new OrganizationScopedRowKeySerializer<>( ENTRY_SERIALIZER );
+        private static final ScopedRowKeySerializer<MapEntryKey> MAP_ENTRY_SERIALIZER =
+                new ScopedRowKeySerializer<>( ENTRY_SERIALIZER );
 
 
         private static final BooleanSerializer BOOLEAN_SERIALIZER = BooleanSerializer.get();
@@ -88,9 +87,9 @@ public class MapSerializationImpl implements MapSerialization {
                 new MultiTennantColumnFamily<>( "Map_Keys", MAP_KEY_SERIALIZER, STRING_SERIALIZER );
 
     /**
-     * Number of buckets to hash across
+     * Number of buckets to hash across.
      */
-    private static final int NUM_BUCKETS = 20;
+    private static final int[] NUM_BUCKETS = {20};
 
     /**
      * How to funnel keys for buckets
@@ -108,7 +107,7 @@ public class MapSerializationImpl implements MapSerialization {
     /**
      * Locator to get us all buckets
      */
-    private static final BucketLocator<String> BUCKET_LOCATOR = new BucketLocator<>( NUM_BUCKETS, MAP_KEY_FUNNEL );
+    private static final ExpandingBucketLocator<String> BUCKET_LOCATOR = new ExpandingBucketLocator<>(MAP_KEY_FUNNEL, NUM_BUCKETS);
 
     private final Keyspace keyspace;
 
@@ -140,7 +139,7 @@ public class MapSerializationImpl implements MapSerialization {
 
         //add it to the keys
 
-        final int bucket = BUCKET_LOCATOR.getBucket( key );
+        final int bucket = BUCKET_LOCATOR.getCurrentBucket( key );
 
         final BucketScopedRowKey< String> keyRowKey =
                 BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket);
@@ -179,7 +178,7 @@ public class MapSerializationImpl implements MapSerialization {
 
         //add it to the keys
 
-        final int bucket = BUCKET_LOCATOR.getBucket( key );
+        final int bucket = BUCKET_LOCATOR.getCurrentBucket( key );
 
         final BucketScopedRowKey< String> keyRowKey =
                 BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket);
@@ -217,7 +216,7 @@ public class MapSerializationImpl implements MapSerialization {
         batch.withRow(MAP_ENTRIES, entryRowKey).putColumn(true, value);
 
         //add it to the keys
-        final int bucket = BUCKET_LOCATOR.getBucket( key );
+        final int bucket = BUCKET_LOCATOR.getCurrentBucket( key );
 
                final BucketScopedRowKey< String> keyRowKey =
                        BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket);
@@ -237,13 +236,16 @@ public class MapSerializationImpl implements MapSerialization {
         //serialize to the entry
         batch.withRow(MAP_ENTRIES, entryRowKey).delete();
 
-        //add it to the keys
+        //add it to the keys, we're not sure which one it may have come from
+       final int[] buckets = BUCKET_LOCATOR.getAllBuckets( key );
 
-        final int bucket = BUCKET_LOCATOR.getBucket( key );
 
-        final BucketScopedRowKey<String> rowKey = BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket );
+        final List<BucketScopedRowKey<String>>
+                rowKeys = BucketScopedRowKey.fromRange( scope.getApplication(), key, buckets );
 
-        batch.withRow( MAP_KEYS, rowKey ).deleteColumn( key );
+        for(BucketScopedRowKey<String> rowKey: rowKeys) {
+            batch.withRow( MAP_KEYS, rowKey ).deleteColumn( key );
+        }
 
         executeBatch( batch );
     }
@@ -272,9 +274,12 @@ public class MapSerializationImpl implements MapSerialization {
 
     private  Column<Boolean> getValue(MapScope scope, String key) {
 
+
+
         //add it to the entry
         final ScopedRowKey<MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
 
+        //now get all columns, including the "old row key value"
         try {
             final Column<Boolean> result = keyspace.prepareQuery( MAP_ENTRIES )
                     .getKey( entryRowKey ).getColumn( true ).execute().getResult();


[2/7] Finished low level serialization refactor

Posted by to...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardedEdgeSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardedEdgeSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardedEdgeSerializationImpl.java
index b0523e1..13f7427 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardedEdgeSerializationImpl.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardedEdgeSerializationImpl.java
@@ -118,11 +118,11 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKey rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
-                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope, rowKey ) ).putColumn( edge, isDeleted );
+                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope.getApplication(), rowKey ) ).putColumn( edge, isDeleted );
 
                 if ( !isDeleted ) {
                     writeEdgeShardStrategy.increment( scope, shard, 1, directedEdgeMeta );
@@ -147,12 +147,12 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKeyType rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
 
-                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope, rowKey ) ).putColumn( edge, isDeleted );
+                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope.getApplication(), rowKey ) ).putColumn( edge, isDeleted );
 
 
                 if ( !isDeleted ) {
@@ -176,11 +176,11 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKey rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
-                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope, rowKey ) ).putColumn( edge, isDeleted );
+                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope.getApplication(), rowKey ) ).putColumn( edge, isDeleted );
 
 
                 if ( !isDeleted ) {
@@ -206,11 +206,11 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKeyType rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
-                batch.withRow( columnFamilies.getTargetNodeSourceTypeCfName(), ScopedRowKey.fromKey( scope, rowKey ) )
+                batch.withRow( columnFamilies.getTargetNodeSourceTypeCfName(), ScopedRowKey.fromKey( scope.getApplication(), rowKey ) )
                      .putColumn( edge, isDeleted );
 
 
@@ -236,10 +236,10 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> columnFamily,
                             final ApplicationScope scope, final EdgeRowKey rowKey, final Long column, final Shard shard,
                             final boolean isDeleted ) {
-                batch.withRow( columnFamilies.getGraphEdgeVersions(), ScopedRowKey.fromKey( scope, rowKey ) )
+                batch.withRow( columnFamilies.getGraphEdgeVersions(), ScopedRowKey.fromKey( scope.getApplication(), rowKey ) )
                      .putColumn( column, isDeleted );
 
 
@@ -260,11 +260,11 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKey rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
-                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope, rowKey ) ).deleteColumn( edge );
+                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope.getApplication(), rowKey ) ).deleteColumn( edge );
                 writeEdgeShardStrategy.increment( scope, shard, -1, directedEdgeMeta );
             }
         }.createBatch( scope, shards, timestamp );
@@ -281,12 +281,12 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKeyType rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
 
-                batch.withRow( columnFamilies.getSourceNodeTargetTypeCfName(), ScopedRowKey.fromKey( scope, rowKey ) )
+                batch.withRow( columnFamilies.getSourceNodeTargetTypeCfName(), ScopedRowKey.fromKey( scope.getApplication(), rowKey ) )
                      .deleteColumn( edge );
                 writeEdgeShardStrategy.increment( scope, shard, -1, directedEdgeMeta );
             }
@@ -303,11 +303,11 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKey rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
-                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope, rowKey ) ).deleteColumn( edge );
+                batch.withRow( columnFamily, ScopedRowKey.fromKey( scope.getApplication(), rowKey ) ).deleteColumn( edge );
                 writeEdgeShardStrategy.increment( scope, shard, -1, directedEdgeMeta );
             }
         }.createBatch( scope, shards, timestamp );
@@ -325,11 +325,11 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily,
                             final ApplicationScope scope, final RowKeyType rowKey, final DirectedEdge edge,
                             final Shard shard, final boolean isDeleted ) {
 
-                batch.withRow( columnFamilies.getTargetNodeSourceTypeCfName(), ScopedRowKey.fromKey( scope, rowKey ) )
+                batch.withRow( columnFamilies.getTargetNodeSourceTypeCfName(), ScopedRowKey.fromKey( scope.getApplication(), rowKey ) )
                      .deleteColumn( edge );
                 writeEdgeShardStrategy.increment( scope, shard, -1, directedEdgeMeta );
             }
@@ -346,10 +346,10 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
             @Override
             void writeEdge( final MutationBatch batch,
-                            final MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> columnFamily,
+                            final MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> columnFamily,
                             final ApplicationScope scope, final EdgeRowKey rowKey, final Long column, final Shard shard,
                             final boolean isDeleted ) {
-                batch.withRow( columnFamilies.getGraphEdgeVersions(), ScopedRowKey.fromKey( scope, rowKey ) )
+                batch.withRow( columnFamilies.getGraphEdgeVersions(), ScopedRowKey.fromKey( scope.getApplication(), rowKey ) )
                      .deleteColumn( column );
                 writeEdgeShardStrategy.increment( scope, shard, -1, directedEdgeMeta );
             }
@@ -367,7 +367,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
         final Id sourceId = search.sourceNode();
         final String type = search.getType();
         final long maxTimestamp = search.getMaxTimestamp();
-        final MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> columnFamily =
+        final MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> columnFamily =
                 columnFamilies.getGraphEdgeVersions();
         final Serializer<Long> serializer = columnFamily.getColumnSerializer();
 
@@ -439,7 +439,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
         final Id sourceId = search.getNode();
         final String type = search.getType();
         final long maxTimestamp = search.getMaxTimestamp();
-        final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily =
+        final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily =
                 columnFamilies.getSourceNodeCfName();
         final Serializer<DirectedEdge> serializer = columnFamily.getColumnSerializer();
 
@@ -496,7 +496,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
         final String type = search.getType();
         final String targetType = search.getIdType();
         final long maxTimestamp = search.getMaxTimestamp();
-        final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily =
+        final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily =
                 columnFamilies.getSourceNodeTargetTypeCfName();
         final Serializer<DirectedEdge> serializer = columnFamily.getColumnSerializer();
 
@@ -545,7 +545,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
         final Id targetId = search.getNode();
         final String type = search.getType();
         final long maxTimestamp = search.getMaxTimestamp();
-        final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily =
+        final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily =
                 columnFamilies.getTargetNodeCfName();
         final Serializer<DirectedEdge> serializer = columnFamily.getColumnSerializer();
 
@@ -598,7 +598,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
         final String sourceType = search.getIdType();
         final String type = search.getType();
         final long maxTimestamp = search.getMaxTimestamp();
-        final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily =
+        final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily =
                 columnFamilies.getTargetNodeSourceTypeCfName();
         final Serializer<DirectedEdge> serializer = columnFamily.getColumnSerializer();
 
@@ -652,7 +652,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
         /**
          * Return the column family used for the write
          */
-        protected abstract MultiTennantColumnFamily<ApplicationScope, R, C> getColumnFamily();
+        protected abstract MultiTennantColumnFamily<ScopedRowKey<R>, C> getColumnFamily();
 
         /**
          * Get the row key
@@ -674,7 +674,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
          * Write the edge with the given data
          */
         abstract void writeEdge( final MutationBatch batch,
-                                 final MultiTennantColumnFamily<ApplicationScope, R, C> columnFamily,
+                                 final MultiTennantColumnFamily<ScopedRowKey<R>, C> columnFamily,
                                  final ApplicationScope scope, final R rowKey, final C column, final Shard shard,
                                  final boolean isDeleted );
 
@@ -691,7 +691,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
 
             final C column = getDirectedEdge();
-            final MultiTennantColumnFamily<ApplicationScope, R, C> columnFamily = getColumnFamily();
+            final MultiTennantColumnFamily<ScopedRowKey<R>, C> columnFamily = getColumnFamily();
             final boolean isDeleted = isDeleted();
 
 
@@ -711,7 +711,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
      */
     private abstract class SourceWriteOp extends RowOp<RowKey, DirectedEdge> {
 
-        private final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily;
+        private final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily;
         private final Id sourceNodeId;
 
 
@@ -736,7 +736,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
 
         @Override
-        protected MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> getColumnFamily() {
+        protected MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> getColumnFamily() {
             return columnFamily;
         }
 
@@ -765,7 +765,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
      */
     private abstract class SourceTargetTypeWriteOp extends RowOp<RowKeyType, DirectedEdge> {
 
-        private final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily;
+        private final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily;
         private final Id sourceNodeId;
         private final String type;
         private Id targetId;
@@ -790,7 +790,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
 
         @Override
-        protected MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> getColumnFamily() {
+        protected MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> getColumnFamily() {
             return columnFamily;
         }
 
@@ -819,7 +819,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
      */
     private abstract class TargetWriteOp extends RowOp<RowKey, DirectedEdge> {
 
-        private final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> columnFamily;
+        private final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> columnFamily;
         private final Id targetNode;
 
 
@@ -844,7 +844,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
 
         @Override
-        protected MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> getColumnFamily() {
+        protected MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> getColumnFamily() {
             return columnFamily;
         }
 
@@ -873,7 +873,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
      */
     private abstract class TargetSourceTypeWriteOp extends RowOp<RowKeyType, DirectedEdge> {
 
-        private final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> columnFamily;
+        private final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> columnFamily;
         private final Id targetNode;
 
         private final Id sourceNode;
@@ -901,7 +901,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
 
         @Override
-        protected MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> getColumnFamily() {
+        protected MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> getColumnFamily() {
             return columnFamily;
         }
 
@@ -930,7 +930,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
      */
     private abstract class EdgeVersions extends RowOp<EdgeRowKey, Long> {
 
-        private final MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> columnFamily;
+        private final MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> columnFamily;
         private final Id targetNode;
 
         private final Id sourceNode;
@@ -958,7 +958,7 @@ public class ShardedEdgeSerializationImpl implements ShardedEdgeSerialization {
 
 
         @Override
-        protected MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> getColumnFamily() {
+        protected MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> getColumnFamily() {
             return columnFamily;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardsColumnIterator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardsColumnIterator.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardsColumnIterator.java
index d99d98b..0020f8c 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardsColumnIterator.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/ShardsColumnIterator.java
@@ -30,7 +30,7 @@ public class ShardsColumnIterator<R, C, T> implements Iterator<T> {
 
     private final EdgeSearcher<R, C, T> searcher;
 
-    private final MultiTennantColumnFamily<ApplicationScope, R, C> cf;
+    private final MultiTennantColumnFamily<ScopedRowKey<R>, C> cf;
 
     private Iterator<T> currentColumnIterator;
 
@@ -42,7 +42,7 @@ public class ShardsColumnIterator<R, C, T> implements Iterator<T> {
 
 
     public ShardsColumnIterator( final EdgeSearcher<R, C, T> searcher,
-                             final MultiTennantColumnFamily<ApplicationScope, R, C> cf, final Keyspace keyspace,
+                             final MultiTennantColumnFamily<ScopedRowKey<R>, C> cf, final Keyspace keyspace,
                              final ConsistencyLevel consistencyLevel, final int pageSize ) {
         this.searcher = searcher;
         this.cf = cf;
@@ -103,12 +103,12 @@ public class ShardsColumnIterator<R, C, T> implements Iterator<T> {
         /**
          * Get our list of slices
          */
-        final List<ScopedRowKey<ApplicationScope, R>> rowKeys = searcher.getRowKeys();
+        final List<ScopedRowKey<R>> rowKeys = searcher.getRowKeys();
 
 
         if(rowKeys.size() == 1){
 
-            final  RowQuery<ScopedRowKey<ApplicationScope, R>, C> query =
+            final  RowQuery<ScopedRowKey<R>, C> query =
                            keyspace.prepareQuery( cf ).setConsistencyLevel( consistencyLevel ).getKey( rowKeys.get( 0 ) )
                                    .autoPaginate( true ).withColumnRange( rangeBuilder.build() );
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
index ddd514b..b7d5636 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SizebasedEdgeColumnFamilies.java
@@ -31,6 +31,7 @@ import org.apache.usergrid.persistence.core.astyanax.ColumnTypes;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
 import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.graph.serialization.impl.shard.DirectedEdge;
 import org.apache.usergrid.persistence.graph.serialization.impl.shard.EdgeColumnFamilies;
@@ -75,17 +76,17 @@ public class SizebasedEdgeColumnFamilies implements EdgeColumnFamilies {
 
 
     //initialize the CF's from our implementation
-    private static final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> SOURCE_NODE_EDGES =
+    private static final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> SOURCE_NODE_EDGES =
             new MultiTennantColumnFamily<>( "Graph_Source_Node_Edges",
                     new OrganizationScopedRowKeySerializer<>( ROW_SERIALIZER ), EDGE_SERIALIZER );
 
 
-    private static final MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> TARGET_NODE_EDGES =
+    private static final MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> TARGET_NODE_EDGES =
             new MultiTennantColumnFamily<>( "Graph_Target_Node_Edges",
                     new OrganizationScopedRowKeySerializer<>( ROW_SERIALIZER ), EDGE_SERIALIZER );
 
 
-    private static final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> SOURCE_NODE_TARGET_TYPE =
+    private static final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> SOURCE_NODE_TARGET_TYPE =
             new MultiTennantColumnFamily<>( "Graph_Source_Node_Target_Type",
                     new OrganizationScopedRowKeySerializer<>( ROW_TYPE_SERIALIZER ), EDGE_SERIALIZER );
 
@@ -93,42 +94,42 @@ public class SizebasedEdgeColumnFamilies implements EdgeColumnFamilies {
     /**
      * The edges that are to the target node with the source type.  The target node is the row key
      */
-    private static final MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> TARGET_NODE_SOURCE_TYPE =
+    private static final MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> TARGET_NODE_SOURCE_TYPE =
             new MultiTennantColumnFamily<>( "Graph_Target_Node_Source_Type",
                     new OrganizationScopedRowKeySerializer<>( ROW_TYPE_SERIALIZER ), EDGE_SERIALIZER );
 
 
-    private static final MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> EDGE_VERSIONS =
+    private static final MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> EDGE_VERSIONS =
             new MultiTennantColumnFamily<>( "Graph_Edge_Versions",
                     new OrganizationScopedRowKeySerializer<>( EDGE_ROW_KEY_SERIALIZER ), LONG_SERIALIZER );
 
 
     @Override
-    public MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> getSourceNodeCfName() {
+    public MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> getSourceNodeCfName() {
         return SOURCE_NODE_EDGES;
     }
 
 
     @Override
-    public MultiTennantColumnFamily<ApplicationScope, RowKey, DirectedEdge> getTargetNodeCfName() {
+    public MultiTennantColumnFamily<ScopedRowKey<RowKey>, DirectedEdge> getTargetNodeCfName() {
         return TARGET_NODE_EDGES;
     }
 
 
     @Override
-    public MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> getSourceNodeTargetTypeCfName() {
+    public MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> getSourceNodeTargetTypeCfName() {
         return SOURCE_NODE_TARGET_TYPE;
     }
 
 
     @Override
-    public MultiTennantColumnFamily<ApplicationScope, RowKeyType, DirectedEdge> getTargetNodeSourceTypeCfName() {
+    public MultiTennantColumnFamily<ScopedRowKey<RowKeyType>, DirectedEdge> getTargetNodeSourceTypeCfName() {
         return TARGET_NODE_SOURCE_TYPE;
     }
 
 
     @Override
-    public MultiTennantColumnFamily<ApplicationScope, EdgeRowKey, Long> getGraphEdgeVersions() {
+    public MultiTennantColumnFamily<ScopedRowKey<EdgeRowKey>, Long> getGraphEdgeVersions() {
         return EDGE_VERSIONS;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5a56dd45/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java b/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
index f95d53f..f43a500 100644
--- a/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
+++ b/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
@@ -18,22 +18,29 @@
  */
 
 package org.apache.usergrid.persistence.map.impl;
+import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 import java.util.UUID;
 
 import com.google.common.base.Preconditions;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 
+import org.apache.usergrid.persistence.core.astyanax.BucketLocator;
 import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
 import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
+import org.apache.usergrid.persistence.core.astyanax.BucketScopedRowKey;
+import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedBucketRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.OrganizationScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.map.MapScope;
 
+import com.google.common.hash.Funnel;
+import com.google.common.hash.PrimitiveSink;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import com.netflix.astyanax.Keyspace;
@@ -51,8 +58,9 @@ import com.netflix.astyanax.serializers.StringSerializer;
 public class MapSerializationImpl implements MapSerialization {
 
     private static final MapKeySerializer KEY_SERIALIZER = new MapKeySerializer();
-        private static final OrganizationScopedRowKeySerializer<String> MAP_KEY_SERIALIZER =
-                new OrganizationScopedRowKeySerializer<>( KEY_SERIALIZER );
+
+        private static final OrganizationScopedBucketRowKeySerializer<String> MAP_KEY_SERIALIZER =
+                new OrganizationScopedBucketRowKeySerializer<>( KEY_SERIALIZER );
 
 
         private static final MapEntrySerializer ENTRY_SERIALIZER = new MapEntrySerializer();
@@ -68,18 +76,39 @@ public class MapSerializationImpl implements MapSerialization {
         /**
          * CFs where the row key contains the source node id
          */
-        private static final MultiTennantColumnFamily<ApplicationScope, MapEntryKey, Boolean> 
-            MAP_ENTRIES = new MultiTennantColumnFamily<>( 
+        private static final MultiTennantColumnFamily<ScopedRowKey<MapEntryKey>, Boolean>
+            MAP_ENTRIES = new MultiTennantColumnFamily<>(
                 "Map_Entries", MAP_ENTRY_SERIALIZER, BOOLEAN_SERIALIZER );
 
 
         /**
          * CFs where the row key contains the source node id
          */
-        private static final MultiTennantColumnFamily<ApplicationScope, String, String> MAP_KEYS =
+        private static final MultiTennantColumnFamily<BucketScopedRowKey<String>, String> MAP_KEYS =
                 new MultiTennantColumnFamily<>( "Map_Keys", MAP_KEY_SERIALIZER, STRING_SERIALIZER );
 
+    /**
+     * Number of buckets to hash across
+     */
+    private static final int NUM_BUCKETS = 20;
+
+    /**
+     * How to funnel keys for buckets
+     */
+    private static final Funnel<String> MAP_KEY_FUNNEL = new Funnel<String>() {
+
+        private Charset UTF8 = Charset.forName( "UTF8" );
+
+        @Override
+        public void funnel( final String key, final PrimitiveSink into ) {
+            into.putString( key, UTF8 );
+        }
+    };
 
+    /**
+     * Locator to get us all buckets
+     */
+    private static final BucketLocator<String> BUCKET_LOCATOR = new BucketLocator<>( NUM_BUCKETS, MAP_KEY_FUNNEL );
 
     private final Keyspace keyspace;
 
@@ -104,15 +133,17 @@ public class MapSerializationImpl implements MapSerialization {
         final MutationBatch batch = keyspace.prepareMutationBatch();
 
         //add it to the entry
-        final ScopedRowKey<ApplicationScope, MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
+        final ScopedRowKey<MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
 
         //serialize to the entry
         batch.withRow(MAP_ENTRIES, entryRowKey).putColumn(true, value);
 
         //add it to the keys
 
-        final ScopedRowKey<ApplicationScope, String> keyRowKey =
-                ScopedRowKey.fromKey((ApplicationScope) scope, key);
+        final int bucket = BUCKET_LOCATOR.getBucket( key );
+
+        final BucketScopedRowKey< String> keyRowKey =
+                BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket);
 
         //serialize to the entry
         batch.withRow(MAP_KEYS, keyRowKey).putColumn(key, true);
@@ -141,15 +172,17 @@ public class MapSerializationImpl implements MapSerialization {
         final MutationBatch batch = keyspace.prepareMutationBatch();
 
         //add it to the entry
-        final ScopedRowKey<ApplicationScope, MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
+        final ScopedRowKey<MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
 
         //serialize to the entry
         batch.withRow(MAP_ENTRIES, entryRowKey).putColumn(true, putUuid);
 
         //add it to the keys
 
-        final ScopedRowKey<ApplicationScope, String> keyRowKey =
-                ScopedRowKey.fromKey((ApplicationScope) scope, key);
+        final int bucket = BUCKET_LOCATOR.getBucket( key );
+
+        final BucketScopedRowKey< String> keyRowKey =
+                BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket);
 
         //serialize to the entry
         batch.withRow(MAP_KEYS, keyRowKey).putColumn(key, true);
@@ -178,15 +211,16 @@ public class MapSerializationImpl implements MapSerialization {
         final MutationBatch batch = keyspace.prepareMutationBatch();
 
         //add it to the entry
-        final ScopedRowKey<ApplicationScope, MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
+        final ScopedRowKey<MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
 
         //serialize to the entry
         batch.withRow(MAP_ENTRIES, entryRowKey).putColumn(true, value);
 
         //add it to the keys
+        final int bucket = BUCKET_LOCATOR.getBucket( key );
 
-        final ScopedRowKey<ApplicationScope, String> keyRowKey =
-                ScopedRowKey.fromKey((ApplicationScope) scope, key);
+               final BucketScopedRowKey< String> keyRowKey =
+                       BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket);
 
         //serialize to the entry
         batch.withRow(MAP_KEYS, keyRowKey).putColumn(key, true);
@@ -198,18 +232,20 @@ public class MapSerializationImpl implements MapSerialization {
     @Override
     public void delete( final MapScope scope, final String key ) {
         final MutationBatch batch = keyspace.prepareMutationBatch();
-        final ScopedRowKey<ApplicationScope, MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
+        final ScopedRowKey<MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
 
         //serialize to the entry
         batch.withRow(MAP_ENTRIES, entryRowKey).delete();
 
         //add it to the keys
 
-        final ScopedRowKey<ApplicationScope, String> keyRowKey = ScopedRowKey.fromKey((ApplicationScope) scope, key);
+        final int bucket = BUCKET_LOCATOR.getBucket( key );
 
-        //serialize to the entry
-        batch.withRow(MAP_KEYS, keyRowKey).delete();
-        executeBatch(batch);
+        final BucketScopedRowKey<String> rowKey = BucketScopedRowKey.fromKey( scope.getApplication(), key, bucket );
+
+        batch.withRow( MAP_KEYS, rowKey ).deleteColumn( key );
+
+        executeBatch( batch );
     }
 
 
@@ -237,7 +273,7 @@ public class MapSerializationImpl implements MapSerialization {
     private  Column<Boolean> getValue(MapScope scope, String key) {
 
         //add it to the entry
-        final ScopedRowKey<ApplicationScope, MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
+        final ScopedRowKey<MapEntryKey> entryRowKey = MapEntryKey.fromKey(scope, key);
 
         try {
             final Column<Boolean> result = keyspace.prepareQuery( MAP_ENTRIES )
@@ -327,11 +363,10 @@ public class MapSerializationImpl implements MapSerialization {
         /**
          * Create a scoped row key from the key
          */
-        public static ScopedRowKey<ApplicationScope, MapEntryKey> fromKey( 
+        public static ScopedRowKey<MapEntryKey> fromKey(
                 final MapScope mapScope, final String key ) {
 
-            return ScopedRowKey.fromKey( 
-                    ( ApplicationScope ) mapScope, new MapEntryKey( mapScope.getName(), key ) );
+            return ScopedRowKey.fromKey( mapScope.getApplication(), new MapEntryKey( mapScope.getName(), key ) );
         }
     }
 }


[4/7] git commit: Added bucket tests

Posted by to...@apache.org.
Added bucket tests


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

Branch: refs/heads/key-row-sharding
Commit: dd84d6bd777352e811fc77574c013c88902a29fe
Parents: 5a56dd4
Author: Todd Nine <tn...@apigee.com>
Authored: Fri Oct 24 12:04:27 2014 -0600
Committer: Todd Nine <tn...@apigee.com>
Committed: Fri Oct 24 12:04:27 2014 -0600

----------------------------------------------------------------------
 .../core/astyanax/BucketLocator.java            | 67 -------------
 .../persistence/core/hash/BucketLocator.java    | 77 +++++++++++++++
 .../core/hash/ExpandingBucketLocator.java       | 99 ++++++++++++++++++++
 .../core/hash/BucketLocatorTest.java            | 66 +++++++++++++
 .../core/hash/ExpandingBucketLocatorTest.java   | 58 ++++++++++++
 5 files changed, 300 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/dd84d6bd/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java
deleted file mode 100644
index 49041f8..0000000
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketLocator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-package org.apache.usergrid.persistence.core.astyanax;/*
- * 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.
- */
-
-
-import com.google.common.hash.Funnel;
-import com.google.common.hash.HashCode;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-
-
-/**
- * Simple utility to locate which bucket an element should be in
- */
-public class BucketLocator<T> {
-
-    /**
-     * Use the murmur 3 hash
-     */
-    private static final HashFunction HASHER = Hashing.murmur3_128();
-
-    private final int totalBuckets;
-    private final Funnel<T> funnel;
-
-
-    public BucketLocator( final int totalBuckets, final Funnel<T> funnel ) {
-        this.totalBuckets = totalBuckets;
-        this.funnel = funnel;
-    }
-
-
-    /**
-     * Locate the bucket number given the value, the funnel and the total buckets.
-     *
-     * Assigns to {@code hashCode} a "bucket" in the range {@code [0, buckets)}, in a uniform manner that minimizes the
-     * need for remapping as {@code buckets} grows. That is, {@code consistentHash(h, n)} equals:
-     *
-     * <ul> <li>{@code n - 1}, with approximate probability {@code 1/n} <li>{@code consistentHash(h, n - 1)}, otherwise
-     * (probability {@code 1 - 1/n}) </ul>
-     *
-     * <p>See the <a href="http://en.wikipedia.org/wiki/Consistent_hashing">wikipedia article on consistent hashing</a>
-     * for more information.
-     */
-    public int getBucket( T value ) {
-
-        final HashCode hashCode = HASHER.hashObject( value, funnel );
-
-        int owningIndex = Hashing.consistentHash( hashCode, totalBuckets );
-
-        return owningIndex;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/dd84d6bd/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java
new file mode 100644
index 0000000..52ffeba
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.hash;
+
+
+
+import com.google.common.hash.Funnel;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+
+
+/**
+ * Simple utility to locate which bucket an element should be located based on it's funnel
+ *
+ */
+public class BucketLocator<T> {
+
+    /**
+     * Use the murmur 3 hash
+     */
+    private static final HashFunction HASHER = Hashing.murmur3_128();
+
+    private final int totalBuckets;
+    private final Funnel<T> funnel;
+
+
+    public BucketLocator(final Funnel<T> funnel, final int totalBuckets ) {
+        this.funnel = funnel;
+        this.totalBuckets = totalBuckets;
+    }
+
+
+    /**
+     * Locate the bucket number given the value, the funnel and the total buckets.
+     *
+     * Assigns to {@code hashCode} a "bucket" in the range {@code [0, buckets)}, in a uniform manner that minimizes the
+     * need for remapping as {@code buckets} grows. That is, {@code consistentHash(h, n)} equals:
+     *
+     * <ul> <li>{@code n - 1}, with approximate probability {@code 1/n} <li>{@code consistentHash(h, n - 1)}, otherwise
+     * (probability {@code 1 - 1/n}) </ul>
+     *
+     * <p>See the <a href="http://en.wikipedia.org/wiki/Consistent_hashing">wikipedia article on consistent hashing</a>
+     * for more information.
+     *
+     * <p>See <a href="http://arxiv.org/pdf/1406.2294v1.pdf">this paper</a> for more details on the algorithm</p>
+     *
+     *
+     * Note that after testing, increasing buckets does NOT yield the expected results.  You will need an algorithm
+     * that manually walks a tree.  See
+     *
+     */
+    public int getBucket( T value ) {
+
+        final HashCode hashCode = HASHER.hashObject( value, funnel );
+
+        int owningIndex = Hashing.consistentHash( hashCode, totalBuckets );
+
+        return owningIndex;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/dd84d6bd/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
new file mode 100644
index 0000000..c9bba2f
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
@@ -0,0 +1,99 @@
+/*
+ *
+ *  * 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.hash;/*
+ * 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.
+ */
+
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.hash.Funnel;
+
+
+/**
+ * An algorithm that will generate all possible keys for different "Levels" of sharding.  For instance, imagine this scheme.
+ *
+ * 1 Shard
+ * 2 Shards
+ * 4 Shards
+ * 8 Shards
+ *
+ * (note that we do not need to expand by 2x each time, this is merely an example).
+ *
+ * When seeking on a string key, for 4 levels of the key, we get 4 different keys due to different shard sizes.
+ * This is faster than seeking ALL shards, since this would result in 15 shards, vs 4 in the example.
+ *
+ * @param <T>
+ */
+public class ExpandingBucketLocator<T> {
+
+    private final BucketLocator<T>[] bucketLocatorList;
+
+
+    /**
+     * Create a new instance with the specified history. For instance, from the javadoc above, the constructor
+     * would contains {8, 4, 3, 2, 1}.  Shards are returned in the size order they are given in the constructor
+     * @param funnel
+     * @param bucketSizes
+     */
+    public ExpandingBucketLocator( final Funnel<T> funnel, final int... bucketSizes ) {
+
+        bucketLocatorList = new BucketLocator[bucketSizes.length];
+
+        for(int i = 0; i < bucketSizes.length; i ++){
+            bucketLocatorList[i] = new BucketLocator<>( funnel, bucketSizes[i] );
+        }
+
+    }
+
+
+    /**
+     * Hash the results, and return them in the same order as specified in the constructor
+     * @param hash
+     * @return
+     */
+    public int[] getBuckets(T hash){
+        int[] results = new int[bucketLocatorList.length];
+
+        for(int i = 0; i < bucketLocatorList.length; i ++){
+          results[i] = bucketLocatorList[i].getBucket( hash );
+        }
+
+        return results;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/dd84d6bd/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java
new file mode 100644
index 0000000..49c306b
--- /dev/null
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.hash;
+
+
+import java.nio.charset.Charset;
+
+import org.junit.Test;
+
+import com.google.common.hash.Funnel;
+import com.google.common.hash.PrimitiveSink;
+
+import junit.framework.TestCase;
+
+import static org.junit.Assert.assertEquals;
+
+
+/**
+ * Simple test that validates hashing is actually consistent as buckets grow
+ */
+public class BucketLocatorTest {
+
+    public static final Funnel<String> STRING_FUNNEL = new Funnel<String>() {
+
+        private Charset UTF8 = Charset.forName( "UTF8" );
+
+
+        @Override
+        public void funnel( final String from, final PrimitiveSink into ) {
+            into.putString( from, UTF8 );
+        }
+    };
+
+
+    @Test
+    public void stringHashing() {
+
+        final String hashValue = "keystring";
+
+        BucketLocator<String> bucketLocator1 = new BucketLocator<>(STRING_FUNNEL,  100 );
+
+        int index1 = bucketLocator1.getBucket( hashValue );
+
+        BucketLocator<String> bucketLocator2 = new BucketLocator<>( STRING_FUNNEL, 100 );
+
+        int index2 = bucketLocator2.getBucket( hashValue );
+
+        assertEquals( "Same index expected", index1, index2 );
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/dd84d6bd/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
new file mode 100644
index 0000000..32deb07
--- /dev/null
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
@@ -0,0 +1,58 @@
+/*
+ *
+ *  * 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.hash;
+
+
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+
+/**
+ * Tests that we get consistent results when hashing
+ */
+public class ExpandingBucketLocatorTest extends TestCase {
+
+
+    @Test
+    public void testConsistency(){
+
+        final ExpandingBucketLocator<String> expandingBucketLocator1 = new ExpandingBucketLocator<>( BucketLocatorTest.STRING_FUNNEL, 20, 10, 1 );
+        final ExpandingBucketLocator<String> expandingBucketLocator2 = new ExpandingBucketLocator<>( BucketLocatorTest.STRING_FUNNEL, 20, 10, 1 );
+
+        final String key = "mytestkey";
+
+
+        int[] results1 = expandingBucketLocator1.getBuckets(key  );
+        int[] results2 = expandingBucketLocator2.getBuckets(key  );
+
+        assertTrue( "Same results returned", Arrays.equals(results1, results2));
+
+        assertTrue("Within bounds", results1[0] <= 19);
+        assertTrue("Within bounds", results1[1] <= 9);
+        assertTrue("Within bounds", results1[2] <= 0);
+
+
+    }
+}


[7/7] git commit: Finished refactor of sharding and versioning. Need to implement versions

Posted by to...@apache.org.
Finished refactor of sharding and versioning.  Need to implement versions


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

Branch: refs/heads/key-row-sharding
Commit: 963fe57c47a9e1de8f7623644fdb6630be67169d
Parents: 6b203de
Author: Todd Nine <tn...@apigee.com>
Authored: Mon Oct 27 17:04:31 2014 -0600
Committer: Todd Nine <tn...@apigee.com>
Committed: Mon Oct 27 17:04:31 2014 -0600

----------------------------------------------------------------------
 .../usergrid/ConcurrentCoreTestSuite.java       |   4 +-
 .../java/org/apache/usergrid/CoreTestSuite.java |   4 +-
 .../SimpleIndexBucketLocatorImplTest.java       | 177 ------
 .../SimpleIndexShardLocatorImplTest.java        | 177 ++++++
 stack/corepersistence/common/pom.xml            |  10 +
 .../core/astyanax/BucketScopedRowKey.java       |  13 +-
 .../astyanax/BucketScopedRowKeySerializer.java  |   3 -
 .../core/astyanax/CassandraConfig.java          |   6 +
 .../core/astyanax/CassandraConfigImpl.java      |  23 +
 .../persistence/core/astyanax/CassandraFig.java |  14 +
 .../core/astyanax/CompositeFieldSerializer.java |   1 +
 .../core/astyanax/IdRowCompositeSerializer.java |   1 +
 .../core/astyanax/MultiRowColumnIterator.java   | 192 ++++--
 .../core/astyanax/ScopedRowKeySerializer.java   |   8 +-
 .../persistence/core/guice/CurrentVersion.java  |  42 ++
 .../persistence/core/guice/PreviousVersion.java |  42 ++
 .../persistence/core/guice/ProxyVersion.java    |  42 ++
 .../persistence/core/hash/BucketLocator.java    |  77 ---
 .../core/hash/ExpandingBucketLocator.java       |  95 ---
 .../core/shard/ExpandingShardLocator.java       |  95 +++
 .../persistence/core/shard/ShardLocator.java    |  77 +++
 .../persistence/core/shard/StringHashUtils.java |  54 ++
 .../core/astyanax/ColumnNameIteratorTest.java   |   6 +
 .../MultiKeyColumnNameIteratorTest.java         |   6 +
 .../astyanax/MultiRowColumnIteratorTest.java    |   6 +
 .../core/hash/BucketLocatorTest.java            |  66 --
 .../core/hash/ExpandingBucketLocatorTest.java   |  63 --
 .../core/shard/ExpandingShardLocatorTest.java   |  65 ++
 .../core/shard/ShardLocatorTest.java            |  64 ++
 .../persistence/graph/guice/GraphModule.java    |   4 +-
 .../impl/EdgeMetadataSerializationImpl.java     | 477 ---------------
 .../impl/EdgeMetadataSerializationV1Impl.java   | 478 +++++++++++++++
 .../impl/EdgeMetadataSerializationV2Impl.java   | 613 +++++++++++++++++++
 .../impl/shard/impl/SourceEdgeSearcher.java     |  22 -
 .../serialize/EdgeShardRowKeySerializer.java    |   2 +
 .../shard/impl/serialize/RowSerializer.java     |   2 +-
 .../shard/impl/serialize/RowTypeSerializer.java |   3 +-
 .../EdgeMetaDataSerializationBridgeTest.java    |  58 ++
 .../EdgeMetaDataSerializationV1Test.java        |  58 ++
 .../EdgeMetaDataSerializationV2Test.java        |  60 ++
 .../EdgeMetadataSerializationTest.java          |   9 +-
 .../map/impl/MapSerializationImpl.java          |  11 +-
 42 files changed, 2180 insertions(+), 1050 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/core/src/test/java/org/apache/usergrid/ConcurrentCoreTestSuite.java
----------------------------------------------------------------------
diff --git a/stack/core/src/test/java/org/apache/usergrid/ConcurrentCoreTestSuite.java b/stack/core/src/test/java/org/apache/usergrid/ConcurrentCoreTestSuite.java
index c003ee1..56c7871 100644
--- a/stack/core/src/test/java/org/apache/usergrid/ConcurrentCoreTestSuite.java
+++ b/stack/core/src/test/java/org/apache/usergrid/ConcurrentCoreTestSuite.java
@@ -29,7 +29,7 @@ import org.apache.usergrid.persistence.QueryUtilsTest;
 import org.apache.usergrid.persistence.SchemaTest;
 import org.apache.usergrid.persistence.UtilsTest;
 import org.apache.usergrid.persistence.cassandra.QueryProcessorTest;
-import org.apache.usergrid.persistence.cassandra.SimpleIndexBucketLocatorImplTest;
+import org.apache.usergrid.persistence.cassandra.SimpleIndexShardLocatorImplTest;
 import org.apache.usergrid.persistence.query.ir.result.IntersectionIteratorTest;
 import org.apache.usergrid.persistence.query.ir.result.SubtractionIteratorTest;
 import org.apache.usergrid.persistence.query.ir.result.UnionIteratorTest;
@@ -38,7 +38,7 @@ import org.apache.usergrid.persistence.query.ir.result.UnionIteratorTest;
 @RunWith(ConcurrentSuite.class)
 @Suite.SuiteClasses({
         ZookeeperLockManagerTest.class, QueuePathsTest.class, QueryProcessorTest.class,
-        SimpleIndexBucketLocatorImplTest.class, EntityTest.class, QueryTest.class, QueryUtilsTest.class,
+        SimpleIndexShardLocatorImplTest.class, EntityTest.class, QueryTest.class, QueryUtilsTest.class,
         SchemaTest.class, UtilsTest.class, IntersectionIteratorTest.class, SubtractionIteratorTest.class,
         UnionIteratorTest.class 
 })

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/core/src/test/java/org/apache/usergrid/CoreTestSuite.java
----------------------------------------------------------------------
diff --git a/stack/core/src/test/java/org/apache/usergrid/CoreTestSuite.java b/stack/core/src/test/java/org/apache/usergrid/CoreTestSuite.java
index 6d93f7f..314b4e5 100644
--- a/stack/core/src/test/java/org/apache/usergrid/CoreTestSuite.java
+++ b/stack/core/src/test/java/org/apache/usergrid/CoreTestSuite.java
@@ -28,7 +28,7 @@ import org.apache.usergrid.persistence.QueryUtilsTest;
 import org.apache.usergrid.persistence.SchemaTest;
 import org.apache.usergrid.persistence.UtilsTest;
 import org.apache.usergrid.persistence.cassandra.QueryProcessorTest;
-import org.apache.usergrid.persistence.cassandra.SimpleIndexBucketLocatorImplTest;
+import org.apache.usergrid.persistence.cassandra.SimpleIndexShardLocatorImplTest;
 import org.apache.usergrid.persistence.query.ir.result.IntersectionIteratorTest;
 import org.apache.usergrid.persistence.query.ir.result.SubtractionIteratorTest;
 import org.apache.usergrid.persistence.query.ir.result.UnionIteratorTest;
@@ -37,7 +37,7 @@ import org.apache.usergrid.persistence.query.ir.result.UnionIteratorTest;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
         ZookeeperLockManagerTest.class, QueuePathsTest.class, QueryProcessorTest.class,
-        SimpleIndexBucketLocatorImplTest.class, EntityTest.class, QueryTest.class, QueryUtilsTest.class,
+        SimpleIndexShardLocatorImplTest.class, EntityTest.class, QueryTest.class, QueryUtilsTest.class,
         SchemaTest.class, UtilsTest.class, IntersectionIteratorTest.class, SubtractionIteratorTest.class,
         UnionIteratorTest.class
 })

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexBucketLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexBucketLocatorImplTest.java b/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexBucketLocatorImplTest.java
deleted file mode 100644
index 2ef3ea1..0000000
--- a/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexBucketLocatorImplTest.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.cassandra;
-
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Test;
-import org.apache.usergrid.cassandra.Concurrent;
-import org.apache.usergrid.persistence.IndexBucketLocator.IndexType;
-import org.apache.usergrid.utils.UUIDUtils;
-
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Timer;
-import com.yammer.metrics.core.TimerContext;
-
-import static org.junit.Assert.assertEquals;
-
-
-/** @author tnine */
-@Concurrent()
-public class SimpleIndexBucketLocatorImplTest {
-    @Test
-    public void oneBucket() {
-
-        UUID appId = UUIDUtils.newTimeUUID();
-        String entityType = "user";
-        String propName = "firstName";
-
-        SimpleIndexBucketLocatorImpl locator = new SimpleIndexBucketLocatorImpl( 1 );
-
-        List<String> buckets = locator.getBuckets( appId, IndexType.COLLECTION, entityType, propName );
-
-        assertEquals( 1, buckets.size() );
-
-        UUID testId1 = UUIDUtils.minTimeUUID( 0l );
-
-        UUID testId2 = UUIDUtils.minTimeUUID( Long.MAX_VALUE / 2 );
-
-        UUID testId3 = UUIDUtils.minTimeUUID( Long.MAX_VALUE );
-
-        String bucket1 = locator.getBucket( appId, IndexType.COLLECTION, testId1, entityType, propName );
-
-        String bucket2 = locator.getBucket( appId, IndexType.COLLECTION, testId2, entityType, propName );
-
-        String bucket3 = locator.getBucket( appId, IndexType.COLLECTION, testId3, entityType, propName );
-
-        assertEquals( bucket1, "000000000000000000000000000000000000000" );
-        assertEquals( bucket2, "000000000000000000000000000000000000000" );
-        assertEquals( bucket3, "000000000000000000000000000000000000000" );
-    }
-
-
-    @Test
-    public void twoBuckets() {
-
-        UUID appId = UUIDUtils.newTimeUUID();
-        String entityType = "user";
-        String propName = "firstName";
-
-        SimpleIndexBucketLocatorImpl locator = new SimpleIndexBucketLocatorImpl( 2 );
-
-        List<String> buckets = locator.getBuckets( appId, IndexType.COLLECTION, entityType, propName );
-
-        assertEquals( 2, buckets.size() );
-
-        UUID testId1 = UUIDUtils.minTimeUUID( 0l );
-
-        UUID testId2 = UUIDUtils.maxTimeUUID( Long.MAX_VALUE / 2 );
-
-        UUID testId3 = UUIDUtils.minTimeUUID( Long.MAX_VALUE );
-
-        String bucket1 = locator.getBucket( appId, IndexType.COLLECTION, testId1, entityType, propName );
-
-        String bucket2 = locator.getBucket( appId, IndexType.COLLECTION, testId2, entityType, propName );
-
-        String bucket3 = locator.getBucket( appId, IndexType.COLLECTION, testId3, entityType, propName );
-
-        assertEquals( bucket1, "000000000000000000000000000000000000000" );
-        assertEquals( bucket2, "085070591730234615865843651857942052863" );
-        assertEquals( bucket3, "000000000000000000000000000000000000000" );
-    }
-
-
-    @Test
-    public void evenDistribution() {
-
-        UUID appId = UUIDUtils.newTimeUUID();
-        String entityType = "user";
-        String propName = "firstName";
-
-        int bucketSize = 20;
-        float distributionPercentage = .05f;
-
-        // test 100 elements
-        SimpleIndexBucketLocatorImpl locator = new SimpleIndexBucketLocatorImpl( bucketSize );
-
-        List<String> buckets = locator.getBuckets( appId, IndexType.COLLECTION, entityType, propName );
-
-        assertEquals( bucketSize, buckets.size() );
-
-        int testSize = 2000000;
-
-        Map<String, Float> counts = new HashMap<String, Float>();
-
-        final Timer hashes =
-                Metrics.newTimer( SimpleIndexBucketLocatorImplTest.class, "responses", TimeUnit.MILLISECONDS,
-                        TimeUnit.SECONDS );
-
-        // ConsoleReporter.enable(1, TimeUnit.SECONDS);
-
-        /**
-         * Loop through each new UUID and add it's hash to our map
-         */
-        for ( int i = 0; i < testSize; i++ ) {
-            UUID id = UUIDUtils.newTimeUUID();
-
-            final TimerContext context = hashes.time();
-
-            String bucket = locator.getBucket( appId, IndexType.COLLECTION, id, entityType, propName );
-
-            context.stop();
-
-            Float count = counts.get( bucket );
-
-            if ( count == null ) {
-                count = 0f;
-            }
-
-            counts.put( bucket, ++count );
-        }
-
-        /**
-         * Check each entry is within +- 5% of every subsequent entry
-         */
-        List<String> keys = new ArrayList<String>( counts.keySet() );
-        int keySize = keys.size();
-
-        assertEquals( bucketSize, keySize );
-
-        for ( int i = 0; i < keySize; i++ ) {
-
-            float sourceCount = counts.get( keys.get( i ) );
-
-            for ( int j = i + 1; j < keySize; j++ ) {
-                float destCount = counts.get( keys.get( j ) );
-
-                // find the maximum allowed value for the assert based on the
-                // largest value in the pair
-                float maxDelta = Math.max( sourceCount, destCount ) * distributionPercentage;
-
-                assertEquals(
-                        String.format( "Not within %f as percentage for keys '%s' and '%s'", distributionPercentage,
-                                keys.get( i ), keys.get( j ) ), sourceCount, destCount, maxDelta );
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexShardLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexShardLocatorImplTest.java b/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexShardLocatorImplTest.java
new file mode 100644
index 0000000..0c9d1f3
--- /dev/null
+++ b/stack/core/src/test/java/org/apache/usergrid/persistence/cassandra/SimpleIndexShardLocatorImplTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.cassandra;
+
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+import org.apache.usergrid.cassandra.Concurrent;
+import org.apache.usergrid.persistence.IndexBucketLocator.IndexType;
+import org.apache.usergrid.utils.UUIDUtils;
+
+import com.yammer.metrics.Metrics;
+import com.yammer.metrics.core.Timer;
+import com.yammer.metrics.core.TimerContext;
+
+import static org.junit.Assert.assertEquals;
+
+
+/** @author tnine */
+@Concurrent()
+public class SimpleIndexShardLocatorImplTest {
+    @Test
+    public void oneBucket() {
+
+        UUID appId = UUIDUtils.newTimeUUID();
+        String entityType = "user";
+        String propName = "firstName";
+
+        SimpleIndexBucketLocatorImpl locator = new SimpleIndexBucketLocatorImpl( 1 );
+
+        List<String> buckets = locator.getBuckets( appId, IndexType.COLLECTION, entityType, propName );
+
+        assertEquals( 1, buckets.size() );
+
+        UUID testId1 = UUIDUtils.minTimeUUID( 0l );
+
+        UUID testId2 = UUIDUtils.minTimeUUID( Long.MAX_VALUE / 2 );
+
+        UUID testId3 = UUIDUtils.minTimeUUID( Long.MAX_VALUE );
+
+        String bucket1 = locator.getBucket( appId, IndexType.COLLECTION, testId1, entityType, propName );
+
+        String bucket2 = locator.getBucket( appId, IndexType.COLLECTION, testId2, entityType, propName );
+
+        String bucket3 = locator.getBucket( appId, IndexType.COLLECTION, testId3, entityType, propName );
+
+        assertEquals( bucket1, "000000000000000000000000000000000000000" );
+        assertEquals( bucket2, "000000000000000000000000000000000000000" );
+        assertEquals( bucket3, "000000000000000000000000000000000000000" );
+    }
+
+
+    @Test
+    public void twoBuckets() {
+
+        UUID appId = UUIDUtils.newTimeUUID();
+        String entityType = "user";
+        String propName = "firstName";
+
+        SimpleIndexBucketLocatorImpl locator = new SimpleIndexBucketLocatorImpl( 2 );
+
+        List<String> buckets = locator.getBuckets( appId, IndexType.COLLECTION, entityType, propName );
+
+        assertEquals( 2, buckets.size() );
+
+        UUID testId1 = UUIDUtils.minTimeUUID( 0l );
+
+        UUID testId2 = UUIDUtils.maxTimeUUID( Long.MAX_VALUE / 2 );
+
+        UUID testId3 = UUIDUtils.minTimeUUID( Long.MAX_VALUE );
+
+        String bucket1 = locator.getBucket( appId, IndexType.COLLECTION, testId1, entityType, propName );
+
+        String bucket2 = locator.getBucket( appId, IndexType.COLLECTION, testId2, entityType, propName );
+
+        String bucket3 = locator.getBucket( appId, IndexType.COLLECTION, testId3, entityType, propName );
+
+        assertEquals( bucket1, "000000000000000000000000000000000000000" );
+        assertEquals( bucket2, "085070591730234615865843651857942052863" );
+        assertEquals( bucket3, "000000000000000000000000000000000000000" );
+    }
+
+
+    @Test
+    public void evenDistribution() {
+
+        UUID appId = UUIDUtils.newTimeUUID();
+        String entityType = "user";
+        String propName = "firstName";
+
+        int bucketSize = 20;
+        float distributionPercentage = .05f;
+
+        // test 100 elements
+        SimpleIndexBucketLocatorImpl locator = new SimpleIndexBucketLocatorImpl( bucketSize );
+
+        List<String> buckets = locator.getBuckets( appId, IndexType.COLLECTION, entityType, propName );
+
+        assertEquals( bucketSize, buckets.size() );
+
+        int testSize = 2000000;
+
+        Map<String, Float> counts = new HashMap<String, Float>();
+
+        final Timer hashes =
+                Metrics.newTimer( SimpleIndexShardLocatorImplTest.class, "responses", TimeUnit.MILLISECONDS,
+                        TimeUnit.SECONDS );
+
+        // ConsoleReporter.enable(1, TimeUnit.SECONDS);
+
+        /**
+         * Loop through each new UUID and add it's hash to our map
+         */
+        for ( int i = 0; i < testSize; i++ ) {
+            UUID id = UUIDUtils.newTimeUUID();
+
+            final TimerContext context = hashes.time();
+
+            String bucket = locator.getBucket( appId, IndexType.COLLECTION, id, entityType, propName );
+
+            context.stop();
+
+            Float count = counts.get( bucket );
+
+            if ( count == null ) {
+                count = 0f;
+            }
+
+            counts.put( bucket, ++count );
+        }
+
+        /**
+         * Check each entry is within +- 5% of every subsequent entry
+         */
+        List<String> keys = new ArrayList<String>( counts.keySet() );
+        int keySize = keys.size();
+
+        assertEquals( bucketSize, keySize );
+
+        for ( int i = 0; i < keySize; i++ ) {
+
+            float sourceCount = counts.get( keys.get( i ) );
+
+            for ( int j = i + 1; j < keySize; j++ ) {
+                float destCount = counts.get( keys.get( j ) );
+
+                // find the maximum allowed value for the assert based on the
+                // largest value in the pair
+                float maxDelta = Math.max( sourceCount, destCount ) * distributionPercentage;
+
+                assertEquals(
+                        String.format( "Not within %f as percentage for keys '%s' and '%s'", distributionPercentage,
+                                keys.get( i ), keys.get( j ) ), sourceCount, destCount, maxDelta );
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/pom.xml
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/pom.xml b/stack/corepersistence/common/pom.xml
index ee6a67b..671aefa 100644
--- a/stack/corepersistence/common/pom.xml
+++ b/stack/corepersistence/common/pom.xml
@@ -71,6 +71,14 @@
       <version>${guicyfig.version}</version>
     </dependency>
 
+
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-collections4</artifactId>
+      <version>4.0</version>
+    </dependency>
+
+
     <!-- Google Guice -->
 
     <dependency>
@@ -144,6 +152,8 @@
     </dependency>
 
 
+
+
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
index fe04875..265afa5 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKey.java
@@ -21,16 +21,13 @@ package org.apache.usergrid.persistence.core.astyanax;/*
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.model.entity.Id;
 
 
 /**
  * A scoped row key that also includes an index, which can be used for consistent hashing.
- *
- * @param <K>
  */
-public class BucketScopedRowKey< K> extends ScopedRowKey< K> {
+public class BucketScopedRowKey<K> extends ScopedRowKey<K> {
 
     private final int bucketNumber;
 
@@ -48,7 +45,6 @@ public class BucketScopedRowKey< K> extends ScopedRowKey< K> {
 
     /**
      * Get the bucket number
-     * @return
      */
     public int getBucketNumber() {
         return bucketNumber;
@@ -96,8 +92,7 @@ public class BucketScopedRowKey< K> extends ScopedRowKey< K> {
     /**
      * Utility function to generate a new key from the scope
      */
-    public static < K> BucketScopedRowKey< K> fromKey( final Id scope, final K key,
-                                                                                    final int bucketNumber ) {
+    public static <K> BucketScopedRowKey<K> fromKey( final Id scope, final K key, final int bucketNumber ) {
         return new BucketScopedRowKey<>( scope, key, bucketNumber );
     }
 
@@ -105,9 +100,9 @@ public class BucketScopedRowKey< K> extends ScopedRowKey< K> {
     /**
      * Create a list of all buckets from [0,  totalBuckets}.  Note that this is an n-1 0 based system
      */
-    public static < K> List<BucketScopedRowKey< K>> fromRange( final Id scope, final K key, final int... buckets ) {
+    public static <K> List<BucketScopedRowKey<K>> fromRange( final Id scope, final K key, final int... buckets ) {
 
-        final List<BucketScopedRowKey< K>> results = new ArrayList<>( buckets.length );
+        final List<BucketScopedRowKey<K>> results = new ArrayList<>( buckets.length  );
 
 
         for ( int i = 0; i < buckets.length; i++ ) {

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java
index 401cc83..cfbcfc7 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/BucketScopedRowKeySerializer.java
@@ -22,15 +22,12 @@ package org.apache.usergrid.persistence.core.astyanax;
 
 import java.nio.ByteBuffer;
 
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
-import org.apache.usergrid.persistence.core.scope.ApplicationScopeImpl;
 import org.apache.usergrid.persistence.model.entity.Id;
 
 import com.netflix.astyanax.model.CompositeBuilder;
 import com.netflix.astyanax.model.CompositeParser;
 import com.netflix.astyanax.model.Composites;
 import com.netflix.astyanax.serializers.AbstractSerializer;
-import com.netflix.astyanax.serializers.IntegerSerializer;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfig.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfig.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfig.java
index 3db4bcb..0ce7df1 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfig.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfig.java
@@ -24,5 +24,11 @@ public interface CassandraConfig {
      */
     public ConsistencyLevel getWriteCL();
 
+    /**
+     * Return the number of shards that has been set in the property file
+     * @return
+     */
+    public int[] getShardSettings();
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfigImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfigImpl.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfigImpl.java
index a0698de..ce80ea2 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfigImpl.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CassandraConfigImpl.java
@@ -38,6 +38,7 @@ public class CassandraConfigImpl implements CassandraConfig {
 
     private ConsistencyLevel readCl;
     private ConsistencyLevel writeCl;
+    private int[] shardSettings;
 
 
     @Inject
@@ -47,6 +48,9 @@ public class CassandraConfigImpl implements CassandraConfig {
 
         this.writeCl = ConsistencyLevel.valueOf( cassandraFig.getWriteCL() );
 
+        this.shardSettings = parseShardSettings( cassandraFig.getShardValues() );
+
+
 
         //add the listeners to update the values
         cassandraFig.addPropertyChangeListener( new PropertyChangeListener() {
@@ -61,6 +65,9 @@ public class CassandraConfigImpl implements CassandraConfig {
                 else if ( CassandraFig.WRITE_CL.equals( propName ) ) {
                     writeCl = ConsistencyLevel.valueOf( evt.getNewValue().toString() );
                 }
+                else if (CassandraFig.SHARD_VALUES.equals(propName)){
+                    shardSettings = parseShardSettings( cassandraFig.getShardValues() );
+                }
             }
         } );
     }
@@ -78,4 +85,20 @@ public class CassandraConfigImpl implements CassandraConfig {
     }
 
 
+    @Override
+    public int[] getShardSettings() {
+      return shardSettings;
+    }
+
+    private int[] parseShardSettings(final String value){
+        final String[] shardHistory = value.split( "," );
+
+        int[] settings = new int [shardHistory.length];
+
+        for(int i = 0; i < shardHistory.length; i ++){
+            settings[i] = Integer.parseInt( shardHistory[i] );
+        }
+
+      return settings;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/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 1d4066d..4cf76f4 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
@@ -37,6 +37,8 @@ public interface CassandraFig extends GuicyFig {
 
     public static final String WRITE_CL = "usergrid.write.cl";
 
+    public static final String SHARD_VALUES = "cassandra.shardvalues";
+
     @Key( "cassandra.hosts" )
     String getHosts();
 
@@ -81,4 +83,16 @@ public interface CassandraFig extends GuicyFig {
     @Key(WRITE_CL)
     String getWriteCL();
 
+    /**
+     * Return the history of all shard values which are immutable.  For instance, if shard values
+     * are initially set to 20 (the default) then increased to 40, the property should contain the string of
+     * "20, 40" so that we can read historic data.
+     *
+     * @return
+     */
+    @Default("20")
+    @Key(SHARD_VALUES)
+    String getShardValues();
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CompositeFieldSerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CompositeFieldSerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CompositeFieldSerializer.java
index 5f5e8e2..e241859 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CompositeFieldSerializer.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/CompositeFieldSerializer.java
@@ -41,4 +41,5 @@ public interface CompositeFieldSerializer<K> {
      * Create an instance from the composite
      */
     public K fromComposite( CompositeParser composite );
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/IdRowCompositeSerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/IdRowCompositeSerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/IdRowCompositeSerializer.java
index 8e518ae..09bde45 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/IdRowCompositeSerializer.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/IdRowCompositeSerializer.java
@@ -60,6 +60,7 @@ public class IdRowCompositeSerializer implements CompositeFieldSerializer<Id> {
     }
 
 
+
     /**
      * Get the singleton serializer
      */

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIterator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIterator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIterator.java
index 0a7cca5..1f8a359 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIterator.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIterator.java
@@ -23,14 +23,19 @@ package org.apache.usergrid.persistence.core.astyanax;
 
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
+import java.util.LinkedHashSet;
 import java.util.NoSuchElementException;
+import java.util.Set;
 import java.util.TreeSet;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.commons.collections4.iterators.PushbackIterator;
+
 import org.apache.usergrid.persistence.core.hystrix.HystrixCassandra;
 
 import com.netflix.astyanax.Keyspace;
@@ -72,7 +77,8 @@ public class MultiRowColumnIterator<R, C, T> implements Iterator<T> {
 
     private T startColumn;
 
-    private boolean moreToFollow;
+
+    private int lastReturnCount;
 
     private Iterator<T> currentColumnIterator;
 
@@ -106,7 +112,7 @@ public class MultiRowColumnIterator<R, C, T> implements Iterator<T> {
         this.rowKeys = rowKeys;
         this.keyspace = keyspace;
         this.consistencyLevel = consistencyLevel;
-        this.moreToFollow = false;
+        this.lastReturnCount = -1;
 
         //        seenResults = new HashMap<>( pageSize * 10 );
     }
@@ -115,7 +121,7 @@ public class MultiRowColumnIterator<R, C, T> implements Iterator<T> {
     @Override
     public boolean hasNext() {
 
-        if ( currentColumnIterator == null || ( !currentColumnIterator.hasNext() && moreToFollow ) ) {
+        if ( currentColumnIterator == null || ( !currentColumnIterator.hasNext() && lastReturnCount == pageSize || lastReturnCount == -1 ) ) {
             advance();
         }
 
@@ -132,7 +138,10 @@ public class MultiRowColumnIterator<R, C, T> implements Iterator<T> {
 
         final T next = currentColumnIterator.next();
 
-        //        LOG.trace( "Emitting {}", next );
+
+        //set our start pointer
+        startColumn = next;
+        lastReturnCount ++;
 
         return next;
     }
@@ -146,14 +155,18 @@ public class MultiRowColumnIterator<R, C, T> implements Iterator<T> {
 
     public void advance() {
 
+        this.lastReturnCount = 0;
 
         /**
          * If the edge is present, we need to being seeking from this
          */
 
+        final boolean skipFirstColumn = startColumn != null;
+
+
 
         //TODO, finalize why this isn't working as expected
-        final int selectSize = startColumn == null ? pageSize : pageSize + 1;
+        final int selectSize = skipFirstColumn ? pageSize + 1 : pageSize;
 
         final RangeBuilder rangeBuilder = new RangeBuilder();
 
@@ -180,73 +193,174 @@ public class MultiRowColumnIterator<R, C, T> implements Iterator<T> {
 
         final Rows<R, C> result = HystrixCassandra.user( query ).getResult();
 
-        final TreeSet<T> mergedResults = new TreeSet<>( comparator );
-
 
         //now aggregate them together
 
-        for ( final R key : result.getKeys() ) {
-            final ColumnList<C> columns = result.getRow( key ).getColumns();
-            final int size = columns.size();
+        //this is an optimization.  It's faster to see if we only have values for one row,
+        // then return the iterator of those columns than
+        //do a merge if only one row has data.
+
+
+        final Iterator<T> mergedResults;
+
+        if ( containsSingleRowOnly( result ) ) {
+            mergedResults = singleRowResult( result );
+        }
+        else {
+            mergedResults = mergeResults( result, selectSize );
+        }
+
+
+
+
+
+        //we've parsed everything truncate to the first pageSize, it's all we can ensure is correct without another
+        //trip back to cassandra
+
+        //discard our first element (maybe)
+
+        PushbackIterator<T> iterator = new PushbackIterator<>( mergedResults );
+
+        //we have a first column to discard, our iterator has a value, but they are not equal, meaning we can't discard it
+        //as we have from the check so we need to push it back into the iterator
+        if(skipFirstColumn && iterator.hasNext()) {
+
+            final T firstResult = iterator.next();
+
+            if(comparator.compare( startColumn, firstResult ) != 0){
+                iterator.pushback( firstResult );
+            }
+
+        }
+
+        currentColumnIterator = iterator;
+
+        LOG.trace( "Finished parsing {} rows for results", rowKeys.size() );
+    }
+
+
+    /**
+     * Return true if we have < 2 rows with columns, false otherwise
+     */
+    private boolean containsSingleRowOnly( final Rows<R, C> result ) {
 
-            int readIndex = 0;
+        int count = 0;
 
-            //skip the first since it's equal and has been set
-            if ( startColumn != null && size > 0 ) {
-                final T returnedValue = columnParser.parseColumn( columns.getColumnByIndex( 0 ) );
+        for ( R key : result.getKeys() ) {
+            if ( result.getRow( key ).getColumns().size() > 0 ) {
+                count++;
 
-                if ( comparator.compare( returnedValue, startColumn ) == 0 ) {
-                    readIndex++;
+                //we have more than 1 row with values, return them
+                if ( count > 1 ) {
+                    return false;
                 }
             }
+        }
+
+        return true;
+    }
+
+
+    /**
+     * A single row is present, only parse the single row
+     * @param result
+     * @return
+     */
+    private Iterator<T> singleRowResult( final Rows<R, C> result ) {
+
 
+        for ( R key : result.getKeys() ) {
+            final ColumnList<C> columnList = result.getRow( key ).getColumns();
 
-//            T previous = null;
+            if ( columnList.size() > 0 ) {
+
+                return new SingleRowIterator(columnList);
+
+            }
+        }
+
+        //we didn't have any results, just return nothing
+        return Collections.<T>emptyList().iterator();
+    }
+
+
+    /**
+     * Multiple rows are present, merge them into a single result set
+     * @param result
+     * @return
+     */
+    private Iterator<T> mergeResults( final Rows<R, C> result, final int maxSize ) {
+
+        final TreeSet<T> mergedResults = new TreeSet<>( comparator );
+
+
+        for ( final R key : result.getKeys() ) {
+            final ColumnList<C> columns = result.getRow( key ).getColumns();
+
+
+            for (final Column<C> column :columns  ) {
 
-            for (; readIndex < size; readIndex++ ) {
-                final Column<C> column = columns.getColumnByIndex( readIndex );
                 final T returnedValue = columnParser.parseColumn( column );
 
                 /**
-                 * DO NOT remove this section of code. If you're seeing inconsistent results during shard transition, you'll
+                 * DO NOT remove this section of code. If you're seeing inconsistent results during shard transition,
+                 * you'll
                  * need to enable this
                  */
-//
-//                if ( previous != null && comparator.compare( previous, returnedValue ) == 0 ) {
-//                    throw new RuntimeException( String.format(
-//                            "Cassandra returned 2 unique columns, but your comparator marked them as equal.  This " +
-//                                    "indicates a bug in your comparator.  Previous value was %s and current value is " +
-//                                    "%s",
-//                            previous, returnedValue ) );
-//                }
-//
-//                previous = returnedValue;
+                //
+                //                if ( previous != null && comparator.compare( previous, returnedValue ) == 0 ) {
+                //                    throw new RuntimeException( String.format(
+                //                            "Cassandra returned 2 unique columns,
+                // but your comparator marked them as equal.  This " +
+                //                                    "indicates a bug in your comparator.  Previous value was %s and
+                // current value is " +
+                //                                    "%s",
+                //                            previous, returnedValue ) );
+                //                }
+                //
+                //                previous = returnedValue;
 
                 mergedResults.add( returnedValue );
 
                 //prune the mergedResults
-                while ( mergedResults.size() > pageSize ) {
+                while ( mergedResults.size() > maxSize ) {
                     mergedResults.pollLast();
                 }
             }
 
-            LOG.trace( "Read {} columns from row key {}", readIndex, key );
             LOG.trace( "Candidate result set size is {}", mergedResults.size() );
+
         }
+        return mergedResults.iterator();
+    }
 
 
-        //we've parsed everything truncate to the first pageSize, it's all we can ensure is correct without another
-        //trip back to cassandra
+    /**
+     * Iterator wrapper that parses as it iterates for single row cases
+     */
+    private class SingleRowIterator implements Iterator<T> {
+
+        private Iterator<Column<C>> columnIterator;
 
-        if(!mergedResults.isEmpty()) {
-            startColumn = mergedResults.last();
+        private SingleRowIterator (final ColumnList<C> columns){
+            this.columnIterator = columns.iterator();
         }
+        @Override
+        public boolean hasNext() {
+            return columnIterator.hasNext();
+        }
+
 
-        moreToFollow = mergedResults.size() == pageSize;
+        @Override
+        public T next() {
+            return columnParser.parseColumn( columnIterator.next() );
+        }
 
-        currentColumnIterator = mergedResults.iterator();
 
-        LOG.trace( "Finished parsing {} rows for a total of {} results", rowKeys.size(), mergedResults.size() );
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException( "Unable to remove single row" );
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java
index 30e1792..b797eb2 100644
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/astyanax/ScopedRowKeySerializer.java
@@ -22,8 +22,6 @@ package org.apache.usergrid.persistence.core.astyanax;
 
 import java.nio.ByteBuffer;
 
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
-import org.apache.usergrid.persistence.core.scope.ApplicationScopeImpl;
 import org.apache.usergrid.persistence.model.entity.Id;
 
 import com.netflix.astyanax.model.CompositeBuilder;
@@ -49,15 +47,13 @@ public class ScopedRowKeySerializer<K> extends AbstractSerializer<ScopedRowKey<K
     private final CompositeFieldSerializer<K> keySerializer;
 
 
-
-
     public ScopedRowKeySerializer( final CompositeFieldSerializer<K> keySerializer ) {
         this.keySerializer = keySerializer;
     }
 
 
     @Override
-    public ByteBuffer toByteBuffer( final ScopedRowKey< K> scopedRowKey ) {
+    public ByteBuffer toByteBuffer( final ScopedRowKey<K> scopedRowKey ) {
 
         final CompositeBuilder builder = Composites.newCompositeBuilder();
 
@@ -80,7 +76,7 @@ public class ScopedRowKeySerializer<K> extends AbstractSerializer<ScopedRowKey<K
 
         final K value = keySerializer.fromComposite( parser );
 
-        return new ScopedRowKey<K>(  orgId, value );
+        return new ScopedRowKey<K>( orgId, value );
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/CurrentVersion.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/CurrentVersion.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/CurrentVersion.java
new file mode 100644
index 0000000..c375df3
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/CurrentVersion.java
@@ -0,0 +1,42 @@
+/*
+ *
+ *  * 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.guice;
+
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import com.google.inject.BindingAnnotation;
+
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.PARAMETER;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+
+/**
+ * Represents 2 versions of an impl.  Generally used for online migration.  This represents the version that is the
+ * current version of the implementation.  I.E the "new" version.
+ */
+@BindingAnnotation
+@Target({ FIELD, PARAMETER, METHOD }) @Retention(RUNTIME)
+public @interface CurrentVersion {}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/PreviousVersion.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/PreviousVersion.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/PreviousVersion.java
new file mode 100644
index 0000000..edda25a
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/PreviousVersion.java
@@ -0,0 +1,42 @@
+/*
+ *
+ *  * 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.guice;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import com.google.inject.BindingAnnotation;
+
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.PARAMETER;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+
+
+/**
+ * Represents 2 versions of an impl.  Generally used for online migration.  This represents the version that is the
+ * previous version of the implementation.  I.E the "old" version.
+ */
+@BindingAnnotation
+@Target({ FIELD, PARAMETER, METHOD }) @Retention(RUNTIME)
+public @interface PreviousVersion {}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/ProxyVersion.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/ProxyVersion.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/ProxyVersion.java
new file mode 100644
index 0000000..5209882
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/guice/ProxyVersion.java
@@ -0,0 +1,42 @@
+/*
+ *
+ *  * 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.guice;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import com.google.inject.BindingAnnotation;
+
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.PARAMETER;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+
+
+/**
+ * Represents 2 versions of an impl.  Generally used for online migration.  This represents the version of the impl
+ * that is responsible for bridging the versions from previous to current during the migration.
+ */
+@BindingAnnotation
+@Target({ FIELD, PARAMETER, METHOD }) @Retention(RUNTIME)
+public @interface ProxyVersion {}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java
deleted file mode 100644
index 52ffeba..0000000
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/BucketLocator.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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.hash;
-
-
-
-import com.google.common.hash.Funnel;
-import com.google.common.hash.HashCode;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-
-
-/**
- * Simple utility to locate which bucket an element should be located based on it's funnel
- *
- */
-public class BucketLocator<T> {
-
-    /**
-     * Use the murmur 3 hash
-     */
-    private static final HashFunction HASHER = Hashing.murmur3_128();
-
-    private final int totalBuckets;
-    private final Funnel<T> funnel;
-
-
-    public BucketLocator(final Funnel<T> funnel, final int totalBuckets ) {
-        this.funnel = funnel;
-        this.totalBuckets = totalBuckets;
-    }
-
-
-    /**
-     * Locate the bucket number given the value, the funnel and the total buckets.
-     *
-     * Assigns to {@code hashCode} a "bucket" in the range {@code [0, buckets)}, in a uniform manner that minimizes the
-     * need for remapping as {@code buckets} grows. That is, {@code consistentHash(h, n)} equals:
-     *
-     * <ul> <li>{@code n - 1}, with approximate probability {@code 1/n} <li>{@code consistentHash(h, n - 1)}, otherwise
-     * (probability {@code 1 - 1/n}) </ul>
-     *
-     * <p>See the <a href="http://en.wikipedia.org/wiki/Consistent_hashing">wikipedia article on consistent hashing</a>
-     * for more information.
-     *
-     * <p>See <a href="http://arxiv.org/pdf/1406.2294v1.pdf">this paper</a> for more details on the algorithm</p>
-     *
-     *
-     * Note that after testing, increasing buckets does NOT yield the expected results.  You will need an algorithm
-     * that manually walks a tree.  See
-     *
-     */
-    public int getBucket( T value ) {
-
-        final HashCode hashCode = HASHER.hashObject( value, funnel );
-
-        int owningIndex = Hashing.consistentHash( hashCode, totalBuckets );
-
-        return owningIndex;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
deleted file mode 100644
index 3d81389..0000000
--- a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocator.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- *
- *  * 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.hash;/*
- * 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.
- */
-
-
-import com.google.common.hash.Funnel;
-
-
-/**
- * An algorithm that will generate all possible keys for different "Levels" of sharding.  For instance, imagine this
- * scheme.
- *
- * 1 Shard 2 Shards 4 Shards 8 Shards
- *
- * (note that we do not need to expand by 2x each time, this is merely an example).
- *
- * When seeking on a string key, for 4 levels of the key, we get 4 different keys due to different shard sizes. This is
- * faster than seeking ALL shards, since this would result in 15 shards, vs 4 in the example.
- */
-public class ExpandingBucketLocator<T> {
-
-    private final BucketLocator<T>[] bucketLocatorList;
-
-
-    /**
-     * Create a new instance with the specified history. For instance, from the javadoc above, the constructor would
-     * contains {8, 4, 3, 2, 1}.  Shards are returned in the size order they are given in the constructor
-     */
-    public ExpandingBucketLocator( final Funnel<T> funnel, final int... bucketSizes ) {
-
-        bucketLocatorList = new BucketLocator[bucketSizes.length];
-
-        for ( int i = 0; i < bucketSizes.length; i++ ) {
-            bucketLocatorList[i] = new BucketLocator<>( funnel, bucketSizes[i] );
-        }
-    }
-
-
-    /**
-     * Hash the results, and return them in the same order as specified in the constructor
-     */
-    public int[] getAllBuckets( T hash ) {
-        int[] results = new int[bucketLocatorList.length];
-
-        for ( int i = 0; i < bucketLocatorList.length; i++ ) {
-            results[i] = bucketLocatorList[i].getBucket( hash );
-        }
-
-        return results;
-    }
-
-
-    /**
-     * Get the current bucket for the hash value.  Hashes from the first element in the list
-     */
-    public int getCurrentBucket( T hash ) {
-        return bucketLocatorList[0].getBucket( hash );
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocator.java
new file mode 100644
index 0000000..02d4d1e
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocator.java
@@ -0,0 +1,95 @@
+/*
+ *
+ *  * 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.shard;/*
+ * 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.
+ */
+
+
+import com.google.common.hash.Funnel;
+
+
+/**
+ * An algorithm that will generate all possible keys for different "Levels" of sharding.  For instance, imagine this
+ * scheme.
+ *
+ * 1 Shard 2 Shards 4 Shards 8 Shards
+ *
+ * (note that we do not need to expand by 2x each time, this is merely an example).
+ *
+ * When seeking on a string key, for 4 levels of the key, we get 4 different keys due to different shard sizes. This is
+ * faster than seeking ALL shards, since this would result in 15 shards, vs 4 in the example.
+ */
+public class ExpandingShardLocator<T> {
+
+    private final ShardLocator<T>[] shardLocatorList;
+
+
+    /**
+     * Create a new instance with the specified history. For instance, from the javadoc above, the constructor would
+     * contains {8, 4, 3, 2, 1}.  Shards are returned in the size order they are given in the constructor
+     */
+    public ExpandingShardLocator( final Funnel<T> funnel, final int... bucketSizes ) {
+
+        shardLocatorList = new ShardLocator[bucketSizes.length];
+
+        for ( int i = 0; i < bucketSizes.length; i++ ) {
+            shardLocatorList[i] = new ShardLocator<>( funnel, bucketSizes[i] );
+        }
+    }
+
+
+    /**
+     * Hash the results, and return them in the same order as specified in the constructor
+     */
+    public int[] getAllBuckets( T hash ) {
+        int[] results = new int[shardLocatorList.length];
+
+        for ( int i = 0; i < shardLocatorList.length; i++ ) {
+            results[i] = shardLocatorList[i].getBucket( hash );
+        }
+
+        return results;
+    }
+
+
+    /**
+     * Get the current bucket for the hash value.  Hashes from the first element in the list
+     */
+    public int getCurrentBucket( T hash ) {
+        return shardLocatorList[0].getBucket( hash );
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ShardLocator.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ShardLocator.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ShardLocator.java
new file mode 100644
index 0000000..e3dc810
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/ShardLocator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.shard;
+
+
+
+import com.google.common.hash.Funnel;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+
+
+/**
+ * Simple utility to locate which bucket an element should be located based on it's funnel
+ *
+ */
+public class ShardLocator<T> {
+
+    /**
+     * Use the murmur 3 hash
+     */
+    private static final HashFunction HASHER = Hashing.murmur3_128();
+
+    private final int totalBuckets;
+    private final Funnel<T> funnel;
+
+
+    public ShardLocator( final Funnel<T> funnel, final int totalBuckets ) {
+        this.funnel = funnel;
+        this.totalBuckets = totalBuckets;
+    }
+
+
+    /**
+     * Locate the bucket number given the value, the funnel and the total buckets.
+     *
+     * Assigns to {@code hashCode} a "bucket" in the range {@code [0, buckets)}, in a uniform manner that minimizes the
+     * need for remapping as {@code buckets} grows. That is, {@code consistentHash(h, n)} equals:
+     *
+     * <ul> <li>{@code n - 1}, with approximate probability {@code 1/n} <li>{@code consistentHash(h, n - 1)}, otherwise
+     * (probability {@code 1 - 1/n}) </ul>
+     *
+     * <p>See the <a href="http://en.wikipedia.org/wiki/Consistent_hashing">wikipedia article on consistent hashing</a>
+     * for more information.
+     *
+     * <p>See <a href="http://arxiv.org/pdf/1406.2294v1.pdf">this paper</a> for more details on the algorithm</p>
+     *
+     *
+     * Note that after testing, increasing buckets does NOT yield the expected results.  You will need an algorithm
+     * that manually walks a tree.  See
+     *
+     */
+    public int getBucket( T value ) {
+
+        final HashCode hashCode = HASHER.hashObject( value, funnel );
+
+        int owningIndex = Hashing.consistentHash( hashCode, totalBuckets );
+
+        return owningIndex;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/StringHashUtils.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/StringHashUtils.java b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/StringHashUtils.java
new file mode 100644
index 0000000..afe94a9
--- /dev/null
+++ b/stack/corepersistence/common/src/main/java/org/apache/usergrid/persistence/core/shard/StringHashUtils.java
@@ -0,0 +1,54 @@
+/*
+ *
+ *  * 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.shard;/*
+ * 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.
+ */
+
+
+import java.nio.charset.Charset;
+
+
+/**
+ * String utils for hashing string bytes
+ */
+public class StringHashUtils {
+
+    /**
+     * The UTF8 charset name
+     */
+    public static final Charset UTF8 = Charset.forName( "UTF8" );
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/ColumnNameIteratorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/ColumnNameIteratorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/ColumnNameIteratorTest.java
index aabab24..2522f33 100644
--- a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/ColumnNameIteratorTest.java
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/ColumnNameIteratorTest.java
@@ -77,6 +77,12 @@ public class ColumnNameIteratorTest {
             public ConsistencyLevel getWriteCL() {
                 return ConsistencyLevel.CL_QUORUM;
             }
+
+
+            @Override
+            public int[] getShardSettings() {
+                return new int[]{20};
+            }
         };
 
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiKeyColumnNameIteratorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiKeyColumnNameIteratorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiKeyColumnNameIteratorTest.java
index 6762588..61b9e54 100644
--- a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiKeyColumnNameIteratorTest.java
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiKeyColumnNameIteratorTest.java
@@ -82,6 +82,12 @@ public class MultiKeyColumnNameIteratorTest {
             public ConsistencyLevel getWriteCL() {
                 return ConsistencyLevel.CL_QUORUM;
             }
+
+
+            @Override
+            public int[] getShardSettings() {
+                return new int[]{20};
+            }
         };
 
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIteratorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIteratorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIteratorTest.java
index c9eeb07..8b4c7b7 100644
--- a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIteratorTest.java
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/astyanax/MultiRowColumnIteratorTest.java
@@ -85,6 +85,12 @@ public class MultiRowColumnIteratorTest {
             public ConsistencyLevel getWriteCL() {
                 return ConsistencyLevel.CL_QUORUM;
             }
+
+
+            @Override
+            public int[] getShardSettings() {
+                return new int[]{20};
+            }
         };
 
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java
deleted file mode 100644
index 49c306b..0000000
--- a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/BucketLocatorTest.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.hash;
-
-
-import java.nio.charset.Charset;
-
-import org.junit.Test;
-
-import com.google.common.hash.Funnel;
-import com.google.common.hash.PrimitiveSink;
-
-import junit.framework.TestCase;
-
-import static org.junit.Assert.assertEquals;
-
-
-/**
- * Simple test that validates hashing is actually consistent as buckets grow
- */
-public class BucketLocatorTest {
-
-    public static final Funnel<String> STRING_FUNNEL = new Funnel<String>() {
-
-        private Charset UTF8 = Charset.forName( "UTF8" );
-
-
-        @Override
-        public void funnel( final String from, final PrimitiveSink into ) {
-            into.putString( from, UTF8 );
-        }
-    };
-
-
-    @Test
-    public void stringHashing() {
-
-        final String hashValue = "keystring";
-
-        BucketLocator<String> bucketLocator1 = new BucketLocator<>(STRING_FUNNEL,  100 );
-
-        int index1 = bucketLocator1.getBucket( hashValue );
-
-        BucketLocator<String> bucketLocator2 = new BucketLocator<>( STRING_FUNNEL, 100 );
-
-        int index2 = bucketLocator2.getBucket( hashValue );
-
-        assertEquals( "Same index expected", index1, index2 );
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
deleted file mode 100644
index ce691fc..0000000
--- a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/hash/ExpandingBucketLocatorTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- *
- *  * 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.hash;
-
-
-import java.util.Arrays;
-
-import org.junit.Test;
-
-import junit.framework.TestCase;
-
-
-/**
- * Tests that we get consistent results when hashing
- */
-public class ExpandingBucketLocatorTest extends TestCase {
-
-
-    @Test
-    public void testConsistency(){
-
-        final ExpandingBucketLocator<String> expandingBucketLocator1 = new ExpandingBucketLocator<>( BucketLocatorTest.STRING_FUNNEL, 20, 10, 1 );
-        final ExpandingBucketLocator<String> expandingBucketLocator2 = new ExpandingBucketLocator<>( BucketLocatorTest.STRING_FUNNEL, 20, 10, 1 );
-
-        final String key = "mytestkey";
-
-
-        int[] results1 = expandingBucketLocator1.getAllBuckets(key  );
-        int[] results2 = expandingBucketLocator2.getAllBuckets(key  );
-
-        assertTrue( "Same results returned", Arrays.equals(results1, results2));
-
-        assertTrue("Within bounds", results1[0] <= 19);
-        assertTrue("Within bounds", results1[1] <= 9);
-        assertTrue("Within bounds", results1[2] <= 0);
-
-        //test the first hash
-        int newestBucket = expandingBucketLocator1.getCurrentBucket( key );
-
-        assertEquals("Same bucket returned", results1[0], newestBucket);
-
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocatorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocatorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocatorTest.java
new file mode 100644
index 0000000..5cbb194
--- /dev/null
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ExpandingShardLocatorTest.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * 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.shard;
+
+
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+
+/**
+ * Tests that we get consistent results when hashing
+ */
+public class ExpandingShardLocatorTest extends TestCase {
+
+
+    @Test
+    public void testConsistency(){
+
+        final ExpandingShardLocator<String>
+                expandingShardLocator1 = new ExpandingShardLocator<>( ShardLocatorTest.STRING_FUNNEL, 20, 10, 1 );
+        final ExpandingShardLocator<String>
+                expandingShardLocator2 = new ExpandingShardLocator<>( ShardLocatorTest.STRING_FUNNEL, 20, 10, 1 );
+
+        final String key = "mytestkey";
+
+
+        int[] results1 = expandingShardLocator1.getAllBuckets(key  );
+        int[] results2 = expandingShardLocator2.getAllBuckets(key  );
+
+        assertTrue( "Same results returned", Arrays.equals(results1, results2));
+
+        assertTrue("Within bounds", results1[0] <= 19);
+        assertTrue("Within bounds", results1[1] <= 9);
+        assertTrue("Within bounds", results1[2] <= 0);
+
+        //test the first hash
+        int newestBucket = expandingShardLocator1.getCurrentBucket( key );
+
+        assertEquals("Same bucket returned", results1[0], newestBucket);
+
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ShardLocatorTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ShardLocatorTest.java b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ShardLocatorTest.java
new file mode 100644
index 0000000..dae85bd
--- /dev/null
+++ b/stack/corepersistence/common/src/test/java/org/apache/usergrid/persistence/core/shard/ShardLocatorTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.shard;
+
+
+import java.nio.charset.Charset;
+
+import org.junit.Test;
+
+import com.google.common.hash.Funnel;
+import com.google.common.hash.PrimitiveSink;
+
+import static org.junit.Assert.assertEquals;
+
+
+/**
+ * Simple test that validates hashing is actually consistent as buckets grow
+ */
+public class ShardLocatorTest {
+
+    public static final Funnel<String> STRING_FUNNEL = new Funnel<String>() {
+
+        private Charset UTF8 = Charset.forName( "UTF8" );
+
+
+        @Override
+        public void funnel( final String from, final PrimitiveSink into ) {
+            into.putString( from, UTF8 );
+        }
+    };
+
+
+    @Test
+    public void stringHashing() {
+
+        final String hashValue = "keystring";
+
+        ShardLocator<String> shardLocator1 = new ShardLocator<>(STRING_FUNNEL,  100 );
+
+        int index1 = shardLocator1.getBucket( hashValue );
+
+        ShardLocator<String> shardLocator2 = new ShardLocator<>( STRING_FUNNEL, 100 );
+
+        int index2 = shardLocator2.getBucket( hashValue );
+
+        assertEquals( "Same index expected", index1, index2 );
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/guice/GraphModule.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/guice/GraphModule.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/guice/GraphModule.java
index 9538be0..41a66c5 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/guice/GraphModule.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/guice/GraphModule.java
@@ -41,7 +41,7 @@ import org.apache.usergrid.persistence.graph.impl.stage.NodeDeleteListenerImpl;
 import org.apache.usergrid.persistence.graph.serialization.EdgeMetadataSerialization;
 import org.apache.usergrid.persistence.graph.serialization.EdgeSerialization;
 import org.apache.usergrid.persistence.graph.serialization.NodeSerialization;
-import org.apache.usergrid.persistence.graph.serialization.impl.EdgeMetadataSerializationImpl;
+import org.apache.usergrid.persistence.graph.serialization.impl.EdgeMetadataSerializationV1Impl;
 import org.apache.usergrid.persistence.graph.serialization.impl.EdgeSerializationImpl;
 import org.apache.usergrid.persistence.graph.serialization.impl.NodeSerializationImpl;
 import org.apache.usergrid.persistence.graph.serialization.impl.shard.EdgeColumnFamilies;
@@ -81,7 +81,7 @@ public class GraphModule extends AbstractModule {
         install( new GuicyFigModule( GraphFig.class ) );
 
 
-        bind( EdgeMetadataSerialization.class ).to( EdgeMetadataSerializationImpl.class );
+        bind( EdgeMetadataSerialization.class ).to( EdgeMetadataSerializationV1Impl.class );
         bind( NodeSerialization.class ).to( NodeSerializationImpl.class );
 
         bind( TimeService.class ).to( TimeServiceImpl.class );


[6/7] Finished refactor of sharding and versioning. Need to implement versions

Posted by to...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
deleted file mode 100644
index 6c20cfe..0000000
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationImpl.java
+++ /dev/null
@@ -1,477 +0,0 @@
-/*
- * 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.graph.serialization.impl;
-
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-
-import org.apache.usergrid.persistence.core.astyanax.CassandraConfig;
-import org.apache.usergrid.persistence.core.astyanax.ColumnNameIterator;
-import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
-import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
-import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
-import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
-import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
-import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
-import org.apache.usergrid.persistence.core.astyanax.StringColumnParser;
-import org.apache.usergrid.persistence.core.migration.Migration;
-import org.apache.usergrid.persistence.core.scope.ApplicationScope;
-import org.apache.usergrid.persistence.core.util.ValidationUtils;
-import org.apache.usergrid.persistence.graph.Edge;
-import org.apache.usergrid.persistence.graph.GraphFig;
-import org.apache.usergrid.persistence.graph.SearchEdgeType;
-import org.apache.usergrid.persistence.graph.SearchIdType;
-import org.apache.usergrid.persistence.graph.serialization.EdgeMetadataSerialization;
-import org.apache.usergrid.persistence.graph.serialization.util.GraphValidation;
-import org.apache.usergrid.persistence.model.entity.Id;
-
-import com.google.common.base.Preconditions;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
-import com.netflix.astyanax.Keyspace;
-import com.netflix.astyanax.MutationBatch;
-import com.netflix.astyanax.model.CompositeBuilder;
-import com.netflix.astyanax.model.CompositeParser;
-import com.netflix.astyanax.query.RowQuery;
-import com.netflix.astyanax.serializers.StringSerializer;
-import com.netflix.astyanax.util.RangeBuilder;
-
-
-/**
- * Class to perform all edge metadata I/O
- */
-@Singleton
-public class EdgeMetadataSerializationImpl implements EdgeMetadataSerialization, Migration {
-
-    private static final byte[] HOLDER = new byte[] { 0 };
-
-
-    //row key serializers
-    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
-    private static final ScopedRowKeySerializer<Id> ROW_KEY_SER =
-            new ScopedRowKeySerializer<Id>( ID_SER );
-
-    private static final StringSerializer STRING_SERIALIZER = StringSerializer.get();
-
-    private static final EdgeTypeRowCompositeSerializer EDGE_SER = new EdgeTypeRowCompositeSerializer();
-    private static final ScopedRowKeySerializer<EdgeIdTypeKey> EDGE_TYPE_ROW_KEY =
-            new ScopedRowKeySerializer<EdgeIdTypeKey>( EDGE_SER );
-
-    private static final StringColumnParser PARSER = StringColumnParser.get();
-
-
-    /**
-     * CFs where the row key contains the source node id
-     */
-    private static final MultiTennantColumnFamily<ScopedRowKey<Id>, String> CF_SOURCE_EDGE_TYPES =
-            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Types", ROW_KEY_SER,
-                    STRING_SERIALIZER );
-
-    //all target id types for source edge type
-    private static final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> CF_SOURCE_EDGE_ID_TYPES =
-            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Id_Types",
-                    EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
-
-    /**
-     * CFs where the row key is the target node id
-     */
-    private static final MultiTennantColumnFamily<ScopedRowKey<Id>, String> CF_TARGET_EDGE_TYPES =
-            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Types", ROW_KEY_SER,
-                    STRING_SERIALIZER );
-
-
-    //all source id types for target edge type
-    private static final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> CF_TARGET_EDGE_ID_TYPES =
-            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Id_Types",
-                    EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
-
-
-    protected final Keyspace keyspace;
-    private final CassandraConfig cassandraConfig;
-    private final GraphFig graphFig;
-
-
-    @Inject
-    public EdgeMetadataSerializationImpl( final Keyspace keyspace, final CassandraConfig cassandraConfig,
-                                          final GraphFig graphFig ) {
-
-        Preconditions.checkNotNull( "cassandraConfig is required", cassandraConfig );
-        Preconditions.checkNotNull( "consistencyFig is required", graphFig );
-        Preconditions.checkNotNull( "keyspace is required", keyspace );
-
-        this.keyspace = keyspace;
-        this.cassandraConfig = cassandraConfig;
-        this.graphFig = graphFig;
-    }
-
-
-    @Override
-    public MutationBatch writeEdge( final ApplicationScope scope, final Edge edge ) {
-
-        ValidationUtils.validateApplicationScope( scope );
-        GraphValidation.validateEdge( edge );
-
-        final Id scopeId = scope.getApplication();
-        final Id source = edge.getSourceNode();
-        final Id target = edge.getTargetNode();
-        final String edgeType = edge.getType();
-        final long timestamp = edge.getTimestamp();
-
-        final MutationBatch batch = keyspace.prepareMutationBatch().withConsistencyLevel( cassandraConfig.getWriteCL() )
-                                            .withTimestamp( timestamp );
-
-
-
-        //add source->target edge type to meta data
-        final ScopedRowKey< Id> sourceKey = new ScopedRowKey<>( scopeId, source );
-
-        batch.withRow( CF_SOURCE_EDGE_TYPES, sourceKey ).putColumn( edgeType, HOLDER );
-
-
-        //write source->target edge type and id type to meta data
-        EdgeIdTypeKey tk = new EdgeIdTypeKey( source, edgeType );
-        final ScopedRowKey<EdgeIdTypeKey> sourceTypeKey =
-                new ScopedRowKey<>( scopeId, tk );
-
-
-        batch.withRow( CF_SOURCE_EDGE_ID_TYPES, sourceTypeKey ).putColumn( target.getType(), HOLDER );
-
-
-        //write target<--source edge type meta data
-        final ScopedRowKey< Id> targetKey = new ScopedRowKey<>( scopeId, target );
-
-
-        batch.withRow( CF_TARGET_EDGE_TYPES, targetKey ).putColumn( edgeType, HOLDER );
-
-
-        //write target<--source edge type and id type to meta data
-        final ScopedRowKey<EdgeIdTypeKey> targetTypeKey =
-                new ScopedRowKey<>( scopeId, new EdgeIdTypeKey( target, edgeType ) );
-
-
-        batch.withRow( CF_TARGET_EDGE_ID_TYPES, targetTypeKey ).putColumn( source.getType(), HOLDER );
-
-
-        return batch;
-    }
-
-
-    @Override
-    public MutationBatch removeEdgeTypeFromSource( final ApplicationScope scope, final Edge edge ) {
-        return removeEdgeTypeFromSource( scope, edge.getSourceNode(), edge.getType(), edge.getTimestamp() );
-    }
-
-
-    @Override
-    public MutationBatch removeEdgeTypeFromSource( final ApplicationScope scope, final Id sourceNode, final String type,
-                                                   final long version ) {
-        return removeEdgeType( scope, sourceNode, type, version, CF_SOURCE_EDGE_TYPES );
-    }
-
-
-    @Override
-    public MutationBatch removeIdTypeFromSource( final ApplicationScope scope, final Edge edge ) {
-        return removeIdTypeFromSource( scope, edge.getSourceNode(), edge.getType(), edge.getTargetNode().getType(),
-                edge.getTimestamp() );
-    }
-
-
-    @Override
-    public MutationBatch removeIdTypeFromSource( final ApplicationScope scope, final Id sourceNode, final String type,
-                                                 final String idType, final long version ) {
-        return removeIdType( scope, sourceNode, idType, type, version, CF_SOURCE_EDGE_ID_TYPES );
-    }
-
-
-    @Override
-    public MutationBatch removeEdgeTypeToTarget( final ApplicationScope scope, final Edge edge ) {
-        return removeEdgeTypeToTarget( scope, edge.getTargetNode(), edge.getType(), edge.getTimestamp() );
-    }
-
-
-    @Override
-    public MutationBatch removeEdgeTypeToTarget( final ApplicationScope scope, final Id targetNode, final String type,
-                                                 final long version ) {
-        return removeEdgeType( scope, targetNode, type, version, CF_TARGET_EDGE_TYPES );
-    }
-
-
-    @Override
-    public MutationBatch removeIdTypeToTarget( final ApplicationScope scope, final Edge edge ) {
-        return removeIdTypeToTarget( scope, edge.getTargetNode(), edge.getType(), edge.getSourceNode().getType(),
-                edge.getTimestamp() );
-    }
-
-
-    @Override
-    public MutationBatch removeIdTypeToTarget( final ApplicationScope scope, final Id targetNode, final String type,
-                                               final String idType, final long version ) {
-        return removeIdType( scope, targetNode, idType, type, version, CF_TARGET_EDGE_ID_TYPES );
-    }
-
-
-    /**
-     * Remove the edge
-     *
-     * @param scope The scope
-     * @param rowKeyId The id to use in the row key
-     * @param edgeType The edge type
-     * @param version The version of the edge
-     * @param cf The column family
-     */
-    private MutationBatch removeEdgeType( final ApplicationScope scope, final Id rowKeyId, final String edgeType,
-                                          final long version,
-                                          final MultiTennantColumnFamily<ScopedRowKey<Id>, String> cf ) {
-
-
-        //write target<--source edge type meta data
-        final ScopedRowKey< Id> rowKey = new ScopedRowKey< Id>( scope.getApplication(), rowKeyId );
-
-        final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
-
-        batch.withRow( cf, rowKey ).deleteColumn( edgeType );
-
-        return batch;
-    }
-
-
-    /**
-     * Remove the id type
-     *
-     * @param scope The scope to use
-     * @param rowId The id to use in the row key
-     * @param idType The id type to use in the column
-     * @param edgeType The edge type to use in the column
-     * @param version The version to use on the column
-     * @param cf The column family to use
-     *
-     * @return A populated mutation with the remove operations
-     */
-    private MutationBatch removeIdType( final ApplicationScope scope, final Id rowId, final String idType,
-                                        final String edgeType, final long version,
-                                        final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> cf ) {
-
-
-        final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
-
-
-        //write target<--source edge type and id type to meta data
-        final ScopedRowKey< EdgeIdTypeKey> rowKey =
-                new ScopedRowKey<>( scope.getApplication(), new EdgeIdTypeKey( rowId, edgeType ) );
-
-
-        batch.withRow( cf, rowKey ).deleteColumn( idType );
-
-        return batch;
-    }
-
-
-    @Override
-    public Iterator<String> getEdgeTypesFromSource( final ApplicationScope scope, final SearchEdgeType search ) {
-        return getEdgeTypes( scope, search, CF_SOURCE_EDGE_TYPES );
-    }
-
-
-    @Override
-    public Iterator<String> getIdTypesFromSource( final ApplicationScope scope, final SearchIdType search ) {
-        return getIdTypes( scope, search, CF_SOURCE_EDGE_ID_TYPES );
-    }
-
-
-    @Override
-    public Iterator<String> getEdgeTypesToTarget( final ApplicationScope scope, final SearchEdgeType search ) {
-        return getEdgeTypes( scope, search, CF_TARGET_EDGE_TYPES );
-    }
-
-
-    /**
-     * Get the edge types from the search criteria.
-     *
-     * @param scope The org scope
-     * @param search The edge type search info
-     * @param cf The column family to execute on
-     */
-    private Iterator<String> getEdgeTypes( final ApplicationScope scope, final SearchEdgeType search,
-                                           final MultiTennantColumnFamily<ScopedRowKey<Id>, String> cf ) {
-        ValidationUtils.validateApplicationScope( scope );
-        GraphValidation.validateSearchEdgeType( search );
-
-
-        final ScopedRowKey< Id> sourceKey = new ScopedRowKey<>( scope.getApplication(), search.getNode() );
-
-
-        //resume from the last if specified.  Also set the range
-
-
-        final RangeBuilder rangeBuilder = createRange( search );
-
-        RowQuery<ScopedRowKey<Id>, String> query =
-                keyspace.prepareQuery( cf ).getKey( sourceKey ).autoPaginate( true )
-                        .withColumnRange( rangeBuilder.build() );
-
-        return new ColumnNameIterator<>( query, PARSER, search.getLast().isPresent() );
-    }
-
-
-    @Override
-    public Iterator<String> getIdTypesToTarget( final ApplicationScope scope, final SearchIdType search ) {
-        return getIdTypes( scope, search, CF_TARGET_EDGE_ID_TYPES );
-    }
-
-
-    /**
-     * Get the id types from the specified column family
-     *
-     * @param scope The organization scope to use
-     * @param search The search criteria
-     * @param cf The column family to search
-     */
-    public Iterator<String> getIdTypes( final ApplicationScope scope, final SearchIdType search,
-                                        final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> cf ) {
-        ValidationUtils.validateApplicationScope( scope );
-        GraphValidation.validateSearchEdgeIdType( search );
-
-
-        final ScopedRowKey<EdgeIdTypeKey> sourceTypeKey =
-                new ScopedRowKey<>( scope.getApplication(), new EdgeIdTypeKey( search.getNode(), search.getEdgeType() ) );
-
-
-        final RangeBuilder rangeBuilder = createRange( search );
-
-
-        RowQuery<ScopedRowKey<EdgeIdTypeKey>, String> query =
-                keyspace.prepareQuery( cf ).getKey( sourceTypeKey ).autoPaginate( true )
-                        .withColumnRange( rangeBuilder.build() );
-
-
-        return new ColumnNameIterator<>( query, PARSER, search.getLast().isPresent() );
-    }
-
-
-    @Override
-    public Collection<MultiTennantColumnFamilyDefinition> getColumnFamilies() {
-        return Arrays.asList( graphCf( CF_SOURCE_EDGE_TYPES ), graphCf( CF_TARGET_EDGE_TYPES ),
-                graphCf( CF_SOURCE_EDGE_ID_TYPES ), graphCf( CF_TARGET_EDGE_ID_TYPES ) );
-    }
-
-
-    /**
-     * Helper to generate an edge definition by the type
-     */
-    private MultiTennantColumnFamilyDefinition graphCf( MultiTennantColumnFamily cf ) {
-        return new MultiTennantColumnFamilyDefinition( cf, BytesType.class.getSimpleName(),
-                UTF8Type.class.getSimpleName(), BytesType.class.getSimpleName(),
-                MultiTennantColumnFamilyDefinition.CacheOption.KEYS );
-    }
-
-
-    /**
-     * Inner class to serialize and edgeIdTypeKey
-     */
-    private static class EdgeTypeRowCompositeSerializer implements CompositeFieldSerializer<EdgeIdTypeKey> {
-
-
-        private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
-
-
-        @Override
-        public void toComposite( final CompositeBuilder builder, final EdgeIdTypeKey value ) {
-            ID_SER.toComposite( builder, value.node );
-
-            builder.addString( value.edgeType );
-        }
-
-
-        @Override
-        public EdgeIdTypeKey fromComposite( final CompositeParser composite ) {
-            final Id id = ID_SER.fromComposite( composite );
-
-            final String edgeType = composite.readString();
-
-            return new EdgeIdTypeKey( id, edgeType );
-        }
-    }
-
-
-    private RangeBuilder createRange( final SearchEdgeType search ) {
-        final RangeBuilder builder = new RangeBuilder().setLimit( graphFig.getScanPageSize() );
-
-
-        //we have a last, it's where we need to start seeking from
-        if ( search.getLast().isPresent() ) {
-            builder.setStart( search.getLast().get() );
-        }
-
-        //no last was set, but we have a prefix, set it
-        else if ( search.prefix().isPresent() ) {
-            builder.setStart( search.prefix().get() );
-        }
-
-
-        //we have a prefix, so make sure we only seek to prefix + max UTF value
-        if ( search.prefix().isPresent() ) {
-            builder.setEnd( search.prefix().get() + "\uffff" );
-        }
-
-
-        return builder;
-    }
-
-
-    //    private void setStart( final SearchEdgeType search, final RangeBuilder builder ) {
-    //        //prefix is set, set our end marker
-    //        if ( search.getLast().isPresent() ) {
-    //            builder.setEnd( search.getLast().get() );
-    //        }
-    //
-    //        else if ( search.prefix().isPresent() ) {
-    //            builder.setStart( search.prefix().get() );
-    //        }
-    //    }
-    //
-    //
-    //    private void setEnd( final SearchEdgeType search, final RangeBuilder builder ) {
-    //        //if our last is set, it takes precendence
-    //
-    //        if ( search.prefix().isPresent() ) {
-    //            builder.setEnd( search.prefix().get() + "\uffff" );
-    //        }
-    //    }
-
-
-    /**
-     * Simple key object for I/O
-     */
-    private static class EdgeIdTypeKey {
-        private final Id node;
-        private final String edgeType;
-
-
-        private EdgeIdTypeKey( final Id node, final String edgeType ) {
-            this.node = node;
-            this.edgeType = edgeType;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV1Impl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV1Impl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV1Impl.java
new file mode 100644
index 0000000..3254dd5
--- /dev/null
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV1Impl.java
@@ -0,0 +1,478 @@
+/*
+ * 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.graph.serialization.impl;
+
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+import org.apache.usergrid.persistence.core.astyanax.CassandraConfig;
+import org.apache.usergrid.persistence.core.astyanax.ColumnNameIterator;
+import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
+import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
+import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
+import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
+import org.apache.usergrid.persistence.core.astyanax.StringColumnParser;
+import org.apache.usergrid.persistence.core.migration.Migration;
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.core.util.ValidationUtils;
+import org.apache.usergrid.persistence.graph.Edge;
+import org.apache.usergrid.persistence.graph.GraphFig;
+import org.apache.usergrid.persistence.graph.SearchEdgeType;
+import org.apache.usergrid.persistence.graph.SearchIdType;
+import org.apache.usergrid.persistence.graph.serialization.EdgeMetadataSerialization;
+import org.apache.usergrid.persistence.graph.serialization.util.GraphValidation;
+import org.apache.usergrid.persistence.model.entity.Id;
+
+import com.google.common.base.Preconditions;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.netflix.astyanax.Keyspace;
+import com.netflix.astyanax.MutationBatch;
+import com.netflix.astyanax.model.CompositeBuilder;
+import com.netflix.astyanax.model.CompositeParser;
+import com.netflix.astyanax.query.RowQuery;
+import com.netflix.astyanax.serializers.StringSerializer;
+import com.netflix.astyanax.util.RangeBuilder;
+
+
+/**
+ * Class to perform all edge metadata I/O
+ */
+@Singleton
+public class EdgeMetadataSerializationV1Impl implements EdgeMetadataSerialization, Migration {
+
+    private static final byte[] HOLDER = new byte[] { 0 };
+
+
+    //row key serializers
+    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
+    private static final ScopedRowKeySerializer<Id> ROW_KEY_SER =
+            new ScopedRowKeySerializer<Id>( ID_SER );
+
+    private static final StringSerializer STRING_SERIALIZER = StringSerializer.get();
+
+    private static final EdgeTypeRowCompositeSerializer EDGE_SER = new EdgeTypeRowCompositeSerializer();
+    private static final ScopedRowKeySerializer<EdgeIdTypeKey> EDGE_TYPE_ROW_KEY =
+            new ScopedRowKeySerializer<EdgeIdTypeKey>( EDGE_SER );
+
+    private static final StringColumnParser PARSER = StringColumnParser.get();
+
+
+    /**
+     * CFs where the row key contains the source node id
+     */
+    private static final MultiTennantColumnFamily<ScopedRowKey<Id>, String> CF_SOURCE_EDGE_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Types", ROW_KEY_SER,
+                    STRING_SERIALIZER );
+
+    //all target id types for source edge type
+    private static final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> CF_SOURCE_EDGE_ID_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Id_Types",
+                    EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
+
+    /**
+     * CFs where the row key is the target node id
+     */
+    private static final MultiTennantColumnFamily<ScopedRowKey<Id>, String> CF_TARGET_EDGE_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Types", ROW_KEY_SER,
+                    STRING_SERIALIZER );
+
+
+    //all source id types for target edge type
+    private static final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> CF_TARGET_EDGE_ID_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Id_Types",
+                    EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
+
+
+    protected final Keyspace keyspace;
+    private final CassandraConfig cassandraConfig;
+    private final GraphFig graphFig;
+
+
+    @Inject
+    public EdgeMetadataSerializationV1Impl( final Keyspace keyspace, final CassandraConfig cassandraConfig,
+                                            final GraphFig graphFig ) {
+
+        Preconditions.checkNotNull( "cassandraConfig is required", cassandraConfig );
+        Preconditions.checkNotNull( "consistencyFig is required", graphFig );
+        Preconditions.checkNotNull( "keyspace is required", keyspace );
+
+        this.keyspace = keyspace;
+        this.cassandraConfig = cassandraConfig;
+        this.graphFig = graphFig;
+    }
+
+
+    @Override
+    public MutationBatch writeEdge( final ApplicationScope scope, final Edge edge ) {
+
+        ValidationUtils.validateApplicationScope( scope );
+        GraphValidation.validateEdge( edge );
+
+        final Id scopeId = scope.getApplication();
+        final Id source = edge.getSourceNode();
+        final Id target = edge.getTargetNode();
+        final String edgeType = edge.getType();
+        final long timestamp = edge.getTimestamp();
+
+        final MutationBatch batch = keyspace.prepareMutationBatch().withConsistencyLevel( cassandraConfig.getWriteCL() )
+                                            .withTimestamp( timestamp );
+
+
+
+        //add source->target edge type to meta data
+        final ScopedRowKey< Id> sourceKey = new ScopedRowKey<>( scopeId, source );
+
+        batch.withRow( CF_SOURCE_EDGE_TYPES, sourceKey ).putColumn( edgeType, HOLDER );
+
+
+        //write source->target edge type and id type to meta data
+        EdgeIdTypeKey tk = new EdgeIdTypeKey( source, edgeType );
+        final ScopedRowKey<EdgeIdTypeKey> sourceTypeKey =
+                new ScopedRowKey<>( scopeId, tk );
+
+
+        batch.withRow( CF_SOURCE_EDGE_ID_TYPES, sourceTypeKey ).putColumn( target.getType(), HOLDER );
+
+
+        //write target<--source edge type meta data
+        final ScopedRowKey< Id> targetKey = new ScopedRowKey<>( scopeId, target );
+
+
+        batch.withRow( CF_TARGET_EDGE_TYPES, targetKey ).putColumn( edgeType, HOLDER );
+
+
+        //write target<--source edge type and id type to meta data
+        final ScopedRowKey<EdgeIdTypeKey> targetTypeKey =
+                new ScopedRowKey<>( scopeId, new EdgeIdTypeKey( target, edgeType ) );
+
+
+        batch.withRow( CF_TARGET_EDGE_ID_TYPES, targetTypeKey ).putColumn( source.getType(), HOLDER );
+
+
+        return batch;
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeFromSource( final ApplicationScope scope, final Edge edge ) {
+        return removeEdgeTypeFromSource( scope, edge.getSourceNode(), edge.getType(), edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeFromSource( final ApplicationScope scope, final Id sourceNode, final String type,
+                                                   final long version ) {
+        return removeEdgeType( scope, sourceNode, type, version, CF_SOURCE_EDGE_TYPES );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeFromSource( final ApplicationScope scope, final Edge edge ) {
+        return removeIdTypeFromSource( scope, edge.getSourceNode(), edge.getType(), edge.getTargetNode().getType(),
+                edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeFromSource( final ApplicationScope scope, final Id sourceNode, final String type,
+                                                 final String idType, final long version ) {
+        return removeIdType( scope, sourceNode, idType, type, version, CF_SOURCE_EDGE_ID_TYPES );
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeToTarget( final ApplicationScope scope, final Edge edge ) {
+        return removeEdgeTypeToTarget( scope, edge.getTargetNode(), edge.getType(), edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeToTarget( final ApplicationScope scope, final Id targetNode, final String type,
+                                                 final long version ) {
+        return removeEdgeType( scope, targetNode, type, version, CF_TARGET_EDGE_TYPES );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeToTarget( final ApplicationScope scope, final Edge edge ) {
+        return removeIdTypeToTarget( scope, edge.getTargetNode(), edge.getType(), edge.getSourceNode().getType(),
+                edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeToTarget( final ApplicationScope scope, final Id targetNode, final String type,
+                                               final String idType, final long version ) {
+        return removeIdType( scope, targetNode, idType, type, version, CF_TARGET_EDGE_ID_TYPES );
+    }
+
+
+    /**
+     * Remove the edge
+     *
+     * @param scope The scope
+     * @param rowKeyId The id to use in the row key
+     * @param edgeType The edge type
+     * @param version The version of the edge
+     * @param cf The column family
+     */
+    private MutationBatch removeEdgeType( final ApplicationScope scope, final Id rowKeyId, final String edgeType,
+                                          final long version,
+                                          final MultiTennantColumnFamily<ScopedRowKey<Id>, String> cf ) {
+
+
+        //write target<--source edge type meta data
+        final ScopedRowKey< Id> rowKey = new ScopedRowKey< Id>( scope.getApplication(), rowKeyId );
+
+        final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
+
+        batch.withRow( cf, rowKey ).deleteColumn( edgeType );
+
+        return batch;
+    }
+
+
+    /**
+     * Remove the id type
+     *
+     * @param scope The scope to use
+     * @param rowId The id to use in the row key
+     * @param idType The id type to use in the column
+     * @param edgeType The edge type to use in the column
+     * @param version The version to use on the column
+     * @param cf The column family to use
+     *
+     * @return A populated mutation with the remove operations
+     */
+    private MutationBatch removeIdType( final ApplicationScope scope, final Id rowId, final String idType,
+                                        final String edgeType, final long version,
+                                        final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> cf ) {
+
+
+        final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
+
+
+        //write target<--source edge type and id type to meta data
+        final ScopedRowKey< EdgeIdTypeKey> rowKey =
+                new ScopedRowKey<>( scope.getApplication(), new EdgeIdTypeKey( rowId, edgeType ) );
+
+
+        batch.withRow( cf, rowKey ).deleteColumn( idType );
+
+        return batch;
+    }
+
+
+    @Override
+    public Iterator<String> getEdgeTypesFromSource( final ApplicationScope scope, final SearchEdgeType search ) {
+        return getEdgeTypes( scope, search, CF_SOURCE_EDGE_TYPES );
+    }
+
+
+    @Override
+    public Iterator<String> getIdTypesFromSource( final ApplicationScope scope, final SearchIdType search ) {
+        return getIdTypes( scope, search, CF_SOURCE_EDGE_ID_TYPES );
+    }
+
+
+    @Override
+    public Iterator<String> getEdgeTypesToTarget( final ApplicationScope scope, final SearchEdgeType search ) {
+        return getEdgeTypes( scope, search, CF_TARGET_EDGE_TYPES );
+    }
+
+
+    /**
+     * Get the edge types from the search criteria.
+     *
+     * @param scope The org scope
+     * @param search The edge type search info
+     * @param cf The column family to execute on
+     */
+    private Iterator<String> getEdgeTypes( final ApplicationScope scope, final SearchEdgeType search,
+                                           final MultiTennantColumnFamily<ScopedRowKey<Id>, String> cf ) {
+        ValidationUtils.validateApplicationScope( scope );
+        GraphValidation.validateSearchEdgeType( search );
+
+
+        final ScopedRowKey< Id> sourceKey = new ScopedRowKey<>( scope.getApplication(), search.getNode() );
+
+
+        //resume from the last if specified.  Also set the range
+
+
+        final RangeBuilder rangeBuilder = createRange( search );
+
+        RowQuery<ScopedRowKey<Id>, String> query =
+                keyspace.prepareQuery( cf ).getKey( sourceKey ).autoPaginate( true )
+                        .withColumnRange( rangeBuilder.build() );
+
+        return new ColumnNameIterator<>( query, PARSER, search.getLast().isPresent() );
+    }
+
+
+    @Override
+    public Iterator<String> getIdTypesToTarget( final ApplicationScope scope, final SearchIdType search ) {
+        return getIdTypes( scope, search, CF_TARGET_EDGE_ID_TYPES );
+    }
+
+
+    /**
+     * Get the id types from the specified column family
+     *
+     * @param scope The organization scope to use
+     * @param search The search criteria
+     * @param cf The column family to search
+     */
+    public Iterator<String> getIdTypes( final ApplicationScope scope, final SearchIdType search,
+                                        final MultiTennantColumnFamily<ScopedRowKey<EdgeIdTypeKey>, String> cf ) {
+        ValidationUtils.validateApplicationScope( scope );
+        GraphValidation.validateSearchEdgeIdType( search );
+
+
+        final ScopedRowKey<EdgeIdTypeKey> sourceTypeKey =
+                new ScopedRowKey<>( scope.getApplication(), new EdgeIdTypeKey( search.getNode(), search.getEdgeType() ) );
+
+
+        final RangeBuilder rangeBuilder = createRange( search );
+
+
+        RowQuery<ScopedRowKey<EdgeIdTypeKey>, String> query =
+                keyspace.prepareQuery( cf ).getKey( sourceTypeKey ).autoPaginate( true )
+                        .withColumnRange( rangeBuilder.build() );
+
+
+        return new ColumnNameIterator<>( query, PARSER, search.getLast().isPresent() );
+    }
+
+
+    @Override
+    public Collection<MultiTennantColumnFamilyDefinition> getColumnFamilies() {
+        return Arrays.asList( graphCf( CF_SOURCE_EDGE_TYPES ), graphCf( CF_TARGET_EDGE_TYPES ),
+                graphCf( CF_SOURCE_EDGE_ID_TYPES ), graphCf( CF_TARGET_EDGE_ID_TYPES ) );
+    }
+
+
+    /**
+     * Helper to generate an edge definition by the type
+     */
+    private MultiTennantColumnFamilyDefinition graphCf( MultiTennantColumnFamily cf ) {
+        return new MultiTennantColumnFamilyDefinition( cf, BytesType.class.getSimpleName(),
+                UTF8Type.class.getSimpleName(), BytesType.class.getSimpleName(),
+                MultiTennantColumnFamilyDefinition.CacheOption.KEYS );
+    }
+
+
+    /**
+     * Inner class to serialize and edgeIdTypeKey
+     */
+    private static class EdgeTypeRowCompositeSerializer implements CompositeFieldSerializer<EdgeIdTypeKey> {
+
+
+        private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
+
+
+        @Override
+        public void toComposite( final CompositeBuilder builder, final EdgeIdTypeKey value ) {
+            ID_SER.toComposite( builder, value.node );
+
+            builder.addString( value.edgeType );
+        }
+
+
+        @Override
+        public EdgeIdTypeKey fromComposite( final CompositeParser composite ) {
+            final Id id = ID_SER.fromComposite( composite );
+
+            final String edgeType = composite.readString();
+
+            return new EdgeIdTypeKey( id, edgeType );
+        }
+
+    }
+
+
+    private RangeBuilder createRange( final SearchEdgeType search ) {
+        final RangeBuilder builder = new RangeBuilder().setLimit( graphFig.getScanPageSize() );
+
+
+        //we have a last, it's where we need to start seeking from
+        if ( search.getLast().isPresent() ) {
+            builder.setStart( search.getLast().get() );
+        }
+
+        //no last was set, but we have a prefix, set it
+        else if ( search.prefix().isPresent() ) {
+            builder.setStart( search.prefix().get() );
+        }
+
+
+        //we have a prefix, so make sure we only seek to prefix + max UTF value
+        if ( search.prefix().isPresent() ) {
+            builder.setEnd( search.prefix().get() + "\uffff" );
+        }
+
+
+        return builder;
+    }
+
+
+    //    private void setStart( final SearchEdgeType search, final RangeBuilder builder ) {
+    //        //prefix is set, set our end marker
+    //        if ( search.getLast().isPresent() ) {
+    //            builder.setEnd( search.getLast().get() );
+    //        }
+    //
+    //        else if ( search.prefix().isPresent() ) {
+    //            builder.setStart( search.prefix().get() );
+    //        }
+    //    }
+    //
+    //
+    //    private void setEnd( final SearchEdgeType search, final RangeBuilder builder ) {
+    //        //if our last is set, it takes precendence
+    //
+    //        if ( search.prefix().isPresent() ) {
+    //            builder.setEnd( search.prefix().get() + "\uffff" );
+    //        }
+    //    }
+
+
+    /**
+     * Simple key object for I/O
+     */
+    private static class EdgeIdTypeKey {
+        private final Id node;
+        private final String edgeType;
+
+
+        private EdgeIdTypeKey( final Id node, final String edgeType ) {
+            this.node = node;
+            this.edgeType = edgeType;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV2Impl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV2Impl.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV2Impl.java
new file mode 100644
index 0000000..d7ffeea
--- /dev/null
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/EdgeMetadataSerializationV2Impl.java
@@ -0,0 +1,613 @@
+/*
+ *
+ *  * 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.graph.serialization.impl;
+
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+import org.apache.usergrid.persistence.core.astyanax.BucketScopedRowKey;
+import org.apache.usergrid.persistence.core.astyanax.BucketScopedRowKeySerializer;
+import org.apache.usergrid.persistence.core.astyanax.CassandraConfig;
+import org.apache.usergrid.persistence.core.astyanax.ColumnSearch;
+import org.apache.usergrid.persistence.core.astyanax.CompositeFieldSerializer;
+import org.apache.usergrid.persistence.core.astyanax.IdRowCompositeSerializer;
+import org.apache.usergrid.persistence.core.astyanax.MultiRowColumnIterator;
+import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamily;
+import org.apache.usergrid.persistence.core.astyanax.MultiTennantColumnFamilyDefinition;
+import org.apache.usergrid.persistence.core.astyanax.StringColumnParser;
+import org.apache.usergrid.persistence.core.migration.Migration;
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.core.shard.ExpandingShardLocator;
+import org.apache.usergrid.persistence.core.shard.StringHashUtils;
+import org.apache.usergrid.persistence.core.util.ValidationUtils;
+import org.apache.usergrid.persistence.graph.Edge;
+import org.apache.usergrid.persistence.graph.GraphFig;
+import org.apache.usergrid.persistence.graph.SearchEdgeType;
+import org.apache.usergrid.persistence.graph.SearchIdType;
+import org.apache.usergrid.persistence.graph.serialization.EdgeMetadataSerialization;
+import org.apache.usergrid.persistence.graph.serialization.util.GraphValidation;
+import org.apache.usergrid.persistence.model.entity.Id;
+
+import com.google.common.base.Preconditions;
+import com.google.common.hash.Funnel;
+import com.google.common.hash.PrimitiveSink;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.netflix.astyanax.Keyspace;
+import com.netflix.astyanax.MutationBatch;
+import com.netflix.astyanax.model.CompositeBuilder;
+import com.netflix.astyanax.model.CompositeParser;
+import com.netflix.astyanax.serializers.StringSerializer;
+import com.netflix.astyanax.util.RangeBuilder;
+
+
+/**
+ * Class to perform all edge metadata I/O
+ */
+@Singleton
+public class EdgeMetadataSerializationV2Impl implements EdgeMetadataSerialization, Migration {
+
+    private static final byte[] HOLDER = new byte[] { 0 };
+
+
+    //row key serializers
+    private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
+
+    private static final BucketScopedRowKeySerializer<Id> ROW_KEY_SER = new BucketScopedRowKeySerializer<>( ID_SER );
+
+    private static final StringSerializer STRING_SERIALIZER = StringSerializer.get();
+
+    private static final EdgeTypeRowCompositeSerializer EDGE_SER = new EdgeTypeRowCompositeSerializer();
+
+    private static final BucketScopedRowKeySerializer<EdgeIdTypeKey> EDGE_TYPE_ROW_KEY =
+            new BucketScopedRowKeySerializer<>( EDGE_SER );
+
+    private static final StringColumnParser PARSER = StringColumnParser.get();
+
+
+    /**
+     * V1 CF's.  We can't delete these until a full migration has been run
+     */
+    /**
+     * CFs where the row key contains the source node id
+     */
+    private static final MultiTennantColumnFamily<BucketScopedRowKey<Id>, String> CF_SOURCE_EDGE_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Types_V2", ROW_KEY_SER, STRING_SERIALIZER );
+
+    //all target id types for source edge type
+    private static final MultiTennantColumnFamily<BucketScopedRowKey<EdgeIdTypeKey>, String> CF_SOURCE_EDGE_ID_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Source_Edge_Id_Types_V2", EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
+
+    /**
+     * CFs where the row key is the target node id
+     */
+    private static final MultiTennantColumnFamily<BucketScopedRowKey<Id>, String> CF_TARGET_EDGE_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Types_V2", ROW_KEY_SER, STRING_SERIALIZER );
+
+
+    //all source id types for target edge type
+    private static final MultiTennantColumnFamily<BucketScopedRowKey<EdgeIdTypeKey>, String> CF_TARGET_EDGE_ID_TYPES =
+            new MultiTennantColumnFamily<>( "Graph_Target_Edge_Id_Types_V2", EDGE_TYPE_ROW_KEY, STRING_SERIALIZER );
+
+
+    private static final Comparator<String> STRING_COMPARATOR = new Comparator<String>() {
+
+        @Override
+        public int compare( final String o1, final String o2 ) {
+            return o1.compareTo( o2 );
+        }
+    };
+
+
+    /**
+     * Funnel for hashing IDS
+     */
+    private static final Funnel<Id> ID_FUNNEL = new Funnel<Id>() {
+
+        @Override
+        public void funnel( final Id from, final PrimitiveSink into ) {
+            final UUID id = from.getUuid();
+            final String type = from.getType();
+
+            into.putLong( id.getMostSignificantBits() );
+            into.putLong( id.getLeastSignificantBits() );
+            into.putString( type, StringHashUtils.UTF8 );
+        }
+    };
+
+    /**
+     * Funnel for hashing IDS
+     */
+    private static final Funnel<EdgeIdTypeKey> EDGE_TYPE_FUNNEL = new Funnel<EdgeIdTypeKey>() {
+
+        @Override
+        public void funnel( final EdgeIdTypeKey from, final PrimitiveSink into ) {
+
+            final UUID id = from.node.getUuid();
+            final String type = from.node.getType();
+
+            into.putLong( id.getMostSignificantBits() );
+            into.putLong( id.getLeastSignificantBits() );
+            into.putString( type, StringHashUtils.UTF8 );
+            into.putString( from.edgeType, StringHashUtils.UTF8 );
+        }
+    };
+
+
+    protected final Keyspace keyspace;
+    private final CassandraConfig cassandraConfig;
+    private final GraphFig graphFig;
+
+    /**
+     * Locator for all id buckets
+     */
+    private ExpandingShardLocator<Id> idExpandingShardLocator;
+
+    /**
+     * Locator for all edge types
+     */
+    private ExpandingShardLocator<EdgeIdTypeKey> edgeTypeExpandingShardLocator;
+
+
+    @Inject
+    public EdgeMetadataSerializationV2Impl( final Keyspace keyspace, final CassandraConfig cassandraConfig,
+                                            final GraphFig graphFig ) {
+
+        Preconditions.checkNotNull( "cassandraConfig is required", cassandraConfig );
+        Preconditions.checkNotNull( "consistencyFig is required", graphFig );
+        Preconditions.checkNotNull( "keyspace is required", keyspace );
+
+        this.keyspace = keyspace;
+        this.cassandraConfig = cassandraConfig;
+        this.graphFig = graphFig;
+
+        //set up the shard locator instances
+        idExpandingShardLocator = new ExpandingShardLocator<>( ID_FUNNEL, cassandraConfig.getShardSettings() );
+
+        edgeTypeExpandingShardLocator =
+                new ExpandingShardLocator<>( EDGE_TYPE_FUNNEL, cassandraConfig.getShardSettings() );
+    }
+
+
+    @Override
+    public MutationBatch writeEdge( final ApplicationScope scope, final Edge edge ) {
+
+        ValidationUtils.validateApplicationScope( scope );
+        GraphValidation.validateEdge( edge );
+
+        final Id scopeId = scope.getApplication();
+        final Id source = edge.getSourceNode();
+        final Id target = edge.getTargetNode();
+        final String edgeType = edge.getType();
+        final long timestamp = edge.getTimestamp();
+
+        final MutationBatch batch = keyspace.prepareMutationBatch().withConsistencyLevel( cassandraConfig.getWriteCL() )
+                                            .withTimestamp( timestamp );
+
+
+        //add source->target edge type to meta data
+        final int sourceKeyBucket = idExpandingShardLocator.getCurrentBucket( source );
+
+
+        final BucketScopedRowKey<Id> sourceKey = BucketScopedRowKey.fromKey( scopeId, source, sourceKeyBucket );
+
+        batch.withRow( CF_SOURCE_EDGE_TYPES, sourceKey ).putColumn( edgeType, HOLDER );
+
+
+        //write source->target edge type and id type to meta data
+        final EdgeIdTypeKey sourceTargetTypeKey = new EdgeIdTypeKey( source, edgeType );
+
+        final int sourceTargetTypeBucket = edgeTypeExpandingShardLocator.getCurrentBucket( sourceTargetTypeKey );
+
+        final BucketScopedRowKey<EdgeIdTypeKey> sourceTypeKey =
+                BucketScopedRowKey.fromKey( scopeId, sourceTargetTypeKey, sourceTargetTypeBucket );
+
+        batch.withRow( CF_SOURCE_EDGE_ID_TYPES, sourceTypeKey ).putColumn( target.getType(), HOLDER );
+
+
+        final int targetKeyBucket = idExpandingShardLocator.getCurrentBucket( target );
+
+        final BucketScopedRowKey<Id> targetKey = BucketScopedRowKey.fromKey( scopeId, target, targetKeyBucket );
+
+        batch.withRow( CF_TARGET_EDGE_TYPES, targetKey ).putColumn( edgeType, HOLDER );
+
+
+        //write target<--source edge type and id type to meta data
+
+        final EdgeIdTypeKey targetSourceTypeKey = new EdgeIdTypeKey( target, edgeType );
+
+        final int targetSourceTypeKeyBucket = edgeTypeExpandingShardLocator.getCurrentBucket( targetSourceTypeKey );
+
+        final BucketScopedRowKey<EdgeIdTypeKey> targetTypeKey =
+                BucketScopedRowKey.fromKey( scopeId, targetSourceTypeKey, targetSourceTypeKeyBucket );
+
+        batch.withRow( CF_TARGET_EDGE_ID_TYPES, targetTypeKey ).putColumn( source.getType(), HOLDER );
+
+
+        return batch;
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeFromSource( final ApplicationScope scope, final Edge edge ) {
+        return removeEdgeTypeFromSource( scope, edge.getSourceNode(), edge.getType(), edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeFromSource( final ApplicationScope scope, final Id sourceNode, final String type,
+                                                   final long version ) {
+        return removeEdgeType( scope, sourceNode, type, version, CF_SOURCE_EDGE_TYPES );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeFromSource( final ApplicationScope scope, final Edge edge ) {
+        return removeIdTypeFromSource( scope, edge.getSourceNode(), edge.getType(), edge.getTargetNode().getType(),
+                edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeFromSource( final ApplicationScope scope, final Id sourceNode, final String type,
+                                                 final String idType, final long version ) {
+        return removeIdType( scope, sourceNode, idType, type, version, CF_SOURCE_EDGE_ID_TYPES );
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeToTarget( final ApplicationScope scope, final Edge edge ) {
+        return removeEdgeTypeToTarget( scope, edge.getTargetNode(), edge.getType(), edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeEdgeTypeToTarget( final ApplicationScope scope, final Id targetNode, final String type,
+                                                 final long version ) {
+        return removeEdgeType( scope, targetNode, type, version, CF_TARGET_EDGE_TYPES );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeToTarget( final ApplicationScope scope, final Edge edge ) {
+        return removeIdTypeToTarget( scope, edge.getTargetNode(), edge.getType(), edge.getSourceNode().getType(),
+                edge.getTimestamp() );
+    }
+
+
+    @Override
+    public MutationBatch removeIdTypeToTarget( final ApplicationScope scope, final Id targetNode, final String type,
+                                               final String idType, final long version ) {
+        return removeIdType( scope, targetNode, idType, type, version, CF_TARGET_EDGE_ID_TYPES );
+    }
+
+
+    /**
+     * Remove the edge
+     *
+     * @param scope The scope
+     * @param rowKeyId The id to use in the row key
+     * @param edgeType The edge type
+     * @param version The version of the edge
+     * @param cf The column family
+     */
+    private MutationBatch removeEdgeType( final ApplicationScope scope, final Id rowKeyId, final String edgeType,
+                                          final long version,
+                                          final MultiTennantColumnFamily<BucketScopedRowKey<Id>, String> cf ) {
+
+
+        //write target<--source edge type meta data
+        final int currentShard = idExpandingShardLocator.getCurrentBucket( rowKeyId );
+
+        final BucketScopedRowKey<Id> rowKey =
+                BucketScopedRowKey.fromKey( scope.getApplication(), rowKeyId, currentShard );
+
+        final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
+
+        batch.withRow( cf, rowKey ).deleteColumn( edgeType );
+
+        return batch;
+    }
+
+
+    /**
+     * Remove the id type
+     *
+     * @param scope The scope to use
+     * @param rowId The id to use in the row key
+     * @param idType The id type to use in the column
+     * @param edgeType The edge type to use in the column
+     * @param version The version to use on the column
+     * @param cf The column family to use
+     *
+     * @return A populated mutation with the remove operations
+     */
+    private MutationBatch removeIdType( final ApplicationScope scope, final Id rowId, final String idType,
+                                        final String edgeType, final long version,
+                                        final MultiTennantColumnFamily<BucketScopedRowKey<EdgeIdTypeKey>, String> cf ) {
+
+
+        final EdgeIdTypeKey edgeIdTypeKey = new EdgeIdTypeKey( rowId, edgeType );
+
+        final int currentShard = edgeTypeExpandingShardLocator.getCurrentBucket( edgeIdTypeKey );
+
+
+        final MutationBatch batch = keyspace.prepareMutationBatch().withTimestamp( version );
+
+
+        //write target<--source edge type and id type to meta data
+        final BucketScopedRowKey<EdgeIdTypeKey> rowKey =
+                BucketScopedRowKey.fromKey( scope.getApplication(), edgeIdTypeKey, currentShard );
+
+
+        batch.withRow( cf, rowKey ).deleteColumn( idType );
+
+        return batch;
+    }
+
+
+    @Override
+    public Iterator<String> getEdgeTypesFromSource( final ApplicationScope scope, final SearchEdgeType search ) {
+        return getEdgeTypes( scope, search, CF_SOURCE_EDGE_TYPES );
+    }
+
+
+    @Override
+    public Iterator<String> getIdTypesFromSource( final ApplicationScope scope, final SearchIdType search ) {
+        return getIdTypes( scope, search, CF_SOURCE_EDGE_ID_TYPES );
+    }
+
+
+    @Override
+    public Iterator<String> getEdgeTypesToTarget( final ApplicationScope scope, final SearchEdgeType search ) {
+        return getEdgeTypes( scope, search, CF_TARGET_EDGE_TYPES );
+    }
+
+
+    /**
+     * Get the edge types from the search criteria.
+     *
+     * @param scope The org scope
+     * @param search The edge type search info
+     * @param cf The column family to execute on
+     */
+    private Iterator<String> getEdgeTypes( final ApplicationScope scope, final SearchEdgeType search,
+                                           final MultiTennantColumnFamily<BucketScopedRowKey<Id>, String> cf ) {
+        ValidationUtils.validateApplicationScope( scope );
+        GraphValidation.validateSearchEdgeType( search );
+
+
+        final Id applicationId = scope.getApplication();
+        final Id searchNode = search.getNode();
+
+        final int[] bucketIds = idExpandingShardLocator.getAllBuckets( searchNode );
+
+
+        //no generics is intentional here
+        final List<BucketScopedRowKey<Id>> buckets =
+                BucketScopedRowKey.fromRange( applicationId, searchNode, bucketIds );
+
+
+        final ColumnSearch<String> columnSearch = createSearch( search );
+
+
+        return new MultiRowColumnIterator( keyspace, cf, cassandraConfig.getReadCL(), PARSER, columnSearch,
+                STRING_COMPARATOR, buckets, graphFig.getScanPageSize() );
+    }
+
+
+    @Override
+    public Iterator<String> getIdTypesToTarget( final ApplicationScope scope, final SearchIdType search ) {
+        return getIdTypes( scope, search, CF_TARGET_EDGE_ID_TYPES );
+    }
+
+
+    /**
+     * Get the id types from the specified column family
+     *
+     * @param scope The organization scope to use
+     * @param search The search criteria
+     * @param cf The column family to search
+     */
+    public Iterator<String> getIdTypes( final ApplicationScope scope, final SearchIdType search,
+                                        final MultiTennantColumnFamily<BucketScopedRowKey<EdgeIdTypeKey>, String> cf ) {
+        ValidationUtils.validateApplicationScope( scope );
+        GraphValidation.validateSearchEdgeIdType( search );
+
+
+        final Id applicationId = scope.getApplication();
+
+        final Id searchNode = search.getNode();
+
+        final EdgeIdTypeKey edgeIdTypeKey = new EdgeIdTypeKey( searchNode, search.getEdgeType() );
+
+        final int[] bucketIds = edgeTypeExpandingShardLocator.getAllBuckets( edgeIdTypeKey );
+
+        //no generics is intentional here
+        final List<BucketScopedRowKey<Id>> buckets =
+                BucketScopedRowKey.fromRange( applicationId, searchNode, bucketIds );
+
+
+        final ColumnSearch<String> columnSearch = createSearch( search );
+
+
+        return new MultiRowColumnIterator( keyspace, cf, cassandraConfig.getReadCL(), PARSER, columnSearch,
+                STRING_COMPARATOR, buckets, graphFig.getScanPageSize() );
+    }
+
+
+    @Override
+    public Collection<MultiTennantColumnFamilyDefinition> getColumnFamilies() {
+        return Arrays.asList( graphCf( CF_SOURCE_EDGE_TYPES ), graphCf( CF_TARGET_EDGE_TYPES ),
+                graphCf( CF_SOURCE_EDGE_ID_TYPES ), graphCf( CF_TARGET_EDGE_ID_TYPES ) );
+    }
+
+
+    /**
+     * Helper to generate an edge definition by the type
+     */
+    private MultiTennantColumnFamilyDefinition graphCf( MultiTennantColumnFamily cf ) {
+        return new MultiTennantColumnFamilyDefinition( cf, BytesType.class.getSimpleName(),
+                UTF8Type.class.getSimpleName(), BytesType.class.getSimpleName(),
+                MultiTennantColumnFamilyDefinition.CacheOption.KEYS );
+    }
+
+
+    /**
+     * Create a new instance of our search
+     */
+    private ColumnSearch<String> createSearch( final SearchEdgeType search ) {
+
+        //resume from the last if specified.  Also set the range
+        return new ColumnSearch<String>() {
+            @Override
+            public void buildRange( final RangeBuilder rangeBuilder, final String value ) {
+                rangeBuilder.setLimit( graphFig.getScanPageSize() );
+
+
+                if ( value != null ) {
+                    rangeBuilder.setStart( value );
+                }
+
+                //we have a last, it's where we need to start seeking from
+                else if ( search.getLast().isPresent() ) {
+                    rangeBuilder.setStart( search.getLast().get() );
+                }
+
+                //no last was set, but we have a prefix, set it
+                else if ( search.prefix().isPresent() ) {
+                    rangeBuilder.setStart( search.prefix().get() );
+                }
+
+
+                //we have a prefix, so make sure we only seek to prefix + max UTF value
+                if ( search.prefix().isPresent() ) {
+                    rangeBuilder.setEnd( search.prefix().get() + "\uffff" );
+                }
+            }
+
+
+            @Override
+            public void buildRange( final RangeBuilder rangeBuilder ) {
+                buildRange( rangeBuilder, null );
+            }
+        };
+    }
+
+
+    /**
+     * Inner class to serialize and edgeIdTypeKey
+     */
+    private static class EdgeTypeRowCompositeSerializer implements CompositeFieldSerializer<EdgeIdTypeKey> {
+
+
+        private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
+
+
+        @Override
+        public void toComposite( final CompositeBuilder builder, final EdgeIdTypeKey value ) {
+            ID_SER.toComposite( builder, value.node );
+
+            builder.addString( value.edgeType );
+        }
+
+
+        @Override
+        public EdgeIdTypeKey fromComposite( final CompositeParser composite ) {
+            final Id id = ID_SER.fromComposite( composite );
+
+            final String edgeType = composite.readString();
+
+            return new EdgeIdTypeKey( id, edgeType );
+        }
+
+    }
+
+    //
+    //    private RangeBuilder createRange( final SearchEdgeType search ) {
+    //        final RangeBuilder builder = new RangeBuilder().setLimit( graphFig.getScanPageSize() );
+    //
+    //
+    //        //we have a last, it's where we need to start seeking from
+    //        if ( search.getLast().isPresent() ) {
+    //            builder.setStart( search.getLast().get() );
+    //        }
+    //
+    //        //no last was set, but we have a prefix, set it
+    //        else if ( search.prefix().isPresent() ) {
+    //            builder.setStart( search.prefix().get() );
+    //        }
+    //
+    //
+    //        //we have a prefix, so make sure we only seek to prefix + max UTF value
+    //        if ( search.prefix().isPresent() ) {
+    //            builder.setEnd( search.prefix().get() + "\uffff" );
+    //        }
+    //
+    //
+    //        return builder;
+    //    }
+
+
+    //    private void setStart( final SearchEdgeType search, final RangeBuilder builder ) {
+    //        //prefix is set, set our end marker
+    //        if ( search.getLast().isPresent() ) {
+    //            builder.setEnd( search.getLast().get() );
+    //        }
+    //
+    //        else if ( search.prefix().isPresent() ) {
+    //            builder.setStart( search.prefix().get() );
+    //        }
+    //    }
+    //
+    //
+    //    private void setEnd( final SearchEdgeType search, final RangeBuilder builder ) {
+    //        //if our last is set, it takes precendence
+    //
+    //        if ( search.prefix().isPresent() ) {
+    //            builder.setEnd( search.prefix().get() + "\uffff" );
+    //        }
+    //    }
+
+
+    /**
+     * Simple key object for I/O
+     */
+    private static class EdgeIdTypeKey {
+        private final Id node;
+        private final String edgeType;
+
+
+        private EdgeIdTypeKey( final Id node, final String edgeType ) {
+            this.node = node;
+            this.edgeType = edgeType;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SourceEdgeSearcher.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SourceEdgeSearcher.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SourceEdgeSearcher.java
deleted file mode 100644
index b33fcaf..0000000
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/SourceEdgeSearcher.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.graph.serialization.impl.shard.impl;
-
-
-public class SourceEdgeSearcher {}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/EdgeShardRowKeySerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/EdgeShardRowKeySerializer.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/EdgeShardRowKeySerializer.java
index 6b1c4e9..dcfc412 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/EdgeShardRowKeySerializer.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/EdgeShardRowKeySerializer.java
@@ -100,4 +100,6 @@ public class EdgeShardRowKeySerializer implements CompositeFieldSerializer<Direc
 
         return  DirectedEdgeMeta.fromStorage( metaType, nodePairs, types );
     }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowSerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowSerializer.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowSerializer.java
index 8376ef1..0760846 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowSerializer.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowSerializer.java
@@ -38,7 +38,6 @@ public class RowSerializer implements CompositeFieldSerializer<RowKey> {
 
     private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
 
-
     @Override
     public void toComposite( final CompositeBuilder builder, final RowKey key ) {
 
@@ -60,4 +59,5 @@ public class RowSerializer implements CompositeFieldSerializer<RowKey> {
 
         return new RowKey( id, edgeType, shard );
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowTypeSerializer.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowTypeSerializer.java b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowTypeSerializer.java
index a67c469..c7aac4e 100644
--- a/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowTypeSerializer.java
+++ b/stack/corepersistence/graph/src/main/java/org/apache/usergrid/persistence/graph/serialization/impl/shard/impl/serialize/RowTypeSerializer.java
@@ -35,7 +35,6 @@ public class RowTypeSerializer implements CompositeFieldSerializer<RowKeyType> {
 
     private static final IdRowCompositeSerializer ID_SER = IdRowCompositeSerializer.get();
 
-
     @Override
     public void toComposite( final CompositeBuilder builder, final RowKeyType keyType ) {
 
@@ -59,4 +58,6 @@ public class RowTypeSerializer implements CompositeFieldSerializer<RowKeyType> {
 
         return new RowKeyType( id, edgeType, idType, shard);
     }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationBridgeTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationBridgeTest.java b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationBridgeTest.java
new file mode 100644
index 0000000..96b3a07
--- /dev/null
+++ b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationBridgeTest.java
@@ -0,0 +1,58 @@
+/*
+ *
+ *  * 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.graph.serialization;/*
+ * 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.
+ */
+
+
+import org.apache.usergrid.persistence.core.guice.CurrentVersion;
+
+import com.google.inject.Inject;
+
+
+public class EdgeMetaDataSerializationBridgeTest extends EdgeMetadataSerializationTest {
+
+
+    @Inject
+    @CurrentVersion
+    protected EdgeMetadataSerialization serialization;
+
+    @Override
+    protected EdgeMetadataSerialization getSerializationImpl() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV1Test.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV1Test.java b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV1Test.java
new file mode 100644
index 0000000..00453d4
--- /dev/null
+++ b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV1Test.java
@@ -0,0 +1,58 @@
+/*
+ *
+ *  * 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.graph.serialization;/*
+ * 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.
+ */
+
+
+import org.apache.usergrid.persistence.core.guice.PreviousVersion;
+
+import com.google.inject.Inject;
+
+
+public class EdgeMetaDataSerializationV1Test extends EdgeMetadataSerializationTest {
+
+
+    @Inject
+    @PreviousVersion
+    protected EdgeMetadataSerialization serialization;
+
+    @Override
+    protected EdgeMetadataSerialization getSerializationImpl() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV2Test.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV2Test.java b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV2Test.java
new file mode 100644
index 0000000..522368b
--- /dev/null
+++ b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetaDataSerializationV2Test.java
@@ -0,0 +1,60 @@
+/*
+ *
+ *  * 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.graph.serialization;/*
+ * 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.
+ */
+
+
+import org.apache.usergrid.persistence.core.guice.CurrentVersion;
+import org.apache.usergrid.persistence.core.guice.PreviousVersion;
+import org.apache.usergrid.persistence.core.guice.ProxyVersion;
+
+import com.google.inject.Inject;
+
+
+public class EdgeMetaDataSerializationV2Test extends EdgeMetadataSerializationTest {
+
+
+    @Inject
+    @ProxyVersion
+    protected EdgeMetadataSerialization serialization;
+
+    @Override
+    protected EdgeMetadataSerialization getSerializationImpl() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetadataSerializationTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetadataSerializationTest.java b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetadataSerializationTest.java
index 517ba03..02f18c9 100644
--- a/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetadataSerializationTest.java
+++ b/stack/corepersistence/graph/src/test/java/org/apache/usergrid/persistence/graph/serialization/EdgeMetadataSerializationTest.java
@@ -64,15 +64,13 @@ import static org.mockito.Mockito.when;
  */
 @RunWith(ITRunner.class)
 @UseModules({ TestGraphModule.class })
-public class EdgeMetadataSerializationTest {
+public abstract class EdgeMetadataSerializationTest {
 
     @Inject
     @Rule
     public MigrationManagerRule migrationManagerRule;
 
 
-    @Inject
-    protected EdgeMetadataSerialization serialization;
 
     @Inject
     protected Keyspace keyspace;
@@ -80,6 +78,7 @@ public class EdgeMetadataSerializationTest {
 
     protected ApplicationScope scope;
 
+    protected EdgeMetadataSerialization serialization;
 
     @Before
     public void setup() {
@@ -91,6 +90,8 @@ public class EdgeMetadataSerializationTest {
         when( orgId.getUuid() ).thenReturn( UUIDGenerator.newTimeUUID() );
 
         when( scope.getApplication() ).thenReturn( orgId );
+
+        serialization = getSerializationImpl();
     }
 
 
@@ -575,4 +576,6 @@ public class EdgeMetadataSerializationTest {
 
         assertTrue( deleted );
     }
+
+    protected abstract EdgeMetadataSerialization getSerializationImpl();
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/963fe57c/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java b/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
index 7f2598b..639f345 100644
--- a/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
+++ b/stack/corepersistence/map/src/main/java/org/apache/usergrid/persistence/map/impl/MapSerializationImpl.java
@@ -18,7 +18,6 @@
  */
 
 package org.apache.usergrid.persistence.map.impl;
-import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -35,7 +34,8 @@ import org.apache.usergrid.persistence.core.astyanax.BucketScopedRowKey;
 import org.apache.usergrid.persistence.core.astyanax.BucketScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKeySerializer;
 import org.apache.usergrid.persistence.core.astyanax.ScopedRowKey;
-import org.apache.usergrid.persistence.core.hash.ExpandingBucketLocator;
+import org.apache.usergrid.persistence.core.shard.ExpandingShardLocator;
+import org.apache.usergrid.persistence.core.shard.StringHashUtils;
 import org.apache.usergrid.persistence.map.MapScope;
 
 import com.google.common.hash.Funnel;
@@ -96,18 +96,19 @@ public class MapSerializationImpl implements MapSerialization {
      */
     private static final Funnel<String> MAP_KEY_FUNNEL = new Funnel<String>() {
 
-        private Charset UTF8 = Charset.forName( "UTF8" );
+
 
         @Override
         public void funnel( final String key, final PrimitiveSink into ) {
-            into.putString( key, UTF8 );
+            into.putString( key, StringHashUtils.UTF8 );
         }
     };
 
     /**
      * Locator to get us all buckets
      */
-    private static final ExpandingBucketLocator<String> BUCKET_LOCATOR = new ExpandingBucketLocator<>(MAP_KEY_FUNNEL, NUM_BUCKETS);
+    private static final ExpandingShardLocator<String>
+            BUCKET_LOCATOR = new ExpandingShardLocator<>(MAP_KEY_FUNNEL, NUM_BUCKETS);
 
     private final Keyspace keyspace;