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/09/11 21:34:33 UTC

[5/5] atlas git commit: ATLAS-2083: Refactor AtlasDefStore classes to reduce code duplication

ATLAS-2083: Refactor Atlas<Type>DefStore classes to reduce code duplication

Signed-off-by: Madhan Neethiraj <ma...@apache.org>
(cherry picked from commit 15d67e52f118ce689724f62ac3c83cfd3128e34f)


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

Branch: refs/heads/branch-0.8
Commit: 5ca7955a1f3bcd2a84381889ad450ca2ea4c6372
Parents: 711baea
Author: Richard Ding <sd...@us.ibm.com>
Authored: Fri Aug 25 11:37:27 2017 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Mon Sep 11 14:34:14 2017 -0700

----------------------------------------------------------------------
 .../graph/AtlasClassificationDefStore.java      | 52 --------------
 .../repository/store/graph/AtlasDefStore.java   | 53 ++++++++++++++
 .../store/graph/AtlasEntityDefStore.java        | 52 --------------
 .../store/graph/AtlasEnumDefStore.java          | 46 ------------
 .../store/graph/AtlasStructDefStore.java        | 52 --------------
 .../store/graph/AtlasTypeDefGraphStore.java     | 51 +++++++-------
 .../store/graph/v1/AtlasAbstractDefStoreV1.java | 34 ++++++++-
 .../graph/v1/AtlasClassificationDefStoreV1.java | 55 +--------------
 .../store/graph/v1/AtlasEntityDefStoreV1.java   | 55 +--------------
 .../store/graph/v1/AtlasEnumDefStoreV1.java     | 48 ++++++-------
 .../store/graph/v1/AtlasStructDefStoreV1.java   | 55 +--------------
 .../graph/v1/AtlasTypeDefGraphStoreV1.java      | 16 ++---
 .../test/java/org/apache/atlas/TestModules.java | 14 +---
 .../store/graph/AtlasEntityDefStoreV1Test.java  | 73 --------------------
 .../graph/v1/AtlasEntityDefStoreV1Test.java     | 72 +++++++++++++++++++
 15 files changed, 224 insertions(+), 504 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasClassificationDefStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasClassificationDefStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasClassificationDefStore.java
deleted file mode 100644
index 5721534..0000000
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasClassificationDefStore.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.atlas.repository.store.graph;
-
-import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.typedef.AtlasClassificationDef;
-
-import java.util.List;
-
-/**
- * Interface for graph persistence store for AtlasClassificationDef
- */
-public interface AtlasClassificationDefStore {
-    Object preCreate(AtlasClassificationDef classificationDef) throws AtlasBaseException;
-
-    AtlasClassificationDef create(AtlasClassificationDef classifiDef, Object preCreateResult) throws AtlasBaseException;
-
-    List<AtlasClassificationDef> getAll() throws AtlasBaseException;
-
-    AtlasClassificationDef getByName(String name) throws AtlasBaseException;
-
-    AtlasClassificationDef getByGuid(String guid) throws AtlasBaseException;
-
-    AtlasClassificationDef update(AtlasClassificationDef classifiDef) throws AtlasBaseException;
-
-    AtlasClassificationDef updateByName(String name, AtlasClassificationDef classifiDef) throws AtlasBaseException;
-
-    AtlasClassificationDef updateByGuid(String guid, AtlasClassificationDef classifiDef) 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;
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasDefStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasDefStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasDefStore.java
new file mode 100644
index 0000000..7df3c86
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasDefStore.java
@@ -0,0 +1,53 @@
+/**
+ * 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 org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+
+import java.util.List;
+
+/**
+ * Interface for graph persistence store for AtlasTypeDef
+ */
+public interface AtlasDefStore<T extends AtlasBaseTypeDef> {
+    AtlasVertex preCreate(T typeDef) throws AtlasBaseException;
+
+    T create(T typeDef, AtlasVertex preCreateResult) throws AtlasBaseException;
+
+    List<T> getAll() throws AtlasBaseException;
+
+    T getByName(String name) throws AtlasBaseException;
+
+    T getByGuid(String guid) throws AtlasBaseException;
+
+    T update(T typeDef) throws AtlasBaseException;
+
+    T updateByName(String name, T typeDef) throws AtlasBaseException;
+
+    T updateByGuid(String guid, T typeDef) throws AtlasBaseException;
+
+    AtlasVertex preDeleteByName(String name) throws AtlasBaseException;
+
+    void deleteByName(String name, AtlasVertex preDeleteResult) throws AtlasBaseException;
+
+    AtlasVertex preDeleteByGuid(String guid) throws AtlasBaseException;
+
+    void deleteByGuid(String guid, AtlasVertex preDeleteResult) throws AtlasBaseException;
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStore.java
deleted file mode 100644
index 7cabb349..0000000
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStore.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.atlas.repository.store.graph;
-
-import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.typedef.AtlasEntityDef;
-
-import java.util.List;
-
-/**
- * Interface for graph persistence store for AtlasEntityDef
- */
-public interface AtlasEntityDefStore {
-    Object preCreate(AtlasEntityDef entityDef) throws AtlasBaseException;
-
-    AtlasEntityDef create(AtlasEntityDef entityDef, Object preCreateResult) throws AtlasBaseException;
-
-    List<AtlasEntityDef> getAll() throws AtlasBaseException;
-
-    AtlasEntityDef getByName(String name) throws AtlasBaseException;
-
-    AtlasEntityDef getByGuid(String guid) throws AtlasBaseException;
-
-    AtlasEntityDef update(AtlasEntityDef entityDef) throws AtlasBaseException;
-
-    AtlasEntityDef updateByName(String name, AtlasEntityDef entityDef) throws AtlasBaseException;
-
-    AtlasEntityDef updateByGuid(String guid, AtlasEntityDef entityDef) 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;
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEnumDefStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEnumDefStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEnumDefStore.java
deleted file mode 100644
index f543eb5..0000000
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEnumDefStore.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.atlas.repository.store.graph;
-
-import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.typedef.AtlasEnumDef;
-
-import java.util.List;
-
-/**
- * Interface for graph persistence store for AtlasEnumDef
- */
-public interface AtlasEnumDefStore {
-    AtlasEnumDef create(AtlasEnumDef enumDef) throws AtlasBaseException;
-
-    List<AtlasEnumDef> getAll() throws AtlasBaseException;
-
-    AtlasEnumDef getByName(String name) throws AtlasBaseException;
-
-    AtlasEnumDef getByGuid(String guid) throws AtlasBaseException;
-
-    AtlasEnumDef update(AtlasEnumDef enumDef) throws AtlasBaseException;
-
-    AtlasEnumDef updateByName(String name, AtlasEnumDef enumDef) throws AtlasBaseException;
-
-    AtlasEnumDef updateByGuid(String guid, AtlasEnumDef enumDef) throws AtlasBaseException;
-
-    void deleteByName(String name) throws AtlasBaseException;
-
-    void deleteByGuid(String guid) throws AtlasBaseException;
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasStructDefStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasStructDefStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasStructDefStore.java
deleted file mode 100644
index 9e74d09..0000000
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasStructDefStore.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.atlas.repository.store.graph;
-
-import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.typedef.AtlasStructDef;
-
-import java.util.List;
-
-/**
- * Interface for graph persistence store for AtlasStructDef
- */
-public interface AtlasStructDefStore {
-    Object preCreate(AtlasStructDef structDef) throws AtlasBaseException;
-
-    AtlasStructDef create(AtlasStructDef structDef, Object preCreateResult) throws AtlasBaseException;
-
-    List<AtlasStructDef> getAll() throws AtlasBaseException;
-
-    AtlasStructDef getByName(String name) throws AtlasBaseException;
-
-    AtlasStructDef getByGuid(String guid) throws AtlasBaseException;
-
-    AtlasStructDef update(AtlasStructDef structDef) throws AtlasBaseException;
-
-    AtlasStructDef updateByName(String name, AtlasStructDef structDef) throws AtlasBaseException;
-
-    AtlasStructDef updateByGuid(String guid, AtlasStructDef structDef) throws AtlasBaseException;
-
-    Object preDeleteByName(String name) throws AtlasBaseException;
-
-    void deleteByName(String name, Object preDeleteResult) throws AtlasBaseException;
-
-    Object preDeleteByGuid(String name) throws AtlasBaseException;
-
-    void deleteByGuid(String guid, Object preDeleteResult) throws AtlasBaseException;
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/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 d024bfd..4669286 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
@@ -32,6 +32,7 @@ import org.apache.atlas.model.typedef.AtlasStructDef;
 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.graphdb.AtlasVertex;
 import org.apache.atlas.repository.util.FilterUtil;
 import org.apache.atlas.store.AtlasTypeDefStore;
 import org.apache.atlas.type.AtlasClassificationType;
@@ -76,13 +77,13 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore {
         this.typeUpdateLockMaxWaitTimeSeconds = AtlasRepositoryConfiguration.getTypeUpdateLockMaxWaitTimeInSeconds();
     }
 
-    protected abstract AtlasEnumDefStore getEnumDefStore(AtlasTypeRegistry typeRegistry);
+    protected abstract AtlasDefStore<AtlasEnumDef> getEnumDefStore(AtlasTypeRegistry typeRegistry);
 
-    protected abstract AtlasStructDefStore getStructDefStore(AtlasTypeRegistry typeRegistry);
+    protected abstract AtlasDefStore<AtlasStructDef> getStructDefStore(AtlasTypeRegistry typeRegistry);
 
-    protected abstract AtlasClassificationDefStore getClassificationDefStore(AtlasTypeRegistry typeRegistry);
+    protected abstract AtlasDefStore<AtlasClassificationDef> getClassificationDefStore(AtlasTypeRegistry typeRegistry);
 
-    protected abstract AtlasEntityDefStore getEntityDefStore(AtlasTypeRegistry typeRegistry);
+    protected abstract AtlasDefStore<AtlasEntityDef> getEntityDefStore(AtlasTypeRegistry typeRegistry);
 
     @Override
     public void init() throws AtlasBaseException {
@@ -438,14 +439,14 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore {
 
         AtlasTransientTypeRegistry ttr = lockTypeRegistryAndReleasePostCommit();
 
-        AtlasEnumDefStore           enumDefStore     = getEnumDefStore(ttr);
-        AtlasStructDefStore         structDefStore   = getStructDefStore(ttr);
-        AtlasClassificationDefStore classifiDefStore = getClassificationDefStore(ttr);
-        AtlasEntityDefStore         entityDefStore   = getEntityDefStore(ttr);
+        AtlasDefStore<AtlasEnumDef>           enumDefStore     = getEnumDefStore(ttr);
+        AtlasDefStore<AtlasStructDef>         structDefStore   = getStructDefStore(ttr);
+        AtlasDefStore<AtlasClassificationDef> classifiDefStore = getClassificationDefStore(ttr);
+        AtlasDefStore<AtlasEntityDef>         entityDefStore   = getEntityDefStore(ttr);
 
-        List<Object> preDeleteStructDefs   = new ArrayList<>();
-        List<Object> preDeleteClassifiDefs = new ArrayList<>();
-        List<Object> preDeleteEntityDefs   = new ArrayList<>();
+        List<AtlasVertex> preDeleteStructDefs   = new ArrayList<>();
+        List<AtlasVertex> preDeleteClassifiDefs = new ArrayList<>();
+        List<AtlasVertex> preDeleteEntityDefs   = new ArrayList<>();
 
         if (CollectionUtils.isNotEmpty(typesDef.getStructDefs())) {
             for (AtlasStructDef structDef : typesDef.getStructDefs()) {
@@ -516,9 +517,9 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore {
         if (CollectionUtils.isNotEmpty(typesDef.getEnumDefs())) {
             for (AtlasEnumDef enumDef : typesDef.getEnumDefs()) {
                 if (StringUtils.isNotBlank(enumDef.getGuid())) {
-                    enumDefStore.deleteByGuid(enumDef.getGuid());
+                    enumDefStore.deleteByGuid(enumDef.getGuid(), null);
                 } else {
-                    enumDefStore.deleteByName(enumDef.getName());
+                    enumDefStore.deleteByName(enumDef.getName(), null);
                 }
             }
         }
@@ -721,18 +722,18 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore {
     private AtlasTypesDef addToGraphStore(AtlasTypesDef typesDef, AtlasTransientTypeRegistry ttr) throws AtlasBaseException {
         AtlasTypesDef ret = new AtlasTypesDef();
 
-        AtlasEnumDefStore           enumDefStore     = getEnumDefStore(ttr);
-        AtlasStructDefStore         structDefStore   = getStructDefStore(ttr);
-        AtlasClassificationDefStore classifiDefStore = getClassificationDefStore(ttr);
-        AtlasEntityDefStore         entityDefStore   = getEntityDefStore(ttr);
+        AtlasDefStore<AtlasEnumDef>           enumDefStore     = getEnumDefStore(ttr);
+        AtlasDefStore<AtlasStructDef>         structDefStore   = getStructDefStore(ttr);
+        AtlasDefStore<AtlasClassificationDef> classifiDefStore = getClassificationDefStore(ttr);
+        AtlasDefStore<AtlasEntityDef>         entityDefStore   = getEntityDefStore(ttr);
 
-        List<Object> preCreateStructDefs   = new ArrayList<>();
-        List<Object> preCreateClassifiDefs = new ArrayList<>();
-        List<Object> preCreateEntityDefs   = new ArrayList<>();
+        List<AtlasVertex> preCreateStructDefs   = new ArrayList<>();
+        List<AtlasVertex> preCreateClassifiDefs = new ArrayList<>();
+        List<AtlasVertex> preCreateEntityDefs   = new ArrayList<>();
 
         if (CollectionUtils.isNotEmpty(typesDef.getEnumDefs())) {
             for (AtlasEnumDef enumDef : typesDef.getEnumDefs()) {
-                AtlasEnumDef createdDef = enumDefStore.create(enumDef);
+                AtlasEnumDef createdDef = enumDefStore.create(enumDef, null);
 
                 ttr.updateGuid(createdDef.getName(), createdDef.getGuid());
 
@@ -800,10 +801,10 @@ public abstract class AtlasTypeDefGraphStore implements AtlasTypeDefStore {
     private AtlasTypesDef updateGraphStore(AtlasTypesDef typesDef, AtlasTransientTypeRegistry ttr) throws AtlasBaseException {
         AtlasTypesDef ret = new AtlasTypesDef();
 
-        AtlasEnumDefStore           enumDefStore     = getEnumDefStore(ttr);
-        AtlasStructDefStore         structDefStore   = getStructDefStore(ttr);
-        AtlasClassificationDefStore classifiDefStore = getClassificationDefStore(ttr);
-        AtlasEntityDefStore         entityDefStore   = getEntityDefStore(ttr);
+        AtlasDefStore<AtlasEnumDef>           enumDefStore     = getEnumDefStore(ttr);
+        AtlasDefStore<AtlasStructDef>         structDefStore   = getStructDefStore(ttr);
+        AtlasDefStore<AtlasClassificationDef> classifiDefStore = getClassificationDefStore(ttr);
+        AtlasDefStore<AtlasEntityDef>         entityDefStore   = getEntityDefStore(ttr);
 
         if (CollectionUtils.isNotEmpty(typesDef.getEnumDefs())) {
             for (AtlasEnumDef enumDef : typesDef.getEnumDefs()) {

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasAbstractDefStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasAbstractDefStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasAbstractDefStoreV1.java
index baf2309..2bf53a1 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasAbstractDefStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasAbstractDefStoreV1.java
@@ -24,6 +24,8 @@ import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
 import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.query.QueryParser;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.atlas.repository.store.graph.AtlasDefStore;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,7 +37,7 @@ import java.util.regex.Pattern;
 /**
  * Abstract typedef-store for v1 format.
  */
-public abstract class AtlasAbstractDefStoreV1 {
+  abstract class AtlasAbstractDefStoreV1 <T extends AtlasBaseTypeDef> implements AtlasDefStore<T> {
     private static final Logger LOG = LoggerFactory.getLogger(AtlasAbstractDefStoreV1.class);
     protected final AtlasTypeDefGraphStoreV1 typeDefStore;
     protected final AtlasTypeRegistry        typeRegistry;
@@ -77,4 +79,34 @@ public abstract class AtlasAbstractDefStoreV1 {
 
         return m.matches();
     }
+
+    @Override
+    public void deleteByName(String name, AtlasVertex preDeleteResult) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasAbstractDefStoreV1.deleteByName({}, {})", name, preDeleteResult);
+        }
+
+        AtlasVertex vertex = (preDeleteResult == null) ? preDeleteByName(name) : preDeleteResult;
+
+        typeDefStore.deleteTypeVertex(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasAbstractDefStoreV1.deleteByName({}, {})", name, preDeleteResult);
+        }
+    }
+
+    @Override
+    public void deleteByGuid(String guid, AtlasVertex preDeleteResult) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> AtlasAbstractDefStoreV1.deleteByGuid({}, {})", guid, preDeleteResult);
+        }
+
+        AtlasVertex vertex = (preDeleteResult == null) ? preDeleteByGuid(guid) : preDeleteResult;
+
+        typeDefStore.deleteTypeVertex(vertex);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasAbstractDefStoreV1.deleteByGuid({}, {})", guid, preDeleteResult);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasClassificationDefStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasClassificationDefStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasClassificationDefStoreV1.java
index 8944504..e3aa4e0 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasClassificationDefStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasClassificationDefStoreV1.java
@@ -23,7 +23,6 @@ import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.typedef.AtlasClassificationDef;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
-import org.apache.atlas.repository.store.graph.AtlasClassificationDefStore;
 import org.apache.atlas.type.AtlasClassificationType;
 import org.apache.atlas.type.AtlasType;
 import org.apache.atlas.type.AtlasTypeRegistry;
@@ -41,7 +40,7 @@ import java.util.regex.Pattern;
 /**
  * ClassificationDef store in v1 format.
  */
-public class AtlasClassificationDefStoreV1 extends AtlasAbstractDefStoreV1 implements AtlasClassificationDefStore {
+class AtlasClassificationDefStoreV1 extends AtlasAbstractDefStoreV1<AtlasClassificationDef> {
     private static final Logger LOG = LoggerFactory.getLogger(AtlasClassificationDefStoreV1.class);
 
     private static final String  TRAIT_NAME_REGEX   = "[a-zA-Z][a-zA-Z0-9_ .]*";
@@ -84,19 +83,13 @@ public class AtlasClassificationDefStoreV1 extends AtlasAbstractDefStoreV1 imple
     }
 
     @Override
-    public AtlasClassificationDef create(AtlasClassificationDef classificationDef, Object preCreateResult)
+    public AtlasClassificationDef create(AtlasClassificationDef classificationDef, AtlasVertex preCreateResult)
         throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> AtlasClassificationDefStoreV1.create({}, {})", classificationDef, preCreateResult);
         }
 
-        AtlasVertex vertex;
-
-        if (preCreateResult == null || !(preCreateResult instanceof AtlasVertex)) {
-            vertex = preCreate(classificationDef);
-        } else {
-            vertex = (AtlasVertex)preCreateResult;
-        }
+        AtlasVertex vertex = (preCreateResult == null) ? preCreate(classificationDef) : preCreateResult;
 
         updateVertexAddReferences(classificationDef, vertex);
 
@@ -281,27 +274,6 @@ public class AtlasClassificationDefStoreV1 extends AtlasAbstractDefStoreV1 imple
     }
 
     @Override
-    public void deleteByName(String name, Object preDeleteResult) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasClassificationDefStoreV1.deleteByName({})", name);
-        }
-
-        AtlasVertex vertex;
-
-        if (preDeleteResult == null || !(preDeleteResult instanceof AtlasVertex)) {
-            vertex = preDeleteByName(name);
-        } else {
-            vertex = (AtlasVertex)preDeleteResult;
-        }
-
-        typeDefStore.deleteTypeVertex(vertex);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("<== AtlasClassificationDefStoreV1.deleteByName({})", name);
-        }
-    }
-
-    @Override
     public AtlasVertex preDeleteByGuid(String guid) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> AtlasClassificationDefStoreV1.preDeleteByGuid({})", guid);
@@ -328,27 +300,6 @@ public class AtlasClassificationDefStoreV1 extends AtlasAbstractDefStoreV1 imple
         return ret;
     }
 
-    @Override
-    public void deleteByGuid(String guid, Object preDeleteResult) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasClassificationDefStoreV1.deleteByGuid({})", guid);
-        }
-
-        AtlasVertex vertex;
-
-        if (preDeleteResult == null || !(preDeleteResult instanceof AtlasVertex)) {
-            vertex = preDeleteByGuid(guid);
-        } else {
-            vertex = (AtlasVertex)preDeleteResult;
-        }
-
-        typeDefStore.deleteTypeVertex(vertex);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("<== AtlasClassificationDefStoreV1.deleteByGuid({})", guid);
-        }
-    }
-
     private void updateVertexPreCreate(AtlasClassificationDef  classificationDef,
                                        AtlasClassificationType classificationType,
                                        AtlasVertex             vertex) throws AtlasBaseException {

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1.java
index 8046234..5fca674 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1.java
@@ -22,7 +22,6 @@ import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.typedef.AtlasEntityDef;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
-import org.apache.atlas.repository.store.graph.AtlasEntityDefStore;
 import org.apache.atlas.type.AtlasEntityType;
 import org.apache.atlas.type.AtlasType;
 import org.apache.atlas.type.AtlasTypeRegistry;
@@ -39,7 +38,7 @@ import java.util.List;
 /**
  * EntityDef store in v1 format.
  */
-public class AtlasEntityDefStoreV1 extends AtlasAbstractDefStoreV1 implements AtlasEntityDefStore {
+public class AtlasEntityDefStoreV1 extends AtlasAbstractDefStoreV1<AtlasEntityDef> {
     private static final Logger LOG = LoggerFactory.getLogger(AtlasEntityDefStoreV1.class);
 
     @Inject
@@ -79,18 +78,12 @@ public class AtlasEntityDefStoreV1 extends AtlasAbstractDefStoreV1 implements At
     }
 
     @Override
-    public AtlasEntityDef create(AtlasEntityDef entityDef, Object preCreateResult) throws AtlasBaseException {
+    public AtlasEntityDef create(AtlasEntityDef entityDef, AtlasVertex preCreateResult) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> AtlasEntityDefStoreV1.create({}, {})", entityDef, preCreateResult);
         }
 
-        AtlasVertex vertex;
-
-        if (preCreateResult == null || !(preCreateResult instanceof AtlasVertex)) {
-            vertex = preCreate(entityDef);
-        } else {
-            vertex = (AtlasVertex)preCreateResult;
-        }
+        AtlasVertex vertex = (preCreateResult == null) ? preCreate(entityDef) : preCreateResult;
 
         updateVertexAddReferences(entityDef, vertex);
 
@@ -275,27 +268,6 @@ public class AtlasEntityDefStoreV1 extends AtlasAbstractDefStoreV1 implements At
     }
 
     @Override
-    public void deleteByName(String name, Object preDeleteResult) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasEntityDefStoreV1.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("<== AtlasEntityDefStoreV1.deleteByName({}, {})", name, preDeleteResult);
-        }
-    }
-
-    @Override
     public AtlasVertex preDeleteByGuid(String guid) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> AtlasEntityDefStoreV1.preDeleteByGuid({})", guid);
@@ -322,27 +294,6 @@ public class AtlasEntityDefStoreV1 extends AtlasAbstractDefStoreV1 implements At
         return ret;
     }
 
-    @Override
-    public void deleteByGuid(String guid, Object preDeleteResult) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasEntityDefStoreV1.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("<== AtlasEntityDefStoreV1.deleteByGuid({}, {})", guid, preDeleteResult);
-        }
-    }
-
     private void updateVertexPreCreate(AtlasEntityDef entityDef, AtlasEntityType entityType, AtlasVertex vertex) throws AtlasBaseException {
         AtlasStructDefStoreV1.updateVertexPreCreate(entityDef, entityType, vertex, typeDefStore);
     }

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEnumDefStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEnumDefStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEnumDefStoreV1.java
index 54d14de..5bd9c12 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEnumDefStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEnumDefStoreV1.java
@@ -23,7 +23,6 @@ import org.apache.atlas.model.typedef.AtlasEnumDef;
 import org.apache.atlas.model.typedef.AtlasEnumDef.AtlasEnumElementDef;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
-import org.apache.atlas.repository.store.graph.AtlasEnumDefStore;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.typesystem.types.DataTypes.TypeCategory;
 import org.apache.commons.collections.CollectionUtils;
@@ -38,7 +37,7 @@ import java.util.List;
 /**
  * EnumDef store in v1 format.
  */
-public class AtlasEnumDefStoreV1 extends AtlasAbstractDefStoreV1 implements AtlasEnumDefStore {
+class AtlasEnumDefStoreV1 extends AtlasAbstractDefStoreV1<AtlasEnumDef> {
     private static final Logger LOG = LoggerFactory.getLogger(AtlasEnumDefStoreV1.class);
 
     public AtlasEnumDefStoreV1(AtlasTypeDefGraphStoreV1 typeDefStore, AtlasTypeRegistry typeRegistry) {
@@ -46,9 +45,9 @@ public class AtlasEnumDefStoreV1 extends AtlasAbstractDefStoreV1 implements Atla
     }
 
     @Override
-    public AtlasEnumDef create(AtlasEnumDef enumDef) throws AtlasBaseException {
+    public AtlasVertex preCreate(AtlasEnumDef enumDef) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasEnumDefStoreV1.create({})", enumDef);
+          LOG.debug("==> AtlasEnumDefStoreV1.preCreate({})", enumDef);
         }
 
         validateType(enumDef);
@@ -63,10 +62,25 @@ public class AtlasEnumDefStoreV1 extends AtlasAbstractDefStoreV1 implements Atla
 
         toVertex(enumDef, vertex);
 
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== AtlasEnumDefStoreV1.preCreate({}): {}", enumDef, vertex);
+        }
+
+        return vertex;
+    }
+
+    @Override
+    public AtlasEnumDef create(AtlasEnumDef enumDef, AtlasVertex preCreateResult) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("==> AtlasEnumDefStoreV1.create({}, {})", enumDef, preCreateResult);
+        }
+
+        AtlasVertex vertex = (preCreateResult == null) ? preCreate(enumDef) : preCreateResult;
+
         AtlasEnumDef ret = toEnumDef(vertex);
 
         if (LOG.isDebugEnabled()) {
-            LOG.debug("<== AtlasEnumDefStoreV1.create({}): {}", enumDef, ret);
+            LOG.debug("<== AtlasEntityDefStoreV1.create({}, {}): {}", enumDef, preCreateResult, ret);
         }
 
         return ret;
@@ -209,41 +223,25 @@ public class AtlasEnumDefStoreV1 extends AtlasAbstractDefStoreV1 implements Atla
     }
 
     @Override
-    public void deleteByName(String name) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasEnumDefStoreV1.deleteByName({})", name);
-        }
-
+    public AtlasVertex preDeleteByName(String name) throws AtlasBaseException {
         AtlasVertex vertex = typeDefStore.findTypeVertexByNameAndCategory(name, TypeCategory.ENUM);
 
         if (vertex == null) {
             throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, name);
         }
 
-        typeDefStore.deleteTypeVertex(vertex);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("<== AtlasEnumDefStoreV1.deleteByName({})", name);
-        }
+        return vertex;
     }
 
     @Override
-    public void deleteByGuid(String guid) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasEnumDefStoreV1.deleteByGuid({})", guid);
-        }
-
+    public AtlasVertex preDeleteByGuid(String guid) throws AtlasBaseException {
         AtlasVertex vertex = typeDefStore.findTypeVertexByGuidAndCategory(guid, TypeCategory.ENUM);
 
         if (vertex == null) {
             throw new AtlasBaseException(AtlasErrorCode.TYPE_GUID_NOT_FOUND, guid);
         }
 
-        typeDefStore.deleteTypeVertex(vertex);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("<== AtlasEnumDefStoreV1.deleteByGuid({})", guid);
-        }
+        return vertex;
     }
 
     private void toVertex(AtlasEnumDef enumDef, AtlasVertex vertex) throws AtlasBaseException {

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
index 1c6cfc7..137fb30 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
@@ -26,7 +26,6 @@ import org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
-import org.apache.atlas.repository.store.graph.AtlasStructDefStore;
 import org.apache.atlas.type.AtlasStructType;
 import org.apache.atlas.type.AtlasStructType.AtlasAttribute;
 import org.apache.atlas.type.AtlasType;
@@ -51,7 +50,7 @@ import java.util.Set;
 /**
  * StructDef store in v1 format.
  */
-public class AtlasStructDefStoreV1 extends AtlasAbstractDefStoreV1 implements AtlasStructDefStore {
+public class AtlasStructDefStoreV1 extends AtlasAbstractDefStoreV1<AtlasStructDef> {
     private static final Logger LOG = LoggerFactory.getLogger(AtlasStructDefStoreV1.class);
 
     public AtlasStructDefStoreV1(AtlasTypeDefGraphStoreV1 typeDefStore, AtlasTypeRegistry typeRegistry) {
@@ -90,18 +89,12 @@ public class AtlasStructDefStoreV1 extends AtlasAbstractDefStoreV1 implements At
     }
 
     @Override
-    public AtlasStructDef create(AtlasStructDef structDef, Object preCreateResult) throws AtlasBaseException {
+    public AtlasStructDef create(AtlasStructDef structDef, AtlasVertex preCreateResult) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> AtlasStructDefStoreV1.create({}, {})", structDef, preCreateResult);
         }
 
-        AtlasVertex vertex;
-
-        if (preCreateResult == null || !(preCreateResult instanceof AtlasVertex)) {
-            vertex = preCreate(structDef);
-        } else {
-            vertex = (AtlasVertex)preCreateResult;
-        }
+        AtlasVertex vertex = (preCreateResult == null) ? preCreate(structDef) : preCreateResult;
 
         if (CollectionUtils.isEmpty(structDef.getAttributeDefs())) {
             throw new AtlasBaseException(AtlasErrorCode.BAD_REQUEST, "Missing attributes for structdef");
@@ -289,27 +282,6 @@ public class AtlasStructDefStoreV1 extends AtlasAbstractDefStoreV1 implements At
     }
 
     @Override
-    public void deleteByName(String name, Object preDeleteResult) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasStructDefStoreV1.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("<== AtlasStructDefStoreV1.deleteByName({}, {})", name, preDeleteResult);
-        }
-    }
-
-    @Override
     public AtlasVertex preDeleteByGuid(String guid) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> AtlasStructDefStoreV1.preDeleteByGuid({})", guid);
@@ -336,27 +308,6 @@ public class AtlasStructDefStoreV1 extends AtlasAbstractDefStoreV1 implements At
         return ret;
     }
 
-    @Override
-    public void deleteByGuid(String guid, Object preDeleteResult) throws AtlasBaseException {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("==> AtlasStructDefStoreV1.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("<== AtlasStructDefStoreV1.deleteByGuid({}, {})", guid, preDeleteResult);
-        }
-    }
-
     private AtlasStructDef toStructDef(AtlasVertex vertex) throws AtlasBaseException {
         AtlasStructDef ret = null;
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/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 49a1f35..6e9ef68 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
@@ -23,17 +23,13 @@ import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.RequestContextV1;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.listener.TypeDefChangeListener;
-import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+import org.apache.atlas.model.typedef.*;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
 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.AtlasStructDefStore;
-import org.apache.atlas.repository.store.graph.AtlasTypeDefGraphStore;
+import org.apache.atlas.repository.store.graph.*;
 import org.apache.atlas.type.AtlasType;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.typesystem.types.DataTypes.TypeCategory;
@@ -79,22 +75,22 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
     }
 
     @Override
-    protected AtlasEnumDefStore getEnumDefStore(AtlasTypeRegistry typeRegistry) {
+    protected AtlasDefStore<AtlasEnumDef> getEnumDefStore(AtlasTypeRegistry typeRegistry) {
         return new AtlasEnumDefStoreV1(this, typeRegistry);
     }
 
     @Override
-    protected AtlasStructDefStore getStructDefStore(AtlasTypeRegistry typeRegistry) {
+    protected AtlasDefStore<AtlasStructDef> getStructDefStore(AtlasTypeRegistry typeRegistry) {
         return new AtlasStructDefStoreV1(this, typeRegistry);
     }
 
     @Override
-    protected AtlasClassificationDefStore getClassificationDefStore(AtlasTypeRegistry typeRegistry) {
+    protected AtlasDefStore<AtlasClassificationDef> getClassificationDefStore(AtlasTypeRegistry typeRegistry) {
         return new AtlasClassificationDefStoreV1(this, typeRegistry);
     }
 
     @Override
-    protected AtlasEntityDefStore getEntityDefStore(AtlasTypeRegistry typeRegistry) {
+    protected AtlasDefStore<AtlasEntityDef> getEntityDefStore(AtlasTypeRegistry typeRegistry) {
         return new AtlasEntityDefStoreV1(this, typeRegistry);
     }
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/test/java/org/apache/atlas/TestModules.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/TestModules.java b/repository/src/test/java/org/apache/atlas/TestModules.java
index 512d994..9efbb77 100644
--- a/repository/src/test/java/org/apache/atlas/TestModules.java
+++ b/repository/src/test/java/org/apache/atlas/TestModules.java
@@ -40,16 +40,8 @@ import org.apache.atlas.repository.graph.HardDeleteHandler;
 import org.apache.atlas.repository.graph.SoftDeleteHandler;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
 import org.apache.atlas.repository.impexp.ExportService;
-import org.apache.atlas.repository.store.graph.AtlasEntityDefStore;
-import org.apache.atlas.repository.store.graph.AtlasEntityStore;
-import org.apache.atlas.repository.store.graph.v1.AtlasEntityChangeNotifier;
-import org.apache.atlas.repository.store.graph.v1.AtlasEntityDefStoreV1;
-import org.apache.atlas.repository.store.graph.v1.AtlasEntityStoreV1;
-import org.apache.atlas.repository.store.graph.v1.AtlasTypeDefGraphStoreV1;
-import org.apache.atlas.repository.store.graph.v1.DeleteHandlerV1;
-import org.apache.atlas.repository.store.graph.v1.EntityGraphMapper;
-import org.apache.atlas.repository.store.graph.v1.HardDeleteHandlerV1;
-import org.apache.atlas.repository.store.graph.v1.SoftDeleteHandlerV1;
+import org.apache.atlas.repository.store.graph.*;
+import org.apache.atlas.repository.store.graph.v1.*;
 import org.apache.atlas.repository.typestore.GraphBackedTypeStore;
 import org.apache.atlas.repository.typestore.ITypeStore;
 import org.apache.atlas.repository.typestore.StoreBackedTypeCache;
@@ -128,8 +120,6 @@ public class TestModules {
             bind(ITypeStore.class).to(GraphBackedTypeStore.class).asEagerSingleton();
             bind(AtlasTypeDefStore.class).to(AtlasTypeDefGraphStoreV1.class).asEagerSingleton();
 
-            //For testing
-            bind(AtlasEntityDefStore.class).to(AtlasEntityDefStoreV1.class).asEagerSingleton();
             bind(AtlasTypeRegistry.class).asEagerSingleton();
             bind(EntityGraphMapper.class).asEagerSingleton();
             bind(ExportService.class).asEagerSingleton();

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStoreV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStoreV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStoreV1Test.java
deleted file mode 100644
index 5f01136..0000000
--- a/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasEntityDefStoreV1Test.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.atlas.repository.store.graph;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.inject.Inject;
-import org.apache.atlas.ApplicationProperties;
-import org.apache.atlas.AtlasErrorCode;
-import org.apache.atlas.AtlasException;
-import org.apache.atlas.TestModules;
-import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.typedef.AtlasEntityDef;
-import org.apache.atlas.repository.graph.AtlasGraphProvider;
-import org.apache.atlas.repository.store.graph.v1.AtlasAbstractDefStoreV1;
-import org.apache.atlas.type.AtlasTypeUtil;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.DataProvider;
-import org.testng.annotations.Guice;
-import org.testng.annotations.Test;
-
-/**
- * Tests for AtlasEntityStoreV1
- */
-@Guice(modules = TestModules.TestOnlyModule.class)
-public class AtlasEntityDefStoreV1Test {
-
-    @Inject
-    private
-    AtlasEntityDefStore entityDefStore;
-
-    @DataProvider
-    public Object[][] invalidAttributeNameWithReservedKeywords(){
-        AtlasEntityDef invalidAttrNameType =
-            AtlasTypeUtil.createClassTypeDef("Invalid_Attribute_Type", "description", ImmutableSet.<String>of(),
-                AtlasTypeUtil.createRequiredAttrDef("order", "string"),
-                AtlasTypeUtil.createRequiredAttrDef("limit", "string"));
-
-        return new Object[][] {{
-            invalidAttrNameType
-        }};
-    }
-
-    @Test(dataProvider = "invalidAttributeNameWithReservedKeywords")
-    public void testCreateTypeWithReservedKeywords(AtlasEntityDef atlasEntityDef) throws AtlasException {
-        try {
-            ApplicationProperties.get().setProperty(AtlasAbstractDefStoreV1.ALLOW_RESERVED_KEYWORDS, false);
-            entityDefStore.create(atlasEntityDef, null);
-        } catch (AtlasBaseException e) {
-            Assert.assertEquals(e.getAtlasErrorCode(), AtlasErrorCode.ATTRIBUTE_NAME_INVALID);
-        }
-    }
-
-    @AfterClass
-    public void clear(){
-        AtlasGraphProvider.cleanup();
-    }
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/5ca7955a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1Test.java
new file mode 100644
index 0000000..95b5761
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityDefStoreV1Test.java
@@ -0,0 +1,72 @@
+/**
+ * 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 com.google.common.collect.ImmutableSet;
+import com.google.inject.Inject;
+import org.apache.atlas.ApplicationProperties;
+import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.TestModules;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.typedef.AtlasEntityDef;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
+import org.apache.atlas.type.AtlasTypeUtil;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+/**
+ * Tests for AtlasEntityStoreV1
+ */
+@Guice(modules = TestModules.TestOnlyModule.class)
+public class AtlasEntityDefStoreV1Test {
+
+    @Inject
+    private
+    AtlasEntityDefStoreV1 entityDefStore;
+
+    @DataProvider
+    public Object[][] invalidAttributeNameWithReservedKeywords(){
+        AtlasEntityDef invalidAttrNameType =
+            AtlasTypeUtil.createClassTypeDef("Invalid_Attribute_Type", "description", ImmutableSet.<String>of(),
+                AtlasTypeUtil.createRequiredAttrDef("order", "string"),
+                AtlasTypeUtil.createRequiredAttrDef("limit", "string"));
+
+        return new Object[][] {{
+            invalidAttrNameType
+        }};
+    }
+
+    @Test(dataProvider = "invalidAttributeNameWithReservedKeywords")
+    public void testCreateTypeWithReservedKeywords(AtlasEntityDef atlasEntityDef) throws AtlasException {
+        try {
+            ApplicationProperties.get().setProperty(AtlasAbstractDefStoreV1.ALLOW_RESERVED_KEYWORDS, false);
+            entityDefStore.create(atlasEntityDef, null);
+        } catch (AtlasBaseException e) {
+            Assert.assertEquals(e.getAtlasErrorCode(), AtlasErrorCode.ATTRIBUTE_NAME_INVALID);
+        }
+    }
+
+    @AfterClass
+    public void clear(){
+        AtlasGraphProvider.cleanup();
+    }
+}