You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ma...@apache.org on 2017/06/11 19:27:14 UTC

[1/2] incubator-atlas git commit: ATLAS-1852: create relationship-def

Repository: incubator-atlas
Updated Branches:
  refs/heads/master e0072e5ff -> 6b9399e00


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java
index 17b7e17..aefd168 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java
@@ -26,24 +26,13 @@ import org.apache.atlas.listener.ActiveStateChangeHandler;
 import org.apache.atlas.listener.ChangedTypeDefs;
 import org.apache.atlas.listener.TypeDefChangeListener;
 import org.apache.atlas.model.SearchFilter;
-import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
-import org.apache.atlas.model.typedef.AtlasClassificationDef;
-import org.apache.atlas.model.typedef.AtlasEntityDef;
-import org.apache.atlas.model.typedef.AtlasEnumDef;
-import org.apache.atlas.model.typedef.AtlasStructDef;
+import org.apache.atlas.model.typedef.*;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef;
-import org.apache.atlas.model.typedef.AtlasTypesDef;
 import org.apache.atlas.repository.util.FilterUtil;
 import org.apache.atlas.store.AtlasTypeDefStore;
-import org.apache.atlas.type.AtlasClassificationType;
-import org.apache.atlas.type.AtlasEntityType;
-import org.apache.atlas.type.AtlasEnumType;
-import org.apache.atlas.type.AtlasStructType;
-import org.apache.atlas.type.AtlasType;
-import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.type.*;
 import org.apache.atlas.type.AtlasTypeRegistry.AtlasTransientTypeRegistry;
-import org.apache.atlas.type.AtlasTypeUtil;
 import org.apache.atlas.util.AtlasRepositoryConfiguration;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.Predicate;
@@ -83,6 +72,8 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
 
     protected abstract AtlasEntityDefStore getEntityDefStore(AtlasTypeRegistry typeRegistry);
 
+    protected abstract AtlasRelationshipDefStore getRelationshipDefStore(AtlasTypeRegistry typeRegistry);
+
     @Override
     public void init() throws AtlasBaseException {
         AtlasTransientTypeRegistry ttr           = null;
@@ -96,7 +87,8 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
             AtlasTypesDef typesDef = new AtlasTypesDef(getEnumDefStore(ttr).getAll(),
                     getStructDefStore(ttr).getAll(),
                     getClassificationDefStore(ttr).getAll(),
-                    getEntityDefStore(ttr).getAll());
+                    getEntityDefStore(ttr).getAll(),
+                    getRelationshipDefStore(ttr).getAll());
 
             rectifyTypeErrorsIfAny(typesDef);
 
@@ -167,6 +159,27 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
 
         return ret;
     }
+    @Override
+    public AtlasRelationshipDef getRelationshipDefByName(String name) throws AtlasBaseException {
+        AtlasRelationshipDef ret = typeRegistry.getRelationshipDefByName(name);
+
+        if (ret == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, name);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasRelationshipDef getRelationshipDefByGuid(String guid) throws AtlasBaseException {
+        AtlasRelationshipDef ret = typeRegistry.getRelationshipDefByGuid(guid);
+
+        if (ret == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_GUID_NOT_FOUND, guid);
+        }
+
+        return ret;
+    }
 
     @Override
     @GraphTransaction
@@ -268,21 +281,36 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
     @GraphTransaction
     public AtlasEntityDef updateEntityDefByGuid(String guid, AtlasEntityDef entityDef) throws AtlasBaseException {
         AtlasTransientTypeRegistry ttr = lockTypeRegistryAndReleasePostCommit();
-
         tryUpdateByGUID(guid, entityDef, ttr);
-
         return getEntityDefStore(ttr).updateByGuid(guid, entityDef);
     }
 
     @Override
     @GraphTransaction
+    public AtlasRelationshipDef updateRelationshipDefByName(String name, AtlasRelationshipDef relationshipDef) throws AtlasBaseException {
+        AtlasTransientTypeRegistry ttr = lockTypeRegistryAndReleasePostCommit();
+        tryUpdateByName(name, relationshipDef, ttr);
+        return getRelationshipDefStore(ttr).updateByName(name, relationshipDef);
+    }
+
+    @Override
+    @GraphTransaction
+    public AtlasRelationshipDef updateRelationshipDefByGuid(String guid, AtlasRelationshipDef relationshipDef) throws AtlasBaseException {
+        AtlasTransientTypeRegistry ttr = lockTypeRegistryAndReleasePostCommit();
+        tryUpdateByGUID(guid, relationshipDef, ttr);
+        return getRelationshipDefStore(ttr).updateByGuid(guid, relationshipDef);
+    }
+
+    @Override
+    @GraphTransaction
     public AtlasTypesDef createTypesDef(AtlasTypesDef typesDef) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasTypeDefGraphStore.createTypesDef(enums={}, structs={}, classifications={}, entities={})",
+            LOG.debug("==> AtlasTypeDefGraphStore.createTypesDef(enums={}, structs={}, classifications={}, entities={}, relationships={})",
                     CollectionUtils.size(typesDef.getEnumDefs()),
                     CollectionUtils.size(typesDef.getStructDefs()),
                     CollectionUtils.size(typesDef.getClassificationDefs()),
-                    CollectionUtils.size(typesDef.getEntityDefs()));
+                    CollectionUtils.size(typesDef.getEntityDefs()),
+                    CollectionUtils.size(typesDef.getRelationshipDefs()));
         }
 
         AtlasTransientTypeRegistry ttr = lockTypeRegistryAndReleasePostCommit();
@@ -292,11 +320,12 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
         AtlasTypesDef ret = addToGraphStore(typesDef, ttr);
 
         if (LOG.isDebugEnabled()) {
-            LOG.debug("<== AtlasTypeDefGraphStore.createTypesDef(enums={}, structs={}, classfications={}, entities={})",
+            LOG.debug("<== AtlasTypeDefGraphStore.createTypesDef(enums={}, structs={}, classfications={}, entities={}, relationships={})",
                     CollectionUtils.size(typesDef.getEnumDefs()),
                     CollectionUtils.size(typesDef.getStructDefs()),
                     CollectionUtils.size(typesDef.getClassificationDefs()),
-                    CollectionUtils.size(typesDef.getEntityDefs()));
+                    CollectionUtils.size(typesDef.getEntityDefs()),
+                    CollectionUtils.size(typesDef.getRelationshipDefs()));
         }
 
         return ret;
@@ -359,11 +388,12 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
     @GraphTransaction
     public AtlasTypesDef updateTypesDef(AtlasTypesDef typesDef) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasTypeDefGraphStore.updateTypesDef(enums={}, structs={}, classfications={}, entities={})",
+            LOG.debug("==> AtlasTypeDefGraphStore.updateTypesDef(enums={}, structs={}, classfications={}, entities={}, relationships{})",
                     CollectionUtils.size(typesDef.getEnumDefs()),
                     CollectionUtils.size(typesDef.getStructDefs()),
                     CollectionUtils.size(typesDef.getClassificationDefs()),
-                    CollectionUtils.size(typesDef.getEntityDefs()));
+                    CollectionUtils.size(typesDef.getEntityDefs()),
+                    CollectionUtils.size(typesDef.getRelationshipDefs()));
         }
 
         AtlasTransientTypeRegistry ttr = lockTypeRegistryAndReleasePostCommit();
@@ -397,11 +427,12 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
     @GraphTransaction
     public void deleteTypesDef(AtlasTypesDef typesDef) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasTypeDefGraphStore.deleteTypesDef(enums={}, structs={}, classfications={}, entities={})",
+            LOG.debug("==> AtlasTypeDefGraphStore.deleteTypesDef(enums={}, structs={}, classfications={}, entities={}, relationships={})",
                     CollectionUtils.size(typesDef.getEnumDefs()),
                     CollectionUtils.size(typesDef.getStructDefs()),
                     CollectionUtils.size(typesDef.getClassificationDefs()),
-                    CollectionUtils.size(typesDef.getEntityDefs()));
+                    CollectionUtils.size(typesDef.getEntityDefs()),
+                    CollectionUtils.size(typesDef.getRelationshipDefs()));
         }
 
         AtlasTransientTypeRegistry ttr = lockTypeRegistryAndReleasePostCommit();
@@ -410,11 +441,25 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
         AtlasStructDefStore         structDefStore   = getStructDefStore(ttr);
         AtlasClassificationDefStore classifiDefStore = getClassificationDefStore(ttr);
         AtlasEntityDefStore         entityDefStore   = getEntityDefStore(ttr);
+        AtlasRelationshipDefStore   relationshipDefStore = getRelationshipDefStore(ttr);
 
         List<Object> preDeleteStructDefs   = new ArrayList<>();
         List<Object> preDeleteClassifiDefs = new ArrayList<>();
         List<Object> preDeleteEntityDefs   = new ArrayList<>();
+        List<Object> preDeleteRelationshipDefs = new ArrayList<>();
 
+        // pre deletes
+
+        // do the relationships first.
+        if (CollectionUtils.isNotEmpty(typesDef.getRelationshipDefs())) {
+            for (AtlasRelationshipDef relationshipDef : typesDef.getRelationshipDefs()) {
+                if (StringUtils.isNotBlank(relationshipDef.getGuid())) {
+                    preDeleteRelationshipDefs.add(relationshipDefStore.preDeleteByGuid(relationshipDef.getGuid()));
+                } else {
+                    preDeleteRelationshipDefs.add(relationshipDefStore.preDeleteByName(relationshipDef.getName()));
+                }
+            }
+        }
         if (CollectionUtils.isNotEmpty(typesDef.getStructDefs())) {
             for (AtlasStructDef structDef : typesDef.getStructDefs()) {
                 if (StringUtils.isNotBlank(structDef.getGuid())) {
@@ -445,6 +490,21 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
             }
         }
 
+        // run the actual deletes
+
+        // run the relationshipDef delete first - in case there is a enumDef or entityDef dependancy that is going to be deleted.
+        if (CollectionUtils.isNotEmpty(typesDef.getRelationshipDefs())) {
+            int i = 0;
+            for (AtlasRelationshipDef relationshipDef : typesDef.getRelationshipDefs()) {
+                if (StringUtils.isNotBlank(relationshipDef.getGuid())) {
+                    relationshipDefStore.deleteByGuid(relationshipDef.getGuid(), preDeleteRelationshipDefs.get(i));
+                } else {
+                    relationshipDefStore.deleteByName(relationshipDef.getName(), preDeleteRelationshipDefs.get(i));
+                }
+                i++;
+            }
+        }
+
         if (CollectionUtils.isNotEmpty(typesDef.getStructDefs())) {
             int i = 0;
             for (AtlasStructDef structDef : typesDef.getStructDefs()) {
@@ -532,6 +592,12 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
             }
         }
 
+        for(AtlasRelationshipType relationshipType : typeRegistry.getAllRelationshipTypes()) {
+            if (searchPredicates.evaluate(relationshipType)) {
+                typesDef.getRelationshipDefs().add(relationshipType.getRelationshipDef());
+            }
+        }
+
         return typesDef;
     }
 
@@ -691,11 +757,14 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
         AtlasStructDefStore         structDefStore   = getStructDefStore(ttr);
         AtlasClassificationDefStore classifiDefStore = getClassificationDefStore(ttr);
         AtlasEntityDefStore         entityDefStore   = getEntityDefStore(ttr);
+        AtlasRelationshipDefStore   relationshipDefStore   = getRelationshipDefStore(ttr);
 
         List<Object> preCreateStructDefs   = new ArrayList<>();
         List<Object> preCreateClassifiDefs = new ArrayList<>();
         List<Object> preCreateEntityDefs   = new ArrayList<>();
+        List<Object> preCreateRelationshipDefs   = new ArrayList<>();
 
+        // for enumerations run the create
         if (CollectionUtils.isNotEmpty(typesDef.getEnumDefs())) {
             for (AtlasEnumDef enumDef : typesDef.getEnumDefs()) {
                 AtlasEnumDef createdDef = enumDefStore.create(enumDef);
@@ -705,6 +774,7 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
                 ret.getEnumDefs().add(createdDef);
             }
         }
+        // run the preCreates
 
         if (CollectionUtils.isNotEmpty(typesDef.getStructDefs())) {
             for (AtlasStructDef structDef : typesDef.getStructDefs()) {
@@ -724,6 +794,12 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
             }
         }
 
+        if (CollectionUtils.isNotEmpty(typesDef.getRelationshipDefs())) {
+            for (AtlasRelationshipDef relationshipDef : typesDef.getRelationshipDefs()) {
+                preCreateRelationshipDefs.add(relationshipDefStore.preCreate(relationshipDef));
+            }
+        }
+
         if (CollectionUtils.isNotEmpty(typesDef.getStructDefs())) {
             int i = 0;
             for (AtlasStructDef structDef : typesDef.getStructDefs()) {
@@ -759,6 +835,17 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore, Activ
                 i++;
             }
         }
+        if (CollectionUtils.isNotEmpty(typesDef.getRelationshipDefs())) {
+            int i = 0;
+            for (AtlasRelationshipDef relationshipDef : typesDef.getRelationshipDefs()) {
+                AtlasRelationshipDef createdDef = relationshipDefStore.create(relationshipDef, preCreateRelationshipDefs.get(i));
+
+                ttr.updateGuid(createdDef.getName(), createdDef.getGuid());
+
+                ret.getRelationshipDefs().add(createdDef);
+                i++;
+            }
+        }
 
         return ret;
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipDefStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipDefStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipDefStoreV1.java
new file mode 100644
index 0000000..96cd8d1
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipDefStoreV1.java
@@ -0,0 +1,409 @@
+/**
+ * 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.atlas.repository.store.graph.v1;
+
+import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef.RelationshipCategory;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef.PropagateTags;
+import org.apache.atlas.model.typedef.AtlasRelationshipEndPointDef;
+import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.atlas.repository.store.graph.AtlasRelationshipDefStore;
+import org.apache.atlas.type.AtlasRelationshipType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.types.DataTypes.TypeCategory;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * RelationshipDef store in v1 format.
+ */
+public class AtlasRelationshipDefStoreV1 extends AtlasAbstractDefStoreV1 implements AtlasRelationshipDefStore {
+    private static final Logger LOG = LoggerFactory.getLogger(AtlasRelationshipDefStoreV1.class);
+
+    public AtlasRelationshipDefStoreV1(AtlasTypeDefGraphStoreV1 typeDefStore, AtlasTypeRegistry typeRegistry) {
+        super(typeDefStore, typeRegistry);
+    }
+
+    @Override
+    public AtlasVertex preCreate(AtlasRelationshipDef relationshipDef) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.preCreate({})", relationshipDef);
+        }
+
+        validateType(relationshipDef);
+
+        AtlasType type = typeRegistry.getType(relationshipDef.getName());
+
+        if (type.getTypeCategory() != org.apache.atlas.model.TypeCategory.RELATIONSHIP) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_MATCH_FAILED, relationshipDef.getName(), TypeCategory.RELATIONSHIP.name());
+        }
+
+        AtlasVertex ret = typeDefStore.findTypeVertexByName(relationshipDef.getName());
+
+        if (ret != null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_ALREADY_EXISTS, relationshipDef.getName());
+        }
+
+        ret = typeDefStore.createTypeVertex(relationshipDef);
+
+        updateVertexPreCreate(relationshipDef, (AtlasRelationshipType) type, ret);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.preCreate({}): {}", relationshipDef, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasRelationshipDef create(AtlasRelationshipDef relationshipDef, Object preCreateResult)
+            throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.create({}, {})", relationshipDef, preCreateResult);
+        }
+
+        AtlasVertex vertex;
+
+        if (preCreateResult == null || !(preCreateResult instanceof AtlasVertex)) {
+            vertex = preCreate(relationshipDef);
+        } else {
+            vertex = (AtlasVertex) preCreateResult;
+        }
+
+        AtlasRelationshipDef ret = toRelationshipDef(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.create({}, {}): {}", relationshipDef, preCreateResult, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public List<AtlasRelationshipDef> getAll() throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.getAll()");
+        }
+
+        List<AtlasRelationshipDef> ret = new ArrayList<>();
+        Iterator<AtlasVertex> vertices = typeDefStore.findTypeVerticesByCategory(TypeCategory.RELATIONSHIP);
+
+        while (vertices.hasNext()) {
+            ret.add(toRelationshipDef(vertices.next()));
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.getAll(): count={}", ret.size());
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasRelationshipDef getByName(String name) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.getByName({})", name);
+        }
+
+        AtlasVertex vertex = typeDefStore.findTypeVertexByNameAndCategory(name, TypeCategory.RELATIONSHIP);
+
+        if (vertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, name);
+        }
+
+        vertex.getProperty(Constants.TYPE_CATEGORY_PROPERTY_KEY, TypeCategory.class);
+
+        AtlasRelationshipDef ret = toRelationshipDef(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.getByName({}): {}", name, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasRelationshipDef getByGuid(String guid) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.getByGuid({})", guid);
+        }
+
+        AtlasVertex vertex = typeDefStore.findTypeVertexByGuidAndCategory(guid, TypeCategory.RELATIONSHIP);
+
+        if (vertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_GUID_NOT_FOUND, guid);
+        }
+
+        AtlasRelationshipDef ret = toRelationshipDef(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.getByGuid({}): {}", guid, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasRelationshipDef update(AtlasRelationshipDef relationshipDef) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.update({})", relationshipDef);
+        }
+
+        validateType(relationshipDef);
+
+        AtlasRelationshipDef ret = StringUtils.isNotBlank(relationshipDef.getGuid())
+                ? updateByGuid(relationshipDef.getGuid(), relationshipDef)
+                : updateByName(relationshipDef.getName(), relationshipDef);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.update({}): {}", relationshipDef, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasRelationshipDef updateByName(String name, AtlasRelationshipDef relationshipDef)
+            throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.updateByName({}, {})", name, relationshipDef);
+        }
+
+        validateType(relationshipDef);
+
+        AtlasType type = typeRegistry.getType(relationshipDef.getName());
+
+        if (type.getTypeCategory() != org.apache.atlas.model.TypeCategory.RELATIONSHIP) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_MATCH_FAILED, relationshipDef.getName(), TypeCategory.RELATIONSHIP.name());
+        }
+
+        AtlasVertex vertex = typeDefStore.findTypeVertexByNameAndCategory(name, TypeCategory.RELATIONSHIP);
+
+        if (vertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, name);
+        }
+
+        updateVertexPreUpdate(relationshipDef, (AtlasRelationshipType) type, vertex);
+
+        AtlasRelationshipDef ret = toRelationshipDef(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.updateByName({}, {}): {}", name, relationshipDef, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasRelationshipDef updateByGuid(String guid, AtlasRelationshipDef relationshipDef)
+            throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.updateByGuid({})", guid);
+        }
+
+        validateType(relationshipDef);
+
+        AtlasType type = typeRegistry.getTypeByGuid(guid);
+
+        if (type.getTypeCategory() != org.apache.atlas.model.TypeCategory.RELATIONSHIP) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_MATCH_FAILED, relationshipDef.getName(), TypeCategory.RELATIONSHIP.name());
+        }
+
+        AtlasVertex vertex = typeDefStore.findTypeVertexByGuidAndCategory(guid, TypeCategory.RELATIONSHIP);
+
+        if (vertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_GUID_NOT_FOUND, guid);
+        }
+
+        updateVertexPreUpdate(relationshipDef, (AtlasRelationshipType) type, vertex);
+        // TODO delete / create edges to entitytypes
+        AtlasRelationshipDef ret = toRelationshipDef(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.updateByGuid({}): {}", guid, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasVertex preDeleteByName(String name) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.preDeleteByName({})", name);
+        }
+
+        AtlasVertex ret = typeDefStore.findTypeVertexByNameAndCategory(name, TypeCategory.RELATIONSHIP);
+
+        if (ret == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, name);
+        }
+
+        if (AtlasGraphUtilsV1.typeHasInstanceVertex(name)) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_HAS_REFERENCES, name);
+        }
+
+        // TODO delete the edges to the other types
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.preDeleteByName({}): {}", name, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public void deleteByName(String name, Object preDeleteResult) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.deleteByName({}, {})", name, preDeleteResult);
+        }
+
+        AtlasVertex vertex;
+
+        if (preDeleteResult == null || !(preDeleteResult instanceof AtlasVertex)) {
+            vertex = preDeleteByName(name);
+        } else {
+            vertex = (AtlasVertex) preDeleteResult;
+        }
+
+        typeDefStore.deleteTypeVertex(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.deleteByName({}, {})", name, preDeleteResult);
+        }
+    }
+
+    @Override
+    public AtlasVertex preDeleteByGuid(String guid) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.preDeleteByGuid({})", guid);
+        }
+
+        AtlasVertex ret = typeDefStore.findTypeVertexByGuidAndCategory(guid, TypeCategory.RELATIONSHIP);
+
+        if (ret == null) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_GUID_NOT_FOUND, guid);
+        }
+
+        String typeName = AtlasGraphUtilsV1.getProperty(ret, Constants.TYPENAME_PROPERTY_KEY, String.class);
+
+        if (AtlasGraphUtilsV1.typeHasInstanceVertex(typeName)) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_HAS_REFERENCES, typeName);
+        }
+
+        // TODO delete the edges to the other types
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.preDeleteByGuid({}): {}", guid, ret);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public void deleteByGuid(String guid, Object preDeleteResult) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasRelationshipDefStoreV1.deleteByGuid({}, {})", guid, preDeleteResult);
+        }
+
+        AtlasVertex vertex;
+
+        if (preDeleteResult == null || !(preDeleteResult instanceof AtlasVertex)) {
+            vertex = preDeleteByGuid(guid);
+        } else {
+            vertex = (AtlasVertex) preDeleteResult;
+        }
+
+        typeDefStore.deleteTypeVertex(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasRelationshipDefStoreV1.deleteByGuid({}, {})", guid, preDeleteResult);
+        }
+    }
+
+    private void updateVertexPreCreate(AtlasRelationshipDef relationshipDef, AtlasRelationshipType relationshipType,
+                                       AtlasVertex vertex) throws AtlasBaseException {
+        AtlasStructDefStoreV1.updateVertexPreCreate(relationshipDef, relationshipType, vertex, typeDefStore);
+        // Update endpoints
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_ENDPOINT1_KEY, AtlasType.toJson(relationshipDef.getEndPointDef1()));
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_ENDPOINT2_KEY, AtlasType.toJson(relationshipDef.getEndPointDef2()));
+        // Update RelationshipCategory
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_CATEGORY_KEY, relationshipDef.getRelationshipCategory().name());
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_TAG_PROPAGATION_KEY, relationshipDef.getPropagateTags().name());
+    }
+
+    private void updateVertexPreUpdate(AtlasRelationshipDef relationshipDef, AtlasRelationshipType relationshipType,
+                                       AtlasVertex vertex) throws AtlasBaseException {
+        AtlasStructDefStoreV1.updateVertexPreUpdate(relationshipDef, relationshipType, vertex, typeDefStore);
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_ENDPOINT1_KEY, AtlasType.toJson(relationshipDef.getEndPointDef1()));
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_ENDPOINT2_KEY, AtlasType.toJson(relationshipDef.getEndPointDef2()));
+        // Update RelationshipCategory
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_CATEGORY_KEY, relationshipDef.getRelationshipCategory().name());
+        vertex.setProperty(Constants.RELATIONSHIPTYPE_TAG_PROPAGATION_KEY, relationshipDef.getPropagateTags().name());
+    }
+
+    private AtlasRelationshipDef toRelationshipDef(AtlasVertex vertex) throws AtlasBaseException {
+        AtlasRelationshipDef ret = null;
+
+        if (vertex != null && typeDefStore.isTypeVertex(vertex, TypeCategory.RELATIONSHIP)) {
+            String name         = vertex.getProperty(Constants.TYPENAME_PROPERTY_KEY, String.class);
+            String description  = vertex.getProperty(Constants.TYPEDESCRIPTION_PROPERTY_KEY, String.class);
+            String version      = vertex.getProperty(Constants.TYPEVERSION_PROPERTY_KEY, String.class);
+            String endPoint1Str = vertex.getProperty(Constants.RELATIONSHIPTYPE_ENDPOINT1_KEY, String.class);
+            String endPoint2Str = vertex.getProperty(Constants.RELATIONSHIPTYPE_ENDPOINT2_KEY, String.class);
+            String relationStr  = vertex.getProperty(Constants.RELATIONSHIPTYPE_CATEGORY_KEY, String.class);
+            String propagateStr = vertex.getProperty(Constants.RELATIONSHIPTYPE_TAG_PROPAGATION_KEY, String.class);
+
+            // set the endpoints
+            AtlasRelationshipEndPointDef endPointDef1 = AtlasType.fromJson(endPoint1Str, AtlasRelationshipEndPointDef.class);
+            AtlasRelationshipEndPointDef endPointDef2 = AtlasType.fromJson(endPoint2Str, AtlasRelationshipEndPointDef.class);
+
+            // set the relationship Category
+            RelationshipCategory relationshipCategory = null;
+            for (RelationshipCategory value : RelationshipCategory.values()) {
+                if (value.name().equals(relationStr)) {
+                    relationshipCategory = value;
+                }
+            }
+
+            // set the propagateTags
+            PropagateTags propagateTags = null;
+            for (PropagateTags value : PropagateTags.values()) {
+                if (value.name().equals(propagateStr)) {
+                    propagateTags = value;
+                }
+            }
+
+            ret = new AtlasRelationshipDef(name, description, version, relationshipCategory,  propagateTags, endPointDef1, endPointDef2);
+
+            // add in the attributes
+            AtlasStructDefStoreV1.toStructDef(vertex, ret, typeDefStore);
+        }
+
+        return ret;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java
index f0c8380..2ba4144 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java
@@ -19,6 +19,17 @@ package org.apache.atlas.repository.store.graph.v1;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+
+import static org.apache.atlas.repository.Constants.TYPE_CATEGORY_PROPERTY_KEY;
+import static org.apache.atlas.repository.Constants.VERTEX_TYPE_PROPERTY_KEY;
+import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.VERTEX_TYPE;
+
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
 import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.listener.TypeDefChangeListener;
@@ -31,6 +42,7 @@ import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.repository.store.graph.AtlasClassificationDefStore;
 import org.apache.atlas.repository.store.graph.AtlasEntityDefStore;
 import org.apache.atlas.repository.store.graph.AtlasEnumDefStore;
+import org.apache.atlas.repository.store.graph.AtlasRelationshipDefStore;
 import org.apache.atlas.repository.store.graph.AtlasStructDefStore;
 import org.apache.atlas.repository.store.graph.AtlasTypeDefGraphStore;
 import org.apache.atlas.type.AtlasType;
@@ -45,16 +57,6 @@ import org.springframework.stereotype.Component;
 import javax.annotation.PostConstruct;
 import javax.inject.Inject;
 import javax.inject.Singleton;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import static org.apache.atlas.repository.Constants.TYPE_CATEGORY_PROPERTY_KEY;
-import static org.apache.atlas.repository.Constants.VERTEX_TYPE_PROPERTY_KEY;
-import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.VERTEX_TYPE;
 
 
 /**
@@ -110,6 +112,12 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
     }
 
     @Override
+    protected AtlasRelationshipDefStore getRelationshipDefStore(AtlasTypeRegistry typeRegistry) {
+        return new AtlasRelationshipDefStoreV1(this, typeRegistry);
+    }
+
+
+    @Override
     @PostConstruct
     public void init() throws AtlasBaseException {
         LOG.debug("==> AtlasTypeDefGraphStoreV1.init()");
@@ -124,34 +132,34 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
     @VisibleForTesting
     public AtlasVertex findTypeVertexByName(String typeName) {
         Iterator results = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE)
-                                             .has(Constants.TYPENAME_PROPERTY_KEY, typeName)
-                                             .vertices().iterator();
+                .has(Constants.TYPENAME_PROPERTY_KEY, typeName)
+                .vertices().iterator();
 
         return (results != null && results.hasNext()) ? (AtlasVertex) results.next() : null;
     }
 
     AtlasVertex findTypeVertexByNameAndCategory(String typeName, TypeCategory category) {
         Iterator results = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE)
-                                             .has(Constants.TYPENAME_PROPERTY_KEY, typeName)
-                                             .has(TYPE_CATEGORY_PROPERTY_KEY, category)
-                                             .vertices().iterator();
+                .has(Constants.TYPENAME_PROPERTY_KEY, typeName)
+                .has(TYPE_CATEGORY_PROPERTY_KEY, category)
+                .vertices().iterator();
 
         return (results != null && results.hasNext()) ? (AtlasVertex) results.next() : null;
     }
 
     AtlasVertex findTypeVertexByGuid(String typeGuid) {
         Iterator<AtlasVertex> vertices = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE)
-                                                      .has(Constants.GUID_PROPERTY_KEY, typeGuid)
-                                                      .vertices().iterator();
+                .has(Constants.GUID_PROPERTY_KEY, typeGuid)
+                .vertices().iterator();
 
         return (vertices != null && vertices.hasNext()) ? vertices.next() : null;
     }
 
     AtlasVertex findTypeVertexByGuidAndCategory(String typeGuid, TypeCategory category) {
         Iterator<AtlasVertex> vertices = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE)
-                                                      .has(Constants.GUID_PROPERTY_KEY, typeGuid)
-                                                      .has(TYPE_CATEGORY_PROPERTY_KEY, category)
-                                                      .vertices().iterator();
+                .has(Constants.GUID_PROPERTY_KEY, typeGuid)
+                .has(TYPE_CATEGORY_PROPERTY_KEY, category)
+                .vertices().iterator();
 
         return (vertices != null && vertices.hasNext()) ? vertices.next() : null;
     }
@@ -159,8 +167,8 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
     Iterator<AtlasVertex> findTypeVerticesByCategory(TypeCategory category) {
 
         return (Iterator<AtlasVertex>) atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE)
-                                                 .has(TYPE_CATEGORY_PROPERTY_KEY, category)
-                                                 .vertices().iterator();
+                .has(TYPE_CATEGORY_PROPERTY_KEY, category)
+                .vertices().iterator();
     }
 
     AtlasVertex createTypeVertex(AtlasBaseTypeDef typeDef) {
@@ -246,7 +254,6 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
         for (AtlasEdge edge : edges) {
             atlasGraph.removeEdge(edge);
         }
-
         atlasGraph.removeVertex(vertex);
     }
 
@@ -343,7 +350,7 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
     }
 
     void createSuperTypeEdges(AtlasVertex vertex, Set<String> superTypes, TypeCategory typeCategory)
-        throws AtlasBaseException {
+            throws AtlasBaseException {
         Set<String> currentSuperTypes = getSuperTypeNames(vertex);
 
         if (CollectionUtils.isNotEmpty(superTypes)) {
@@ -385,6 +392,9 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
 
             case ENUM:
                 return TypeCategory.ENUM;
+
+            case RELATIONSHIP:
+                return TypeCategory.RELATIONSHIP;
         }
 
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/typesystem/src/main/java/org/apache/atlas/typesystem/types/DataTypes.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/DataTypes.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/DataTypes.java
index 21d5f1a..f9f4abe 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/DataTypes.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/DataTypes.java
@@ -86,7 +86,8 @@ public class DataTypes {
         MAP,
         STRUCT,
         TRAIT,
-        CLASS
+        CLASS,
+        RELATIONSHIP
     }
 
     public static abstract class PrimitiveType<T> extends AbstractDataType<T> {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/webapp/src/main/java/org/apache/atlas/examples/CreateTypesFromJsonFileUtil.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/examples/CreateTypesFromJsonFileUtil.java b/webapp/src/main/java/org/apache/atlas/examples/CreateTypesFromJsonFileUtil.java
new file mode 100644
index 0000000..82d6f2e
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/examples/CreateTypesFromJsonFileUtil.java
@@ -0,0 +1,109 @@
+/**
+ * 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.atlas.examples;
+
+import java.io.Console;
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+
+import org.apache.atlas.ApplicationProperties;
+import org.apache.atlas.AtlasBaseClient;
+import org.apache.atlas.AtlasClientV2;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.utils.AuthenticationUtil;
+import org.apache.commons.configuration.Configuration;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * A driver that sets up types supplied in a file.  
+ */
+public class CreateTypesFromJsonFileUtil  extends AtlasBaseClient{
+    public static final String ATLAS_REST_ADDRESS = "atlas.rest.address";
+
+    public static void main(String[] args) throws Exception {
+
+        Console console = System.console();
+        if (console == null) {
+            System.err.println("No console.");
+            System.exit(1);
+        }
+        String[] basicAuthUsernamePassword = null;
+        if (!AuthenticationUtil.isKerberosAuthenticationEnabled()) {
+            basicAuthUsernamePassword = AuthenticationUtil.getBasicAuthenticationInput();
+        }
+        AtlasClientV2 atlasClientV2 = getAtlasClientV2(args, basicAuthUsernamePassword);
+
+
+        String createFileName = console.readLine("Enter fileName containing TypeDefs for create:- ");
+        File createFile = new File(createFileName);
+        String createJsonStr = new String( Files.readAllBytes(createFile.toPath()), StandardCharsets.UTF_8);
+
+        System.err.println("create json is :\n" + createJsonStr);
+
+        runTypeCreation(createJsonStr,atlasClientV2);
+//        String updateFileName = console.readLine("Enter fileName containing TypeDefs for update:- ");
+//        File updateFile = new File(updateFileName);
+//        String updateJsonStr = new String( Files.readAllBytes(updateFile.toPath()), StandardCharsets.UTF_8);
+//        System.err.println("update json is :\n" + updateJsonStr);
+//        runTypeUpdate(updateJsonStr,atlasClientV2);
+    }
+
+    @VisibleForTesting
+    static void runTypeCreation(String jsonStr,AtlasClientV2 atlasClientV2) throws Exception {
+        AtlasTypesDef typesDef = AtlasType.fromJson(jsonStr, AtlasTypesDef.class);
+        atlasClientV2.createAtlasTypeDefs(typesDef);
+    }
+    @VisibleForTesting
+    static void runTypeUpdate(String jsonStr,AtlasClientV2 atlasClientV2) throws Exception {
+        AtlasTypesDef typesDef = AtlasType.fromJson(jsonStr, AtlasTypesDef.class);
+        atlasClientV2.updateAtlasTypeDefs(typesDef);
+    }
+
+    private static AtlasClientV2 getAtlasClientV2(String[] args, String[] basicAuthUsernamePassword) throws AtlasException {
+        String[] urls = getServerUrl(args);
+
+        AtlasClientV2 atlasClientV2;
+
+        if (!AuthenticationUtil.isKerberosAuthenticationEnabled()) {
+            atlasClientV2 =new AtlasClientV2(urls,basicAuthUsernamePassword);
+        } else {
+            atlasClientV2 = new AtlasClientV2(urls);
+        }
+        return atlasClientV2;
+    }
+
+    static String[] getServerUrl(String[] args) throws AtlasException {
+        if (args.length > 0) {
+            return args[0].split(",");
+        }
+
+        Configuration configuration = ApplicationProperties.get();
+        String[] urls = configuration.getStringArray(ATLAS_REST_ADDRESS);
+        if (urls == null || urls.length == 0) {
+            System.out.println("Usage: quick_start.py <atlas endpoint of format <http/https>://<atlas-fqdn>:<atlas port> like http://localhost:21000>");
+            System.exit(-1);
+        }
+
+        return urls;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/webapp/src/main/java/org/apache/atlas/examples/UpdateTypesFromJsonFileUtil.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/examples/UpdateTypesFromJsonFileUtil.java b/webapp/src/main/java/org/apache/atlas/examples/UpdateTypesFromJsonFileUtil.java
new file mode 100644
index 0000000..3b67706
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/examples/UpdateTypesFromJsonFileUtil.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.examples;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.atlas.ApplicationProperties;
+import org.apache.atlas.AtlasBaseClient;
+import org.apache.atlas.AtlasClientV2;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.utils.AuthenticationUtil;
+import org.apache.commons.configuration.Configuration;
+
+import java.io.Console;
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+
+/**
+ * A driver that sets up types supplied in a file.  
+ */
+public class UpdateTypesFromJsonFileUtil extends AtlasBaseClient{
+    public static final String ATLAS_REST_ADDRESS = "atlas.rest.address";
+
+    public static void main(String[] args) throws Exception {
+
+        Console console = System.console();
+        if (console == null) {
+            System.err.println("No console.");
+            System.exit(1);
+        }
+        String[] basicAuthUsernamePassword = null;
+        if (!AuthenticationUtil.isKerberosAuthenticationEnabled()) {
+            basicAuthUsernamePassword = AuthenticationUtil.getBasicAuthenticationInput();
+        }
+        AtlasClientV2 atlasClientV2 = getAtlasClientV2(args, basicAuthUsernamePassword);
+
+
+        String createFileName = console.readLine("Enter fileName containing TypeDefs for create:- ");
+        File createFile = new File(createFileName);
+        String createJsonStr = new String( Files.readAllBytes(createFile.toPath()), StandardCharsets.UTF_8);
+
+        System.err.println("create json is :\n" + createJsonStr);
+
+        runTypeCreation(createJsonStr,atlasClientV2);
+//        String updateFileName = console.readLine("Enter fileName containing TypeDefs for update:- ");
+//        File updateFile = new File(updateFileName);
+//        String updateJsonStr = new String( Files.readAllBytes(updateFile.toPath()), StandardCharsets.UTF_8);
+//        System.err.println("update json is :\n" + updateJsonStr);
+//        runTypeUpdate(updateJsonStr,atlasClientV2);
+    }
+
+    @VisibleForTesting
+    static void runTypeCreation(String jsonStr,AtlasClientV2 atlasClientV2) throws Exception {
+        AtlasTypesDef typesDef = AtlasType.fromJson(jsonStr, AtlasTypesDef.class);
+        atlasClientV2.createAtlasTypeDefs(typesDef);
+    }
+    @VisibleForTesting
+    static void runTypeUpdate(String jsonStr,AtlasClientV2 atlasClientV2) throws Exception {
+        AtlasTypesDef typesDef = AtlasType.fromJson(jsonStr, AtlasTypesDef.class);
+        atlasClientV2.updateAtlasTypeDefs(typesDef);
+    }
+
+    private static AtlasClientV2 getAtlasClientV2(String[] args, String[] basicAuthUsernamePassword) throws AtlasException {
+        String[] urls = getServerUrl(args);
+
+        AtlasClientV2 atlasClientV2;
+
+        if (!AuthenticationUtil.isKerberosAuthenticationEnabled()) {
+            atlasClientV2 =new AtlasClientV2(urls,basicAuthUsernamePassword);
+        } else {
+            atlasClientV2 = new AtlasClientV2(urls);
+        }
+        return atlasClientV2;
+    }
+
+    static String[] getServerUrl(String[] args) throws AtlasException {
+        if (args.length > 0) {
+            return args[0].split(",");
+        }
+
+        Configuration configuration = ApplicationProperties.get();
+        String[] urls = configuration.getStringArray(ATLAS_REST_ADDRESS);
+        if (urls == null || urls.length == 0) {
+            System.out.println("Usage: quick_start.py <atlas endpoint of format <http/https>://<atlas-fqdn>:<atlas port> like http://localhost:21000>");
+            System.exit(-1);
+        }
+
+        return urls;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
index 08121d8..f70593a 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
@@ -266,7 +266,7 @@ public class TypesResource {
      * Return the list of type names in the type system which match the specified filter.
      *
      * @return list of type names
-     * @param typeCategory returns types whose category is the given typeCategory
+     * @param typeCategory returns types whose relationshipCategory is the given typeCategory
      * @param supertype returns types which contain the given supertype
      * @param notsupertype returns types which do not contain the given supertype
      *

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java b/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java
index c32f36e..59ea338 100644
--- a/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java
+++ b/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java
@@ -19,13 +19,7 @@ package org.apache.atlas.web.rest;
 
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.SearchFilter;
-import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
-import org.apache.atlas.model.typedef.AtlasClassificationDef;
-import org.apache.atlas.model.typedef.AtlasEntityDef;
-import org.apache.atlas.model.typedef.AtlasEnumDef;
-import org.apache.atlas.model.typedef.AtlasStructDef;
-import org.apache.atlas.model.typedef.AtlasTypeDefHeader;
-import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.model.typedef.*;
 import org.apache.atlas.store.AtlasTypeDefStore;
 import org.apache.atlas.type.AtlasTypeUtil;
 import org.apache.atlas.utils.AtlasPerfTracer;
@@ -37,14 +31,7 @@ import org.springframework.stereotype.Service;
 import javax.inject.Inject;
 import javax.inject.Singleton;
 import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
+import javax.ws.rs.*;
 import javax.ws.rs.core.Context;
 import java.util.List;
 import java.util.Set;
@@ -269,7 +256,39 @@ public class TypesREST {
 
         return ret;
     }
+    /**
+     * Get the relationship definition by it's name (unique)
+     * @param name relationship name
+     * @return relationship definition
+     * @throws AtlasBaseException
+     * @HTTP 200 On successful lookup of the the relationship definition by it's name
+     * @HTTP 404 On Failed lookup for the given name
+     */
+    @GET
+    @Path("/relationshipdef/name/{name}")
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public AtlasRelationshipDef getRelationshipDefByName(@PathParam("name") String name) throws AtlasBaseException {
+        AtlasRelationshipDef ret = typeDefStore.getRelationshipDefByName(name);
+
+        return ret;
+    }
 
+    /**
+     * Get the relationship definition for the given guid
+     * @param guid relationship guid
+     * @return relationship definition
+     * @throws AtlasBaseException
+     * @HTTP 200 On successful lookup of the the relationship definition by it's guid
+     * @HTTP 404 On Failed lookup for the given guid
+     */
+    @GET
+    @Path("/relationshipdef/guid/{guid}")
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public AtlasRelationshipDef getRelationshipDefByGuid(@PathParam("guid") String guid) throws AtlasBaseException {
+        AtlasRelationshipDef ret = typeDefStore.getRelationshipDefByGuid(guid);
+
+        return ret;
+    }
     /* Bulk API operation */
 
     /**


[2/2] incubator-atlas git commit: ATLAS-1852: create relationship-def

Posted by ma...@apache.org.
ATLAS-1852: create relationship-def

Signed-off-by: Madhan Neethiraj <ma...@apache.org>


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

Branch: refs/heads/master
Commit: 6b9399e00dcd2fb2b231c91064dfe15f46feaf1d
Parents: e0072e5
Author: David Radley <da...@uk.ibm.com>
Authored: Sun Jun 11 11:57:50 2017 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Sun Jun 11 12:26:19 2017 -0700

----------------------------------------------------------------------
 .../org/apache/atlas/repository/Constants.java  |   6 +
 .../java/org/apache/atlas/AtlasErrorCode.java   |   9 +-
 .../org/apache/atlas/model/TypeCategory.java    |   2 +-
 .../atlas/model/typedef/AtlasBaseTypeDef.java   |  16 +
 .../model/typedef/AtlasRelationshipDef.java     | 279 +++++++++++++
 .../typedef/AtlasRelationshipEndPointDef.java   | 193 +++++++++
 .../atlas/model/typedef/AtlasTypesDef.java      |  66 ++-
 .../apache/atlas/store/AtlasTypeDefStore.java   |  10 +
 .../atlas/type/AtlasRelationshipType.java       | 151 +++++++
 .../apache/atlas/type/AtlasTypeRegistry.java    |  99 +++--
 .../org/apache/atlas/type/AtlasTypeUtil.java    |  39 +-
 .../org/apache/atlas/model/ModelTestUtil.java   |  68 ++-
 .../model/typedef/TestAtlasRelationshipDef.java |  96 +++++
 .../atlas/type/TestAtlasRelationshipType.java   | 107 +++++
 .../store/graph/AtlasRelationshipDefStore.java  |  52 +++
 .../store/graph/AtlasTypeDefGraphStore.java     | 135 ++++--
 .../graph/v1/AtlasRelationshipDefStoreV1.java   | 409 +++++++++++++++++++
 .../graph/v1/AtlasTypeDefGraphStoreV1.java      |  58 +--
 .../atlas/typesystem/types/DataTypes.java       |   3 +-
 .../examples/CreateTypesFromJsonFileUtil.java   | 109 +++++
 .../examples/UpdateTypesFromJsonFileUtil.java   | 108 +++++
 .../atlas/web/resources/TypesResource.java      |   2 +-
 .../org/apache/atlas/web/rest/TypesREST.java    |  49 ++-
 23 files changed, 1923 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/common/src/main/java/org/apache/atlas/repository/Constants.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/atlas/repository/Constants.java b/common/src/main/java/org/apache/atlas/repository/Constants.java
index bcdf08c..b07934b 100644
--- a/common/src/main/java/org/apache/atlas/repository/Constants.java
+++ b/common/src/main/java/org/apache/atlas/repository/Constants.java
@@ -57,6 +57,11 @@ public final class Constants {
     public static final String TYPEVERSION_PROPERTY_KEY = INTERNAL_PROPERTY_KEY_PREFIX + "type.version";
     public static final String TYPEOPTIONS_PROPERTY_KEY = INTERNAL_PROPERTY_KEY_PREFIX + "type.options";
 
+    // relationship def constants
+    public static final String RELATIONSHIPTYPE_ENDPOINT1_KEY = "endPointDef1";
+    public static final String RELATIONSHIPTYPE_ENDPOINT2_KEY = "endPointDef2";
+    public static final String RELATIONSHIPTYPE_CATEGORY_KEY = "relationshipCategory";
+    public static final String RELATIONSHIPTYPE_TAG_PROPAGATION_KEY = "tagPropagation";
     /**
      * Trait names property key and index name.
      */
@@ -92,6 +97,7 @@ public final class Constants {
     public static final String QUALIFIED_NAME = "Referenceable.qualifiedName";
     public static final String TYPE_NAME_PROPERTY_KEY = INTERNAL_PROPERTY_KEY_PREFIX + "typeName";
 
+
     private Constants() {
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
index d723b2a..ca2f3d0 100644
--- a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
+++ b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
@@ -71,7 +71,14 @@ public enum AtlasErrorCode {
     BAD_REQUEST(400, "ATLAS-400-00-029", "{0}"),
     PARAMETER_PARSING_FAILED(400, "ATLAS-400-00-02A", "Parameter parsing failed at: {0}"),
     MISSING_MANDATORY_ATTRIBUTE(400, "ATLAS-400-00-02B", "Mandatory field {0}.{1} has empty/null value"),
-
+    RELATIONSHIPDEF_INSUFFICIENT_ENDPOINTS(400,  "ATLAS-400-00-02C", "Relationship def {0} creation attempted without 2 end points"),
+    RELATIONSHIPDEF_DOUBLE_CONTAINERS(400,  "ATLAS-400-00-02D", "Relationship def {0} creation attempted with both end points as containers"),
+    RELATIONSHIPDEF_UNSUPPORTED_ATTRIBUTE_TYPE(400,  "ATLAS-400-00-02F", "Cannot set an Attribute with type {0} on relationship def {1}, as it is not a primitive type "),
+    RELATIONSHIPDEF_ASSOCIATION_AND_CONTAINER(400,  "ATLAS-400-00-030", "ASSOCIATION relationship def {0} creation attempted with an endpoint specifying isContainer"),
+    RELATIONSHIPDEF_COMPOSITION_NO_CONTAINER(400,  "ATLAS-400-00-031", "COMPOSITION relationship def {0} creation attempted without an endpoint specifying isContainer"),
+    RELATIONSHIPDEF_AGGREGATION_NO_CONTAINER(400,  "ATLAS-400-00-032", "AGGREGATION relationship def {0} creation attempted without an endpoint specifying isContainer"),
+    RELATIONSHIPDEF_COMPOSITION_SET_CONTAINER(400,  "ATLAS-400-00-033", "COMPOSITION relationship def {0} cannot have a SET cardinality and be a container"),
+    RELATIONSHIPDEF_LIST_ON_ENDPOINT(400,  "ATLAS-400-00-034", "relationship def {0} cannot have a LIST cardinality on an endpoint"),
     // All Not found enums go here
     TYPE_NAME_NOT_FOUND(404, "ATLAS-404-00-001", "Given typename {0} was invalid"),
     TYPE_GUID_NOT_FOUND(404, "ATLAS-404-00-002", "Given type guid {0} was invalid"),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/model/TypeCategory.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/TypeCategory.java b/intg/src/main/java/org/apache/atlas/model/TypeCategory.java
index e47a8a7..f06f64f 100644
--- a/intg/src/main/java/org/apache/atlas/model/TypeCategory.java
+++ b/intg/src/main/java/org/apache/atlas/model/TypeCategory.java
@@ -18,5 +18,5 @@
 package org.apache.atlas.model;
 
 public enum TypeCategory {
-    PRIMITIVE, OBJECT_ID_TYPE, ENUM, STRUCT, CLASSIFICATION, ENTITY, ARRAY, MAP
+    PRIMITIVE, OBJECT_ID_TYPE, ENUM, STRUCT, CLASSIFICATION, ENTITY, ARRAY, MAP, RELATIONSHIP
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/model/typedef/AtlasBaseTypeDef.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/typedef/AtlasBaseTypeDef.java b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasBaseTypeDef.java
index 7308eb7..1ccab22 100644
--- a/intg/src/main/java/org/apache/atlas/model/typedef/AtlasBaseTypeDef.java
+++ b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasBaseTypeDef.java
@@ -91,6 +91,22 @@ public abstract class AtlasBaseTypeDef implements java.io.Serializable {
         ATLAS_TYPE_BIGDECIMAL,
         ATLAS_TYPE_STRING,
     };
+    /**
+     * The list of types that are valid for relationships. These are the
+     * primitive attributes and date.
+     */
+    public static final String[] ATLAS_RELATIONSHIP_ATTRIBUTE_TYPES = { ATLAS_TYPE_BOOLEAN,
+            ATLAS_TYPE_BYTE,
+            ATLAS_TYPE_SHORT,
+            ATLAS_TYPE_INT,
+            ATLAS_TYPE_LONG,
+            ATLAS_TYPE_FLOAT,
+            ATLAS_TYPE_DOUBLE,
+            ATLAS_TYPE_BIGINTEGER,
+            ATLAS_TYPE_BIGDECIMAL,
+            ATLAS_TYPE_STRING,
+            ATLAS_TYPE_DATE
+    };
 
     public static final String[] ATLAS_BUILTIN_TYPES = {
         ATLAS_TYPE_BOOLEAN,

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipDef.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipDef.java b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipDef.java
new file mode 100644
index 0000000..eb8330e
--- /dev/null
+++ b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipDef.java
@@ -0,0 +1,279 @@
+/**
+ * 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.atlas.model.typedef;
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.codehaus.jackson.annotate.JsonAutoDetect;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
+
+/**
+ * AtlasRelationshipDef is a TypeDef that defines a relationship.
+ *
+ * As with other typeDefs the AtlasRelationshipDef has a name. Once created the RelationshipDef has a guid.
+ * The name and the guid are the 2 ways that the RelationshipDef is identified.
+ *
+ * RelationshipDefs have 2 endpoints, each of which specify cardinality, an EntityDef type name and name and optionally
+ * whether the endpoint is a container.
+ * RelationshipDefs can have AttributeDefs - though only primitive types are allowed.
+ * RelationshipDefs have a relationshipCategory specifying the UML type of relationship required
+ * RelationshipDefs also have a PropogateTag - indicating which way tags could flow over the relationships.
+ *
+ * The way EntityDefs and RelationshipDefs are intended to be used is that EntityDefs will define AttributeDefs these AttributeDefs
+ * will not specify an EntityDef type name as their types.
+ *
+ * RelationshipDefs introduce new atributes to the entity instances. For example
+ * EntityDef A might have attributes attr1,attr2,attr3
+ * EntityDef B might have attributes attr4,attr5,attr6
+ * RelationshipDef AtoB might define 2 endpoints
+ *  endpoint1:  type A, name attr7
+ *  endpoint1:  type B, name attr8
+ *
+ * When an instance of EntityDef A is created, it will have attributes attr1,attr2,attr3,attr7
+ * When an instance of EntityDef B is created, it will have attributes attr4,attr5,attr6,attr8
+ *
+ * In this way relationshipDefs can be authored separately from entityDefs and can inject relationship attributes into
+ * the entity instances
+ *
+ */
+@JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.PROPERTY)
+public class AtlasRelationshipDef extends AtlasStructDef implements java.io.Serializable {
+    private static final long serialVersionUID = 1L;
+
+
+    /**
+     * The Relationship category determines the style of relationship around containment and lifecycle.
+     * UML terminology is used for the values.
+     * ASSOCIATION is a relationship with no containment.
+     * COMPOSITION and AGGREGATION are containment relationships.
+     * The difference being in the lifecycles of the container and its children. In the COMPOSITION case,
+     * the children cannot exist without the container. For AGGREGATION, the life cycles
+     * of the container and children are totally independant.
+     */
+    public enum RelationshipCategory {
+        ASSOCIATION, AGGREGATION, COMPOSITION
+    };
+
+    /**
+     * PropagateTags indicates whether tags should propagate across the relationship instance.
+     * Tags can propagate:
+     * NONE - not at all
+     * ONE_TO_TWO - from endpoint 1 to 2
+     * TWO_TO_ONE - from endpoint 2 to 1
+     * BOTH - both ways
+     *
+     * Care needs to be taken when specifying. The use cases we are aware of this flag being useful are :
+     *
+     * - propagating confidentiality classifications from a table to columns - ONE_TO_TWO could be used here
+     * - propagating classifications around Glossary synonyms - BOTH could be used here.
+     *
+     * There is an expectation that further enhancements will allow more granular control of tag propagation and will
+     * address how to resolve conflicts.
+     */
+    public enum PropagateTags {
+        NONE, ONE_TO_TWO, TWO_TO_ONE, BOTH
+    };
+
+    private RelationshipCategory         relationshipCategory;
+    private PropagateTags                propagateTags;
+    private AtlasRelationshipEndPointDef endPointDef1;
+    private AtlasRelationshipEndPointDef endPointDef2;
+
+    /**
+     * AtlasRelationshipDef contructor
+     * @throws AtlasBaseException
+     */
+    public AtlasRelationshipDef() throws AtlasBaseException {
+        this(null, null, null, null,null, null, null);
+    }
+
+    /**
+     * Create a relationshipDef without attributeDefs
+     * @param name
+     *            - the name of the relationship type
+     * @param description
+     *            - an optional description
+     * @param typeVersion
+     *            - version - that defaults to 1.0
+     * @param relationshipCategory
+     *            - there are 3 sorts of relationship category ASSOCIATION, COMPOSITION
+     *            and AGGREGATION
+     * @param propagatetags
+     *            -
+     * @param endPointDef1
+     *            - first endpoint. As endpoint specifies an entity
+     *            type and an attribute name. the attribute name then appears in
+     *            the relationship instance
+     * @param endPointDef2
+     *            - second endpoint. The endpoints are defined as 1
+     *            ad 2 to avoid implying a direction. So we do not use to and
+     *            from.
+     * @throws AtlasBaseException
+     */
+    public AtlasRelationshipDef(String name, String description, String typeVersion,
+                                RelationshipCategory relationshipCategory,
+                                PropagateTags propagatetags,
+                                AtlasRelationshipEndPointDef endPointDef1,
+                                AtlasRelationshipEndPointDef endPointDef2) throws AtlasBaseException {
+        this(name, description, typeVersion, relationshipCategory,propagatetags, endPointDef1, endPointDef2,
+             new ArrayList<AtlasAttributeDef>());
+    }
+
+    /**
+     * Create a relationshipDef with attributeDefs
+     * @param name
+     *            - the name of the relationship type
+     * @param description
+     *            - an optional description
+     * @param typeVersion
+     *            - version - that defaults to 1.0
+     * @param relationshipCategory
+     *            - there are 3 sorts of relationship category ASSOCIATION, COMPOSITION
+     *            and AGGREGATION
+     * @param propagatetags
+     *            -
+     * @param endPointDef1
+     *            - First endpoint. As endpoint specifies an entity
+     *            type and an attribute name. the attribute name then appears in
+     *            the relationship instance
+     * @param endPointDef2
+     *            - Second endpoint. The endpoints are defined as 1
+     *            ad 2 to avoid implying a direction. So we do not use to and
+     *            from.
+     * @param attributeDefs
+     *            - these are the attributes on the relationship itself.
+     */
+    public AtlasRelationshipDef(String name, String description, String typeVersion,
+                                RelationshipCategory relationshipCategory,
+                                PropagateTags propagatetags, AtlasRelationshipEndPointDef endPointDef1,
+                                AtlasRelationshipEndPointDef endPointDef2, List<AtlasAttributeDef> attributeDefs)
+            {
+        super(TypeCategory.RELATIONSHIP, name, description, typeVersion, attributeDefs, null);
+
+        setRelationshipCategory(relationshipCategory);
+        setPropagateTags(propagatetags);
+        setEndPointDef1(endPointDef1);
+        setEndPointDef2(endPointDef2);
+    }
+
+    public void setRelationshipCategory(RelationshipCategory relationshipCategory) {
+        this.relationshipCategory = relationshipCategory;
+    }
+
+    public RelationshipCategory getRelationshipCategory() {
+        return this.relationshipCategory;
+    }
+
+    public void setPropagateTags(PropagateTags propagateTags) {
+        this.propagateTags=propagateTags;
+    }
+
+    public PropagateTags getPropagateTags() {
+        return this.propagateTags;
+    }
+
+    public void setEndPointDef1(AtlasRelationshipEndPointDef endPointDef1) {
+        this.endPointDef1 = endPointDef1;
+    }
+
+    public AtlasRelationshipEndPointDef getEndPointDef1() {
+        return this.endPointDef1;
+    }
+
+    public void setEndPointDef2(AtlasRelationshipEndPointDef endPointDef2) {
+        this.endPointDef2 = endPointDef2;
+    }
+
+    public AtlasRelationshipEndPointDef getEndPointDef2() {
+        return this.endPointDef2;
+    }
+
+    public AtlasRelationshipDef(AtlasRelationshipDef other) throws AtlasBaseException {
+        super(other);
+
+        if (other != null) {
+            setRelationshipCategory(other.getRelationshipCategory());
+            setPropagateTags(other.getPropagateTags());
+            setEndPointDef1(other.getEndPointDef1());
+            setEndPointDef2(other.getEndPointDef2());
+        }
+    }
+    @Override
+    public StringBuilder toString(StringBuilder sb) {
+        if (sb == null) {
+            sb = new StringBuilder();
+        }
+
+        sb.append("AtlasRelationshipDef{");
+        super.toString(sb);
+        sb.append(',');
+        sb.append(this.relationshipCategory);
+        sb.append(',');
+        sb.append(this.propagateTags);
+        sb.append(',');
+        sb.append(this.endPointDef1.toString());
+        sb.append(',');
+        sb.append(this.endPointDef2.toString());
+        sb.append('}');
+        return sb;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        //AttributeDefs are checked in the super
+        if (!super.equals(o))
+            return false;
+        AtlasRelationshipDef that = (AtlasRelationshipDef) o;
+        if (!Objects.equals(relationshipCategory, that.getRelationshipCategory()))
+            return false;
+        if (!Objects.equals(propagateTags, that.getPropagateTags()))
+            return false;
+        if (!Objects.equals(endPointDef1, that.getEndPointDef1()))
+            return false;
+        return (Objects.equals(endPointDef2, that.getEndPointDef2()));
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(super.hashCode(), relationshipCategory, propagateTags, endPointDef1, endPointDef2);
+    }
+
+    @Override
+    public String toString() {
+        return toString(new StringBuilder()).toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipEndPointDef.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipEndPointDef.java b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipEndPointDef.java
new file mode 100644
index 0000000..dde8416
--- /dev/null
+++ b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasRelationshipEndPointDef.java
@@ -0,0 +1,193 @@
+/**
+ * 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.atlas.model.typedef;
+
+import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality;
+import org.codehaus.jackson.annotate.JsonAutoDetect;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
+
+/**
+ * The relationshipEndPointsDef represents an end of the relationship. The end of the relationship is defined by a type, an
+ * attribute name, cardinality and whether it  is the container end of the relationship.
+ */
+@JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.PROPERTY)
+public class AtlasRelationshipEndPointDef implements Serializable {
+    private static final long serialVersionUID = 1L;
+    /**
+     * The type associated with the endpoint.
+     */
+    private String type;
+    /**
+     * The name of the attribute for this endpoint
+     */
+    private String name;
+
+    /**
+     * When set this indicates that this end is the container end
+     */
+    private boolean isContainer;
+    /**
+     * This is the cardinality of the end point
+     */
+    private Cardinality cardinality;
+
+    /**
+     * Base constructor
+     */
+    public AtlasRelationshipEndPointDef() {
+        this(null, null, Cardinality.SINGLE, false);
+    }
+
+    /**
+     *
+     * @param typeName
+     *   - The name of an entityDef type
+     * @param name
+     *   - The name of the new attribute that the entity instance will pick up.
+     * @param cardinality
+     *   - this indicates whether the end point is SINGLE (1) or SET (many)
+     */
+    public AtlasRelationshipEndPointDef(String typeName, String name, Cardinality cardinality) {
+        this(typeName, name, cardinality, false);
+    }
+
+    /**
+     *
+     * @param typeName
+     *   - The name of an entityDef type
+     * @param name
+     *   - The name of the new attribute that the entity instance will pick up.
+     * @param cardinality
+     *   - whether the end point is SINGLE (1) or SET (many)
+     * @param isContainer
+     *   - whether the end point is a container or not
+     */
+    public AtlasRelationshipEndPointDef(String typeName, String name, Cardinality cardinality, boolean isContainer) {
+        setType(typeName);
+        setName(name);
+        setCardinality(cardinality);
+        setIsContainer(isContainer);
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public String getName() {
+        return this.name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * set whether this endpoint is a container or not.
+     * @param isContainer
+     */
+    public void setIsContainer(boolean isContainer) {
+        this.isContainer = isContainer;
+    }
+
+    public boolean getIsContainer() {
+        return isContainer;
+    }
+
+    /**
+     * set the cardinality SINGLE or SET on the endpoint.
+     * @param cardinality
+     */
+    public void setCardinality(AtlasStructDef.AtlasAttributeDef.Cardinality cardinality) {
+        this.cardinality = cardinality;
+    }
+
+    /**
+     *
+     * @return the cardinality
+     */
+    public Cardinality getCardinality() {
+        return this.cardinality;
+    }
+
+    /**
+     * Construct using an existing AtlasRelationshipEndPointDef
+     * @param other
+     */
+    public AtlasRelationshipEndPointDef(AtlasRelationshipEndPointDef other) {
+        if (other != null) {
+            setType(other.getType());
+            setName(other.getName());
+            setIsContainer(other.getIsContainer());
+            setCardinality(other.getCardinality());
+        }
+    }
+
+    public StringBuilder toString(StringBuilder sb) {
+        if (sb == null) {
+            sb = new StringBuilder();
+        }
+
+        sb.append("AtlasRelationshipEndPointsDef{");
+        sb.append("type='").append(type).append('\'');
+        sb.append(", name==>'").append(name).append('\'');
+        sb.append(", isContainer==>'").append(isContainer).append('\'');
+        sb.append(", cardinality==>'").append(cardinality).append('\'');
+        sb.append('}');
+
+        return sb;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        AtlasRelationshipEndPointDef that = (AtlasRelationshipEndPointDef) o;
+        return Objects.equals(type, that.type) && Objects.equals(name, that.name)
+                && (isContainer == that.isContainer) && (cardinality == that.cardinality);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(type, getName(), isContainer, cardinality);
+    }
+
+    @Override
+    public String toString() {
+        return toString(new StringBuilder()).toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/model/typedef/AtlasTypesDef.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/typedef/AtlasTypesDef.java b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasTypesDef.java
index af95bff..66ae503 100644
--- a/intg/src/main/java/org/apache/atlas/model/typedef/AtlasTypesDef.java
+++ b/intg/src/main/java/org/apache/atlas/model/typedef/AtlasTypesDef.java
@@ -17,11 +17,8 @@
  */
 package org.apache.atlas.model.typedef;
 
-import org.apache.commons.collections.CollectionUtils;
-import org.codehaus.jackson.annotate.JsonAutoDetect;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -31,8 +28,11 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
-import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
+import org.apache.commons.collections.CollectionUtils;
+import org.codehaus.jackson.annotate.JsonAutoDetect;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 @JsonAutoDetect(getterVisibility=PUBLIC_ONLY, setterVisibility=PUBLIC_ONLY, fieldVisibility=NONE)
 @JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
@@ -44,29 +44,50 @@ public class AtlasTypesDef {
     private List<AtlasStructDef>         structDefs;
     private List<AtlasClassificationDef> classificationDefs;
     private List<AtlasEntityDef>         entityDefs;
+    private List<AtlasRelationshipDef>   relationshipDefs;
 
     public AtlasTypesDef() {
         enumDefs           = new ArrayList<>();
         structDefs         = new ArrayList<>();
         classificationDefs = new ArrayList<>();
         entityDefs         = new ArrayList<>();
-    }
-
+        relationshipDefs   = new ArrayList<>();
+    }
+
+    /**
+     * tolerate typeDef creations that do not contain relationshipDefs, so that
+     * the older calls will still work.
+     * @param enumDefs
+     * @param structDefs
+     * @param classificationDefs
+     * @param entityDefs
+     */
+    public AtlasTypesDef(List<AtlasEnumDef> enumDefs, List<AtlasStructDef> structDefs,
+                         List<AtlasClassificationDef> classificationDefs, List<AtlasEntityDef> entityDefs) {
+       this(enumDefs, structDefs, classificationDefs, entityDefs,new ArrayList<AtlasRelationshipDef>());
+    }
+    /**
+     * Create the TypesDef. This created definitions for each of the types.
+     * @param enumDefs
+     * @param structDefs
+     * @param classificationDefs
+     * @param entityDefs
+     * @param relationshipDefs
+     */
     public AtlasTypesDef(List<AtlasEnumDef>           enumDefs,
                          List<AtlasStructDef>         structDefs,
                          List<AtlasClassificationDef> classificationDefs,
-                         List<AtlasEntityDef>         entityDefs) {
+                         List<AtlasEntityDef>         entityDefs,
+                         List<AtlasRelationshipDef>   relationshipDefs) {
         this.enumDefs           = enumDefs;
         this.structDefs         = structDefs;
         this.classificationDefs = classificationDefs;
         this.entityDefs         = entityDefs;
+        this.relationshipDefs   = relationshipDefs;
     }
-
-
     public List<AtlasEnumDef> getEnumDefs() {
         return enumDefs;
     }
-
     public void setEnumDefs(List<AtlasEnumDef> enumDefs) {
         this.enumDefs = enumDefs;
     }
@@ -94,7 +115,13 @@ public class AtlasTypesDef {
     public void setClassificationDefs(List<AtlasClassificationDef> classificationDefs) {
         this.classificationDefs = classificationDefs;
     }
+    public List<AtlasRelationshipDef> getRelationshipDefs() {
+        return relationshipDefs;
+    }
 
+    public void setRelationshipDefs(List<AtlasRelationshipDef> relationshipDefs) {
+        this.relationshipDefs = relationshipDefs;
+    }
 
     public boolean hasClassificationDef(String name) {
         return hasTypeDef(classificationDefs, name);
@@ -111,6 +138,9 @@ public class AtlasTypesDef {
     public boolean hasEntityDef(String name) {
         return hasTypeDef(entityDefs, name);
     }
+    public boolean hasRelationshipDef(String name) {
+        return hasTypeDef(relationshipDefs, name);
+    }
 
 
     private <T extends AtlasBaseTypeDef> boolean hasTypeDef(Collection<T> typeDefs, String name) {
@@ -130,7 +160,8 @@ public class AtlasTypesDef {
         return CollectionUtils.isEmpty(enumDefs) &&
                 CollectionUtils.isEmpty(structDefs) &&
                 CollectionUtils.isEmpty(classificationDefs) &&
-                CollectionUtils.isEmpty(entityDefs);
+                CollectionUtils.isEmpty(entityDefs) &&
+                CollectionUtils.isEmpty(relationshipDefs);
     }
 
     public void clear() {
@@ -149,8 +180,10 @@ public class AtlasTypesDef {
         if (entityDefs != null) {
             entityDefs.clear();
         }
+        if (relationshipDefs != null) {
+            relationshipDefs.clear();
+        }
     }
-
     public StringBuilder toString(StringBuilder sb) {
         if (sb == null) {
             sb = new StringBuilder();
@@ -169,6 +202,9 @@ public class AtlasTypesDef {
         sb.append("entityDefs={");
         AtlasBaseTypeDef.dumpObjects(entityDefs, sb);
         sb.append("}");
+        sb.append("relationshipDefs={");
+        AtlasBaseTypeDef.dumpObjects(relationshipDefs, sb);
+        sb.append("}");
 
         return sb;
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/store/AtlasTypeDefStore.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/store/AtlasTypeDefStore.java b/intg/src/main/java/org/apache/atlas/store/AtlasTypeDefStore.java
index 198bd8f..e1c5a7f 100644
--- a/intg/src/main/java/org/apache/atlas/store/AtlasTypeDefStore.java
+++ b/intg/src/main/java/org/apache/atlas/store/AtlasTypeDefStore.java
@@ -24,6 +24,7 @@ import org.apache.atlas.model.typedef.AtlasClassificationDef;
 import org.apache.atlas.model.typedef.AtlasClassificationDef.AtlasClassificationDefs;
 import org.apache.atlas.model.typedef.AtlasEntityDef;
 import org.apache.atlas.model.typedef.AtlasEnumDef;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef;
 import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.model.typedef.AtlasTypesDef;
 
@@ -76,6 +77,15 @@ public interface AtlasTypeDefStore {
     AtlasEntityDef updateEntityDefByName(String name, AtlasEntityDef entityDef) throws AtlasBaseException;
 
     AtlasEntityDef updateEntityDefByGuid(String guid, AtlasEntityDef entityDef) throws AtlasBaseException;
+    /* RelationshipDef operations */
+
+    AtlasRelationshipDef getRelationshipDefByName(String name) throws AtlasBaseException;
+
+    AtlasRelationshipDef getRelationshipDefByGuid(String guid) throws AtlasBaseException;
+
+    AtlasRelationshipDef updateRelationshipDefByName(String name, AtlasRelationshipDef structDef) throws AtlasBaseException;
+
+    AtlasRelationshipDef updateRelationshipDefByGuid(String guid, AtlasRelationshipDef structDef) throws AtlasBaseException;
 
     /* Bulk Operations */
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/type/AtlasRelationshipType.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasRelationshipType.java b/intg/src/main/java/org/apache/atlas/type/AtlasRelationshipType.java
new file mode 100644
index 0000000..6328108
--- /dev/null
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasRelationshipType.java
@@ -0,0 +1,151 @@
+/**
+ * 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.atlas.type;
+
+
+import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef.RelationshipCategory;
+import org.apache.atlas.model.typedef.AtlasRelationshipEndPointDef;
+import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * class that implements behaviour of an relationship-type.
+ */
+public class AtlasRelationshipType extends AtlasStructType {
+    private static final Logger LOG = LoggerFactory.getLogger(AtlasRelationshipType.class);
+
+    private final AtlasRelationshipDef relationshipDef;
+
+    public AtlasRelationshipType(AtlasRelationshipDef relationshipDef) {
+        super(relationshipDef);
+
+        this.relationshipDef = relationshipDef;
+    }
+
+    public AtlasRelationshipType(AtlasRelationshipDef relationshipDef, AtlasTypeRegistry typeRegistry) throws AtlasBaseException {
+        super(relationshipDef);
+
+        this.relationshipDef = relationshipDef;
+
+        resolveReferences(typeRegistry);
+    }
+    public AtlasRelationshipDef getRelationshipDef() { return relationshipDef; }
+
+    @Override
+    public void resolveReferences(AtlasTypeRegistry typeRegistry) throws AtlasBaseException {
+        super.resolveReferences(typeRegistry);
+
+        validateAtlasRelationshipDef(this.relationshipDef);
+    }
+
+    @Override
+    public boolean isValidValue(Object obj) {
+        boolean ret = true;
+
+        if (obj != null) {
+           validateAtlasRelationshipType((AtlasRelationshipType) obj);
+           ret = super.isValidValue(obj);
+        }
+
+        return ret;
+    }
+
+    @Override
+    public boolean isValidValueForUpdate(Object obj) {
+        boolean ret = true;
+
+        if (obj != null) {
+            validateAtlasRelationshipType((AtlasRelationshipType) obj);
+            ret = super.isValidValueForUpdate(obj);
+        }
+
+        return ret;
+    }
+    /**
+     * Validate the fields in the the RelationshipType are consistent with respect to themselves.
+     * @param type
+     * @throws AtlasBaseException
+     */
+    private boolean validateAtlasRelationshipType(AtlasRelationshipType type) {
+        boolean isValid = false;
+        try {
+            validateAtlasRelationshipDef(type.getRelationshipDef());
+            isValid = true;
+        } catch (AtlasBaseException abe) {
+            LOG.error("Validation error for AtlasRelationshipType", abe);
+        }
+        return isValid;
+    }
+
+    /**
+     * Throw an exception so we can junit easily.
+     * @param relationshipDef
+     * @throws AtlasBaseException
+     */
+    public static void validateAtlasRelationshipDef(AtlasRelationshipDef relationshipDef) throws AtlasBaseException {
+        AtlasRelationshipEndPointDef endPointDef1 = relationshipDef.getEndPointDef1();
+        AtlasRelationshipEndPointDef endPointDef2 = relationshipDef.getEndPointDef2();
+        boolean                      isContainer1 = endPointDef1.getIsContainer();
+        boolean                      isContainer2 = endPointDef2.getIsContainer();
+        RelationshipCategory relationshipCategory = relationshipDef.getRelationshipCategory();
+        String                       name         = relationshipDef.getName();
+
+        if (isContainer1 && isContainer2) {
+            // we support 0 or 1 of these flags.
+            throw new AtlasBaseException(AtlasErrorCode.RELATIONSHIPDEF_DOUBLE_CONTAINERS, name);
+        }
+        if ((isContainer1 || isContainer2)) {
+            // we have an isContainer defined in an endpoint
+            if (relationshipCategory == RelationshipCategory.ASSOCIATION) {
+                // associations are not containment relaitonships - so do not allow an endpoiint with isContainer
+                throw new AtlasBaseException(AtlasErrorCode.RELATIONSHIPDEF_ASSOCIATION_AND_CONTAINER, name);
+            }
+        } else {
+            // we do not have an isContainer defined in an endpoint
+            if (relationshipCategory == RelationshipCategory.COMPOSITION) {
+                // COMPOSITION needs one endpoint to be the container.
+                throw new AtlasBaseException(AtlasErrorCode.RELATIONSHIPDEF_COMPOSITION_NO_CONTAINER, name);
+            } else if (relationshipCategory == RelationshipCategory.AGGREGATION) {
+                // AGGREGATION needs one endpoint to be the container.
+                throw new AtlasBaseException(AtlasErrorCode.RELATIONSHIPDEF_AGGREGATION_NO_CONTAINER, name);
+            }
+        }
+        if (relationshipCategory == RelationshipCategory.COMPOSITION) {
+            // composition containers should not be multiple cardinality
+            if (endPointDef1 != null &&
+                    endPointDef1.getCardinality() == AtlasAttributeDef.Cardinality.SET &&
+                    endPointDef1.getIsContainer()) {
+                throw new AtlasBaseException(AtlasErrorCode.RELATIONSHIPDEF_COMPOSITION_SET_CONTAINER, name);
+            }
+            if (endPointDef2 != null && endPointDef2 != null &&
+                    endPointDef2.getCardinality() == AtlasAttributeDef.Cardinality.SET &&
+                    endPointDef2.getIsContainer()) {
+                throw new AtlasBaseException(AtlasErrorCode.RELATIONSHIPDEF_COMPOSITION_SET_CONTAINER, name);
+            }
+        }
+        if ((endPointDef1 != null && endPointDef1.getCardinality() == AtlasAttributeDef.Cardinality.LIST) ||
+                (endPointDef2 != null && endPointDef2.getCardinality() == AtlasAttributeDef.Cardinality.LIST)) {
+            throw new AtlasBaseException(AtlasErrorCode.RELATIONSHIPDEF_LIST_ON_ENDPOINT, name);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/type/AtlasTypeRegistry.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasTypeRegistry.java b/intg/src/main/java/org/apache/atlas/type/AtlasTypeRegistry.java
index 1b3526b..aebd4d1 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasTypeRegistry.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasTypeRegistry.java
@@ -17,12 +17,30 @@
  */
 package org.apache.atlas.type;
 
+import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_ARRAY_PREFIX;
+import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_ARRAY_SUFFIX;
+import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_KEY_VAL_SEP;
+import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_PREFIX;
+import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_SUFFIX;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+import javax.inject.Singleton;
+
 import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
 import org.apache.atlas.model.typedef.AtlasClassificationDef;
 import org.apache.atlas.model.typedef.AtlasEntityDef;
 import org.apache.atlas.model.typedef.AtlasEnumDef;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef;
 import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.model.typedef.AtlasTypesDef;
 import org.apache.commons.collections.CollectionUtils;
@@ -30,19 +48,6 @@ import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.stereotype.Component;
-
-import javax.inject.Singleton;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.*;
-
 /**
  * registry for all types defined in Atlas.
  */
@@ -183,7 +188,6 @@ public class AtlasTypeRegistry {
 
 
     public Collection<AtlasEntityDef> getAllEntityDefs() { return registryData.entityDefs.getAll(); }
-
     public AtlasEntityDef getEntityDefByGuid(String guid) {
         return registryData.entityDefs.getTypeDefByGuid(guid);
     }
@@ -193,12 +197,20 @@ public class AtlasTypeRegistry {
     }
 
     public Collection<String> getAllEntityDefNames() { return registryData.entityDefs.getAllNames(); }
-
     public Collection<AtlasEntityType> getAllEntityTypes() { return registryData.entityDefs.getAllTypes(); }
-
     public AtlasEntityType getEntityTypeByName(String name) { return registryData.entityDefs.getTypeByName(name); }
+    /**
+     * @return relationshipTypes
+     */
+    public Collection<AtlasRelationshipType> getAllRelationshipTypes() { return registryData.relationshipDefs.getAllTypes(); }
 
-
+    public AtlasRelationshipDef getRelationshipDefByGuid(String guid) {
+        return registryData.relationshipDefs.getTypeDefByGuid(guid);
+    }
+    public AtlasRelationshipDef getRelationshipDefByName(String name) {
+        return registryData.relationshipDefs.getTypeDefByName(name);
+    }
+    public AtlasRelationshipType getRelationshipTypeByName(String name) { return registryData.relationshipDefs.getTypeByName(name); }
     public AtlasTransientTypeRegistry lockTypeRegistryForUpdate() throws AtlasBaseException {
         return lockTypeRegistryForUpdate(DEFAULT_LOCK_MAX_WAIT_TIME_IN_SECONDS);
     }
@@ -218,6 +230,7 @@ public class AtlasTypeRegistry {
         final TypeDefCache<AtlasStructDef, AtlasStructType>                   structDefs;
         final TypeDefCache<AtlasClassificationDef, AtlasClassificationType>   classificationDefs;
         final TypeDefCache<AtlasEntityDef, AtlasEntityType>                   entityDefs;
+        final TypeDefCache<AtlasRelationshipDef, AtlasRelationshipType>       relationshipDefs;
         final TypeDefCache<? extends AtlasBaseTypeDef, ? extends AtlasType>[] allDefCaches;
 
         RegistryData() {
@@ -226,7 +239,8 @@ public class AtlasTypeRegistry {
             structDefs         = new TypeDefCache<>(allTypes);
             classificationDefs = new TypeDefCache<>(allTypes);
             entityDefs         = new TypeDefCache<>(allTypes);
-            allDefCaches       = new TypeDefCache[] { enumDefs, structDefs, classificationDefs, entityDefs };
+            relationshipDefs   = new TypeDefCache<>(allTypes);
+            allDefCaches       = new TypeDefCache[] { enumDefs, structDefs, classificationDefs, entityDefs, relationshipDefs };
 
             init();
         }
@@ -284,6 +298,7 @@ public class AtlasTypeRegistry {
                 structDefs.updateGuid(typeName, guid);
                 classificationDefs.updateGuid(typeName, guid);
                 entityDefs.updateGuid(typeName, guid);
+                relationshipDefs.updateGuid(typeName, guid);
             }
         }
 
@@ -293,6 +308,7 @@ public class AtlasTypeRegistry {
                 structDefs.removeTypeDefByGuid(guid);
                 classificationDefs.removeTypeDefByGuid(guid);
                 entityDefs.removeTypeDefByGuid(guid);
+                relationshipDefs.removeTypeDefByGuid(guid);
             }
         }
 
@@ -302,6 +318,7 @@ public class AtlasTypeRegistry {
                 structDefs.removeTypeDefByName(typeName);
                 classificationDefs.removeTypeDefByName(typeName);
                 entityDefs.removeTypeDefByName(typeName);
+                relationshipDefs.removeTypeDefByName(typeName);
             }
         }
 
@@ -311,6 +328,7 @@ public class AtlasTypeRegistry {
             structDefs.clear();
             classificationDefs.clear();
             entityDefs.clear();
+            relationshipDefs.clear();
 
             init();
         }
@@ -403,6 +421,7 @@ public class AtlasTypeRegistry {
                 addTypesWithNoRefResolve(typesDef.getStructDefs());
                 addTypesWithNoRefResolve(typesDef.getClassificationDefs());
                 addTypesWithNoRefResolve(typesDef.getEntityDefs());
+                addTypesWithNoRefResolve(typesDef.getRelationshipDefs());
 
                 resolveReferences();
             }
@@ -502,6 +521,7 @@ public class AtlasTypeRegistry {
                 updateTypesWithNoRefResolve(typesDef.getStructDefs());
                 updateTypesWithNoRefResolve(typesDef.getClassificationDefs());
                 updateTypesWithNoRefResolve(typesDef.getEntityDefs());
+                updateTypesWithNoRefResolve(typesDef.getRelationshipDefs());
             }
 
             if (LOG.isDebugEnabled()) {
@@ -515,6 +535,7 @@ public class AtlasTypeRegistry {
                 removeTypesWithNoRefResolve(typesDef.getStructDefs());
                 removeTypesWithNoRefResolve(typesDef.getClassificationDefs());
                 removeTypesWithNoRefResolve(typesDef.getEntityDefs());
+                removeTypesWithNoRefResolve(typesDef.getRelationshipDefs());
 
                 resolveReferences();
             }
@@ -546,6 +567,9 @@ public class AtlasTypeRegistry {
                 case ENTITY:
                     registryData.entityDefs.removeTypeDefByName(typeDef.getName());
                     break;
+                case RELATIONSHIP:
+                    registryData.relationshipDefs.removeTypeDefByName(typeDef.getName());
+                    break;
             }
             deletedTypes.add(typeDef);
         }
@@ -564,6 +588,9 @@ public class AtlasTypeRegistry {
                 case ENTITY:
                     registryData.entityDefs.removeTypeDefByGuid(typeDef.getGuid());
                     break;
+                case RELATIONSHIP:
+                    registryData.relationshipDefs.removeTypeDefByGuid(typeDef.getGuid());
+                    break;
             }
             deletedTypes.add(typeDef);
         }
@@ -641,11 +668,15 @@ public class AtlasTypeRegistry {
                     AtlasClassificationDef classificationDef = (AtlasClassificationDef) typeDef;
 
                     registryData.classificationDefs.addType(classificationDef,
-                                                            new AtlasClassificationType(classificationDef));
+                            new AtlasClassificationType(classificationDef));
                 } else if (typeDef.getClass().equals(AtlasEntityDef.class)) {
                     AtlasEntityDef entityDef = (AtlasEntityDef) typeDef;
 
                     registryData.entityDefs.addType(entityDef, new AtlasEntityType(entityDef));
+                } else if (typeDef.getClass().equals(AtlasRelationshipDef.class)) {
+                    AtlasRelationshipDef relationshipDef = (AtlasRelationshipDef) typeDef;
+
+                    registryData.relationshipDefs.addType(relationshipDef, new AtlasRelationshipType(relationshipDef));
                 }
 
                 addedTypes.add(typeDef);
@@ -659,7 +690,7 @@ public class AtlasTypeRegistry {
         private void addTypesWithNoRefResolve(Collection<? extends AtlasBaseTypeDef> typeDefs) throws AtlasBaseException {
             if (LOG.isDebugEnabled()) {
                 LOG.debug("==> AtlasTypeRegistry.addTypesWithNoRefResolve(length={})",
-                          (typeDefs == null ? 0 : typeDefs.size()));
+                        (typeDefs == null ? 0 : typeDefs.size()));
             }
 
             if (CollectionUtils.isNotEmpty(typeDefs)) {
@@ -670,7 +701,7 @@ public class AtlasTypeRegistry {
 
             if (LOG.isDebugEnabled()) {
                 LOG.debug("<== AtlasTypeRegistry.addTypesWithNoRefResolve(length={})",
-                          (typeDefs == null ? 0 : typeDefs.size()));
+                        (typeDefs == null ? 0 : typeDefs.size()));
             }
         }
 
@@ -714,12 +745,17 @@ public class AtlasTypeRegistry {
 
                     registryData.classificationDefs.removeTypeDefByGuid(guid);
                     registryData.classificationDefs.addType(classificationDef,
-                                                            new AtlasClassificationType(classificationDef));
+                            new AtlasClassificationType(classificationDef));
                 } else if (typeDef.getClass().equals(AtlasEntityDef.class)) {
                     AtlasEntityDef entityDef = (AtlasEntityDef) typeDef;
 
                     registryData.entityDefs.removeTypeDefByGuid(guid);
                     registryData.entityDefs.addType(entityDef, new AtlasEntityType(entityDef));
+                } else if (typeDef.getClass().equals(AtlasRelationshipDef.class)) {
+                    AtlasRelationshipDef relationshipDef = (AtlasRelationshipDef) typeDef;
+
+                    registryData.relationshipDefs.removeTypeDefByGuid(guid);
+                    registryData.relationshipDefs.addType(relationshipDef, new AtlasRelationshipType(relationshipDef));
                 }
 
                 updatedTypes.add(typeDef);
@@ -751,12 +787,17 @@ public class AtlasTypeRegistry {
 
                     registryData.classificationDefs.removeTypeDefByName(name);
                     registryData.classificationDefs.addType(classificationDef,
-                                                            new AtlasClassificationType(classificationDef));
+                            new AtlasClassificationType(classificationDef));
                 } else if (typeDef.getClass().equals(AtlasEntityDef.class)) {
                     AtlasEntityDef entityDef = (AtlasEntityDef) typeDef;
 
                     registryData.entityDefs.removeTypeDefByName(name);
                     registryData.entityDefs.addType(entityDef, new AtlasEntityType(entityDef));
+                } else if (typeDef.getClass().equals(AtlasRelationshipDef.class)) {
+                    AtlasRelationshipDef relationshipDef = (AtlasRelationshipDef) typeDef;
+
+                    registryData.relationshipDefs.removeTypeDefByName(name);
+                    registryData.relationshipDefs.addType(relationshipDef, new AtlasRelationshipType(relationshipDef));
                 }
 
                 updatedTypes.add(typeDef);
@@ -770,7 +811,7 @@ public class AtlasTypeRegistry {
         private void updateTypesWithNoRefResolve(Collection<? extends AtlasBaseTypeDef> typeDefs) {
             if (LOG.isDebugEnabled()) {
                 LOG.debug("==> AtlasTypeRegistry.updateTypesWithNoRefResolve(length={})",
-                                                                             (typeDefs == null ? 0 : typeDefs.size()));
+                        (typeDefs == null ? 0 : typeDefs.size()));
             }
 
             if (CollectionUtils.isNotEmpty(typeDefs)) {
@@ -781,7 +822,7 @@ public class AtlasTypeRegistry {
 
             if (LOG.isDebugEnabled()) {
                 LOG.debug("<== AtlasTypeRegistry.updateTypesWithNoRefResolve(length={})",
-                                                                              (typeDefs == null ? 0 : typeDefs.size()));
+                        (typeDefs == null ? 0 : typeDefs.size()));
             }
         }
     }
@@ -808,7 +849,7 @@ public class AtlasTypeRegistry {
                 }
             } else {
                 LOG.warn("lockTypeRegistryForUpdate(): already locked. currentLockCount={}",
-                         typeRegistryUpdateLock.getHoldCount());
+                        typeRegistryUpdateLock.getHoldCount());
             }
 
             try {
@@ -842,8 +883,8 @@ public class AtlasTypeRegistry {
                 try {
                     if (typeRegistryUnderUpdate != ttr) {
                         LOG.error("releaseTypeRegistryForUpdate(): incorrect typeRegistry returned for release" +
-                                  ": found=" + ttr + "; expected=" + typeRegistryUnderUpdate,
-                                  new Exception().fillInStackTrace());
+                                        ": found=" + ttr + "; expected=" + typeRegistryUnderUpdate,
+                                new Exception().fillInStackTrace());
                     } else if (typeRegistryUpdateLock.getHoldCount() == 1) {
                         if (ttr != null && commitUpdates) {
                             typeRegistry.registryData = ttr.registryData;
@@ -861,7 +902,7 @@ public class AtlasTypeRegistry {
                 }
             } else {
                 LOG.error("releaseTypeRegistryForUpdate(): current thread does not hold the lock",
-                          new Exception().fillInStackTrace());
+                        new Exception().fillInStackTrace());
             }
 
             LOG.debug("<== releaseTypeRegistryForUpdate()");

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java b/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
index c0135f5..3125668 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
@@ -18,47 +18,23 @@
 package org.apache.atlas.type;
 
 import com.google.common.collect.ImmutableSet;
-
-import org.apache.atlas.AtlasErrorCode;
-import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
 import org.apache.atlas.model.instance.AtlasObjectId;
-import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
-import org.apache.atlas.model.typedef.AtlasClassificationDef;
-import org.apache.atlas.model.typedef.AtlasEntityDef;
-import org.apache.atlas.model.typedef.AtlasEnumDef;
+import org.apache.atlas.model.typedef.*;
 import org.apache.atlas.model.typedef.AtlasEnumDef.AtlasEnumElementDef;
-import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef;
-import org.apache.atlas.model.typedef.AtlasTypeDefHeader;
-import org.apache.atlas.model.typedef.AtlasTypesDef;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang.StringUtils;
 
-
-import java.util.Arrays;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
+import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_ARRAY_PREFIX;
-import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_ARRAY_SUFFIX;
-import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_KEY_VAL_SEP;
-import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_PREFIX;
-import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_SUFFIX;
+import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.*;
 
 /**
  * Utility methods for AtlasType/AtlasTypeDef.
@@ -294,6 +270,11 @@ public class AtlasTypeUtil {
                 headerList.add(new AtlasTypeDefHeader(entityDef));
             }
         }
+        if (CollectionUtils.isNotEmpty(typesDef.getRelationshipDefs())) {
+            for (AtlasRelationshipDef relationshipDef : typesDef.getRelationshipDefs()) {
+                headerList.add(new AtlasTypeDefHeader(relationshipDef));
+            }
+        }
 
         return headerList;
     }
@@ -390,6 +371,10 @@ public class AtlasTypeUtil {
             sb.append("entityDefs=[");
             dumpTypeNames(typesDef.getEntityDefs(), sb);
             sb.append("]");
+
+            sb.append("relationshipDefs=[");
+            dumpTypeNames(typesDef.getRelationshipDefs(), sb);
+            sb.append("]");
         }
         sb.append("}");
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/test/java/org/apache/atlas/model/ModelTestUtil.java
----------------------------------------------------------------------
diff --git a/intg/src/test/java/org/apache/atlas/model/ModelTestUtil.java b/intg/src/test/java/org/apache/atlas/model/ModelTestUtil.java
index 084bcc4..e1ca889 100644
--- a/intg/src/test/java/org/apache/atlas/model/ModelTestUtil.java
+++ b/intg/src/test/java/org/apache/atlas/model/ModelTestUtil.java
@@ -21,12 +21,8 @@ import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.instance.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.instance.AtlasStruct;
-import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
-import org.apache.atlas.model.typedef.AtlasClassificationDef;
-import org.apache.atlas.model.typedef.AtlasEntityDef;
-import org.apache.atlas.model.typedef.AtlasEnumDef;
+import org.apache.atlas.model.typedef.*;
 import org.apache.atlas.model.typedef.AtlasEnumDef.AtlasEnumElementDef;
-import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
 import org.apache.atlas.type.AtlasClassificationType;
 import org.apache.atlas.type.AtlasEntityType;
@@ -43,6 +39,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_BUILTIN_TYPES;
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_PRIMITIVE_TYPES;
+import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_RELATIONSHIP_ATTRIBUTE_TYPES;
 
 
 public final class  ModelTestUtil {
@@ -441,6 +438,67 @@ public final class  ModelTestUtil {
         return ret;
     }
 
+    /**
+     * Valid types for attributes in relationships. This has good coverage of all attribute type and includes enums
+     * maps and arrays.
+     * This test does not use getRandomBuiltInType() style - so that it is deterministic.
+     *
+     * It does cover very nested maps / arrays.
+     * @param attrNamePrefix
+     * @return
+     */
+    public static List<AtlasAttributeDef> newAttributeDefsWithAllBuiltInTypesForRelationship(String attrNamePrefix) {
+        List<AtlasAttributeDef> ret = new ArrayList<>();
+
+        // add enum types
+        ret.add(getAttributeDef(attrNamePrefix, ENUM_DEF.getName()));
+        ret.add(getAttributeDef(attrNamePrefix, ENUM_DEF_WITH_NO_DEFAULT.getName()));
+        // add array of enum types
+        ret.add(getAttributeDef(attrNamePrefix, AtlasBaseTypeDef.getArrayTypeName(ENUM_DEF.getName())));
+        ret.add(getAttributeDef(attrNamePrefix, AtlasBaseTypeDef.getArrayTypeName(ENUM_DEF_WITH_NO_DEFAULT.getName())));
+
+
+        for (String attributeType : ATLAS_RELATIONSHIP_ATTRIBUTE_TYPES) {
+
+            // simple attributes
+            ret.add(getAttributeDef(attrNamePrefix, attributeType));
+            // array
+            ret.add(getAttributeDef(attrNamePrefix, AtlasBaseTypeDef.getArrayTypeName(attributeType)));
+            // map types with enum as key
+            ret.add(getAttributeDef(attrNamePrefix,
+                    AtlasBaseTypeDef.getMapTypeName(ENUM_DEF.getName(), attributeType)));
+            // map types with enum as key no default
+            ret.add(getAttributeDef(attrNamePrefix,
+                    AtlasBaseTypeDef.getMapTypeName(ENUM_DEF_WITH_NO_DEFAULT.getName(), attributeType)));
+             // map types attribute as key enum no default as value
+            ret.add(getAttributeDef(attrNamePrefix,
+                    AtlasBaseTypeDef.getMapTypeName(attributeType, ENUM_DEF_WITH_NO_DEFAULT.getName())));
+            // map types with enum as value
+            ret.add(getAttributeDef(attrNamePrefix,
+                    AtlasBaseTypeDef.getMapTypeName(attributeType, ENUM_DEF.getName())));
+
+            for (String attributeType2 : ATLAS_RELATIONSHIP_ATTRIBUTE_TYPES) {
+                // add map types
+                ret.add(getAttributeDef(attrNamePrefix,
+                        AtlasBaseTypeDef.getMapTypeName(attributeType, attributeType2)));
+                // add array of arrays
+                ret.add(getAttributeDef(attrNamePrefix,
+                        AtlasBaseTypeDef.getArrayTypeName(AtlasBaseTypeDef.getArrayTypeName(attributeType2))));
+                // add array of maps
+                ret.add(getAttributeDef(attrNamePrefix, AtlasBaseTypeDef.getArrayTypeName(
+                        AtlasBaseTypeDef.getMapTypeName(attributeType, attributeType2))));
+                // add map of arrays
+                ret.add(getAttributeDef(attrNamePrefix, AtlasBaseTypeDef.getMapTypeName(attributeType,
+                        AtlasBaseTypeDef.getArrayTypeName(attributeType2))));
+                // add map of maps
+                ret.add(getAttributeDef(attrNamePrefix, AtlasBaseTypeDef.getMapTypeName(attributeType,
+                        AtlasBaseTypeDef.getMapTypeName(attributeType, attributeType2))));
+            }
+        }
+
+        return ret;
+    }
+
     public static String getDefaultAttributeName(String attrType) {
         return PREFIX_ATTRIBUTE_NAME + attrType;
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/test/java/org/apache/atlas/model/typedef/TestAtlasRelationshipDef.java
----------------------------------------------------------------------
diff --git a/intg/src/test/java/org/apache/atlas/model/typedef/TestAtlasRelationshipDef.java b/intg/src/test/java/org/apache/atlas/model/typedef/TestAtlasRelationshipDef.java
new file mode 100644
index 0000000..78efa6e
--- /dev/null
+++ b/intg/src/test/java/org/apache/atlas/model/typedef/TestAtlasRelationshipDef.java
@@ -0,0 +1,96 @@
+/**
+ * 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.atlas.model.typedef;
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.ModelTestUtil;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef.RelationshipCategory;
+import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality;
+import org.apache.atlas.type.AtlasType;
+import org.testng.annotations.Test;
+
+import java.util.List;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotEquals;
+import static org.testng.Assert.assertNotNull;
+
+public class TestAtlasRelationshipDef {
+
+    private static final String PREFIX_ATTRIBUTE_NAME = "reltests-";
+    private List<AtlasStructDef.AtlasAttributeDef> attributeDefs;
+
+    @Test
+    public void testRelationshipDefSerDeEmpty() throws AtlasBaseException {
+
+        AtlasRelationshipEndPointDef ep1 = new AtlasRelationshipEndPointDef("typeA", "attr1", Cardinality.SINGLE);
+        AtlasRelationshipEndPointDef ep2 = new AtlasRelationshipEndPointDef("typeB", "attr2", Cardinality.SINGLE);
+        AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                RelationshipCategory.ASSOCIATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+
+        String jsonString = AtlasType.toJson(relationshipDef);
+        System.out.println(jsonString);
+        assertNotNull(jsonString);
+
+        AtlasRelationshipDef relationshipDef2 = AtlasType.fromJson(jsonString, AtlasRelationshipDef.class);
+        String jsonString2 = AtlasType.toJson(relationshipDef2);
+
+        assertEquals(jsonString, jsonString2);
+        assertEquals(relationshipDef2, relationshipDef,
+                "Incorrect serialization/deserialization of AtlasRelationshipDef");
+    }
+
+    @Test
+    public void testRelationshipDefSerDeAttributes() throws AtlasBaseException {
+
+        AtlasRelationshipEndPointDef ep1 = new AtlasRelationshipEndPointDef("typeA", "attr1", Cardinality.SINGLE);
+        AtlasRelationshipEndPointDef ep2 = new AtlasRelationshipEndPointDef("typeB", "attr2", Cardinality.SINGLE);
+        AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                RelationshipCategory.ASSOCIATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+        relationshipDef.setAttributeDefs(
+                ModelTestUtil.newAttributeDefsWithAllBuiltInTypesForRelationship(PREFIX_ATTRIBUTE_NAME));
+        String jsonString = AtlasType.toJson(relationshipDef);
+        assertNotNull(jsonString);
+
+        AtlasRelationshipDef relationshipDef2 = AtlasType.fromJson(jsonString, AtlasRelationshipDef.class);
+        String jsonString2 = AtlasType.toJson(relationshipDef2);
+
+        assertEquals(jsonString, jsonString2);
+        assertEquals(relationshipDef2, relationshipDef,
+                "Incorrect serialization/deserialization of AtlasRelationshipDef");
+    }
+    @Test
+    public void testRelationshipEquals() throws AtlasBaseException {
+
+        AtlasRelationshipEndPointDef ep1 = new AtlasRelationshipEndPointDef("typeA", "attr1", Cardinality.SINGLE);
+        AtlasRelationshipEndPointDef ep2 = new AtlasRelationshipEndPointDef("typeB", "attr2", Cardinality.SINGLE);
+        AtlasRelationshipDef relationshipDef1 = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                RelationshipCategory.ASSOCIATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+        List<AtlasStructDef.AtlasAttributeDef> attributeDefs = ModelTestUtil.newAttributeDefsWithAllBuiltInTypesForRelationship(PREFIX_ATTRIBUTE_NAME);
+        relationshipDef1.setAttributeDefs(attributeDefs);
+        AtlasRelationshipDef relationshipDef2 = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                RelationshipCategory.ASSOCIATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+        relationshipDef2.setAttributeDefs(attributeDefs);
+        assertEquals(relationshipDef1,relationshipDef2);
+
+        AtlasRelationshipDef relationshipDef3 = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                RelationshipCategory.ASSOCIATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+
+        assertNotEquals(relationshipDef1,relationshipDef3);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/intg/src/test/java/org/apache/atlas/type/TestAtlasRelationshipType.java
----------------------------------------------------------------------
diff --git a/intg/src/test/java/org/apache/atlas/type/TestAtlasRelationshipType.java b/intg/src/test/java/org/apache/atlas/type/TestAtlasRelationshipType.java
new file mode 100644
index 0000000..6af374a
--- /dev/null
+++ b/intg/src/test/java/org/apache/atlas/type/TestAtlasRelationshipType.java
@@ -0,0 +1,107 @@
+/**
+ * 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.atlas.type;
+
+import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef;
+import org.apache.atlas.model.typedef.AtlasRelationshipEndPointDef;
+import org.apache.atlas.model.typedef.AtlasStructDef;
+import org.testng.annotations.Test;
+
+import static org.testng.AssertJUnit.fail;
+public class TestAtlasRelationshipType {
+    @Test
+    public void testvalidateAtlasRelationshipDef() throws AtlasBaseException {
+        AtlasRelationshipEndPointDef ep1 = new AtlasRelationshipEndPointDef("typeA", "attr1", AtlasStructDef.AtlasAttributeDef.Cardinality.SINGLE);
+        AtlasRelationshipEndPointDef ep2 = new AtlasRelationshipEndPointDef("typeB", "attr2", AtlasStructDef.AtlasAttributeDef.Cardinality.SINGLE);
+        AtlasRelationshipEndPointDef ep3 = new AtlasRelationshipEndPointDef("typeC", "attr2", AtlasStructDef.AtlasAttributeDef.Cardinality.SINGLE, true);
+        AtlasRelationshipEndPointDef ep4 = new AtlasRelationshipEndPointDef("typeD", "attr2", AtlasStructDef.AtlasAttributeDef.Cardinality.SINGLE, true);
+        AtlasRelationshipEndPointDef ep5 = new AtlasRelationshipEndPointDef("typeA", "attr1", AtlasStructDef.AtlasAttributeDef.Cardinality.SET,true);
+        AtlasRelationshipEndPointDef ep6 = new AtlasRelationshipEndPointDef("typeA", "attr1", AtlasStructDef.AtlasAttributeDef.Cardinality.LIST,true);
+        AtlasRelationshipDef relationshipDef1 = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                AtlasRelationshipDef.RelationshipCategory.ASSOCIATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+        AtlasRelationshipDef relationshipDef2 = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                AtlasRelationshipDef.RelationshipCategory.COMPOSITION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep3);
+        AtlasRelationshipDef relationshipDef3 = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                AtlasRelationshipDef.RelationshipCategory.AGGREGATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep3);
+
+        try {
+            AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                    AtlasRelationshipDef.RelationshipCategory.ASSOCIATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep3, ep2);
+            AtlasRelationshipType.validateAtlasRelationshipDef(relationshipDef);
+            fail("This call is expected to fail");
+        } catch (AtlasBaseException abe) {
+            if (!abe.getAtlasErrorCode().equals(AtlasErrorCode.RELATIONSHIPDEF_ASSOCIATION_AND_CONTAINER)) {
+                fail("This call expected a different error");
+            }
+        }
+        try {
+            AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                    AtlasRelationshipDef.RelationshipCategory.COMPOSITION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+            AtlasRelationshipType.validateAtlasRelationshipDef(relationshipDef);
+            fail("This call is expected to fail");
+        } catch (AtlasBaseException abe) {
+            if (!abe.getAtlasErrorCode().equals(AtlasErrorCode.RELATIONSHIPDEF_COMPOSITION_NO_CONTAINER)) {
+                fail("This call expected a different error");
+            }
+        }
+        try {
+            AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                    AtlasRelationshipDef.RelationshipCategory.AGGREGATION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep2);
+            AtlasRelationshipType.validateAtlasRelationshipDef(relationshipDef);
+            fail("This call is expected to fail");
+        } catch (AtlasBaseException abe) {
+            if (!abe.getAtlasErrorCode().equals(AtlasErrorCode.RELATIONSHIPDEF_AGGREGATION_NO_CONTAINER)) {
+                fail("This call expected a different error");
+            }
+        }
+
+        try {
+            AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                    AtlasRelationshipDef.RelationshipCategory.COMPOSITION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep5);
+            AtlasRelationshipType.validateAtlasRelationshipDef(relationshipDef);
+            fail("This call is expected to fail");
+        } catch (AtlasBaseException abe) {
+            if (!abe.getAtlasErrorCode().equals(AtlasErrorCode.RELATIONSHIPDEF_COMPOSITION_SET_CONTAINER)) {
+                fail("This call expected a different error");
+            }
+        }
+        try {
+            AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                    AtlasRelationshipDef.RelationshipCategory.COMPOSITION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep1, ep6);
+            AtlasRelationshipType.validateAtlasRelationshipDef(relationshipDef);
+            fail("This call is expected to fail");
+        } catch (AtlasBaseException abe) {
+            if (!abe.getAtlasErrorCode().equals(AtlasErrorCode.RELATIONSHIPDEF_LIST_ON_ENDPOINT)) {
+                fail("This call expected a different error");
+            }
+        }
+        try {
+            AtlasRelationshipDef relationshipDef = new AtlasRelationshipDef("emptyRelationshipDef", "desc 1", "version1",
+                    AtlasRelationshipDef.RelationshipCategory.COMPOSITION, AtlasRelationshipDef.PropagateTags.ONE_TO_TWO, ep6, ep1);
+            AtlasRelationshipType.validateAtlasRelationshipDef(relationshipDef);
+            fail("This call is expected to fail");
+        } catch (AtlasBaseException abe) {
+            if (!abe.getAtlasErrorCode().equals(AtlasErrorCode.RELATIONSHIPDEF_LIST_ON_ENDPOINT)) {
+                fail("This call expected a different error");
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/6b9399e0/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasRelationshipDefStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasRelationshipDefStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasRelationshipDefStore.java
new file mode 100644
index 0000000..df67958
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasRelationshipDefStore.java
@@ -0,0 +1,52 @@
+/**
+ * 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.atlas.repository.store.graph;
+
+import java.util.List;
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.typedef.AtlasRelationshipDef;
+
+/**
+ * Interface for graph persistence store for AtlasRelationshipDef
+ */
+public interface AtlasRelationshipDefStore {
+    Object preCreate(AtlasRelationshipDef relationshipDef) throws AtlasBaseException;
+
+    AtlasRelationshipDef create(AtlasRelationshipDef relationshipDef, Object preCreateResult) throws AtlasBaseException;
+
+    List<AtlasRelationshipDef> getAll() throws AtlasBaseException;
+
+    AtlasRelationshipDef getByName(String name) throws AtlasBaseException;
+
+    AtlasRelationshipDef getByGuid(String guid) throws AtlasBaseException;
+
+    AtlasRelationshipDef update(AtlasRelationshipDef relationshipDef) throws AtlasBaseException;
+
+    AtlasRelationshipDef updateByName(String name, AtlasRelationshipDef relationshipDef) throws AtlasBaseException;
+
+    AtlasRelationshipDef updateByGuid(String guid, AtlasRelationshipDef relationshipDef) throws AtlasBaseException;
+
+    Object preDeleteByName(String name) throws AtlasBaseException;
+
+    void deleteByName(String name, Object preDeleteResult) throws AtlasBaseException;
+
+    Object preDeleteByGuid(String guid) throws AtlasBaseException;
+
+    void deleteByGuid(String guid, Object preDeleteResult) throws AtlasBaseException;
+}