You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@olingo.apache.org by ko...@apache.org on 2013/12/13 10:04:26 UTC

[7/9] [OLINGO-63] Uri Parser: adoptions in lexer and parser, create UriParserImpl to convert the parsetree into internal structures, moved parser antlr part from .../olingo/producer/... to .../olingo/odata4/producer

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriParserImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriParserImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriParserImpl.java
index 737aa00..2ea0db9 100644
--- a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriParserImpl.java
+++ b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriParserImpl.java
@@ -16,203 +16,374 @@
  * specific language governing permissions and limitations
  * under the License.
  ******************************************************************************/
+
 package org.apache.olingo.odata4.producer.core.uri;
 
-import org.antlr.v4.runtime.ANTLRInputStream;
-import org.antlr.v4.runtime.CommonTokenStream;
-import org.antlr.v4.runtime.DefaultErrorStrategy;
-import org.antlr.v4.runtime.atn.PredictionMode;
-import org.antlr.v4.runtime.tree.ParseTree;
+import java.util.List;
+
 import org.apache.olingo.odata4.commons.api.edm.Edm;
+import org.apache.olingo.odata4.commons.api.edm.EdmAction;
 import org.apache.olingo.odata4.commons.api.edm.EdmActionImport;
+import org.apache.olingo.odata4.commons.api.edm.EdmComplexType;
+import org.apache.olingo.odata4.commons.api.edm.EdmElement;
 import org.apache.olingo.odata4.commons.api.edm.EdmEntityContainer;
 import org.apache.olingo.odata4.commons.api.edm.EdmEntitySet;
+import org.apache.olingo.odata4.commons.api.edm.EdmEntityType;
+import org.apache.olingo.odata4.commons.api.edm.EdmFunction;
 import org.apache.olingo.odata4.commons.api.edm.EdmFunctionImport;
-import org.apache.olingo.odata4.commons.api.edm.EdmNamed;
+import org.apache.olingo.odata4.commons.api.edm.EdmNavigationProperty;
+import org.apache.olingo.odata4.commons.api.edm.EdmProperty;
 import org.apache.olingo.odata4.commons.api.edm.EdmSingleton;
-import org.apache.olingo.odata4.producer.api.uri.UriInfoKind;
-import org.apache.olingo.odata4.producer.api.uri.UriPathInfoKind;
-import org.apache.olingo.producer.core.uri.antlr.UriLexer;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.AllAltContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.BatchAltContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.CrossjoinAltContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.EntityAltContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.MetadataAltContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.OdataRelativeUriContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.PathSegmentContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.PathSegmentsAltContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.PathSegmentsContext;
-import org.apache.olingo.producer.core.uri.antlr.UriParserParser.ResourcePathAltContext;
+import org.apache.olingo.odata4.commons.api.edm.EdmStructuralType;
+import org.apache.olingo.odata4.commons.api.edm.EdmType;
+import org.apache.olingo.odata4.commons.api.edm.helper.FullQualifiedName;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.AltBatchContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.AltEntityCastContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.AltEntityContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.AltMetadataContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.AltResourcePathContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.NameValueListContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.NameValueOptListContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.NameValuePairContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.OdataRelativeUriContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.OdataRelativeUriEOFContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.PathSegmentContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.PathSegmentsContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.QueryOptionsContext;
+import org.apache.olingo.odata4.producer.core.uri.antlr.UriParserParser.ResourcePathContext;
 
 public class UriParserImpl {
+  private Edm edm = null;
+  private EdmEntityContainer edmEntityContainer = null;
 
-  private EdmEntityContainer entityContainer = null;
+  public UriParserImpl(Edm edm) {
+    this.edm = edm;
+    this.edmEntityContainer = edm.getEntityContainer(null);
+  }
 
-  public UriInfoImpl readUri(final String uri, final Edm edm) {
-    entityContainer = edm.getEntityContainer(null);// "RefScenario","Container1"
+  public UriInfoImpl ParseUri(String uri) throws UriParserException {
+    OdataRelativeUriEOFContext root = ParserAdapter.parseInput(uri);
+    return readODataRelativeUriEOF(root);
+  }
 
-    UriInfoImpl ret = new UriInfoImpl();
-    OdataRelativeUriContext root = parseUri(uri);
+  private UriInfoImpl readODataRelativeUriEOF(OdataRelativeUriEOFContext node) {
+    OdataRelativeUriContext first = (OdataRelativeUriContext) node.getChild(0);
+    return readODataRelativeUri(first);
+  }
 
-    ret = readODataRelativeUri(root);
-    return ret;
+  private UriInfoImpl readODataRelativeUri(OdataRelativeUriContext node) {
+    if (node instanceof AltBatchContext) {
+      return new UriInfoImplBatch();
+    } else if (node instanceof AltEntityContext) {
+      // TODO read the entity options
+      return new UriInfoImplEntity();
+    } else if (node instanceof AltEntityCastContext) {
+      // TODO read the entity options and the cast ns.odi
+      return new UriInfoImplEntity();
+    } else if (node instanceof AltMetadataContext) {
+      // TODO read the metadata queryparameter and fragment
+      return new UriInfoImplMetadata();
+    } else if (node instanceof AltResourcePathContext) {
+      return readAltResourcePath((AltResourcePathContext) node);
+    }
+    return null;
   }
 
-  UriInfoImpl readODataRelativeUri(final OdataRelativeUriContext root) {
-    root.getChildCount();
+  private UriInfoImpl readAltResourcePath(AltResourcePathContext node) {
+    ResourcePathContext rpc = (ResourcePathContext) node.getChild(0);
+    QueryOptionsContext qoc = (QueryOptionsContext) node.getChild(2); // is null if there are no options
 
-    if (root instanceof BatchAltContext) {
-      return new UriInfoImpl().setKind(UriInfoKind.batch);
+    if (rpc.vPSs != null) {
+      return readPathSegments(rpc.vPSs);
+    } else if (rpc.vCJ != null) {
+      return new UriInfoImplCrossjoin();
+    } else if (rpc.vAll != null) {
+      return new UriInfoImplAll();
+    }
 
-    } else if (root instanceof EntityAltContext) {
-      // TODO implement
-    } else if (root instanceof MetadataAltContext) {
-      // TODO implement
-    } else if (root instanceof ResourcePathAltContext) {
+    return null;
+  }
 
-      return readResourcePath(root);
+  private UriInfoImpl readPathSegments(PathSegmentsContext pathSegments) {
+    int iSegment = 0;
+    UriInfoImplPath infoImpl = new UriInfoImplPath();
+    PathSegmentContext firstChild = (PathSegmentContext) pathSegments.vlPS.get(iSegment);
+    UriPathInfoImpl firstPathInfo = readFirstPathSegment(infoImpl, firstChild);
 
-    }
+    iSegment++;
 
-    return null;
+    UriPathInfoImpl prevPathInfo = firstPathInfo;
+
+    while (iSegment < pathSegments.vlPS.size()) {
+      PathSegmentContext nextChild = (PathSegmentContext) pathSegments.vlPS.get(iSegment);
+      prevPathInfo = readNextPathSegment(infoImpl, nextChild, prevPathInfo);
+      iSegment++;
+    }
+    return infoImpl;
   }
 
-  private UriInfoImpl readResourcePath(final OdataRelativeUriContext root) {
-    ParseTree firstChild = root.getChild(0);
+  private UriPathInfoImpl readNextPathSegment(UriInfoImplPath infoImpl, PathSegmentContext pathSegment,
+      UriPathInfoImpl prevPathInfo) {
+
+    UriPathInfoImpl pathInfo = null;
+
+    String odi = pathSegment.vODI.getText(); // not optional
+
+    // check for properties
+    if (pathSegment.vNS == null) {
+
+      EdmType prevType = prevPathInfo.getType();
+      if (prevType instanceof EdmStructuralType) {
+        EdmStructuralType prevStructType = (EdmStructuralType) prevType;
+
+        EdmElement element = prevStructType.getProperty(odi);
+        if (element == null) {
+          // TODO exception property not found
+        }
+
+        if (element instanceof EdmProperty) {
+          prevPathInfo.addProperty((EdmProperty) element);
+          return prevPathInfo;
+        } else if (element instanceof EdmNavigationProperty) {
+
+          prevPathInfo.addNavigationProperty((EdmNavigationProperty) element);
+
+          UriPathInfoNavEntitySet pathInfoNav = new UriPathInfoNavEntitySet();
+          pathInfoNav.addSourceNavigationProperty((EdmNavigationProperty) element);
+          infoImpl.addPathInfo(pathInfoNav);
+          return pathInfoNav;
+
+        } else {
+
+        }
+
+      }
+
+    } else {
+      // check for namespace
+      String namespace = pathSegment.vNS.getText();
+      namespace = namespace.substring(0, namespace.length() - 1);
+
+      FullQualifiedName fullName = new FullQualifiedName(namespace, odi);
+
+      // check for typecasts
+      if (prevPathInfo.getType() instanceof EdmEntityType) {
+        EdmEntityType et = edm.getEntityType(fullName);
+        if (et != null) {
+          prevPathInfo.addTypeFilter(et);
+
+          if (pathSegment.vlVPO.size() != 0) {
+            UriKeyPredicateList keyPred = readKeyPredicateList(
+                pathSegment.vlVPO.get(0), (EdmEntityType) prevPathInfo.getType());
+            prevPathInfo.setKeyPredicates(keyPred);
+          }
+
+          return prevPathInfo;
+        }
+      } else if (prevPathInfo.getType() instanceof EdmComplexType) {
+        EdmComplexType ct = edm.getComplexType(fullName);
+        if (ct != null) {
+          prevPathInfo.addTypeFilter(ct);
+          return prevPathInfo;
+        }
+      }
 
-    if (firstChild instanceof AllAltContext) {
-      return new UriInfoImpl().setKind(UriInfoKind.all);
-    } else if (firstChild instanceof CrossjoinAltContext) {
-      // TODO read ODIs behind crossjoin
-      return new UriInfoImpl().setKind(UriInfoKind.crossjoin);
-    } else if (firstChild instanceof PathSegmentsAltContext) {
-      return readPathSegments((PathSegmentsAltContext) firstChild);
+      // check for bound action
+      if (pathSegment.vlVPO == null) {
+        EdmAction action = edm.getAction(fullName, prevPathInfo.getFullType(), false);
+        pathInfo = new UriPathInfoActionImpl().setAction(action);
+        infoImpl.addPathInfo(pathInfo);
+        return pathInfo;
+      } else {
+        // check for bound functions
+        UriParameterlist parameterList = readParameterList(pathSegment.vlVPO.get(0));
+
+        EdmFunction function = edm.getFunction(fullName, prevPathInfo.getFullType(), false, parameterList.getNames());
+        if (function != null) {
+
+          UriPathInfoFunctionImpl pathInfoFunction = new UriPathInfoFunctionImpl();
+          pathInfoFunction.setFunction(function);
+          pathInfoFunction.setParameters(parameterList);
+
+          if (pathSegment.vlVPO.size() > 1) {
+            UriKeyPredicateList keyPred = readKeyPredicateList(
+                pathSegment.vlVPO.get(1), (EdmEntityType) prevPathInfo.getType());
+            pathInfoFunction.setKeyPredicates(keyPred);
+          }
+
+          infoImpl.addPathInfo(pathInfo);
+
+          return pathInfo;
+        }
+      }
+
+      // Exception unknown typeFilter/action or function
     }
+
     return null;
   }
 
-  private UriInfoImpl readPathSegments(final PathSegmentsAltContext pathSegmentsAlt) {
-    PathSegmentsContext firstChild = (PathSegmentsContext) pathSegmentsAlt.getChild(0);
+  private UriPathInfoImpl readFirstPathSegment(UriInfoImplPath infoImpl, PathSegmentContext pathSegment) {
+    UriPathInfoImpl pathInfo = null;
 
-    UriInfoImpl uriInfo = new UriInfoImpl();
+    // assert pathSegment.vNS = null;
+    String odi = pathSegment.vODI.getText(); // not optional
 
-    readFirstPathSegment(uriInfo, firstChild.ps.get(0));
+    // EntitySet
+    EdmEntitySet edmES = edmEntityContainer.getEntitySet(odi);
+    if (edmES != null) {
+      pathInfo = readEntitySet(pathSegment, edmES);
+    }
 
-    for (int i = 1; i < firstChild.ps.size(); i++) {
+    // Singleton
+    EdmSingleton edmSI = edmEntityContainer.getSingleton(odi);
+    if (edmSI != null) {
+      pathInfo = readSingleton(pathSegment, edmSI);
+    }
 
+    // FunctionImport
+    EdmFunctionImport edmFI = edmEntityContainer.getFunctionImport(odi);
+    if (edmFI != null) {
+      pathInfo = readFunctionImport(pathSegment, edmFI);
     }
 
-    return null;
+    // ActionImport
+    EdmActionImport edmAI = edmEntityContainer.getActionImport(odi);
+    if (edmAI != null) {
+      pathInfo = readActionImport(pathSegment, edmAI);
+    }
+
+    infoImpl.addPathInfo(pathInfo);
+
+    return pathInfo;
+  }
+
+  private UriPathInfoImpl readActionImport(PathSegmentContext pathSegment, EdmActionImport edmFI) {
+    UriPathInfoActionImpl uriPathInfo = new UriPathInfoActionImpl();
+
+    EdmAction action = edmFI.getAction();
+    uriPathInfo.setAction(action);
+
+    int num = pathSegment.vlVPO.size();
+    if (num == 2) {
+      // TODO exception action parameters not allowed
+    } else if (num == 1) {
+
+      if (uriPathInfo.isCollection() == true) {
+        if (uriPathInfo.getType() instanceof EdmEntityType) {
+          uriPathInfo.setKeyPredicates(
+              readKeyPredicateList(pathSegment.vlVPO.get(0), (EdmEntityType) uriPathInfo.getType()));
+        } else {
+          // TODO exception action keypreticates not allowed
+        }
+      } else {
+        // TODO exception action parameters not allowed
+      }
+    }
+
+    return uriPathInfo;
   }
 
-  private void readFirstPathSegment(final UriInfoImpl uriInfo, final PathSegmentContext ctx) {
-    /*
-     * if (ctx.ns != null) {//TODO implement
-     * // Error: First pathsegment can not be qualified. Allowed is entityset|function...
-     * }
-     */
+  private UriPathInfoImpl readFunctionImport(PathSegmentContext pathSegment, EdmFunctionImport edmFI) {
+    UriPathInfoFunctionImpl uriPathInfo = new UriPathInfoFunctionImpl();
 
-    /*
-     * if (ctx.odi == null) {//TODO implement
-     * // Error: First pathsegment must contain an odata identifier
-     * }
-     */
+    if (pathSegment.vlVPO == null) {
+      // TODO exception function parameters missing
+    }
 
-    // get element "odataIdentifier" from EDM
-    EdmNamed edmObject = null;// entityContainer.getElement(odataIdentifier);
+    UriParameterlist parameterList = readParameterList(pathSegment.vlVPO.get(0));
+    EdmFunction function = edmFI.getFunction(parameterList.getNames());
+    uriPathInfo.setFunction(function);
+    uriPathInfo.setParameters(parameterList);
 
-    if (edmObject instanceof EdmEntitySet) {
+    if (pathSegment.vlVPO.size() > 1) {
 
-      // is EdmEntitySet
+      if (!(uriPathInfo.getType() instanceof EdmEntityType)) {
+        // TODO exception illegally used keypredicates on function impored returning not an entityset
+      }
+      uriPathInfo.setKeyPredicates(
+          readKeyPredicateList(pathSegment.vlVPO.get(1), (EdmEntityType) uriPathInfo.getType()));
+    }
 
-      EdmEntitySet entityset = (EdmEntitySet) edmObject;
-      UriPathInfoEntitySetImpl pathInfo = new UriPathInfoEntitySetImpl();
-      pathInfo.setKind(UriPathInfoKind.entitySet);
-      pathInfo.setEntityContainer(entityContainer);
+    return null;
+  }
 
-      pathInfo.setTargetEntityset(entityset);
-      pathInfo.setTargetType(entityset.getEntityType());
-      pathInfo.setCollection(true);
+  private UriPathInfoImpl readSingleton(PathSegmentContext pathSegment, EdmSingleton edmSI) {
+    UriPathInfoSingletonImpl uriPathInfo = new UriPathInfoSingletonImpl();
 
-      // TODO check if kp may have been collected into fp
-      /*
-       * if (ctx.kp != null) {
-       * //pathInfo.setKeyPredicates(readkeypredicates(ctx.kp, entityset.getEntityType()));
-       * pathInfo.setCollection(false);
-       * }
-       */
-      uriInfo.addUriPathInfo(pathInfo);
-      return;
-    } else if (edmObject instanceof EdmSingleton) {
+    uriPathInfo.setSingleton(edmSI);
+    return uriPathInfo;
+  }
 
-      // is EdmSingleton
+  private UriPathInfoImpl readEntitySet(PathSegmentContext pathSegment, EdmEntitySet edmES) {
 
-      EdmSingleton singleton = (EdmSingleton) edmObject;
-      UriPathInfoSigletonImpl pathInfo = new UriPathInfoSigletonImpl(); // TODO change to UriPathInfoImplEntitySet
-      pathInfo.setKind(UriPathInfoKind.singleton);
-      pathInfo.setEntityContainer(entityContainer);
-      pathInfo.setTargetType(singleton.getEntityType());
-      // pathInfo.targetType = singleton.getEntityType();
-      pathInfo.setCollection(false);
+    UriPathInfoEntitySetImpl uriPathInfo = new UriPathInfoEntitySetImpl();
+    uriPathInfo.setEntitSet(edmES);
 
-      uriInfo.addUriPathInfo(pathInfo);
-      return;
-    } else if (edmObject instanceof EdmActionImport) {
+    // KeyPredicates
+    if (pathSegment.vlVPO != null) {
+      if (pathSegment.vlVPO.size() == 1) {
+        uriPathInfo.setKeyPredicates(readKeyPredicateList(pathSegment.vlVPO.get(0), edmES.getEntityType()));
 
-      // is EdmActionImport
+      } else if (pathSegment.vlVPO.size() > 1) {
+        // TODO exception ( to much key predicates)
+      }
 
-      UriPathInfoActionImportImpl pathInfo = new UriPathInfoActionImportImpl();
-      pathInfo.setKind(UriPathInfoKind.actionImport);
+    }
+    return uriPathInfo;
+  }
 
-      uriInfo.addUriPathInfo(pathInfo);
-      return;
+  private UriKeyPredicateList readKeyPredicateList(NameValueOptListContext parameterList, EdmEntityType entityType) {
+    if (parameterList.vVO != null) {
+      String value = parameterList.vVO.vV.getText();
+      List<String> kp = entityType.getKeyPredicateNames();
 
-    } else if (edmObject instanceof EdmFunctionImport) {
+      if (kp.size() != 1) {
+        // TODO exception "for using a value only keyPredicate there must be exact ONE defined keyProperty
+      }
 
-      // is EdmFunctionImport
+      String keyName = kp.get(0); // there yhoul
 
-      UriPathInfoImplFunctionImport pathInfo = new UriPathInfoImplFunctionImport();
-      pathInfo.setKind(UriPathInfoKind.functioncall);
+      return new UriKeyPredicateList().add(keyName, value);
+    }
 
-      /*
-       * if (ctx.fp != null) {
-       * pathInfo.setFunctionParameter(readFunctionParameters(uriInfo, ctx.fp));
-       * }
-       */
-      /*
-       * if (ctx.kp != null) {
-       * pathInfo.setKeyPredicates(readkeypredicates(ctx.kp, fi.getReturnedEntitySet().getEntityType()));
-       * }
-       */
+    NameValueListContext vNVL = parameterList.vNVL;
+    if (vNVL == null) {
+      // TODO throw exception empty keypredicates not allowed
+    }
+    UriKeyPredicateList uriPrameterList1 = new UriKeyPredicateList();
 
-      uriInfo.addUriPathInfo(pathInfo);
-      return;
+    for (NameValuePairContext nvl : vNVL.vNVP) {
+      String name = nvl.vODI.getText();
+      String value = nvl.vVAL.getText();
+      uriPrameterList1.add(name, value);
     }
+    return uriPrameterList1;
 
   }
 
-  private OdataRelativeUriContext parseUri(final String uri) {
-
-    ANTLRInputStream input = new ANTLRInputStream(uri);
+  private UriParameterlist readParameterList(NameValueOptListContext parameterList) {
+    if (parameterList.vVO != null) {
+      // TODO throw error "Value Only" not allowed for function/action parameters, only in keypredicates
+      return null;
+    }
 
-    // UriLexer lexer = new UriLexer(input);
-    UriLexer lexer = new UriLexer(input);
+    NameValueListContext vNVL = parameterList.vNVL;
+    UriParameterlist uriPrameterList1 = new UriParameterlist();
 
-    CommonTokenStream tokens = new CommonTokenStream(lexer);
-    UriParserParser parser = new UriParserParser(tokens);
+    for (NameValuePairContext nvl : vNVL.vNVP) {
+      String name = nvl.vODI.getText();
+      if (nvl.vVAL != null) {
+        String value = nvl.vVAL.getText();
+        uriPrameterList1.add(name, value, null);
+      } else {
+        String alias = nvl.vALI.getText();
+        uriPrameterList1.add(name, null, alias);
+      }
 
-    // parser.addErrorListener(new ErrorHandler());
-    // if (stage == 1) {
-    // //see https://github.com/antlr/antlr4/issues/192
-    // parser.setErrorHandler(new BailErrorStrategy());
-    // parser.getInterpreter().setPredictionMode(PredictionMode.LL);
-    // } else {
-    parser.setErrorHandler(new DefaultErrorStrategy());
-    parser.getInterpreter().setPredictionMode(PredictionMode.SLL);
-    // }
+    }
+    return uriPrameterList1;
 
-    // parser.d
-    return parser.odataRelativeUri();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImpl.java
new file mode 100644
index 0000000..50aa554
--- /dev/null
+++ b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImpl.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * 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.olingo.odata4.producer.core.uri;
+
+import org.apache.olingo.odata4.commons.api.edm.EdmAction;
+import org.apache.olingo.odata4.producer.api.uri.UriPathInfoKind;
+
+/**
+ * Covers Functionimports and BoundFunction in URI
+ */
+public class UriPathInfoActionImpl extends UriPathInfoImpl {
+
+  private EdmAction action;
+
+  public UriPathInfoActionImpl() {
+    this.setKind(UriPathInfoKind.action);
+  }
+
+  public UriPathInfoActionImpl setAction(EdmAction action) {
+
+    this.action = action;
+    this.setType(action.getReturnType().getType());
+    this.setCollection(action.getReturnType().isCollection());
+
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImportImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImportImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImportImpl.java
deleted file mode 100644
index 23dede3..0000000
--- a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoActionImportImpl.java
+++ /dev/null
@@ -1,24 +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.olingo.odata4.producer.core.uri;
-
-public class UriPathInfoActionImportImpl extends UriPathInfoImpl {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoEntitySetImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoEntitySetImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoEntitySetImpl.java
index 8d80e1e..f47b66f 100644
--- a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoEntitySetImpl.java
+++ b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoEntitySetImpl.java
@@ -19,32 +19,22 @@
 
 package org.apache.olingo.odata4.producer.core.uri;
 
-import java.util.List;
-
 import org.apache.olingo.odata4.commons.api.edm.EdmEntitySet;
-import org.apache.olingo.odata4.producer.api.uri.KeyPredicate;
+import org.apache.olingo.odata4.producer.api.uri.UriPathInfoKind;
 
 public class UriPathInfoEntitySetImpl extends UriPathInfoImpl {
+  EdmEntitySet edmEntitySet = null;
 
-  private EdmEntitySet targetEntityset;
-
-  private List<KeyPredicate> keyPredicates;
-
-  public EdmEntitySet getTargetEntityset() {
-    return targetEntityset;
+  public UriPathInfoEntitySetImpl() {
+    this.setKind(UriPathInfoKind.entitySet);
   }
 
-  // TODO add to Interface UriPathInfoEntitySet
-  public void setTargetEntityset(final EdmEntitySet targetEntityset) {
-    this.targetEntityset = targetEntityset;
-  }
+  public UriPathInfoEntitySetImpl setEntitSet(EdmEntitySet edmES) {
 
-  public List<KeyPredicate> getKeyPredicates() {
-    return keyPredicates;
-  }
+    this.edmEntitySet = edmES;
+    this.setType(edmES.getEntityType());
 
-  public void setKeyPredicates(final List<KeyPredicate> keyPredicates) {
-    this.keyPredicates = keyPredicates;
+    return this;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoFunctionImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoFunctionImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoFunctionImpl.java
new file mode 100644
index 0000000..3d607bd
--- /dev/null
+++ b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoFunctionImpl.java
@@ -0,0 +1,57 @@
+/*******************************************************************************
+ * 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.olingo.odata4.producer.core.uri;
+
+import org.apache.olingo.odata4.commons.api.edm.EdmFunction;
+import org.apache.olingo.odata4.producer.api.uri.UriPathInfoKind;
+
+/**
+ * Covers Functionimports and BoundFunction in URI
+ */
+public class UriPathInfoFunctionImpl extends UriPathInfoImpl {
+
+  private UriParameterlist parameters;
+  private EdmFunction function;
+  private UriKeyPredicateList keyPredicates;
+
+  public UriPathInfoFunctionImpl() {
+    this.setKind(UriPathInfoKind.function);
+  }
+
+  public UriPathInfoFunctionImpl setParameters(UriParameterlist parameters) {
+    this.parameters = parameters;
+    return this;
+  }
+
+  public UriParameterlist getParameters() {
+    return parameters;
+  }
+
+  public UriPathInfoFunctionImpl setFunction(EdmFunction function) {
+    this.function = function;
+    super.setType(function.getReturnType().getType());
+    super.setCollection(function.getReturnType().isCollection());
+    return this;
+  }
+
+  public UriPathInfoFunctionImpl setKeyPredicates(UriKeyPredicateList keyPredicates) {
+    this.keyPredicates = keyPredicates;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImpl.java
index 4a2636d..1a4f67c 100644
--- a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImpl.java
+++ b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImpl.java
@@ -18,58 +18,161 @@
  ******************************************************************************/
 package org.apache.olingo.odata4.producer.core.uri;
 
-import org.apache.olingo.odata4.commons.api.edm.EdmEntityContainer;
-import org.apache.olingo.odata4.commons.api.edm.EdmEntityType;
-import org.apache.olingo.odata4.producer.api.uri.UriPathInfo;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.olingo.odata4.commons.api.edm.EdmElement;
+import org.apache.olingo.odata4.commons.api.edm.EdmNavigationProperty;
+import org.apache.olingo.odata4.commons.api.edm.EdmProperty;
+import org.apache.olingo.odata4.commons.api.edm.EdmStructuralType;
+import org.apache.olingo.odata4.commons.api.edm.EdmType;
+import org.apache.olingo.odata4.commons.api.edm.helper.FullQualifiedName;
 import org.apache.olingo.odata4.producer.api.uri.UriPathInfoKind;
-//import org.apache.olingo.api.commons.InlineCount;
-//import org.apache.olingo.api.uri.NavigationPropertySegment;
-//import org.apache.olingo.api.uri.NavigationSegment;
-//import org.apache.olingo.api.uri.SelectItem;
-//import org.apache.olingo.api.uri.expression.FilterExpression;
-//import org.apache.olingo.api.uri.expression.OrderByExpression;
-
-/**
- *  
- */
-public class UriPathInfoImpl implements UriPathInfo {
+
+public abstract class UriPathInfoImpl {
+
+  private EdmType initialType = null;
+  private EdmType finalType = null;
+
   private UriPathInfoKind kind;
-  private EdmEntityContainer entityContainer;
+  private EdmType collectionTypeFilter = null;
+  private UriKeyPredicateList keyPredicates = null;
+  private EdmType singleTypeFilter = null;
+
+  private class PathListItem {
+    private EdmElement property; // ia EdmProperty or EdmNavigationProperty
+
+    private EdmType initialType;
+    private EdmType finalType;
+    private boolean isCollection;
+  }
+
+  private List<PathListItem> pathList = null;
   private boolean isCollection;
-  private EdmEntityType targetType;
 
-  @Override
-  public EdmEntityContainer getEntityContainer() {
-    return entityContainer;
+  public UriPathInfoImpl setType(EdmType edmType) {
+    this.initialType = edmType;
+    this.finalType = edmType;
+    return this;
+  }
+
+  public EdmType getType() {
+    return finalType;
+  }
+
+  public EdmType getInitialType() {
+    return initialType;
   }
 
-  public void setEntityContainer(final EdmEntityContainer entityContainer) {
-    this.entityContainer = entityContainer;
+  public FullQualifiedName getFullType() {
+    return new FullQualifiedName(finalType.getNamespace(), finalType.getName());
+  }
+
+  public UriPathInfoImpl setKind(UriPathInfoKind kind) {
+    this.kind = kind;
+    return this;
   }
 
-  @Override
   public UriPathInfoKind getKind() {
     return kind;
   }
 
-  public void setKind(final UriPathInfoKind kind) {
-    this.kind = kind;
+  public UriPathInfoImpl setKeyPredicates(UriKeyPredicateList keyPredicates) {
+    if ( this.isCollection()!= true) {
+      // throw exception
+    }
+    this.keyPredicates = keyPredicates;
+    this.setCollection(false);
+    return this;
   }
 
-  @Override
-  public boolean isCollection() {
-    return isCollection;
+  public UriKeyPredicateList getKeyPredicates() {
+    return this.keyPredicates;
   }
 
-  public void setCollection(final boolean isCollection) {
-    this.isCollection = isCollection;
+  public UriPathInfoImpl addTypeFilter(EdmStructuralType targetType) {
+    // TODO if there is a navigation path the type filter musst be applied to the last
+    if (pathList == null) {
+      if (keyPredicates == null) {
+        if (collectionTypeFilter != null) {
+          // TODO exception Type filters are not directy chainable
+        }
+        if (targetType.compatibleTo((EdmStructuralType) finalType)) {
+          collectionTypeFilter = targetType;
+          finalType = targetType;
+        } else {
+          // TODO throw exception
+        }
+      } else {
+        if (singleTypeFilter != null) {
+          // TODO exception Type filters are not directy chainable
+        }
+        if (targetType.compatibleTo((EdmStructuralType) finalType)) {
+          singleTypeFilter = targetType;
+          finalType = targetType;
+        } else {
+          // TODO throw exception
+        }
+      }
+    } else {
+      PathListItem last = pathList.get(pathList.size() - 1);
+
+      if (targetType.compatibleTo(last.finalType)) {
+        last.finalType = targetType;
+      }
+    }
+    return this;
+  }
+
+  public UriPathInfoImpl addProperty(EdmProperty property) {
+    if (pathList == null) {
+      pathList = new ArrayList<PathListItem>();
+    }
+
+    PathListItem newItem = new PathListItem();
+    newItem.property = property;
+    newItem.initialType = property.getType();
+    newItem.finalType = property.getType();
+    newItem.isCollection = property.isCollection();
+    pathList.add(newItem);
+
+    this.finalType = newItem.finalType;
+    this.isCollection = newItem.isCollection;
+    return this;
+  }
+
+  public UriPathInfoImpl addNavigationProperty(EdmNavigationProperty property) {
+    if (pathList == null) {
+      pathList = new ArrayList<PathListItem>();
+    }
+    PathListItem newItem = new PathListItem();
+    newItem.property = property;
+    newItem.initialType = property.getType();
+    newItem.finalType = property.getType();
+    newItem.isCollection = property.isCollection();
+    pathList.add(newItem);
+
+    this.finalType = newItem.finalType;
+    this.isCollection = newItem.isCollection;
+    return this;
   }
 
-  public EdmEntityType getTargetType() {
-    return targetType;
+  public int getPropertyCount() {
+    return pathList.size();
   }
 
-  public void setTargetType(final EdmEntityType targetType) {
-    this.targetType = targetType;
+  public EdmElement getProperty(int index) {
+    return pathList.get(index).property;
+
+  }
+  
+  public UriPathInfoImpl setCollection(boolean isCollection) {
+    this.isCollection = isCollection;
+    return this;
+  }
+
+  public boolean isCollection() {
+
+    return isCollection;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImplFunctionImport.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImplFunctionImport.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImplFunctionImport.java
deleted file mode 100644
index c25b1f6..0000000
--- a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoImplFunctionImport.java
+++ /dev/null
@@ -1,48 +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.olingo.odata4.producer.core.uri;
-
-import java.util.List;
-
-import org.apache.olingo.odata4.producer.api.uri.KeyPredicate;
-
-public class UriPathInfoImplFunctionImport extends UriPathInfoImpl {
-
-  // TODO change to proper Type
-  private Object functionParameter;
-  private List<KeyPredicate> keyPredicates;
-
-  public Object getFunctionParameter() {
-    return functionParameter;
-  }
-
-  public void setFunctionParameter(final Object functionParameter) {
-    this.functionParameter = functionParameter;
-  }
-
-  public List<KeyPredicate> getKeyPredicates() {
-    return keyPredicates;
-  }
-
-  public void setKeyPredicates(final List<KeyPredicate> keyPredicates) {
-    this.keyPredicates = keyPredicates;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoNavEntitySet.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoNavEntitySet.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoNavEntitySet.java
new file mode 100644
index 0000000..3383e69
--- /dev/null
+++ b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoNavEntitySet.java
@@ -0,0 +1,38 @@
+/*******************************************************************************
+ * 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.olingo.odata4.producer.core.uri;
+
+import org.apache.olingo.odata4.commons.api.edm.EdmNavigationProperty;
+import org.apache.olingo.odata4.producer.api.uri.UriPathInfoKind;
+
+public class UriPathInfoNavEntitySet extends UriPathInfoImpl {
+
+  private EdmNavigationProperty sourceNavigationProperty;
+
+  public UriPathInfoNavEntitySet() {
+    this.setKind(UriPathInfoKind.navEntitySet);
+  }
+
+  public UriPathInfoNavEntitySet addSourceNavigationProperty(EdmNavigationProperty sourceNavigationProperty) {
+    this.sourceNavigationProperty = sourceNavigationProperty;
+    this.setType(sourceNavigationProperty.getType());
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSigletonImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSigletonImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSigletonImpl.java
deleted file mode 100644
index 3311a47..0000000
--- a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSigletonImpl.java
+++ /dev/null
@@ -1,24 +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.olingo.odata4.producer.core.uri;
-
-public class UriPathInfoSigletonImpl extends UriPathInfoImpl {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-olingo-odata4/blob/10ac7ee3/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSingletonImpl.java
----------------------------------------------------------------------
diff --git a/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSingletonImpl.java b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSingletonImpl.java
new file mode 100644
index 0000000..c6aa2b0
--- /dev/null
+++ b/odata4-lib/odata4-producer-core/src/main/java/org/apache/olingo/odata4/producer/core/uri/UriPathInfoSingletonImpl.java
@@ -0,0 +1,39 @@
+/*******************************************************************************
+ * 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.olingo.odata4.producer.core.uri;
+
+import org.apache.olingo.odata4.commons.api.edm.EdmSingleton;
+import org.apache.olingo.odata4.producer.api.uri.UriPathInfoKind;
+
+public class UriPathInfoSingletonImpl extends UriPathInfoImpl {
+
+  private EdmSingleton singleton;
+  
+  public UriPathInfoSingletonImpl() {
+    this.setKind(UriPathInfoKind.singleton);
+  }
+
+  public UriPathInfoSingletonImpl setSingleton(EdmSingleton singleton) {
+        
+    this.singleton = singleton;
+    this.setType(singleton.getEntityType());
+    return this;
+  }
+
+}