You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@olingo.apache.org by ch...@apache.org on 2016/10/19 12:34:12 UTC

[2/2] olingo-odata4 git commit: [OLINGO-1037] Support GeoTypes in Json

[OLINGO-1037] Support GeoTypes in Json


Project: http://git-wip-us.apache.org/repos/asf/olingo-odata4/repo
Commit: http://git-wip-us.apache.org/repos/asf/olingo-odata4/commit/cc0a6a51
Tree: http://git-wip-us.apache.org/repos/asf/olingo-odata4/tree/cc0a6a51
Diff: http://git-wip-us.apache.org/repos/asf/olingo-odata4/diff/cc0a6a51

Branch: refs/heads/master
Commit: cc0a6a51a8d0b1a885b32d4b72e471b862343447
Parents: 5255c33
Author: Christian Amend <ch...@sap.com>
Authored: Wed Oct 19 14:33:34 2016 +0200
Committer: Christian Amend <ch...@sap.com>
Committed: Wed Oct 19 14:33:34 2016 +0200

----------------------------------------------------------------------
 .../json/ODataJsonDeserializer.java             | 190 ++++-
 .../serializer/json/ODataJsonSerializer.java    | 126 +++-
 .../json/ODataJsonDeserializerEntityTest.java   | 283 +++++++-
 .../json/ODataJsonSerializerTest.java           | 701 ++++++++++++-------
 4 files changed, 990 insertions(+), 310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/olingo-odata4/blob/cc0a6a51/lib/server-core/src/main/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializer.java
----------------------------------------------------------------------
diff --git a/lib/server-core/src/main/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializer.java b/lib/server-core/src/main/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializer.java
index d30d973..9106613 100644
--- a/lib/server-core/src/main/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializer.java
+++ b/lib/server-core/src/main/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializer.java
@@ -20,9 +20,12 @@ package org.apache.olingo.server.core.deserializer.json;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.math.BigDecimal;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -53,6 +56,14 @@ import org.apache.olingo.commons.api.edm.EdmType;
 import org.apache.olingo.commons.api.edm.EdmTypeDefinition;
 import org.apache.olingo.commons.api.edm.FullQualifiedName;
 import org.apache.olingo.commons.api.edm.constants.EdmTypeKind;
+import org.apache.olingo.commons.api.edm.geo.Geospatial;
+import org.apache.olingo.commons.api.edm.geo.GeospatialCollection;
+import org.apache.olingo.commons.api.edm.geo.LineString;
+import org.apache.olingo.commons.api.edm.geo.MultiLineString;
+import org.apache.olingo.commons.api.edm.geo.MultiPoint;
+import org.apache.olingo.commons.api.edm.geo.MultiPolygon;
+import org.apache.olingo.commons.api.edm.geo.Point;
+import org.apache.olingo.commons.api.edm.geo.Polygon;
 import org.apache.olingo.commons.api.format.ContentType;
 import org.apache.olingo.server.api.ServiceMetadata;
 import org.apache.olingo.server.api.deserializer.DeserializerException;
@@ -76,6 +87,19 @@ import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class ODataJsonDeserializer implements ODataDeserializer {
 
+  private static final Map<String, Class<? extends Geospatial>> jsonNameToGeoDataType;
+  static {
+    Map<String, Class<? extends Geospatial>> temp = new HashMap<String, Class<? extends Geospatial>>();
+    temp.put(Constants.ELEM_POINT, Point.class);
+    temp.put(Constants.ELEM_MULTIPOINT, MultiPoint.class);
+    temp.put(Constants.ELEM_LINESTRING, LineString.class);
+    temp.put("MultiLineString", MultiLineString.class);
+    temp.put(Constants.ELEM_POLYGON, Polygon.class);
+    temp.put("MultiPolygon", MultiPolygon.class);
+    temp.put("GeometryCollection", GeospatialCollection.class);
+    jsonNameToGeoDataType = Collections.unmodifiableMap(temp);
+  }
+
   private static final String ODATA_ANNOTATION_MARKER = "@";
   private static final String ODATA_CONTROL_INFORMATION_PREFIX = "@odata.";
 
@@ -594,16 +618,21 @@ public class ODataJsonDeserializer implements ODataDeserializer {
   private Object readPrimitiveValue(final String name, final EdmPrimitiveType type,
       final boolean isNullable, final Integer maxLength, final Integer precision, final Integer scale,
       final boolean isUnicode, final EdmMapping mapping, final JsonNode jsonNode) throws DeserializerException {
-    checkForValueNode(name, jsonNode);
     if (isValidNull(name, isNullable, jsonNode)) {
       return null;
     }
+    final boolean isGeoType = type.getName().startsWith("Geo");
+    if (!isGeoType) {
+      checkForValueNode(name, jsonNode);
+    }
     checkJsonTypeBasedOnPrimitiveType(name, type, jsonNode);
-    Class<?> javaClass = getJavaClassForPrimitiveType(mapping, type);
     try {
+      if (isGeoType) {
+        return readPrimitiveGeoValue(name, type, (ObjectNode) jsonNode);
+      }
       return type.valueOfString(jsonNode.asText(),
           isNullable, maxLength, precision, scale, isUnicode,
-          javaClass);
+          getJavaClassForPrimitiveType(mapping, type));
     } catch (final EdmPrimitiveTypeException e) {
       throw new DeserializerException(
           "Invalid value: " + jsonNode.asText() + " for property: " + name, e,
@@ -625,6 +654,132 @@ public class ODataJsonDeserializer implements ODataDeserializer {
   }
 
   /**
+   * Reads a geospatial JSON value following the GeoJSON specification defined in RFC 7946.
+   * @param name property name
+   * @param type EDM type of the value
+   *             (can be <code>null</code> for recursive calls while parsing a GeometryCollection)
+   */
+  private Geospatial readPrimitiveGeoValue(final String name, final EdmPrimitiveType type, ObjectNode jsonNode)
+      throws DeserializerException, EdmPrimitiveTypeException {
+    JsonNode typeNode = jsonNode.remove(Constants.ATTR_TYPE);
+    if (typeNode != null && typeNode.isTextual()) {
+      final Class<? extends Geospatial> geoDataType = jsonNameToGeoDataType.get(typeNode.asText());
+      if (geoDataType != null && (type == null || geoDataType.equals(type.getDefaultType()))) {
+        final JsonNode topNode = jsonNode.remove(
+            geoDataType.equals(GeospatialCollection.class) ? Constants.JSON_GEOMETRIES : Constants.JSON_COORDINATES);
+
+        // The "crs" member mentioned in some versions of the OData specification is not part of GeoJSON.
+        // It used to be used to specify the coordinate reference system.
+        // TODO: Is it OK to follow RFC 7946 strictly and not allow this element from its obsolete predecessor?
+        assertJsonNodeIsEmpty(jsonNode);
+
+        if (topNode != null && topNode.isArray()) {
+          final Geospatial.Dimension dimension = type == null || type.getName().startsWith("Geometry") ?
+              Geospatial.Dimension.GEOMETRY :
+              Geospatial.Dimension.GEOGRAPHY;
+          if (geoDataType.equals(Point.class)) {
+            return readGeoPointValue(name, dimension, topNode);
+          } else if (geoDataType.equals(MultiPoint.class)) {
+            return new MultiPoint(dimension, null, readGeoPointValues(name, dimension, 0, false, topNode));
+          } else if (geoDataType.equals(LineString.class)) {
+            // Although a line string with less than two points is not really one, the OData specification says:
+            // "The coordinates member of a LineString can have zero or more positions".
+            // Therefore the required minimal size of the points array currently is zero.
+            return new LineString(dimension, null, readGeoPointValues(name, dimension, 0, false, topNode));
+          } else if (geoDataType.equals(MultiLineString.class)) {
+            List<LineString> lines = new ArrayList<LineString>();
+            for (final JsonNode element : topNode) {
+              // Line strings can be empty (see above).
+              lines.add(new LineString(dimension, null, readGeoPointValues(name, dimension, 0, false, element)));
+            }
+            return new MultiLineString(dimension, null, lines);
+          } else if (geoDataType.equals(Polygon.class)) {
+            return readGeoPolygon(name, dimension, topNode);
+          } else if (geoDataType.equals(MultiPolygon.class)) {
+            List<Polygon> polygons = new ArrayList<Polygon>();
+            for (final JsonNode element : topNode) {
+              polygons.add(readGeoPolygon(name, dimension, element));
+            }
+            return new MultiPolygon(dimension, null, polygons);
+          } else if (geoDataType.equals(GeospatialCollection.class)) {
+            List<Geospatial> elements = new ArrayList<Geospatial>();
+            for (final JsonNode element : topNode) {
+              if (element.isObject()) {
+                elements.add(readPrimitiveGeoValue(name, null, (ObjectNode) element));
+              } else {
+                throw new DeserializerException("Invalid value '" + element + "' in property: " + name,
+                    DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY, name);
+              }
+            }
+            return new GeospatialCollection(dimension, null, elements);
+          }
+        }
+      }
+    }
+    throw new DeserializerException("Invalid value '" + jsonNode + "' for property: " + name,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY, name);
+  }
+
+  private Point readGeoPointValue(final String name, final Geospatial.Dimension dimension, JsonNode node)
+      throws DeserializerException, EdmPrimitiveTypeException {
+    if (node.isArray() && (node.size() ==2 || node.size() == 3)
+        && node.get(0).isNumber() && node.get(1).isNumber() && (node.get(2) == null || node.get(2).isNumber())) {
+      Point point = new Point(dimension, null);
+      point.setX(getDoubleValue(node.get(0).asText()));
+      point.setY(getDoubleValue(node.get(1).asText()));
+      if (node.get(2) != null) {
+        point.setZ(getDoubleValue(node.get(2).asText()));
+      }
+      return point;
+    }
+    throw new DeserializerException("Invalid point value '" + node + "' in property: " + name,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY, name);
+  }
+
+  private double getDoubleValue(final String value) throws EdmPrimitiveTypeException {
+    final BigDecimal bigDecimalValue = new BigDecimal(value);
+    final Double result = bigDecimalValue.doubleValue();
+    // "Real" infinite values cannot occur, so we can throw an exception
+    // if the conversion to a double results in an infinite value.
+    // An exception is also thrown if the number cannot be stored in a double without loss.
+    if (result.isInfinite() || BigDecimal.valueOf(result).compareTo(bigDecimalValue) != 0) {
+      throw new EdmPrimitiveTypeException("The literal '" + value + "' has illegal content.");
+    }
+    return result;
+  }
+
+  private List<Point> readGeoPointValues(final String name, final Geospatial.Dimension dimension,
+      final int minimalSize, final boolean closed, JsonNode node)
+      throws DeserializerException, EdmPrimitiveTypeException {
+    if (node.isArray()) {
+      List<Point> points = new ArrayList<Point>();
+      for (final JsonNode element : node) {
+        points.add(readGeoPointValue(name, dimension, element));
+      }
+      if (points.size() >= minimalSize
+          && (!closed || points.get(points.size() - 1).equals(points.get(0)))) {
+          return points;
+      }
+    }
+    throw new DeserializerException("Invalid point values '" + node + "' in property: " + name,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY, name);
+  }
+
+  private Polygon readGeoPolygon(final String name, final Geospatial.Dimension dimension, JsonNode node)
+      throws DeserializerException, EdmPrimitiveTypeException {
+    // GeoJSON would allow for more than one interior polygon (hole).
+    // But there is no place in the data object to store this information so for now we throw an error.
+    // There could be a more strict verification that the lines describe boundaries and have the correct winding order.
+    if (node.isArray() && (node.size() == 1 || node.size() == 2)) {
+      return new Polygon(dimension, null,
+          node.size() > 1 ? readGeoPointValues(name, dimension, 4, true, node.get(1)) : null,
+          readGeoPointValues(name, dimension, 4, true, node.get(0)));
+    }
+    throw new DeserializerException("Invalid polygon values '" + node + "' in property: " + name,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY, name);
+  }
+
+  /**
    * Returns the primitive type's default class or the manually mapped class if present.
    * @param mapping
    * @param edmPrimitiveType
@@ -704,7 +859,8 @@ public class ODataJsonDeserializer implements ODataDeserializer {
       valid = matchTextualCase(jsonNode, primKind)
           || matchNumberCase(jsonNode, primKind)
           || matchBooleanCase(jsonNode, primKind)
-          || matchIEEENumberCase(jsonNode, primKind);
+          || matchIEEENumberCase(jsonNode, primKind)
+          || jsonNode.isObject() && name.startsWith("Geo");
     }
     if (!valid) {
       throw new DeserializerException(
@@ -837,17 +993,11 @@ public class ODataJsonDeserializer implements ODataDeserializer {
               DeserializerException.MessageKeys.UNKNOWN_CONTENT);
         }
 
-        EdmStructuredType currentEdmType = null;
-        if (edmType instanceof EdmEntityType) {
-          currentEdmType = serviceMetadata.getEdm()
-              .getEntityType(new FullQualifiedName(odataType));
-        } else {
-          currentEdmType = serviceMetadata.getEdm()
-              .getComplexType(new FullQualifiedName(odataType));
-        }
+        final EdmStructuredType currentEdmType = edmType.getKind() == EdmTypeKind.ENTITY ?
+            serviceMetadata.getEdm().getEntityType(new FullQualifiedName(odataType)) :
+            serviceMetadata.getEdm().getComplexType(new FullQualifiedName(odataType));
         if (!isAssignable(edmType, currentEdmType)) {
-          throw new DeserializerException(
-              "Odata type " + odataType + " not allowed here",
+          throw new DeserializerException("Odata type " + odataType + " not allowed here",
               DeserializerException.MessageKeys.UNKNOWN_CONTENT);
         }
 
@@ -859,14 +1009,8 @@ public class ODataJsonDeserializer implements ODataDeserializer {
 
   private boolean isAssignable(final EdmStructuredType edmStructuredType,
       final EdmStructuredType edmStructuredTypeToAssign) {
-    if (edmStructuredTypeToAssign == null) {
-      return false;
-    } else if (edmStructuredType.getFullQualifiedName()
-        .equals(edmStructuredTypeToAssign.getFullQualifiedName())) {
-      return true;
-    } else {
-      return isAssignable(edmStructuredType,
-          edmStructuredTypeToAssign.getBaseType());
-    }
+    return edmStructuredTypeToAssign != null
+        && (edmStructuredType.getFullQualifiedName().equals(edmStructuredTypeToAssign.getFullQualifiedName())
+            || isAssignable(edmStructuredType, edmStructuredTypeToAssign.getBaseType()));
   }
 }

http://git-wip-us.apache.org/repos/asf/olingo-odata4/blob/cc0a6a51/lib/server-core/src/main/java/org/apache/olingo/server/core/serializer/json/ODataJsonSerializer.java
----------------------------------------------------------------------
diff --git a/lib/server-core/src/main/java/org/apache/olingo/server/core/serializer/json/ODataJsonSerializer.java b/lib/server-core/src/main/java/org/apache/olingo/server/core/serializer/json/ODataJsonSerializer.java
index 57af7a9..a86bdf0 100644
--- a/lib/server-core/src/main/java/org/apache/olingo/server/core/serializer/json/ODataJsonSerializer.java
+++ b/lib/server-core/src/main/java/org/apache/olingo/server/core/serializer/json/ODataJsonSerializer.java
@@ -21,8 +21,10 @@ package org.apache.olingo.server.core.serializer.json;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Collections;
+import java.util.EnumMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.olingo.commons.api.Constants;
@@ -47,6 +49,15 @@ import org.apache.olingo.commons.api.edm.EdmStructuredType;
 import org.apache.olingo.commons.api.edm.EdmType;
 import org.apache.olingo.commons.api.edm.FullQualifiedName;
 import org.apache.olingo.commons.api.edm.constants.EdmTypeKind;
+import org.apache.olingo.commons.api.edm.geo.ComposedGeospatial;
+import org.apache.olingo.commons.api.edm.geo.Geospatial;
+import org.apache.olingo.commons.api.edm.geo.GeospatialCollection;
+import org.apache.olingo.commons.api.edm.geo.LineString;
+import org.apache.olingo.commons.api.edm.geo.MultiLineString;
+import org.apache.olingo.commons.api.edm.geo.MultiPoint;
+import org.apache.olingo.commons.api.edm.geo.MultiPolygon;
+import org.apache.olingo.commons.api.edm.geo.Point;
+import org.apache.olingo.commons.api.edm.geo.Polygon;
 import org.apache.olingo.commons.api.format.ContentType;
 import org.apache.olingo.commons.core.edm.primitivetype.EdmPrimitiveTypeFactory;
 import org.apache.olingo.server.api.ODataServerError;
@@ -81,6 +92,19 @@ import com.fasterxml.jackson.core.JsonGenerator;
 
 public class ODataJsonSerializer extends AbstractODataSerializer {
 
+  private static final Map<Geospatial.Type, String> geoValueTypeToJsonName;
+  static {
+    Map<Geospatial.Type, String> temp = new EnumMap<Geospatial.Type, String>(Geospatial.Type.class);
+    temp.put(Geospatial.Type.POINT, Constants.ELEM_POINT);
+    temp.put(Geospatial.Type.MULTIPOINT, Constants.ELEM_MULTIPOINT);
+    temp.put(Geospatial.Type.LINESTRING, Constants.ELEM_LINESTRING);
+    temp.put(Geospatial.Type.MULTILINESTRING, "MultiLineString");
+    temp.put(Geospatial.Type.POLYGON, Constants.ELEM_POLYGON);
+    temp.put(Geospatial.Type.MULTIPOLYGON, "MultiPolygon");
+    temp.put(Geospatial.Type.GEOSPATIALCOLLECTION, "GeometryCollection");
+    geoValueTypeToJsonName = Collections.unmodifiableMap(temp);
+  }
+
   private final boolean isIEEE754Compatible;
   private final boolean isODataMetadataNone;
   private final boolean isODataMetadataFull;
@@ -701,6 +725,7 @@ public class ODataJsonSerializer extends AbstractODataSerializer {
       switch (property.getValueType()) {
       case COLLECTION_PRIMITIVE:
       case COLLECTION_ENUM:
+      case COLLECTION_GEOSPATIAL:
         try {
           writePrimitiveValue(property.getName(), type, value, isNullable,
               maxLength, precision, scale, isUnicode, json);
@@ -710,9 +735,6 @@ public class ODataJsonSerializer extends AbstractODataSerializer {
               property.getName(), property.getValue().toString());
         }
         break;
-      case COLLECTION_GEOSPATIAL:
-        throw new SerializerException("Property type not yet supported!",
-            SerializerException.MessageKeys.UNSUPPORTED_PROPERTY_TYPE, property.getName());
       default:
         throw new SerializerException("Property type not yet supported!",
             SerializerException.MessageKeys.UNSUPPORTED_PROPERTY_TYPE, property.getName());
@@ -753,8 +775,7 @@ public class ODataJsonSerializer extends AbstractODataSerializer {
       writePrimitiveValue(property.getName(), type, property.asPrimitive(),
           isNullable, maxLength, precision, scale, isUnicode, json);
     } else if (property.isGeospatial()) {
-      throw new SerializerException("Property type not yet supported!",
-          SerializerException.MessageKeys.UNSUPPORTED_PROPERTY_TYPE, property.getName());
+      writeGeoValue(property.getName(), type, property.asGeospatial(), isNullable, json);
     } else if (property.isEnum()) {
       writePrimitiveValue(property.getName(), type, property.asEnum(),
           isNullable, maxLength, precision, scale, isUnicode, json);
@@ -808,12 +829,105 @@ public class ODataJsonSerializer extends AbstractODataSerializer {
     }
   }
 
+  /** Writes a geospatial value following the GeoJSON specification defined in RFC 7946. */
+  protected void writeGeoValue(final String name, final EdmPrimitiveType type, final Geospatial geoValue,
+      final Boolean isNullable, JsonGenerator json)
+      throws EdmPrimitiveTypeException, IOException, SerializerException {
+    if (geoValue == null) {
+      if (isNullable == null || isNullable) {
+        json.writeNull();
+      } else {
+        throw new EdmPrimitiveTypeException("The literal 'null' is not allowed.");
+      }
+    } else {
+      if (!type.getDefaultType().isAssignableFrom(geoValue.getClass())) {
+        throw new EdmPrimitiveTypeException("The value type " + geoValue.getClass() + " is not supported.");
+      }
+      if (geoValue.getSrid() != null && geoValue.getSrid().isNotDefault()) {
+        throw new SerializerException("Non-standard SRID not supported!",
+            SerializerException.MessageKeys.WRONG_PROPERTY_VALUE, name, geoValue.toString());
+      }
+      json.writeStartObject();
+      json.writeStringField(Constants.ATTR_TYPE, geoValueTypeToJsonName.get(geoValue.getGeoType()));
+      json.writeFieldName(geoValue.getGeoType() == Geospatial.Type.GEOSPATIALCOLLECTION ?
+          Constants.JSON_GEOMETRIES :
+          Constants.JSON_COORDINATES);
+      json.writeStartArray();
+      switch (geoValue.getGeoType()) {
+      case POINT:
+        writeGeoPoint(json, (Point) geoValue);
+        break;
+      case MULTIPOINT:
+        writeGeoPoints(json, (MultiPoint) geoValue);
+        break;
+      case LINESTRING:
+        writeGeoPoints(json, (LineString) geoValue);
+        break;
+      case MULTILINESTRING:
+        for (final LineString lineString : (MultiLineString) geoValue) {
+          json.writeStartArray();
+          writeGeoPoints(json, lineString);
+          json.writeEndArray();
+        }
+        break;
+      case POLYGON:
+        writeGeoPolygon(json, (Polygon) geoValue);
+        break;
+      case MULTIPOLYGON:
+        for (final Polygon polygon : (MultiPolygon) geoValue) {
+          json.writeStartArray();
+          writeGeoPolygon(json, polygon);
+          json.writeEndArray();
+        }
+        break;
+      case GEOSPATIALCOLLECTION:
+        for (final Geospatial element : (GeospatialCollection) geoValue) {
+          writeGeoValue(name, EdmPrimitiveTypeFactory.getInstance(element.getEdmPrimitiveTypeKind()),
+              element, isNullable, json);
+        }
+        break;
+      }
+      json.writeEndArray();
+      json.writeEndObject();
+    }
+  }
+
+  private void writeGeoPoint(JsonGenerator json, final Point point) throws IOException {
+    json.writeNumber(point.getX());
+    json.writeNumber(point.getY());
+    if (point.getZ() != 0) {
+      json.writeNumber(point.getZ());
+    }
+  }
+
+  private void writeGeoPoints(JsonGenerator json, final ComposedGeospatial<Point> points) throws IOException {
+    for (final Point point : points) {
+      json.writeStartArray();
+      writeGeoPoint(json, point);
+      json.writeEndArray();
+    }
+  }
+
+  // TODO: There could be a more strict verification that the lines describe boundaries
+  //       and have the correct winding order.
+  //       But arguably the better place for this is the constructor of the Polygon object.
+  private void writeGeoPolygon(JsonGenerator json, final Polygon polygon) throws IOException {
+    json.writeStartArray();
+    writeGeoPoints(json, polygon.getExterior());
+    json.writeEndArray();
+    if (!polygon.getInterior().isEmpty()) {
+      json.writeStartArray();
+      writeGeoPoints(json, polygon.getInterior());
+      json.writeEndArray();
+    }
+  }
+
   protected void writeComplexValue(final ServiceMetadata metadata,
       final EdmComplexType type, final List<Property> properties,
       final Set<List<String>> selectedPaths, final JsonGenerator json)
       throws IOException, SerializerException {
 
-      for (final String propertyName : type.getPropertyNames()) {
+    for (final String propertyName : type.getPropertyNames()) {
       final Property property = findProperty(propertyName, properties);
       if (selectedPaths == null || ExpandSelectHelper.isSelected(selectedPaths, propertyName)) {
         writeProperty(metadata, (EdmProperty) type.getProperty(propertyName), property,

http://git-wip-us.apache.org/repos/asf/olingo-odata4/blob/cc0a6a51/lib/server-test/src/test/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializerEntityTest.java
----------------------------------------------------------------------
diff --git a/lib/server-test/src/test/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializerEntityTest.java b/lib/server-test/src/test/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializerEntityTest.java
index d91f3ff..2f18594 100644
--- a/lib/server-test/src/test/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializerEntityTest.java
+++ b/lib/server-test/src/test/java/org/apache/olingo/server/core/deserializer/json/ODataJsonDeserializerEntityTest.java
@@ -31,7 +31,9 @@ import java.io.ByteArrayInputStream;
 import java.io.InputStream;
 import java.math.BigDecimal;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Date;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.olingo.commons.api.Constants;
@@ -44,6 +46,14 @@ import org.apache.olingo.commons.api.edm.EdmEntityType;
 import org.apache.olingo.commons.api.edm.EdmPrimitiveTypeKind;
 import org.apache.olingo.commons.api.edm.EdmProperty;
 import org.apache.olingo.commons.api.edm.FullQualifiedName;
+import org.apache.olingo.commons.api.edm.geo.Geospatial;
+import org.apache.olingo.commons.api.edm.geo.GeospatialCollection;
+import org.apache.olingo.commons.api.edm.geo.LineString;
+import org.apache.olingo.commons.api.edm.geo.MultiLineString;
+import org.apache.olingo.commons.api.edm.geo.MultiPoint;
+import org.apache.olingo.commons.api.edm.geo.MultiPolygon;
+import org.apache.olingo.commons.api.edm.geo.Point;
+import org.apache.olingo.commons.api.edm.geo.Polygon;
 import org.apache.olingo.commons.api.edm.provider.CsdlMapping;
 import org.apache.olingo.commons.api.format.ContentType;
 import org.apache.olingo.server.api.OData;
@@ -53,11 +63,13 @@ import org.apache.olingo.server.api.deserializer.ODataDeserializer;
 import org.apache.olingo.server.core.deserializer.AbstractODataDeserializerTest;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTest {
 
   private static final ContentType CONTENT_TYPE_JSON_IEEE754Compatible =
       ContentType.create(ContentType.JSON, ContentType.PARAMETER_IEEE754_COMPATIBLE, "true");
+  private static final OData odata = OData.newInstance();
 
   @Test
   public void emptyEntity() throws Exception {
@@ -658,6 +670,231 @@ public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTe
   }
 
   @Test
+  public void geoPoint() throws Exception {
+    final EdmEntityType entityType = mockEntityType(EdmPrimitiveTypeKind.GeometryPoint);
+    final String preamble = "{\"" + entityType.getPropertyNames().get(0) + "\":{";
+    final Entity entity = deserialize(preamble + "\"type\":\"Point\",\"coordinates\":[1.25,2.75]}}",
+        entityType);
+    assertEquals(1, entity.getProperties().size());
+    assertTrue(entity.getProperties().get(0).getValue() instanceof Point);
+    final Point point = (Point) entity.getProperties().get(0).getValue();
+    assertEquals(Geospatial.Dimension.GEOMETRY, point.getDimension());
+    assertEquals(1.25, point.getX(), 0);
+    assertEquals(2.75, point.getY(), 0);
+
+    expectException(preamble + "}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":1}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"point\",\"coordinates\":null}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"LineString\"}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\"}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coord\":[]}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.UNKNOWN_CONTENT);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":\"1 2\"}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":{\"x\":1,\"y\":2}}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[]}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[1]}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[\"1\",2]}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[1,\"2\"]}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[1,2,\"3\"]}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[1,2,3,4]}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[12345678901234567,2]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException(preamble + "\"type\":\"Point\",\"coordinates\":[1,2],\"extra\":\"extra\"}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.UNKNOWN_CONTENT);
+  }
+
+  @Test
+  public void geoMultiPoint() throws Exception {
+    final EdmEntityType entityType = mockEntityType(EdmPrimitiveTypeKind.GeographyMultiPoint);
+    final Entity entity = deserialize("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"MultiPoint\",\"coordinates\":[[2.5,3.125,99],[3.5,4.125],[4.5,5.125]]}}",
+        entityType);
+    assertTrue(entity.getProperties().get(0).getValue() instanceof MultiPoint);
+    final MultiPoint multiPoint = (MultiPoint) entity.getProperties().get(0).getValue();
+    assertEquals(Geospatial.Dimension.GEOGRAPHY, multiPoint.getDimension());
+    Iterator<Point> iterator = multiPoint.iterator();
+    final Point point1 = iterator.next();
+    assertEquals(Geospatial.Dimension.GEOGRAPHY, point1.getDimension());
+    assertEquals(2.5, point1.getX(), 0);
+    assertEquals(3.125, point1.getY(), 0);
+    assertEquals(99, point1.getZ(), 0);
+    final Point point2 = iterator.next();
+    assertEquals(Geospatial.Dimension.GEOGRAPHY, point2.getDimension());
+    assertEquals(3.5, point2.getX(), 0);
+    assertEquals(4.125, point2.getY(), 0);
+    final Point point3 = iterator.next();
+    assertEquals(Geospatial.Dimension.GEOGRAPHY, point3.getDimension());
+    assertEquals(4.5, point3.getX(), 0);
+    assertEquals(5.125, point3.getY(), 0);
+    assertFalse(iterator.hasNext());
+
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"MultiPoint\",\"coordinates\":[{\"x\":1,\"y\":2}]}}",
+        entityType, ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+  }
+
+  @Test
+  public void geoLineString() throws Exception {
+    final EdmEntityType entityType = mockEntityType(EdmPrimitiveTypeKind.GeometryLineString);
+    final Entity entity = deserialize("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"LineString\",\"coordinates\":[[1.0,1.0],[2.0,2.0]]}}",
+        entityType);
+    assertTrue(entity.getProperties().get(0).getValue() instanceof LineString);
+    final LineString lineString = (LineString) entity.getProperties().get(0).getValue();
+    assertEquals(Geospatial.Dimension.GEOMETRY, lineString.getDimension());
+    Iterator<Point> iterator = lineString.iterator();
+    final Point point1 = iterator.next();
+    assertEquals(Geospatial.Dimension.GEOMETRY, point1.getDimension());
+    assertEquals(1, point1.getX(), 0);
+    assertEquals(1, point1.getY(), 0);
+    final Point point2 = iterator.next();
+    assertEquals(Geospatial.Dimension.GEOMETRY, point2.getDimension());
+    assertEquals(2, point2.getX(), 0);
+    assertEquals(2, point2.getY(), 0);
+    assertFalse(iterator.hasNext());
+
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{\"type\":\"LineString\"}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    // A proper line string has at least two points but the OData specification has another opinion
+    // so the following negative test would fail.
+    //    expectException("{\"" + entityType.getPropertyNames().get(0)
+    //        + "\":{\"type\":\"LineString\",\"coordinates\":[[1,2]]}}", entityType, ContentType.JSON,
+    //        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException("{\"" + entityType.getPropertyNames().get(0)
+        + "\":{\"type\":\"LineString\",\"coordinates\":null}}", entityType, ContentType.JSON,
+        DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+  }
+
+  @Test
+  public void geoMultiLineString() throws Exception {
+    final EdmEntityType entityType = mockEntityType(EdmPrimitiveTypeKind.GeometryMultiLineString);
+    final Entity entity = deserialize("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"MultiLineString\",\"coordinates\":["
+        + "[[1.0,1.0],[2.0,2.0],[3.0,3.0],[4.0,4.0],[5.0,5.0]],"
+        + "[[99.5,101.5],[150.0,151.25]]]}}",
+        entityType);
+    assertTrue(entity.getProperties().get(0).getValue() instanceof MultiLineString);
+    final MultiLineString multiLineString = (MultiLineString) entity.getProperties().get(0).getValue();
+    assertEquals(Geospatial.Dimension.GEOMETRY, multiLineString.getDimension());
+    assertEquals(1, multiLineString.iterator().next().iterator().next().getY(), 0);
+
+    expectException("{\"" + entityType.getPropertyNames().get(0)
+        + "\":{\"type\":\"MultiLineString\",\"coordinates\":null}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException("{\"" + entityType.getPropertyNames().get(0)
+        + "\":{\"type\":\"MultiLineString\",\"coordinates\":\"1 2 3 4\"}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException("{\"" + entityType.getPropertyNames().get(0)
+        + "\":{\"type\":\"MultiLineString\",\"coordinates\":[{\"first\":[[1,2],[3,4]]}]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+  }
+
+  @Test
+  public void geoPolygon() throws Exception {
+    final EdmEntityType entityType = mockEntityType(EdmPrimitiveTypeKind.GeometryPolygon);
+    Entity entity = deserialize("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"Polygon\",\"coordinates\":[[[0.0,0.0],[3.0,0.0],[3.0,3.0],[0.0,3.0],[0.0,0.0]],"
+        + "[[1.0,1.0],[1.0,2.0],[2.0,2.0],[2.0,1.0],[1.0,1.0]]]}}",
+        entityType);
+    assertTrue(entity.getProperties().get(0).getValue() instanceof Polygon);
+    Polygon polygon = (Polygon) entity.getProperties().get(0).getValue();
+    assertEquals(Geospatial.Dimension.GEOMETRY, polygon.getDimension());
+    assertEquals(0, polygon.getExterior().iterator().next().getX(), 0);
+    assertEquals(1, polygon.getInterior().iterator().next().getY(), 0);
+
+    entity = deserialize("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"Polygon\",\"coordinates\":[[[0,0],[3,0],[3,3],[0,3],[0,0]]]}}",
+        entityType);
+    polygon = (Polygon) entity.getProperties().get(0).getValue();
+    assertTrue(polygon.getInterior().isEmpty());
+
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"Polygon\",\"coordinates\":{\"ext\":[[0,0],[3,0],[0,3],[0,0]]}}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"Polygon\",\"coordinates\":[[[0,0],[3,0],[3,3],[0,3]]]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"Polygon\",\"coordinates\":[[[0,0],[3,0],[3,3],[0,3],[42,87]]]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"Polygon\",\"coordinates\":[[[0,0],[3,0],[3,3],[0,3],[0,0]],"
+        + "[[1,1],[1,2],[2,2],[2,1],[1,1]],"
+        + "[[1,1],[1,2],[2,2],[2,1],[1,1]]]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+  }
+
+  @Test
+  public void geoMultiPolygon() throws Exception {
+    final EdmEntityType entityType = mockEntityType(EdmPrimitiveTypeKind.GeometryMultiPolygon);
+    final Entity entity = deserialize("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"MultiPolygon\",\"coordinates\":["
+        + "[[[0.0,0.0],[3.0,0.0],[3.0,3.0],[0.0,3.0],[0.0,0.0]],"
+        + "[[1.0,1.0],[1.0,2.0],[2.0,2.0],[2.0,1.0],[1.0,1.0]]],"
+        + "[[[0.0,0.0],[30.0,0.0],[0.0,30.0],[0.0,0.0]]]]}}",
+        entityType);
+    final MultiPolygon multiPolygon = (MultiPolygon) entity.getProperties().get(0).getValue();
+    assertEquals(1, multiPolygon.iterator().next().getInterior().iterator().next().getX(), 0);
+
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"MultiPolygon\",\"coordinates\":[{\"first\":[[[0,0],[3,0],[3,3],[0,3],[0,0]]]}]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+  }
+
+  @Test
+  public void geoCollection() throws Exception {
+    final EdmEntityType entityType = mockEntityType(EdmPrimitiveTypeKind.GeometryCollection);
+    final Entity entity = deserialize("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"GeometryCollection\",\"geometries\":["
+        + "{\"type\":\"Point\",\"coordinates\":[100.0,0.0]},"
+        + "{\"type\":\"LineString\",\"coordinates\":[[101.0,0.0],[102.0,1.0]]}]}}",
+        entityType);
+    assertTrue(entity.getProperties().get(0).getValue() instanceof GeospatialCollection);
+    GeospatialCollection collection = (GeospatialCollection) entity.getProperties().get(0).getValue();
+    assertEquals(Geospatial.Dimension.GEOMETRY, collection.getDimension());
+    Iterator<Geospatial> iterator = collection.iterator();
+    final Geospatial point = iterator.next();
+    assertEquals(Geospatial.Dimension.GEOMETRY, point.getDimension());
+    assertEquals(Geospatial.Type.POINT, point.getGeoType());
+    assertEquals(100, ((Point) point).getX(), 0);
+    final Geospatial line = iterator.next();
+    assertEquals(Geospatial.Dimension.GEOMETRY, line.getDimension());
+    assertEquals(Geospatial.Type.LINESTRING, line.getGeoType());
+    assertEquals(101, ((LineString) line).iterator().next().getX(), 0);
+
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"GeometryCollection\",\"coordinates\":[0,0]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.UNKNOWN_CONTENT);
+    expectException("{\"" + entityType.getPropertyNames().get(0) + "\":{"
+        + "\"type\":\"GeometryCollection\",\"geometries\":[[0,0]]}}", entityType,
+        ContentType.JSON, DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
+  }
+
+  private EdmEntityType mockEntityType(final EdmPrimitiveTypeKind typeKind) {
+    EdmProperty property = Mockito.mock(EdmProperty.class);
+    final String name = "Property" + typeKind.name();
+    Mockito.when(property.getType()).thenReturn(odata.createPrimitiveTypeInstance(typeKind));
+    EdmEntityType entityType = Mockito.mock(EdmEntityType.class);
+    Mockito.when(entityType.getFullQualifiedName()).thenReturn(new FullQualifiedName(NAMESPACE, "entityType"));
+    Mockito.when(entityType.getPropertyNames()).thenReturn(Arrays.asList(name));
+    Mockito.when(entityType.getProperty(name)).thenReturn(property);
+    return entityType;
+  }
+
+  @Test
   public void mappingTest() throws Exception {
     EdmEntityType entityType = mock(EdmEntityType.class);
     when(entityType.getFullQualifiedName()).thenReturn(new FullQualifiedName("namespace", "name"));
@@ -670,22 +907,21 @@ public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTe
     EdmProperty propertyDate = mock(EdmProperty.class);
     when(propertyDate.getName()).thenReturn("PropertyDate");
     when(propertyDate.getMapping()).thenReturn(mapping);
-    when(propertyDate.getType()).thenReturn(
-        OData.newInstance().createPrimitiveTypeInstance(EdmPrimitiveTypeKind.Date));
+    when(propertyDate.getType()).thenReturn(odata.createPrimitiveTypeInstance(EdmPrimitiveTypeKind.Date));
     when(entityType.getProperty("PropertyDate")).thenReturn(propertyDate);
 
     EdmProperty propertyDateTimeOffset = mock(EdmProperty.class);
     when(propertyDateTimeOffset.getName()).thenReturn("PropertyDateTimeOffset");
     when(propertyDateTimeOffset.getMapping()).thenReturn(mapping);
     when(propertyDateTimeOffset.getType()).thenReturn(
-        OData.newInstance().createPrimitiveTypeInstance(EdmPrimitiveTypeKind.DateTimeOffset));
+        odata.createPrimitiveTypeInstance(EdmPrimitiveTypeKind.DateTimeOffset));
     when(entityType.getProperty("PropertyDateTimeOffset")).thenReturn(propertyDateTimeOffset);
 
     String entityString =
         "{\"PropertyDate\":\"2012-12-03\","
             + "\"PropertyDateTimeOffset\":\"2012-12-03T07:16:23Z\"}";
     InputStream stream = new ByteArrayInputStream(entityString.getBytes());
-    ODataDeserializer deserializer = OData.newInstance().createDeserializer(ContentType.JSON, metadata);
+    ODataDeserializer deserializer = odata.createDeserializer(ContentType.JSON, metadata);
     Entity entity = deserializer.entity(stream, entityType).getEntity();
     assertNotNull(entity);
     List<Property> properties = entity.getProperties();
@@ -844,7 +1080,7 @@ public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTe
   }
 
   @Test
-  public void unkownContentInEntity() throws Exception {
+  public void unknownContentInEntity() throws Exception {
     final String entityString = "{"
         + "\"PropertyInt16\":32767,"
         + "\"unknown\": 12,"
@@ -860,7 +1096,7 @@ public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTe
   }
 
   @Test
-  public void unkownContentInComplexProperty() throws Exception {
+  public void unknownContentInComplexProperty() throws Exception {
     final String entityString = "{"
         + "\"PropertyInt16\":32767,"
         + "\"CollPropertyString\":"
@@ -875,7 +1111,7 @@ public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTe
   }
 
   @Test
-  public void unkownContentInComplexCollectionProperty() throws Exception {
+  public void unknownContentInComplexCollectionProperty() throws Exception {
     final String entityString = "{"
         + "\"PropertyInt16\":32767,"
         + "\"CollPropertyString\":"
@@ -1363,15 +1599,18 @@ public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTe
 
   protected static Entity deserialize(final InputStream stream, final String entityTypeName,
       final ContentType contentType) throws DeserializerException {
-    return OData.newInstance().createDeserializer(contentType, metadata)
-        .entity(stream, edm.getEntityType(new FullQualifiedName(NAMESPACE, entityTypeName)))
-        .getEntity();
+    return deserializeWithResult(stream, entityTypeName, contentType).getEntity();
   }
-  
+
   protected static DeserializerResult deserializeWithResult(final InputStream stream, final String entityTypeName,
       final ContentType contentType) throws DeserializerException {
-    return OData.newInstance().createDeserializer(contentType, metadata)
-        .entity(stream, edm.getEntityType(new FullQualifiedName(NAMESPACE, entityTypeName)));
+    final EdmEntityType entityType = edm.getEntityType(new FullQualifiedName(NAMESPACE, entityTypeName));
+    return deserializeWithResult(stream, entityType, contentType);
+  }
+
+  protected static DeserializerResult deserializeWithResult(final InputStream stream, final EdmEntityType entityType,
+      final ContentType contentType) throws DeserializerException {
+    return odata.createDeserializer(contentType, metadata).entity(stream, entityType);
   }
 
   private static Entity deserialize(final String entityString, final String entityTypeName,
@@ -1384,22 +1623,34 @@ public class ODataJsonDeserializerEntityTest extends AbstractODataDeserializerTe
     return deserialize(entityString, entityTypeName, ContentType.JSON);
   }
 
+  private Entity deserialize(final String entityString, final EdmEntityType entityType)
+      throws DeserializerException {
+    return deserializeWithResult(new ByteArrayInputStream(entityString.getBytes()), entityType, ContentType.JSON)
+        .getEntity();
+  }
+
   private static void checkPropertyJsonType(final String entityString) throws DeserializerException {
     expectException(entityString, "ETAllPrim", DeserializerException.MessageKeys.INVALID_VALUE_FOR_PROPERTY);
   }
 
   protected static void expectException(final String entityString, final String entityTypeName,
+      final ContentType contentType, final DeserializerException.MessageKeys messageKey) {
+    expectException(entityString,
+        edm.getEntityType(new FullQualifiedName(NAMESPACE, entityTypeName)), contentType, messageKey);
+  }
+
+  protected static void expectException(final String entityString, final String entityTypeName,
       final DeserializerException.MessageKeys messageKey) {
     expectException(entityString, entityTypeName, ContentType.JSON, messageKey);
   }
 
-  private static void expectException(final String entityString, final String entityTypeName,
+  private static void expectException(final String entityString, final EdmEntityType entityType,
       final ContentType contentType, final DeserializerException.MessageKeys messageKey) {
     try {
-      deserialize(entityString, entityTypeName, contentType);
+      deserializeWithResult(new ByteArrayInputStream(entityString.getBytes()), entityType, contentType);
       fail("Expected exception not thrown.");
     } catch (final DeserializerException e) {
       assertEquals(messageKey, e.getMessageKey());
     }
-  } 
+  }
 }