You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@usergrid.apache.org by sf...@apache.org on 2015/07/27 23:20:28 UTC

incubator-usergrid git commit: convert to lat lon for ES otherwise its latitude longitude

Repository: incubator-usergrid
Updated Branches:
  refs/heads/USERGRID-880 [created] ab059f649


convert to lat lon for ES otherwise its latitude longitude


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

Branch: refs/heads/USERGRID-880
Commit: ab059f64990a030bbe2a079b80850a3c2ca3f263
Parents: 4a7ce9a
Author: Shawn Feldman <sf...@apache.org>
Authored: Mon Jul 27 15:20:15 2015 -0600
Committer: Shawn Feldman <sf...@apache.org>
Committed: Mon Jul 27 15:20:15 2015 -0600

----------------------------------------------------------------------
 .../corepersistence/util/CpEntityMapUtils.java  | 286 +------------------
 .../org/apache/usergrid/persistence/Schema.java |  18 +-
 .../org/apache/usergrid/persistence/GeoIT.java  |   6 +-
 .../model/collection/SchemaManager.java         |  35 +++
 .../model/entity/EntityToMapConverter.java      |  61 ++--
 .../model/entity/MapToEntityConverter.java      | 137 +++++----
 .../index/impl/EntityMappingParser.java         |  14 +-
 .../index/impl/EntityToMapConverterTest.java    |  12 +-
 8 files changed, 181 insertions(+), 388 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/core/src/main/java/org/apache/usergrid/corepersistence/util/CpEntityMapUtils.java
----------------------------------------------------------------------
diff --git a/stack/core/src/main/java/org/apache/usergrid/corepersistence/util/CpEntityMapUtils.java b/stack/core/src/main/java/org/apache/usergrid/corepersistence/util/CpEntityMapUtils.java
index 80c7857..07078fc 100644
--- a/stack/core/src/main/java/org/apache/usergrid/corepersistence/util/CpEntityMapUtils.java
+++ b/stack/core/src/main/java/org/apache/usergrid/corepersistence/util/CpEntityMapUtils.java
@@ -33,7 +33,10 @@ import java.util.UUID;
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.datatype.guava.GuavaModule;
 import org.apache.usergrid.persistence.Schema;
+import org.apache.usergrid.persistence.model.collection.SchemaManager;
 import org.apache.usergrid.persistence.model.entity.Entity;
+import org.apache.usergrid.persistence.model.entity.EntityToMapConverter;
+import org.apache.usergrid.persistence.model.entity.MapToEntityConverter;
 import org.apache.usergrid.persistence.model.entity.SimpleId;
 import org.apache.usergrid.persistence.model.field.*;
 import org.apache.usergrid.persistence.model.field.value.EntityObject;
@@ -51,12 +54,9 @@ import org.slf4j.LoggerFactory;
  */
 public class CpEntityMapUtils {
 
-    private static final Logger logger = LoggerFactory.getLogger( CpEntityMapUtils.class );
-
-    public static JsonFactory jsonFactory = new JsonFactory();
-    public static ObjectMapper objectMapper = new ObjectMapper(  jsonFactory )
-        .registerModule(new GuavaModule());
 
+    private static final MapToEntityConverter mapConverter = new MapToEntityConverter();
+    private static final EntityToMapConverter entityConverter = new EntityToMapConverter();
 
     /**
      * Convert a usergrid 1.0 entity into a usergrid 2.0 entity
@@ -78,7 +78,7 @@ public class CpEntityMapUtils {
 
 
     public static Entity fromMap( Map<String, Object> map, String entityType, boolean topLevel ) {
-        return fromMap( null, map, entityType, topLevel );
+        return fromMap(null, map, entityType, topLevel);
     }
 
     public static Entity fromMap(
@@ -88,67 +88,8 @@ public class CpEntityMapUtils {
             entity = new Entity();
         }
 
-        for ( String fieldName : map.keySet() ) {
-
-            Object value = map.get( fieldName );
-            boolean unique = Schema.getDefaultSchema().isPropertyUnique(entityType, fieldName);
-
-            if ( value instanceof String ) {
-                entity.setField( new StringField( fieldName, (String)value, unique && topLevel ));
-
-            } else if ( value instanceof Boolean ) {
-                entity.setField( new BooleanField( fieldName, (Boolean)value, unique && topLevel ));
-
-            } else if ( value instanceof Integer ) {
-                entity.setField( new IntegerField( fieldName, (Integer)value, unique && topLevel ));
-
-            } else if ( value instanceof Double ) {
-                entity.setField( new DoubleField( fieldName, (Double)value, unique && topLevel ));
-
-		    } else if ( value instanceof Float ) {
-                entity.setField( new FloatField( fieldName, (Float)value, unique && topLevel ));
-
-            } else if ( value instanceof Long ) {
-                entity.setField( new LongField( fieldName, (Long)value, unique && topLevel ));
-
-            } else if ( value instanceof List) {
-                entity.setField( listToListField( fieldName, (List)value, entityType ));
-
-            } else if ( value instanceof UUID) {
-                entity.setField( new UUIDField( fieldName, (UUID)value, unique && topLevel ));
-
-            } else if ( value instanceof Map ) {
-                processMapValue( value, fieldName, entity, entityType);
-
-            } else if ( value instanceof Enum ) {
-                entity.setField( new StringField( fieldName, value.toString(), unique && topLevel ));
-
-			} else if ( value != null ) {
-//                String valueSerialized;
-//                try {
-//                    valueSerialized = objectMapper.writeValueAsString(value);
-//                }
-//                catch ( JsonProcessingException e ) {
-//                    throw new RuntimeException( "Can't serialize object ",e );
-//                }
-//
-//                SerializedObjectField bf = new SerializedObjectField( fieldName, valueSerialized, value.getClass() );
-//                entity.setField( bf );
-                byte[] valueSerialized;
-                try {
-                    valueSerialized = objectMapper.writeValueAsBytes( value );
-                }
-                catch ( JsonProcessingException e ) {
-                    throw new RuntimeException( "Can't serialize object ",e );
-                }
-
-                ByteBuffer byteBuffer = ByteBuffer.wrap(valueSerialized);
-                ByteArrayField bf = new ByteArrayField( fieldName, byteBuffer.array(), value.getClass() );
-                entity.setField( bf );
-            }
-        }
-
-        return entity;
+        SchemaManager schemaManager = Schema.getDefaultSchema();
+        return mapConverter.fromMap(entity,map, schemaManager, entityType,topLevel);
     }
 
 
@@ -160,217 +101,8 @@ public class CpEntityMapUtils {
      */
     public static Map toMap(EntityObject entity) {
 
-        Map<String, Object> entityMap = new TreeMap<>();
-
-        for (Object f : entity.getFields().toArray()) {
-            Field field = (Field) f;
-
-            if (f instanceof ListField || f instanceof ArrayField) {
-                List list = (List) field.getValue();
-                entityMap.put(field.getName(),
-                        new ArrayList( processCollectionForMap(list)));
-
-            } else if (f instanceof SetField) {
-                Set set = (Set) field.getValue();
-                entityMap.put(field.getName(),
-                        new ArrayList( processCollectionForMap(set)));
-
-            } else if (f instanceof EntityObjectField) {
-                EntityObject eo = (EntityObject) field.getValue();
-                entityMap.put( field.getName(), toMap(eo)); // recursion
-
-            } else if (f instanceof StringField) {
-                entityMap.put(field.getName(), ((String) field.getValue()));
-
-            } else if (f instanceof LocationField) {
-                LocationField locField = (LocationField) f;
-                Map<String, Object> locMap = new HashMap<String, Object>();
-
-                // field names lat and lon trigger ElasticSearch geo location
-                locMap.put("lat", locField.getValue().getLatitude());
-                locMap.put("lon", locField.getValue().getLongitude());
-                entityMap.put( field.getName(), field.getValue());
-
-            } else if (f instanceof ByteArrayField) {
-                    ByteArrayField bf = ( ByteArrayField ) f;
-
-                    byte[] serilizedObj =  bf.getValue();
-                    Object o;
-                    try {
-                        o = objectMapper.readValue( serilizedObj, bf.getClassinfo() );
-                    }
-                    catch ( IOException e ) {
-                        throw new RuntimeException( "Can't deserialize object ",e );
-                    }
-                    entityMap.put( bf.getName(), o );
-            }
-            else if (f instanceof SerializedObjectField) {
-                SerializedObjectField bf = (SerializedObjectField) f;
-
-                String serilizedObj =  bf.getValue();
-                Object o;
-                try {
-                    o = objectMapper.readValue( serilizedObj, bf.getClassinfo() );
-                }
-                catch ( IOException e ) {
-                    throw new RuntimeException( "Can't deserialize object "+serilizedObj,e );
-                }
-                entityMap.put( bf.getName(), o );
-            }else {
-                entityMap.put( field.getName(), field.getValue());
-            }
-        }
-
-        return entityMap;
-    }
-
-    private static void processMapValue(
-        Object value, String fieldName, Entity entity, String entityType) {
-
-        // is the map really a location element?
-        if ("location" .equals(fieldName.toString().toLowerCase()) ) {
-
-            // get the object to inspect
-            Map<String, Object> origMap = (Map<String, Object>) value;
-            Map<String, Object> m = new HashMap<String, Object>();
-
-            // Tests expect us to treat "Longitude" the same as "longitude"
-            for ( String key : origMap.keySet() ) {
-                m.put( key.toLowerCase(), origMap.get(key) );
-            }
-
-            // Expect at least two fields in a Location object
-            if (m.size() >= 2) {
+        return entityConverter.toMap(entity);
 
-                Double lat = null;
-                Double lon = null;
-
-                // check the properties to make sure they are set and are doubles
-                if (m.get("latitude") != null && m.get("longitude") != null) {
-                    try {
-                        lat = Double.parseDouble(m.get("latitude").toString());
-                        lon = Double.parseDouble(m.get("longitude").toString());
-
-                    } catch (NumberFormatException ignored) {
-                        throw new IllegalArgumentException(
-                            "Latitude and longitude must be doubles (e.g. 32.1234).");
-                    }
-                } else if (m.get("lat") != null && m.get("lon") != null) {
-                    try {
-                        lat = Double.parseDouble(m.get("lat").toString());
-                        lon = Double.parseDouble(m.get("lon").toString());
-                    } catch (NumberFormatException ignored) {
-                        throw new IllegalArgumentException(""
-                            + "Latitude and longitude must be doubles (e.g. 32.1234).");
-                    }
-                } else {
-                    throw new IllegalArgumentException("Location properties require two fields - "
-                        + "latitude and longitude, or lat and lon");
-                }
-
-                if (lat != null && lon != null) {
-                    entity.setField( new LocationField(fieldName, new Location(lat, lon)));
-                } else {
-                    throw new IllegalArgumentException( "Unable to parse location field properties "
-                        + "- make sure they conform - lat and lon, and should be doubles.");
-                }
-            } else {
-                throw new IllegalArgumentException("Location properties requires two fields - "
-                    + "latitude and longitude, or lat and lon.");
-            }
-        } else {
-            // not a location element, process it as map
-            entity.setField(new EntityObjectField(fieldName,
-                fromMap((Map<String, Object>) value, entityType, false))); // recursion
-        }
-    }
-
-
-    private static ListField listToListField( String fieldName, List list, String entityType ) {
-
-        if (list.isEmpty()) {
-            return new ArrayField( fieldName );
-        }
-
-        Object sample = list.get(0);
-
-        if ( sample instanceof Map ) {
-            return new ArrayField<Entity>( fieldName, processListForField( list, entityType ));
-
-        } else if ( sample instanceof List ) {
-            return new ArrayField<List>( fieldName, processListForField( list, entityType ));
-        } else if ( sample instanceof String ) {
-            return new ArrayField<String>( fieldName, (List<String>)list );
-
-        } else if ( sample instanceof Boolean ) {
-            return new ArrayField<Boolean>( fieldName, (List<Boolean>)list );
-
-        } else if ( sample instanceof Integer ) {
-            return new ArrayField<Integer>( fieldName, (List<Integer>)list );
-
-        } else if ( sample instanceof Double ) {
-            return new ArrayField<Double>( fieldName, (List<Double>)list );
-
-        } else if ( sample instanceof Long ) {
-            return new ArrayField<Long>( fieldName, (List<Long>)list );
-
-        } else {
-            throw new RuntimeException("Unknown type " + sample.getClass().getName());
-        }
-    }
-
-
-    private static List processListForField( List list, String entityType ) {
-        if ( list.isEmpty() ) {
-            return list;
-        }
-        Object sample = list.get(0);
-
-        if ( sample instanceof Map ) {
-            List<Entity> newList = new ArrayList<Entity>();
-            for ( Map<String, Object> map : (List<Map<String, Object>>)list ) {
-                newList.add( fromMap( map, entityType, false ) );
-            }
-            return newList;
-
-        } else if ( sample instanceof List ) {
-            return processListForField( list, entityType ); // recursion
-
-        } else {
-            return list;
-        }
-    }
-    private static Collection processCollectionForMap(Collection c) {
-        if (c.isEmpty()) {
-            return c;
-        }
-        List processed = new ArrayList();
-        Object sample = c.iterator().next();
-
-        if (sample instanceof Entity) {
-            for (Object o : c.toArray()) {
-                Entity e = (Entity) o;
-                processed.add(toMap(e));
-            }
-
-        } else if (sample instanceof List) {
-            for (Object o : c.toArray()) {
-                List list = (List) o;
-                processed.add(processCollectionForMap(list)); // recursion;
-            }
-
-        } else if (sample instanceof Set) {
-            for (Object o : c.toArray()) {
-                Set set = (Set) o;
-                processed.add(processCollectionForMap(set)); // recursion;
-            }
-
-        } else {
-            for (Object o : c.toArray()) {
-                processed.add(o);
-            }
-        }
-        return processed;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/core/src/main/java/org/apache/usergrid/persistence/Schema.java
----------------------------------------------------------------------
diff --git a/stack/core/src/main/java/org/apache/usergrid/persistence/Schema.java b/stack/core/src/main/java/org/apache/usergrid/persistence/Schema.java
index b42995a..d1ac6dc 100644
--- a/stack/core/src/main/java/org/apache/usergrid/persistence/Schema.java
+++ b/stack/core/src/main/java/org/apache/usergrid/persistence/Schema.java
@@ -41,6 +41,7 @@ import javax.crypto.KeyGenerator;
 import javax.crypto.SecretKey;
 import javax.crypto.spec.SecretKeySpec;
 
+import org.apache.usergrid.persistence.model.collection.SchemaManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.config.BeanDefinition;
@@ -95,7 +96,7 @@ import static org.apache.usergrid.utils.StringUtils.stringOrSubstringAfterLast;
  *
  * @author edanuff
  */
-public class Schema {
+public class Schema implements SchemaManager {
 
     private static final Logger logger = LoggerFactory.getLogger( Schema.class );
 
@@ -697,7 +698,8 @@ public class Schema {
 
 
     /** @return value */
-    public boolean isPropertyMutable( String entityType, String propertyName ) {
+    @Override
+    public boolean isPropertyMutable(String entityType, String propertyName) {
 
         EntityInfo entity = getEntityInfo( entityType );
         return entity != null && entity.isPropertyMutable(propertyName);
@@ -705,7 +707,8 @@ public class Schema {
     }
 
 
-    public boolean isPropertyUnique( String entityType, String propertyName ) {
+    @Override
+    public boolean isPropertyUnique(String entityType, String propertyName) {
 
         EntityInfo entity = getEntityInfo( entityType );
         return entity != null && entity.isPropertyUnique(propertyName);
@@ -713,7 +716,8 @@ public class Schema {
     }
 
 
-    public boolean isPropertyIndexed( String entityType, String propertyName ) {
+    @Override
+    public boolean isPropertyIndexed(String entityType, String propertyName) {
 
         EntityInfo entity = getEntityInfo( entityType );
         return entity == null || !entity.hasProperty(propertyName) || entity.isPropertyIndexed(propertyName);
@@ -721,7 +725,8 @@ public class Schema {
     }
 
 
-    public boolean isPropertyFulltextIndexed( String entityType, String propertyName ) {
+    @Override
+    public boolean isPropertyFulltextIndexed(String entityType, String propertyName) {
 
         EntityInfo entity = getEntityInfo( entityType );
         return entity == null || !entity.hasProperty(propertyName) || entity.isPropertyFulltextIndexed(propertyName);
@@ -729,7 +734,8 @@ public class Schema {
     }
 
 
-    public boolean isPropertyTimestamp( String entityType, String propertyName ) {
+    @Override
+    public boolean isPropertyTimestamp(String entityType, String propertyName) {
 
         EntityInfo entity = getEntityInfo( entityType );
         return entity != null && entity.isPropertyTimestamp(propertyName);

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/core/src/test/java/org/apache/usergrid/persistence/GeoIT.java
----------------------------------------------------------------------
diff --git a/stack/core/src/test/java/org/apache/usergrid/persistence/GeoIT.java b/stack/core/src/test/java/org/apache/usergrid/persistence/GeoIT.java
index 4529140..8d71826 100644
--- a/stack/core/src/test/java/org/apache/usergrid/persistence/GeoIT.java
+++ b/stack/core/src/test/java/org/apache/usergrid/persistence/GeoIT.java
@@ -104,7 +104,9 @@ public class GeoIT extends AbstractCoreIT {
 
         //3. Remove the entity's location
         properties.remove("location");
+        user.getDynamicProperties().remove("location");
         em.updateProperties(user, properties);
+        em.update(user);
         app.refreshIndex();
 
         //4. Repeat the query, expecting no results
@@ -241,8 +243,10 @@ public class GeoIT extends AbstractCoreIT {
         assertEquals("total number of 'stores'", LOCATION_PROPERTIES.size(), listResults.size());
         //3. verify each entity has geo data
         for (Entity entity : listResults.entities) {
-            Location location = (Location) entity.getProperty("location");
+            Map location =  (Map)entity.getProperty("location");
             assertNotNull(location);
+            assertNotNull(location.get("longitude"));
+            assertNotNull(location.get("latitude"));
         }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/collection/SchemaManager.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/collection/SchemaManager.java b/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/collection/SchemaManager.java
new file mode 100644
index 0000000..452d65e
--- /dev/null
+++ b/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/collection/SchemaManager.java
@@ -0,0 +1,35 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  *  contributor license agreements.  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.  For additional information regarding
+ *  * copyright in this work, please see the NOTICE file in the top level
+ *  * directory of this distribution.
+ *
+ */
+package org.apache.usergrid.persistence.model.collection;
+
+/**
+ * interface encapsulates schema structure
+ */
+public interface SchemaManager {
+    boolean isPropertyMutable(String entityType, String propertyName);
+
+    boolean isPropertyUnique(String entityType, String propertyName);
+
+    boolean isPropertyIndexed(String entityType, String propertyName);
+
+    boolean isPropertyFulltextIndexed(String entityType, String propertyName);
+
+    boolean isPropertyTimestamp(String entityType, String propertyName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/EntityToMapConverter.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/EntityToMapConverter.java b/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/EntityToMapConverter.java
index c7bd90b..9eb50c7 100644
--- a/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/EntityToMapConverter.java
+++ b/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/EntityToMapConverter.java
@@ -20,23 +20,11 @@ package org.apache.usergrid.persistence.model.entity;
 
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.Stack;
-
-import org.apache.usergrid.persistence.model.field.AbstractField;
-import org.apache.usergrid.persistence.model.field.ArrayField;
-import org.apache.usergrid.persistence.model.field.ByteArrayField;
-import org.apache.usergrid.persistence.model.field.EntityObjectField;
-import org.apache.usergrid.persistence.model.field.Field;
-import org.apache.usergrid.persistence.model.field.ListField;
-import org.apache.usergrid.persistence.model.field.LocationField;
-import org.apache.usergrid.persistence.model.field.SetField;
+import java.util.*;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.datatype.guava.GuavaModule;
+import org.apache.usergrid.persistence.model.field.*;
 import org.apache.usergrid.persistence.model.field.value.EntityObject;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -46,27 +34,31 @@ import com.fasterxml.jackson.databind.ObjectMapper;
  * abstract conversion to Map<String,Object> form EntityObject
  */
 public class EntityToMapConverter {
-    public static final String LAT = "lat";
-    public static final String LON = "lon";
+    public static final String LAT = "latitude";
+    public static final String LON = "longitude";
+    private final JsonFactory jsonFactory = new JsonFactory();
+    private final ObjectMapper objectMapper = new ObjectMapper(jsonFactory).registerModule(new GuavaModule());
+
+    /**
+     * Convert Entity to Map, adding version_ug_field and a {name}_ug_analyzed field for each StringField.
+     */
 
-    public static ObjectMapper objectMapper = new ObjectMapper();
 
-    private final Stack<String> fieldName = new Stack();
 
     /**
-     * Convert Entity to Map, adding version_ug_field and a {name}_ug_analyzed field for each StringField.
+     * hacky impl, for outbound implementations longitude needs to be \ "longitude" and not "lon"
+     * @param entityObject
+     * @return
      */
 
+
     public EntityMap toMap( EntityObject entityObject ) {
         EntityMap map = new EntityMap();
         return toMap(entityObject, map);
     }
 
-
-    public EntityMap toMap( EntityObject entity, EntityMap entityMap ) {
-
+    private EntityMap toMap( EntityObject entity, EntityMap entityMap ) {
         for ( Field field : entity.getFields() ) {
-
             if ( field instanceof ListField || field instanceof ArrayField  || field instanceof SetField) {
                 Collection list = ( Collection ) field.getValue();
                 entityMap.put( field.getName(), processCollection( list )  );
@@ -96,16 +88,25 @@ public class EntityToMapConverter {
                     throw new RuntimeException( "Can't deserialize object ", e );
                 }
                 entityMap.put( bf.getName(), o );
-            }
-            else {
-                entityMap.put( field.getName(), field.getValue() );
+            }else if (field instanceof SerializedObjectField) {
+                SerializedObjectField bf = (SerializedObjectField) field;
+
+                String serilizedObj = bf.getValue();
+                Object o;
+                try {
+                    o = objectMapper.readValue(serilizedObj, bf.getClassinfo());
+                } catch (IOException e) {
+                    throw new RuntimeException("Can't deserialize object " + serilizedObj, e);
+                }
+                entityMap.put(bf.getName(), o);
+            } else {
+                entityMap.put(field.getName(), field.getValue());
             }
         }
 
         return entityMap;
     }
 
-
     /**
      * Process the collection for our map
      * @param c

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/MapToEntityConverter.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/MapToEntityConverter.java b/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/MapToEntityConverter.java
index 6bd6cb6..8f79de5 100644
--- a/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/MapToEntityConverter.java
+++ b/stack/corepersistence/model/src/main/java/org/apache/usergrid/persistence/model/entity/MapToEntityConverter.java
@@ -1,7 +1,10 @@
 package org.apache.usergrid.persistence.model.entity;
 
+import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.guava.GuavaModule;
+import org.apache.usergrid.persistence.model.collection.SchemaManager;
 import org.apache.usergrid.persistence.model.field.*;
 import org.apache.usergrid.persistence.model.field.value.Location;
 
@@ -11,49 +14,51 @@ import java.util.*;
 
 public class MapToEntityConverter{
 
-    private ObjectMapper objectMapper = new ObjectMapper();
+    private final JsonFactory jsonFactory = new JsonFactory();
+    private final ObjectMapper objectMapper = new ObjectMapper(jsonFactory).registerModule(new GuavaModule());
 
     public  Entity fromMap( Map<String, Object> map,  boolean topLevel ) {
 
         Entity  entity = new Entity();
-        return fromMap( entity, map, topLevel );
+        return fromMap( entity, map, null, null, topLevel );
     }
 
-    public Entity fromMap( Entity entity, Map<String, Object> map,  boolean topLevel ) {
+    public Entity fromMap(final Entity entity,final  Map<String, Object> map,final  SchemaManager schemaManager, final String entityType, boolean topLevel) {
 
         for ( String fieldName : map.keySet() ) {
 
-            Object value = map.get(fieldName);
+            Object value = map.get( fieldName );
+            boolean unique = schemaManager == null ? topLevel :  topLevel && schemaManager.isPropertyUnique(entityType, fieldName);
 
             if ( value instanceof String ) {
-                entity.setField( new StringField( fieldName, (String)value, topLevel ));
+                entity.setField( new StringField( fieldName, (String)value, unique  ));
 
             } else if ( value instanceof Boolean ) {
-                entity.setField( new BooleanField( fieldName, (Boolean)value, topLevel ));
+                entity.setField( new BooleanField( fieldName, (Boolean)value, unique  ));
 
             } else if ( value instanceof Integer ) {
-                entity.setField( new IntegerField( fieldName, (Integer)value, topLevel ));
+                entity.setField( new IntegerField( fieldName, (Integer)value, unique ));
 
             } else if ( value instanceof Double ) {
-                entity.setField( new DoubleField( fieldName, (Double)value, topLevel ));
+                entity.setField( new DoubleField( fieldName, (Double)value, unique  ));
 
             } else if ( value instanceof Float ) {
-                entity.setField( new FloatField( fieldName, (Float)value, topLevel ));
+                entity.setField( new FloatField( fieldName, (Float)value, unique ));
 
             } else if ( value instanceof Long ) {
-                entity.setField( new LongField( fieldName, (Long)value, topLevel ));
+                entity.setField( new LongField( fieldName, (Long)value, unique ));
 
             } else if ( value instanceof List) {
                 entity.setField( listToListField( fieldName, (List)value ));
 
             } else if ( value instanceof UUID) {
-                entity.setField( new UUIDField( fieldName, (UUID)value, topLevel ));
+                entity.setField( new UUIDField( fieldName, (UUID)value, unique ));
 
             } else if ( value instanceof Map ) {
                 processMapValue( value, fieldName, entity);
 
             } else if ( value instanceof Enum ) {
-                entity.setField( new StringField( fieldName, value.toString(), topLevel ));
+                entity.setField( new StringField( fieldName, value.toString(), unique ));
 
             } else if ( value != null ) {
                 byte[] valueSerialized;
@@ -63,15 +68,14 @@ public class MapToEntityConverter{
                 catch ( JsonProcessingException e ) {
                     throw new RuntimeException( "Can't serialize object ",e );
                 }
-                catch ( IOException e ) {
-                    throw new RuntimeException( "Can't serialize object ",e );
-                }
-                ByteBuffer byteBuffer = ByteBuffer.wrap( valueSerialized );
+
+                ByteBuffer byteBuffer = ByteBuffer.wrap(valueSerialized);
                 ByteArrayField bf = new ByteArrayField( fieldName, byteBuffer.array(), value.getClass() );
                 entity.setField( bf );
             }
         }
 
+
         return entity;
     }
 
@@ -136,59 +140,70 @@ public class MapToEntityConverter{
 
         // is the map really a location element?
         if ("location" .equals(fieldName.toString().toLowerCase()) ) {
+            processLocationField((Map<String, Object>) value, fieldName, entity);
 
-            // get the object to inspect
-            Map<String, Object> origMap = (Map<String, Object>) value;
-            Map<String, Object> m = new HashMap<String, Object>();
 
-            // Tests expect us to treat "Longitude" the same as "longitude"
-            for ( String key : origMap.keySet() ) {
-                m.put( key.toLowerCase(), origMap.get(key) );
-            }
+        } else {
+            // not a location element, process it as map
+            entity.setField(new EntityObjectField(fieldName,
+                    fromMap((Map<String, Object>) value, false))); // recursion
+        }
+    }
 
-            // Expect at least two fields in a Location object
-            if (m.size() >= 2) {
-
-                Double lat = null;
-                Double lon = null;
-
-                // check the properties to make sure they are set and are doubles
-                if (m.get("latitude") != null && m.get("longitude") != null) {
-                    try {
-                        lat = Double.parseDouble(m.get("latitude").toString());
-                        lon = Double.parseDouble(m.get("longitude").toString());
-
-                    } catch (NumberFormatException ignored) {
-                        throw new IllegalArgumentException(
-                                "Latitude and longitude must be doubles (e.g. 32.1234).");
-                    }
-                } else if (m.get("lat") != null && m.get("lon") != null) {
-                    try {
-                        lat = Double.parseDouble(m.get("lat").toString());
-                        lon = Double.parseDouble(m.get("lon").toString());
-                    } catch (NumberFormatException ignored) {
-                        throw new IllegalArgumentException(""
-                                + "Latitude and longitude must be doubles (e.g. 32.1234).");
-                    }
-                } else {
-                    throw new IllegalArgumentException("Location properties require two fields - "
-                            + "latitude and longitude, or lat and lon");
-                }
+    /**
+     * for location we need to parse two formats potentially and convert to a typed field
+     * @param value
+     * @param fieldName
+     * @param entity
+     */
+    private void processLocationField(Map<String, Object> value, String fieldName, Entity entity) {
+        // get the object to inspect
+        Map<String, Object> origMap = value;
+        Map<String, Object> m = new HashMap<String, Object>();
+
+        // Tests expect us to treat "Longitude" the same as "longitude"
+        for ( String key : origMap.keySet() ) {
+            m.put( key.toLowerCase(), origMap.get(key) );
+        }
 
-                if (lat != null && lon != null) {
-                    entity.setField( new LocationField(fieldName, new Location(lat, lon)));
-                } else {
-                    throw new IllegalArgumentException( "Unable to parse location field properties "
-                            + "- make sure they conform - lat and lon, and should be doubles.");
+        // Expect at least two fields in a Location object
+        if (m.size() >= 2) {
+
+            Double lat = null;
+            Double lon = null;
+
+            // check the properties to make sure they are set and are doubles
+            if (m.get("latitude") != null && m.get("longitude") != null) {
+                try {
+                    lat = Double.parseDouble(m.get("latitude").toString());
+                    lon = Double.parseDouble(m.get("longitude").toString());
+
+                } catch (NumberFormatException ignored) {
+                    throw new IllegalArgumentException(
+                            "Latitude and longitude must be doubles (e.g. 32.1234).");
+                }
+            } else if (m.get("lat") != null && m.get("lon") != null) {
+                try {
+                    lat = Double.parseDouble(m.get("lat").toString());
+                    lon = Double.parseDouble(m.get("lon").toString());
+                } catch (NumberFormatException ignored) {
+                    throw new IllegalArgumentException(""
+                            + "Latitude and longitude must be doubles (e.g. 32.1234).");
                 }
             } else {
-                throw new IllegalArgumentException("Location properties requires two fields - "
-                        + "latitude and longitude, or lat and lon.");
+                throw new IllegalArgumentException("Location properties require two fields - "
+                        + "latitude and longitude, or lat and lon");
+            }
+
+            if (lat != null && lon != null) {
+                entity.setField( new LocationField(fieldName, new Location(lat, lon)));
+            } else {
+                throw new IllegalArgumentException( "Unable to parse location field properties "
+                        + "- make sure they conform - lat and lon, and should be doubles.");
             }
         } else {
-            // not a location element, process it as map
-            entity.setField(new EntityObjectField(fieldName,
-                    fromMap((Map<String, Object>) value, false))); // recursion
+            throw new IllegalArgumentException("Location properties requires two fields - "
+                    + "latitude and longitude, or lat and lon.");
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EntityMappingParser.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EntityMappingParser.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EntityMappingParser.java
index 9adc35d..bfefaa9 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EntityMappingParser.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EntityMappingParser.java
@@ -24,12 +24,7 @@
 package org.apache.usergrid.persistence.index.impl;
 
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Stack;
-import java.util.UUID;
+import java.util.*;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -139,7 +134,12 @@ public class EntityMappingParser implements FieldParser {
         if ( value instanceof Map ) {
             //if it's a location, then create a location field.
             if ( EntityMap.isLocationField( (Map)value ) ) {
-                fields.add( EntityField.create( fieldStack.peek(), ( Map ) value ) );
+                Map<String,Object> map = ( Map ) value;
+                Map<String,Object> location = new HashMap<>(2);
+                //normalize location field to use lat/lon for es
+                location.put("lat",map.get("latitude"));
+                location.put("lon",map.get("longitude"));
+                fields.add( EntityField.create( fieldStack.peek(), location) );
                 return;
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/ab059f64/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/impl/EntityToMapConverterTest.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/impl/EntityToMapConverterTest.java b/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/impl/EntityToMapConverterTest.java
index 96f9169..2fb32cd 100644
--- a/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/impl/EntityToMapConverterTest.java
+++ b/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/impl/EntityToMapConverterTest.java
@@ -187,9 +187,9 @@ public class EntityToMapConverterTest {
             final Map<String, Double> latLong = ( Map<String, Double> ) entityField.get( IndexingUtils.FIELD_LOCATION );
 
             assertEquals( Double.valueOf( 10 ),
-                latLong.get( org.apache.usergrid.persistence.model.entity.EntityToMapConverter.LAT ) );
+                latLong.get( "lat" ) );
             assertEquals( Double.valueOf( 20 ),
-                latLong.get( org.apache.usergrid.persistence.model.entity.EntityToMapConverter.LON ) );
+                latLong.get( "lon" ) );
         } );
     }
 
@@ -404,9 +404,9 @@ public class EntityToMapConverterTest {
             final Map<String, Double> latLong = ( Map<String, Double> ) entityField.get( IndexingUtils.FIELD_LOCATION );
 
             assertEquals( Double.valueOf( 10 ),
-                latLong.get( org.apache.usergrid.persistence.model.entity.EntityToMapConverter.LAT ) );
+                latLong.get( "lat" ) );
             assertEquals( Double.valueOf( 20 ),
-                latLong.get( org.apache.usergrid.persistence.model.entity.EntityToMapConverter.LON ) );
+                latLong.get( "lon" ) );
         } );
     }
 
@@ -518,9 +518,9 @@ public class EntityToMapConverterTest {
             final Map<String, Double> latLong = ( Map<String, Double> ) entityField.get( IndexingUtils.FIELD_LOCATION );
 
             assertEquals( Double.valueOf( 10 ),
-                latLong.get( org.apache.usergrid.persistence.model.entity.EntityToMapConverter.LAT ) );
+                latLong.get("lat" ) );
             assertEquals( Double.valueOf( 20 ),
-                latLong.get( org.apache.usergrid.persistence.model.entity.EntityToMapConverter.LON ) );
+                latLong.get("lon" ) );
         } );
     }