You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by am...@apache.org on 2018/02/16 22:31:18 UTC

atlas git commit: ATLAS-2422: Export Improvement: Add ability to export all entities of the specified type.

Repository: atlas
Updated Branches:
  refs/heads/branch-0.8 435338ccc -> 11ed3ccd3


ATLAS-2422: Export Improvement: Add ability to export all entities of the specified type.


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

Branch: refs/heads/branch-0.8
Commit: 11ed3ccd32290020c34d55b415f5fb69901e5105
Parents: 435338c
Author: Ashutosh Mestry <am...@hortonworks.com>
Authored: Fri Feb 16 14:30:44 2018 -0800
Committer: Ashutosh Mestry <am...@hortonworks.com>
Committed: Fri Feb 16 14:30:50 2018 -0800

----------------------------------------------------------------------
 .../atlas/model/impexp/AtlasExportRequest.java  |   1 +
 .../atlas/repository/impexp/ExportService.java  | 116 +++++++++++--------
 .../atlas/util/AtlasGremlin2QueryProvider.java  |   2 +
 .../atlas/util/AtlasGremlinQueryProvider.java   |   1 +
 .../repository/impexp/ExportServiceTest.java    |  41 ++++++-
 .../impexp/ImportServiceReportingTest.java      |  43 -------
 .../impexp/ImportServiceTestUtils.java          | 116 -------------------
 7 files changed, 112 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/atlas/blob/11ed3ccd/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportRequest.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportRequest.java b/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportRequest.java
index a015e9b..f1ed822 100644
--- a/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportRequest.java
+++ b/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportRequest.java
@@ -54,6 +54,7 @@ public class AtlasExportRequest implements Serializable {
     public static final String MATCH_TYPE_ENDS_WITH   = "endsWith";
     public static final String MATCH_TYPE_CONTAINS    = "contains";
     public static final String MATCH_TYPE_MATCHES     = "matches";
+    public static final String MATCH_TYPE_FOR_TYPE     = "forType";
 
     private List<AtlasObjectId> itemsToExport = new ArrayList<>();
     private Map<String, Object> options       = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/atlas/blob/11ed3ccd/repository/src/main/java/org/apache/atlas/repository/impexp/ExportService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/impexp/ExportService.java b/repository/src/main/java/org/apache/atlas/repository/impexp/ExportService.java
index a88c09e..ce792e0 100644
--- a/repository/src/main/java/org/apache/atlas/repository/impexp/ExportService.java
+++ b/repository/src/main/java/org/apache/atlas/repository/impexp/ExportService.java
@@ -60,13 +60,7 @@ import org.springframework.stereotype.Component;
 import javax.inject.Inject;
 import javax.script.ScriptEngine;
 import javax.script.ScriptException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 import static org.apache.atlas.model.impexp.AtlasExportRequest.*;
 
@@ -182,19 +176,19 @@ public class ExportService {
         return overall;
     }
 
-    private AtlasExportResult.OperationStatus processObjectId(AtlasObjectId item, ExportContext context) throws AtlasServiceException, AtlasException, AtlasBaseException {
+    private AtlasExportResult.OperationStatus processObjectId(AtlasObjectId item, ExportContext context) {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> processObjectId({})", item);
         }
 
         try {
-            List<AtlasEntityWithExtInfo> entities = getStartingEntity(item, context);
-            if(entities.size() == 0) {
+            List<String> entityGuids = getStartingEntity(item, context);
+            if(entityGuids.size() == 0) {
                 return AtlasExportResult.OperationStatus.FAIL;
             }
 
-            for (AtlasEntityWithExtInfo entityWithExtInfo : entities) {
-                processEntity(entityWithExtInfo.getEntity().getGuid(), context);
+            for (String guid : entityGuids) {
+                processEntity(guid, context);
             }
 
             while (!context.guidsToProcess.isEmpty()) {
@@ -221,74 +215,102 @@ public class ExportService {
         return AtlasExportResult.OperationStatus.SUCCESS;
     }
 
-    private List<AtlasEntityWithExtInfo> getStartingEntity(AtlasObjectId item, ExportContext context) throws AtlasBaseException {
-        List<AtlasEntityWithExtInfo> ret = new ArrayList<>();
+    private List<String> getStartingEntity(AtlasObjectId item, ExportContext context) throws AtlasBaseException {
+        List<String> ret = null;
 
         if (StringUtils.isNotEmpty(item.getGuid())) {
-            AtlasEntityWithExtInfo entity = entityGraphRetriever.toAtlasEntityWithExtInfo(item);
+            ret = Collections.singletonList(item.getGuid());
+        } else if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_FOR_TYPE) && StringUtils.isNotEmpty(item.getTypeName())) {
+            final String queryTemplate = getQueryTemplateForMatchType(context);
 
-            if (entity != null) {
-                ret = Collections.singletonList(entity);
-            }
+            setupBindingsForTypeName(context, item.getTypeName());
+
+            ret = executeGremlinQueryForGuids(queryTemplate, context);
         } else if (StringUtils.isNotEmpty(item.getTypeName()) && MapUtils.isNotEmpty(item.getUniqueAttributes())) {
-            String          typeName   = item.getTypeName();
-            AtlasEntityType entityType = typeRegistry.getEntityTypeByName(typeName);
+            final String          queryTemplate = getQueryTemplateForMatchType(context);
+            final String          typeName      = item.getTypeName();
+            final AtlasEntityType entityType    = typeRegistry.getEntityTypeByName(typeName);
 
             if (entityType == null) {
                 throw new AtlasBaseException(AtlasErrorCode.UNKNOWN_TYPENAME, typeName);
             }
 
-            final String queryTemplate;
-            if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_STARTS_WITH)) {
-                queryTemplate = gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_STARTS_WITH);
-            } else if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_ENDS_WITH)) {
-                queryTemplate = gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_ENDS_WITH);
-            } else if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_CONTAINS)) {
-                queryTemplate = gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_CONTAINS);
-            } else if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_MATCHES)) {
-                queryTemplate = gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_MATCHES);
-            } else { // default
-                queryTemplate = gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_DEFAULT);
-            }
-
             for (Map.Entry<String, Object> e : item.getUniqueAttributes().entrySet()) {
                 String attrName  = e.getKey();
                 Object attrValue = e.getValue();
 
                 AtlasAttribute attribute = entityType.getAttribute(attrName);
-
                 if (attribute == null || attrValue == null) {
                     continue;
                 }
 
-                context.bindings.clear();
-                context.bindings.put("typeName", typeName);
-                context.bindings.put("attrName", attribute.getQualifiedName());
-                context.bindings.put("attrValue", attrValue);
+                setupBindingsForTypeNameAttrNameAttrValue(context, typeName, attrValue, attribute);
 
                 List<String> guids = executeGremlinQueryForGuids(queryTemplate, context);
 
                 if (CollectionUtils.isNotEmpty(guids)) {
-                    for (String guid : guids) {
-                        AtlasEntityWithExtInfo entityWithExtInfo = entityGraphRetriever.toAtlasEntityWithExtInfo(guid);
+                    if (ret == null) {
+                        ret = new ArrayList<>();
+                    }
 
-                        if (entityWithExtInfo == null) {
-                            continue;
+                    for (String guid : guids) {
+                        if (!ret.contains(guid)) {
+                            ret.add(guid);
                         }
-
-                        ret.add(entityWithExtInfo);
                     }
                 }
-
-                break;
             }
+        }
 
-            LOG.info("export(item={}; matchType={}, fetchType={}): found {} entities", item, context.matchType, context.fetchType, ret.size());
+        if (ret == null) {
+            ret = Collections.emptyList();
         }
 
+        logInfoStartingEntitiesFound(item, context, ret);
         return ret;
     }
 
+    private void logInfoStartingEntitiesFound(AtlasObjectId item, ExportContext context, List<String> ret) {
+        LOG.info("export(item={}; matchType={}, fetchType={}): found {} entities", item, context.matchType, context.fetchType, ret.size());
+    }
+
+    private void setupBindingsForTypeName(ExportContext context, String typeName) {
+        context.bindings.clear();
+        context.bindings.put("typeName", new HashSet<String>(Arrays.asList(StringUtils.split(typeName,","))));
+    }
+
+    private void setupBindingsForTypeNameAttrNameAttrValue(ExportContext context,
+                                                           String typeName, Object attrValue, AtlasAttribute attribute) {
+        context.bindings.clear();
+        context.bindings.put("typeName", typeName);
+        context.bindings.put("attrName", attribute.getQualifiedName());
+        context.bindings.put("attrValue", attrValue);
+    }
+
+    private String getQueryTemplateForMatchType(ExportContext context) {
+        if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_STARTS_WITH)) {
+            return gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_STARTS_WITH);
+        }
+
+        if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_ENDS_WITH)) {
+            return gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_ENDS_WITH);
+        }
+
+        if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_CONTAINS)) {
+            return gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_CONTAINS);
+        }
+
+        if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_MATCHES)) {
+            return gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_MATCHES);
+        }
+
+        if (StringUtils.equalsIgnoreCase(context.matchType, MATCH_TYPE_FOR_TYPE)) {
+            return gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_ALL_FOR_TYPE);
+        }
+
+        return gremlinQueryProvider.getQuery(AtlasGremlinQuery.EXPORT_TYPE_DEFAULT);
+    }
+
     private void processEntity(String guid, ExportContext context) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> processEntity({})", guid);

http://git-wip-us.apache.org/repos/asf/atlas/blob/11ed3ccd/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java b/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
index 9cffdb9..5cacbfd 100644
--- a/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
+++ b/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
@@ -43,6 +43,8 @@ public class AtlasGremlin2QueryProvider extends AtlasGremlinQueryProvider {
                 return "g.V('__guid', startGuid).inE().outV().has('__guid').transform{[__guid:it.__guid,isProcess:(it.__superTypeNames != null) ? it.__superTypeNames.contains('Process') : false ]}.dedup().toList()";
             case EXPORT_BY_GUID_CONNECTED_OUT_EDGE:
                 return "g.V('__guid', startGuid).outE().inV().has('__guid').transform{[__guid:it.__guid,isProcess:(it.__superTypeNames != null) ? it.__superTypeNames.contains('Process') : false ]}.dedup().toList()";
+            case EXPORT_TYPE_ALL_FOR_TYPE:
+                return "g.V().has('__typeName',T.in,typeName).has('__guid').__guid.toList()";
             case EXPORT_TYPE_STARTS_WITH:
                 return "g.V().has('__typeName',typeName).filter({it.getProperty(attrName).startsWith(attrValue)}).has('__guid').__guid.toList()";
             case EXPORT_TYPE_ENDS_WITH:

http://git-wip-us.apache.org/repos/asf/atlas/blob/11ed3ccd/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java b/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
index e4898bd..646628f 100644
--- a/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
+++ b/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
@@ -45,6 +45,7 @@ public abstract class AtlasGremlinQueryProvider {
         EXPORT_BY_GUID_FULL,
         EXPORT_BY_GUID_CONNECTED_IN_EDGE,
         EXPORT_BY_GUID_CONNECTED_OUT_EDGE,
+        EXPORT_TYPE_ALL_FOR_TYPE,
         EXPORT_TYPE_STARTS_WITH,
         EXPORT_TYPE_ENDS_WITH,
         EXPORT_TYPE_CONTAINS,

http://git-wip-us.apache.org/repos/asf/atlas/blob/11ed3ccd/repository/src/test/java/org/apache/atlas/repository/impexp/ExportServiceTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/impexp/ExportServiceTest.java b/repository/src/test/java/org/apache/atlas/repository/impexp/ExportServiceTest.java
index 7901ef6..8b68bba 100644
--- a/repository/src/test/java/org/apache/atlas/repository/impexp/ExportServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/impexp/ExportServiceTest.java
@@ -36,6 +36,7 @@ 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.SoftDeleteHandlerV1;
 import org.apache.atlas.store.AtlasTypeDefStore;
+import org.apache.atlas.type.AtlasType;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -261,8 +262,6 @@ public class ExportServiceTest {
 
     @Test
     public void verifyOverallStatus() throws Exception {
-
-//        ExportService service = new ExportService(typeRegistry);
         assertEquals(AtlasExportResult.OperationStatus.FAIL, exportService.getOverallOperationStatus());
 
         assertEquals(AtlasExportResult.OperationStatus.SUCCESS, exportService.getOverallOperationStatus(AtlasExportResult.OperationStatus.SUCCESS));
@@ -299,6 +298,26 @@ public class ExportServiceTest {
         assertEquals(AtlasExportResult.OperationStatus.FAIL, zipSource.getExportResult().getOperationStatus());
     }
 
+    @Test
+    public void requestForTypeFull() {
+        AtlasExportRequest req = getRequestForTypeFull("Department,Employee");
+
+        assertNotNull(req);
+        assertEquals(req.getItemsToExport().size(), 1);
+        assertEquals(req.getOptions().get(AtlasExportRequest.OPTION_ATTR_MATCH_TYPE), "forType");
+    }
+
+    @Test
+    public void verifyTypeFull() throws AtlasBaseException, IOException {
+        ZipSource zipSource = runExportWithParameters(getRequestForTypeFull("Department,Employee,Manager"));
+        verifyExportForFullEmployeeData(zipSource);
+    }
+
+    private AtlasExportRequest getRequestForTypeFull(String type) {
+        String jsonRequest = "{ \"itemsToExport\": [ { \"typeName\": \"%s\" } ], \"options\": {  \"fetchType\": \"FULL\", \"matchType\": \"forType\"} }";
+        return AtlasType.fromJson(String.format(jsonRequest, type), AtlasExportRequest.class);
+    }
+
     private void tamperEmployeeRequest(AtlasExportRequest request) {
         AtlasObjectId objectId = request.getItemsToExport().get(0);
         objectId.getUniqueAttributes().remove("name");
@@ -323,6 +342,24 @@ public class ExportServiceTest {
         verifyTypeDefs(zipSource);
     }
 
+    private void verifyExportForFullEmployeeData(ZipSource zipSource) throws AtlasBaseException {
+        final List<String> expectedEntityTypes = Arrays.asList(new String[]{"Manager", "Employee", "Department"});
+
+        assertNotNull(zipSource.getCreationOrder());
+        assertEquals(zipSource.getCreationOrder().size(), 1);
+        assertTrue(zipSource.hasNext());
+
+        while (zipSource.hasNext()) {
+            AtlasEntity entity = zipSource.next();
+
+            assertNotNull(entity);
+            assertEquals(AtlasEntity.Status.ACTIVE, entity.getStatus());
+            assertTrue(expectedEntityTypes.contains(entity.getTypeName()));
+        }
+
+        verifyTypeDefs(zipSource);
+    }
+
     private void verifyExportForHrData(ZipSource zipSource) throws IOException, AtlasBaseException {
         assertNotNull(zipSource.getCreationOrder());
         assertTrue(zipSource.getCreationOrder().size() == 1);

http://git-wip-us.apache.org/repos/asf/atlas/blob/11ed3ccd/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceReportingTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceReportingTest.java b/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceReportingTest.java
deleted file mode 100644
index 0aaaa70..0000000
--- a/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceReportingTest.java
+++ /dev/null
@@ -1,43 +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.impexp;
-
-import com.google.inject.Inject;
-import org.apache.atlas.TestModules;
-import org.apache.atlas.repository.store.graph.AtlasEntityStore;
-import org.apache.atlas.store.AtlasTypeDefStore;
-import org.apache.atlas.type.AtlasTypeRegistry;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testng.annotations.Guice;
-
-@Guice(modules = TestModules.TestOnlyModule.class)
-public class ImportServiceReportingTest {
-    private static final Logger LOG = LoggerFactory.getLogger(ImportServiceReportingTest.class);
-
-    @Inject
-    AtlasTypeRegistry typeRegistry;
-
-    @Inject
-    private AtlasTypeDefStore typeDefStore;
-
-    @Inject
-    private AtlasEntityStore entityStore;
-
-
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/11ed3ccd/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceTestUtils.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceTestUtils.java b/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceTestUtils.java
deleted file mode 100644
index 7289512..0000000
--- a/repository/src/test/java/org/apache/atlas/repository/impexp/ImportServiceTestUtils.java
+++ /dev/null
@@ -1,116 +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.impexp;
-
-import com.google.common.collect.Sets;
-import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.impexp.AtlasExportResult;
-import org.apache.atlas.model.impexp.AtlasImportRequest;
-import org.apache.atlas.model.impexp.AtlasImportResult;
-import org.apache.atlas.model.typedef.AtlasTypesDef;
-import org.apache.atlas.repository.store.bootstrap.AtlasTypeDefStoreInitializer;
-import org.apache.atlas.store.AtlasTypeDefStore;
-import org.apache.atlas.type.AtlasType;
-import org.apache.atlas.type.AtlasTypeRegistry;
-import org.testng.Assert;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class ImportServiceTestUtils {
-
-    public static void verifyImportedEntities(List<String> creationOrder, List<String> processedEntities) {
-        Set<String> lhs = com.google.common.collect.Sets.newHashSet(creationOrder);
-        Set<String> rhs = com.google.common.collect.Sets.newHashSet(processedEntities);
-        Set<String> difference = Sets.difference(lhs, rhs);
-
-        Assert.assertNotNull(difference);
-        Assert.assertEquals(difference.size(), 0);
-    }
-
-    public static void verifyImportedMetrics(AtlasExportResult exportResult, AtlasImportResult importResult) {
-        Map<String, Integer> metricsForCompare = getImportMetricsForCompare(importResult);
-        for (Map.Entry<String, Integer> entry : exportResult.getMetrics().entrySet()) {
-            if(entry.getKey().startsWith("entity") == false ||
-                    entry.getKey().contains("withExtInfo") ||
-                    entry.getKey().contains("Column") ||
-                    entry.getKey().contains("StorageDesc")) continue;
-
-            Assert.assertTrue(metricsForCompare.containsKey(entry.getKey()));
-            Assert.assertEquals(entry.getValue(), metricsForCompare.get(entry.getKey()));
-        }
-    }
-
-    private static Map<String,Integer> getImportMetricsForCompare(AtlasImportResult result) {
-        Map<String, Integer> r = new HashMap<>();
-        for (Map.Entry<String, Integer> entry : result.getMetrics().entrySet()) {
-            r.put(entry.getKey().replace(":updated", "").replace(":created", ""), entry.getValue());
-        }
-
-        return r;
-    }
-
-
-    public static void loadModelFromJson(String fileName, AtlasTypeDefStore typeDefStore, AtlasTypeRegistry typeRegistry) throws IOException, AtlasBaseException {
-        AtlasTypesDef typesFromJson = getAtlasTypesDefFromFile(fileName);
-        createTypesAsNeeded(typesFromJson, typeDefStore, typeRegistry);
-    }
-
-    private static void createTypesAsNeeded(AtlasTypesDef typesFromJson, AtlasTypeDefStore typeDefStore, AtlasTypeRegistry typeRegistry) throws AtlasBaseException {
-        AtlasTypesDef typesToCreate = AtlasTypeDefStoreInitializer.getTypesToCreate(typesFromJson, typeRegistry);
-
-        if (!typesToCreate.isEmpty()) {
-            typeDefStore.createTypesDef(typesToCreate);
-        }
-    }
-
-    private static AtlasTypesDef getAtlasTypesDefFromFile(String fileName) throws IOException {
-        String sampleTypes = ZipFileResourceTestUtils.getModelJson(fileName);
-        return AtlasType.fromJson(sampleTypes, AtlasTypesDef.class);
-    }
-
-    public static AtlasImportRequest getDefaultImportRequest() {
-        return new AtlasImportRequest();
-    }
-
-
-    public static AtlasImportResult runImportWithParameters(ImportService importService, AtlasImportRequest request, ZipSource source) throws AtlasBaseException, IOException {
-        final String requestingIP = "1.0.0.0";
-        final String hostName = "localhost";
-        final String userName = "admin";
-
-        AtlasImportResult result = importService.run(source, request, userName, hostName, requestingIP);
-        Assert.assertEquals(result.getOperationStatus(), AtlasImportResult.OperationStatus.SUCCESS);
-        return result;
-    }
-
-    public static void runAndVerifyQuickStart_v1_Import(ImportService importService, ZipSource zipSource) throws AtlasBaseException, IOException {
-        AtlasExportResult exportResult = zipSource.getExportResult();
-        List<String> creationOrder = zipSource.getCreationOrder();
-
-        AtlasImportRequest request = getDefaultImportRequest();
-        AtlasImportResult result = runImportWithParameters(importService, request, zipSource);
-
-        Assert.assertNotNull(result);
-        verifyImportedMetrics(exportResult, result);
-        verifyImportedEntities(creationOrder, result.getProcessedEntities());
-    }
-}