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/02/10 02:34:32 UTC

incubator-atlas git commit: ATLAS-1503: implementation of REST API for export

Repository: incubator-atlas
Updated Branches:
  refs/heads/master b63d6316e -> 9cb56c541


ATLAS-1503: implementation of REST API for export

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/9cb56c54
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/9cb56c54
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/9cb56c54

Branch: refs/heads/master
Commit: 9cb56c541e26eb35ff3ab486de31cb20076fe1fe
Parents: b63d631
Author: ashutoshm <am...@hortonworks.com>
Authored: Thu Feb 9 13:17:28 2017 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Thu Feb 9 18:34:20 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/atlas/AtlasErrorCode.java   |   3 +-
 .../atlas/model/impexp/AtlasExportRequest.java  |  94 +++++++
 .../atlas/model/impexp/AtlasExportResult.java   | 251 +++++++++++++++++++
 .../atlas/model/typedef/AtlasTypesDef.java      |  96 ++++++-
 .../store/graph/v1/AtlasEntityStoreV1.java      |  22 +-
 .../store/graph/v1/AtlasGraphUtilsV1.java       |  30 ++-
 .../store/graph/v1/EntityGraphRetriever.java    |  32 ++-
 .../atlas/web/resources/AdminResource.java      |  85 +++++--
 .../atlas/web/resources/ExportService.java      | 202 +++++++++++++++
 .../org/apache/atlas/web/resources/ZipSink.java | 109 ++++++++
 .../org/apache/atlas/web/util/Servlets.java     |  25 ++
 .../atlas/web/resources/AdminExportTest.java    |  25 ++
 .../atlas/web/resources/AdminResourceTest.java  |   4 +-
 13 files changed, 922 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/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 7b11b70..aed5140 100644
--- a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
+++ b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
@@ -59,7 +59,8 @@ public enum AtlasErrorCode {
     CONSTRAINT_INVERSE_REF_INVERSE_ATTRIBUTE_NON_EXISTING(400, "ATLAS40028E", "{0}.{1}: invalid {2} constraint. Inverse attribute {3}.{4} does not exist"),
     CONSTRAINT_INVERSE_REF_INVERSE_ATTRIBUTE_INVALID_TYPE(400, "ATLAS40029E", "{0}.{1}: invalid {2} constraint. Inverse attribute {3}.{4} is not an entity type"),
     CONSTRAINT_OWNED_REF_ATTRIBUTE_INVALID_TYPE(400, "ATLAS40030E", "{0}.{1}: invalid {2} constraint. Attribute {3} is not an entity type"),
-    CANNOT_MAP_ATTRIBUTE(400, "ATLAS40027E", "cannot map attribute: {0} of type: {1} from vertex"),
+    CANNOT_MAP_ATTRIBUTE(400, "ATLAS40031E", "cannot map attribute: {0} of type: {1} from vertex"),
+    INVALID_OBJECT_ID(400, "ATLAS40032E", "ObjectId is not valid {0}"),
 
     // All Not found enums go here
     TYPE_NAME_NOT_FOUND(404, "ATLAS4041E", "Given typename {0} was invalid"),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/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
new file mode 100644
index 0000000..dcb2765
--- /dev/null
+++ b/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportRequest.java
@@ -0,0 +1,94 @@
+/**
+ * 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.impexp;
+
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+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 static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
+
+
+@JsonAutoDetect(getterVisibility=PUBLIC_ONLY, setterVisibility=PUBLIC_ONLY, fieldVisibility=NONE)
+@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown=true)
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.PROPERTY)
+public class AtlasExportRequest implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public static final String EXPORT_PARAM_OPTION             = "FORMAT";
+    public static final String EXPORT_PARAM_OPTION_FORMAT_JSON = "JSON";
+    public static final String EXPORT_PARAM_OPTION_FORMAT_ZIP  = "ZIP";
+
+
+    private List<AtlasObjectId> itemsToExport = new ArrayList<>();
+    private Map<String, Object> options       = new HashMap<>();
+
+    public List<AtlasObjectId> getItemsToExport() {
+        return itemsToExport;
+    }
+
+    public void setItemsToExport(List<AtlasObjectId> itemsToExport) {
+        this.itemsToExport = itemsToExport;
+    }
+
+    public Map<String, Object> getOptions() {
+        return options;
+    }
+
+    public void setOptions(Map<String, Object> options) {
+        this.options = options;
+    }
+
+    public StringBuilder toString(StringBuilder sb) {
+        if (sb == null) {
+            sb = new StringBuilder();
+        }
+
+        sb.append("AtlasExportRequest{");
+        sb.append("itemsToExport={");
+        AtlasBaseTypeDef.dumpObjects(itemsToExport, sb);
+        sb.append("}");
+        sb.append("options={");
+        AtlasBaseTypeDef.dumpObjects(options, sb);
+        sb.append("}");
+        sb.append("}");
+
+        return sb;
+    }
+
+    @Override
+    public String toString() {
+        return toString(new StringBuilder()).toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportResult.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportResult.java b/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportResult.java
new file mode 100644
index 0000000..d89dd36
--- /dev/null
+++ b/intg/src/main/java/org/apache/atlas/model/impexp/AtlasExportResult.java
@@ -0,0 +1,251 @@
+/**
+ * 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.impexp;
+
+
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+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.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
+
+
+@JsonAutoDetect(getterVisibility=PUBLIC_ONLY, setterVisibility=PUBLIC_ONLY, fieldVisibility=NONE)
+@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown=true)
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.PROPERTY)
+public class AtlasExportResult implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public final static String ENTITY_COUNT = "entityCount";
+
+    public enum OperationStatus {
+        SUCCESS, PARTIAL_SUCCESS, FAIL
+    }
+
+    private AtlasExportRequest   request;
+    private String               userName;
+    private String               clientIpAddress;
+    private String               hostName;
+    private long                 timeStamp;
+    private Map<String, Integer> metrics;
+    private AtlasExportData      data;
+    private OperationStatus      operationStatus;
+
+
+    public AtlasExportResult() {
+        this(null, null, null, null, System.currentTimeMillis());
+    }
+
+    public AtlasExportResult(AtlasExportRequest request,
+                             String userName, String clientIpAddress, String hostName, long timeStamp) {
+        this.request         = request;
+        this.userName        = userName;
+        this.clientIpAddress = clientIpAddress;
+        this.hostName        = hostName;
+        this.timeStamp       = timeStamp;
+        this.metrics         = new HashMap<>();
+        this.operationStatus = OperationStatus.FAIL;
+        this.data            = new AtlasExportData();
+    }
+
+    public AtlasExportRequest getRequest() {
+        return request;
+    }
+
+    public void setRequest(AtlasExportRequest request) {
+        this.request = request;
+    }
+
+    public String getUserName() {
+        return userName;
+    }
+
+    public void setUserName(String userName) {
+        this.userName = userName;
+    }
+
+    public String getClientIpAddress() {
+        return clientIpAddress;
+    }
+
+    public void setClientIpAddress(String clientIpAddress) {
+        this.clientIpAddress = clientIpAddress;
+    }
+
+    public String getHostName() {
+        return hostName;
+    }
+
+    public void setHostName(String hostName) {
+        this.hostName = hostName;
+    }
+
+    public long getTimeStamp() {
+        return timeStamp;
+    }
+
+    public void setTimeStamp(long timeStamp) {
+        this.timeStamp = timeStamp;
+    }
+
+    public Map<String, Integer> getMetrics() {
+        return metrics;
+    }
+
+    public void setMetrics(Map<String, Integer> metrics) {
+        this.metrics = metrics;
+    }
+
+    public AtlasExportData getData() {
+        return data;
+    }
+
+    public void setData(AtlasExportData data) {
+        this.data = data;
+    }
+
+    public OperationStatus getOperationStatus() {
+        return operationStatus;
+    }
+
+    public void setOperationStatus(OperationStatus operationStatus) {
+        this.operationStatus = operationStatus;
+    }
+
+
+    public void incrementMeticsCounter(String key) {
+        incrementMeticsCounter(key, 1);
+    }
+
+    public void incrementMeticsCounter(String key, int incrementBy) {
+        int currentValue = metrics.containsKey(key) ? metrics.get(key) : 0;
+
+        metrics.put(key, currentValue + incrementBy);
+    }
+
+    public StringBuilder toString(StringBuilder sb) {
+        if (sb == null) {
+            sb = new StringBuilder();
+        }
+
+        sb.append("AtlasExportResult{");
+        sb.append("request={").append(request).append("}");
+        sb.append(", userName='").append(userName).append("'");
+        sb.append(", clientIpAddress='").append(clientIpAddress).append("'");
+        sb.append(", hostName='").append(hostName).append("'");
+        sb.append(", timeStamp='").append(timeStamp).append("'");
+        sb.append(", metrics={");
+        AtlasBaseTypeDef.dumpObjects(metrics, sb);
+        sb.append("}");
+
+        sb.append(", data='").append(data).append("'");
+        sb.append(", operationStatus='").append(operationStatus).append("'");
+        sb.append("}");
+
+        return sb;
+    }
+
+    @Override
+    public String toString() {
+        return toString(new StringBuilder()).toString();
+    }
+
+    @JsonAutoDetect(getterVisibility=PUBLIC_ONLY, setterVisibility=PUBLIC_ONLY, fieldVisibility=NONE)
+    @JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+    @JsonIgnoreProperties(ignoreUnknown=true)
+    @XmlRootElement
+    @XmlAccessorType(XmlAccessType.PROPERTY)
+    public static class AtlasExportData implements Serializable{
+        private static final long serialVersionUID = 1L;
+
+        private AtlasTypesDef             typesDef;
+        private Map<String, AtlasEntity>  entities;
+        private Map<String, List<String>> entityCreationOrder;
+
+
+        public AtlasExportData() {
+            typesDef            = new AtlasTypesDef();
+            entities            = new HashMap<>();
+            entityCreationOrder = new HashMap<>();
+        }
+
+        public AtlasTypesDef getTypesDef() { return typesDef; }
+
+        public void setTypesDef(AtlasTypesDef typesDef) { this.typesDef = typesDef; }
+
+        public Map<String, AtlasEntity> getEntities() { return entities; }
+
+        public void setEntities(Map<String, AtlasEntity> entities) { this.entities = entities; }
+
+        public Map<String, List<String>> getEntityCreationOrder() { return entityCreationOrder; }
+
+        public void setEntityCreationOrder(Map<String, List<String>> entityCreationOrder) { this.entityCreationOrder = entityCreationOrder; }
+
+
+        public void clear() {
+            if (entityCreationOrder != null) {
+                entityCreationOrder.clear();
+            }
+
+            if (typesDef != null) {
+                typesDef.clear();
+            }
+
+            if (entities != null) {
+                entities.clear();
+            }
+        }
+
+        public StringBuilder toString(StringBuilder sb) {
+            if (sb == null) {
+                sb = new StringBuilder();
+            }
+
+            sb.append("AtlasExportData{");
+            sb.append("typesDef={").append(typesDef).append("}");
+            sb.append("entities={");
+            AtlasBaseTypeDef.dumpObjects(entities, sb);
+            sb.append("}");
+            sb.append("entityCreationOrder={");
+            AtlasBaseTypeDef.dumpObjects(entityCreationOrder, sb);
+            sb.append("}");
+            sb.append("}");
+
+            return sb;
+        }
+
+        @Override
+        public String toString() {
+            return toString(new StringBuilder()).toString();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/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 899e53f..af95bff 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
@@ -24,6 +24,7 @@ import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -45,21 +46,23 @@ public class AtlasTypesDef {
     private List<AtlasEntityDef>         entityDefs;
 
     public AtlasTypesDef() {
-        enumDefs = new ArrayList<>();
-        structDefs = new ArrayList<>();
+        enumDefs           = new ArrayList<>();
+        structDefs         = new ArrayList<>();
         classificationDefs = new ArrayList<>();
-        entityDefs = new ArrayList<>();
+        entityDefs         = new ArrayList<>();
     }
 
-    public AtlasTypesDef(List<AtlasEnumDef> enumDefs, List<AtlasStructDef> structDefs,
+    public AtlasTypesDef(List<AtlasEnumDef>           enumDefs,
+                         List<AtlasStructDef>         structDefs,
                          List<AtlasClassificationDef> classificationDefs,
-                         List<AtlasEntityDef> entityDefs) {
+                         List<AtlasEntityDef>         entityDefs) {
         this.enumDefs           = enumDefs;
         this.structDefs         = structDefs;
         this.classificationDefs = classificationDefs;
         this.entityDefs         = entityDefs;
     }
 
+
     public List<AtlasEnumDef> getEnumDefs() {
         return enumDefs;
     }
@@ -80,10 +83,6 @@ public class AtlasTypesDef {
         return classificationDefs;
     }
 
-    public void setClassificationDefs(List<AtlasClassificationDef> classificationDefs) {
-        this.classificationDefs = classificationDefs;
-    }
-
     public List<AtlasEntityDef> getEntityDefs() {
         return entityDefs;
     }
@@ -92,6 +91,40 @@ public class AtlasTypesDef {
         this.entityDefs = entityDefs;
     }
 
+    public void setClassificationDefs(List<AtlasClassificationDef> classificationDefs) {
+        this.classificationDefs = classificationDefs;
+    }
+
+
+    public boolean hasClassificationDef(String name) {
+        return hasTypeDef(classificationDefs, name);
+    }
+
+    public boolean hasEnumDef(String name) {
+        return hasTypeDef(enumDefs, name);
+    }
+
+    public boolean hasStructDef(String name) {
+        return hasTypeDef(structDefs, name);
+    }
+
+    public boolean hasEntityDef(String name) {
+        return hasTypeDef(entityDefs, name);
+    }
+
+
+    private <T extends AtlasBaseTypeDef> boolean hasTypeDef(Collection<T> typeDefs, String name) {
+        if (CollectionUtils.isNotEmpty(typeDefs)) {
+            for (T typeDef : typeDefs) {
+                if (typeDef.getName().equals(name)) {
+                    return true;
+                }
+            }
+        }
+
+        return false;
+    }
+
     @JsonIgnore
     public boolean isEmpty() {
         return CollectionUtils.isEmpty(enumDefs) &&
@@ -99,4 +132,49 @@ public class AtlasTypesDef {
                 CollectionUtils.isEmpty(classificationDefs) &&
                 CollectionUtils.isEmpty(entityDefs);
     }
+
+    public void clear() {
+        if (enumDefs != null) {
+            enumDefs.clear();
+        }
+
+        if (structDefs != null) {
+            structDefs.clear();
+        }
+
+        if (classificationDefs != null) {
+            classificationDefs.clear();
+        }
+
+        if (entityDefs != null) {
+            entityDefs.clear();
+        }
+    }
+
+    public StringBuilder toString(StringBuilder sb) {
+        if (sb == null) {
+            sb = new StringBuilder();
+        }
+
+        sb.append("AtlasTypesDef{");
+        sb.append("enumDefs={");
+        AtlasBaseTypeDef.dumpObjects(enumDefs, sb);
+        sb.append("}");
+        sb.append("structDefs={");
+        AtlasBaseTypeDef.dumpObjects(structDefs, sb);
+        sb.append("}");
+        sb.append("classificationDefs={");
+        AtlasBaseTypeDef.dumpObjects(classificationDefs, sb);
+        sb.append("}");
+        sb.append("entityDefs={");
+        AtlasBaseTypeDef.dumpObjects(entityDefs, sb);
+        sb.append("}");
+
+        return sb;
+    }
+
+    @Override
+    public String toString() {
+        return toString(new StringBuilder()).toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
index 365f3cd..566207b 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
@@ -91,29 +91,11 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
             LOG.debug("Retrieving entity with type={} and attributes={}: values={}", entityTypeName, uniqAttributes);
         }
 
-        AtlasGraphQuery query = graph.query();
-
-        for (Map.Entry<String, Object> e : uniqAttributes.entrySet()) {
-            String attrName = e.getKey();
-            Object attrValue = e.getValue();
-
-            query = query.has(entityType.getQualifiedAttributeName(attrName), attrValue);
-        }
-
-        Iterator<AtlasVertex> result = query.has(Constants.ENTITY_TYPE_PROPERTY_KEY, entityTypeName)
-                                            .has(Constants.STATE_PROPERTY_KEY, Status.ACTIVE.name())
-                                            .vertices().iterator();
-        AtlasVertex entityVertex = result.hasNext() ? result.next() : null;
-
-        if (entityVertex == null) {
-            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND, entityTypeName, uniqAttributes.keySet().toString(), uniqAttributes.values().toString());
-        }
-
-        String guid = GraphHelper.getGuid(entityVertex);
+        AtlasVertex entityVertex = AtlasGraphUtilsV1.getVertexByUniqueAttributes(entityType, uniqAttributes);
 
         EntityGraphRetriever entityRetriever = new EntityGraphRetriever(typeRegistry);
 
-        return entityRetriever.toAtlasEntityWithExtInfo(guid);
+        return entityRetriever.toAtlasEntityWithExtInfo(entityVertex);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
index bddac7a..b17cf90 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
@@ -26,19 +26,20 @@ import org.apache.atlas.model.TypeCategory;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
 import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
 import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasElement;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.atlas.type.AtlasEntityType;
 import org.apache.atlas.type.AtlasStructType;
 import org.apache.atlas.type.AtlasType;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
+import java.util.*;
 
 /**
  * Utility methods for Graph.
@@ -199,6 +200,29 @@ public class AtlasGraphUtilsV1 {
         return returnType.cast(property);
     }
 
+    public static AtlasVertex getVertexByUniqueAttributes(AtlasEntityType entityType, Map<String, Object> uniqAttributes) throws AtlasBaseException {
+        AtlasGraphQuery query = AtlasGraphProvider.getGraphInstance().query();
+
+        for (Map.Entry<String, Object> e : uniqAttributes.entrySet()) {
+            String attrName = e.getKey();
+            Object attrValue = e.getValue();
+
+            query = query.has(entityType.getQualifiedAttributeName(attrName), attrValue);
+        }
+
+        Iterator<AtlasVertex> result = query.has(Constants.ENTITY_TYPE_PROPERTY_KEY, entityType.getTypeName())
+                                            .has(Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name())
+                                            .vertices().iterator();
+        AtlasVertex entityVertex = result.hasNext() ? result.next() : null;
+
+        if (entityVertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND, entityType.getTypeName(),
+                                         uniqAttributes.keySet().toString(), uniqAttributes.values().toString());
+        }
+
+        return entityVertex;
+    }
+
     private static String toString(AtlasElement element) {
         if (element instanceof AtlasVertex) {
             return toString((AtlasVertex) element);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
index e3c1f5c..255c52c 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
@@ -31,12 +31,8 @@ import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
-import org.apache.atlas.type.AtlasArrayType;
-import org.apache.atlas.type.AtlasMapType;
-import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.*;
 import org.apache.atlas.type.AtlasStructType.AtlasAttribute;
-import org.apache.atlas.type.AtlasType;
-import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.commons.collections.CollectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -78,6 +74,10 @@ public final class EntityGraphRetriever {
         return toAtlasEntity(getEntityVertex(guid));
     }
 
+    public AtlasEntity toAtlasEntity(AtlasObjectId objId) throws AtlasBaseException {
+        return toAtlasEntity(getEntityVertex(objId));
+    }
+
     public AtlasEntity toAtlasEntity(AtlasVertex entityVertex) throws AtlasBaseException {
         return mapVertexToAtlasEntity(entityVertex, null);
     }
@@ -86,6 +86,10 @@ public final class EntityGraphRetriever {
         return toAtlasEntityWithExtInfo(getEntityVertex(guid));
     }
 
+    public AtlasEntityWithExtInfo toAtlasEntityWithExtInfo(AtlasObjectId objId) throws AtlasBaseException {
+        return toAtlasEntityWithExtInfo(getEntityVertex(objId));
+    }
+
     public AtlasEntityWithExtInfo toAtlasEntityWithExtInfo(AtlasVertex entityVertex) throws AtlasBaseException {
         AtlasEntityExtInfo     entityExtInfo = new AtlasEntityExtInfo();
         AtlasEntity            entity        = mapVertexToAtlasEntity(entityVertex, entityExtInfo);
@@ -104,6 +108,24 @@ public final class EntityGraphRetriever {
         }
     }
 
+    private AtlasVertex getEntityVertex(AtlasObjectId objId) throws AtlasBaseException {
+        try {
+            if (! objId.isValid()) {
+                throw new AtlasBaseException(AtlasErrorCode.INVALID_OBJECT_ID, objId.toString());
+            }
+            if (objId.isAssignedGuid()) {
+                return graphHelper.getVertexForGUID(objId.getGuid());
+            } else {
+                AtlasEntityType     entityType     = typeRegistry.getEntityTypeByName(objId.getTypeName());
+                Map<String, Object> uniqAttributes = objId.getUniqueAttributes();
+
+                return AtlasGraphUtilsV1.getVertexByUniqueAttributes(entityType, uniqAttributes);
+            }
+        } catch (AtlasException excp) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, objId.toString());
+        }
+    }
+
     private AtlasEntity mapVertexToAtlasEntity(AtlasVertex entityVertex, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException {
         String      guid   = GraphHelper.getGuid(entityVertex);
         AtlasEntity entity = entityExtInfo != null ? entityExtInfo.getEntity(guid) : null;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
index 3bdcc87..c1450c1 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
@@ -20,11 +20,17 @@ package org.apache.atlas.web.resources;
 
 import com.google.inject.Inject;
 import org.apache.atlas.AtlasClient;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.discovery.DiscoveryService;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.impexp.AtlasExportRequest;
+import org.apache.atlas.model.impexp.AtlasExportResult;
 import org.apache.atlas.model.metrics.AtlasMetrics;
 import org.apache.atlas.services.MetricsService;
 import org.apache.atlas.authorize.AtlasActionTypes;
 import org.apache.atlas.authorize.AtlasResourceTypes;
 import org.apache.atlas.authorize.simple.AtlasAuthorizationUtils;
+import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.web.filters.AtlasCSRFPreventionFilter;
 import org.apache.atlas.web.service.ServiceState;
 import org.apache.atlas.web.util.Servlets;
@@ -40,16 +46,17 @@ import org.springframework.security.core.GrantedAuthority;
 import org.springframework.security.core.context.SecurityContextHolder;
 
 import javax.inject.Singleton;
-import javax.ws.rs.GET;
-import javax.ws.rs.Path;
-import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.*;
+import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.atlas.web.adapters.AtlasInstanceRestAdapters.toAtlasBaseException;
 
 /**
  * Jersey Resource for admin operations.
@@ -59,6 +66,12 @@ import java.util.Set;
 public class AdminResource {
     private static final Logger LOG = LoggerFactory.getLogger(AdminResource.class);
 
+    @Context
+    private HttpServletRequest httpServletRequest;
+
+    @Context
+    private HttpServletResponse httpServletResponse;
+
     private static final String isCSRF_ENABLED = "atlas.rest-csrf.enabled";
     private static final String BROWSER_USER_AGENT_PARAM = "atlas.rest-csrf.browser-useragents-regex";
     private static final String CUSTOM_METHODS_TO_IGNORE_PARAM = "atlas.rest-csrf.methods-to-ignore";
@@ -69,13 +82,19 @@ public class AdminResource {
     private static final String editableEntityTypes = "atlas.ui.editable.entity.types";
     private static final String DEFAULT_EDITABLE_ENTITY_TYPES = "hdfs_path,hdfs_path,hbase_table,hbase_column,hbase_column_family,kafka_topic";
     private Response version;
-    private ServiceState serviceState;
-    private MetricsService metricsService;
+
+    private final ServiceState      serviceState;
+    private final MetricsService    metricsService;
+    private final DiscoveryService  discoveryService;
+    private final AtlasTypeRegistry typeRegistry;
 
     @Inject
-    public AdminResource(ServiceState serviceState, MetricsService metricsService) {
-        this.serviceState = serviceState;
-        this.metricsService = metricsService;
+    public AdminResource(ServiceState serviceState, MetricsService metricsService,
+                         DiscoveryService discoveryService, AtlasTypeRegistry typeRegistry) {
+        this.serviceState     = serviceState;
+        this.metricsService   = metricsService;
+        this.discoveryService = discoveryService;
+        this.typeRegistry     = typeRegistry;
     }
 
     /**
@@ -249,6 +268,40 @@ public class AdminResource {
         return metrics;
     }
 
+    @POST
+    @Path("/export")
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    public Response export(AtlasExportRequest request) throws AtlasBaseException {
+        ZipSink exportSink = null;
+        try {
+            exportSink = new ZipSink();
+            ExportService exportService = new ExportService(this.typeRegistry);
+
+            AtlasExportResult result = exportService.run(exportSink, request, Servlets.getUserName(httpServletRequest),
+                                                         Servlets.getHostName(httpServletRequest),
+                                                         Servlets.getRequestIpAddress(httpServletRequest));
+
+            exportSink.close();
+
+            ServletOutputStream outStream = httpServletResponse.getOutputStream();
+            exportSink.writeTo(outStream);
+
+            httpServletResponse.setContentType("application/zip");
+            httpServletResponse.setHeader("Content-Disposition",
+                                          "attachment; filename=" + result.getClass().getSimpleName());
+
+            outStream.flush();
+            return Response.ok().build();
+        } catch (AtlasException | IOException ex) {
+            LOG.error("export() failed", ex);
+
+            throw toAtlasBaseException(new AtlasException(ex));
+        } finally {
+            if (exportSink != null)
+                exportSink.close();
+        }
+    }
+
     private String getEditableEntityTypes(PropertiesConfiguration config) {
         String ret = DEFAULT_EDITABLE_ENTITY_TYPES;
 
@@ -256,11 +309,11 @@ public class AdminResource {
             Object value = config.getProperty(editableEntityTypes);
 
             if (value instanceof String) {
-                ret = (String)value;
+                ret = (String) value;
             } else if (value instanceof Collection) {
                 StringBuilder sb = new StringBuilder();
 
-                for (Object elem : ((Collection)value)) {
+                for (Object elem : ((Collection) value)) {
                     if (sb.length() > 0) {
                         sb.append(",");
                     }
@@ -274,4 +327,4 @@ public class AdminResource {
 
         return ret;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/webapp/src/main/java/org/apache/atlas/web/resources/ExportService.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/ExportService.java b/webapp/src/main/java/org/apache/atlas/web/resources/ExportService.java
new file mode 100644
index 0000000..4499b9c
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/ExportService.java
@@ -0,0 +1,202 @@
+/**
+ * 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.web.resources;
+
+import org.apache.atlas.model.instance.AtlasClassification;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.store.graph.v1.EntityGraphRetriever;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.AtlasServiceException;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.impexp.*;
+import org.apache.atlas.model.typedef.AtlasClassificationDef;
+import org.apache.atlas.model.typedef.AtlasEntityDef;
+import org.apache.commons.collections.CollectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.script.ScriptException;
+import java.util.*;
+
+
+public class ExportService {
+    private static final Logger LOG = LoggerFactory.getLogger(ExportService.class);
+
+    private final AtlasTypeRegistry    typeRegistry;
+    private final AtlasGraph           atlasGraph;
+    private final EntityGraphRetriever entityGraphRetriever;
+
+    public ExportService(final AtlasTypeRegistry typeRegistry) {
+
+        this.typeRegistry         = typeRegistry;
+        this.entityGraphRetriever = new EntityGraphRetriever(this.typeRegistry);
+        this.atlasGraph           = AtlasGraphProvider.getGraphInstance();
+    }
+
+    private class ExportContext {
+        final Map<String, Boolean> entitiesToBeProcessed = new HashMap<>();
+        final AtlasExportResult    result;
+        final ZipSink              sink;
+        long                       numOfEntitiesExported = 0;
+
+        ExportContext(AtlasExportResult result, ZipSink sink) {
+            this.result = result;
+            this.sink   = sink;
+        }
+    }
+
+    public AtlasExportResult run(ZipSink exportSink, AtlasExportRequest request, String userName, String hostName,
+                                 String requestingIP) throws AtlasException {
+
+        ExportContext context = new ExportContext(new AtlasExportResult(request, userName, hostName, requestingIP, System.currentTimeMillis()), exportSink);
+
+        try {
+            LOG.info("==> export(user={}, from={})", userName, requestingIP);
+
+            int i = 0;
+            for (AtlasObjectId item : request.getItemsToExport()) {
+                process(Integer.toString(i++), item, context);
+            }
+
+            context.sink.setExportOrder(context.result.getData().getEntityCreationOrder());
+            context.sink.setTypesDef(context.result.getData().getTypesDef());
+
+            context.result.getData().clear();
+            context.result.setOperationStatus(AtlasExportResult.OperationStatus.SUCCESS);
+            context.sink.setResult(context.result);
+        }
+        catch(Exception ex) {
+            LOG.error("Operation failed: ", ex);
+        } finally {
+            LOG.info("<== export(user={}, from={}): status {}", userName, requestingIP, context.result.getOperationStatus());
+        }
+
+        return context.result;
+    }
+
+    private void process(String folder, AtlasObjectId item, ExportContext context) throws AtlasServiceException, AtlasException, AtlasBaseException {
+        try {
+            AtlasEntity  entity = entityGraphRetriever.toAtlasEntity(item);
+            List<String> queue  = populateConnectedEntities(entity.getGuid(), context);
+
+            process(entity, context);
+
+            for (String guid : queue) {
+                if(context.entitiesToBeProcessed.get(guid)) {
+                    continue;
+                }
+
+                process(entityGraphRetriever.toAtlasEntity(guid), context);
+            }
+
+            context.result.getData().getEntityCreationOrder().put(folder, queue);
+        } catch (AtlasBaseException e) {
+            context.result.setOperationStatus(AtlasExportResult.OperationStatus.PARTIAL_SUCCESS);
+
+            LOG.error("Fetching entity failed for: {}", item);
+        }
+    }
+
+    private void process(AtlasEntity entity, ExportContext context) throws AtlasBaseException, AtlasException {
+        addTypesAsNeeded(entity.getTypeName(), context);
+        addClassificationsAsNeeded(entity, context);
+        addEntity(entity, context);
+    }
+
+    private void addEntity(AtlasEntity entity, ExportContext context) throws AtlasException, AtlasBaseException {
+        context.entitiesToBeProcessed.put(entity.getGuid(), true);
+        context.sink.add(entity);
+
+        context.result.incrementMeticsCounter(String.format("entity:%s", entity.getTypeName()));
+        context.result.incrementMeticsCounter("Entities");
+
+        context.numOfEntitiesExported++;
+
+        if (context.numOfEntitiesExported % 10 == 0) {
+            LOG.info("export(): in progress.. number of entities exported: {}", context.numOfEntitiesExported);
+        }
+    }
+
+    private List<String> populateConnectedEntities(String startGuid, ExportContext context) {
+        final String gremlinQuery = "g.V('__guid', '%s').bothE().bothV().has('__guid').__guid.toList()";
+
+        Map<String, Boolean> entitiesToBeProcessed = context.entitiesToBeProcessed;
+
+        List<String> queue = new ArrayList<>();
+
+        entitiesToBeProcessed.put(startGuid, false);
+        queue.add(startGuid);
+
+        for (int i=0; i < queue.size(); i++) {
+            String currentGuid = queue.get(i);
+
+            try {
+                List<String> result = (List<String>) atlasGraph.executeGremlinScript(
+                                                        String.format(gremlinQuery, currentGuid), false);
+
+                for (String guid : result) {
+                    if (entitiesToBeProcessed.containsKey(guid)) {
+                        continue;
+                    }
+
+                    entitiesToBeProcessed.put(guid, false);
+                    queue.add(guid);
+                }
+            } catch (ScriptException e) {
+                LOG.error("Child entities could not be added for %s", currentGuid);
+            }
+        }
+
+        return queue;
+    }
+
+    private void addClassificationsAsNeeded(AtlasEntity entity, ExportContext context) {
+        AtlasExportResult result   = context.result;
+        AtlasTypesDef     typesDef = result.getData().getTypesDef();
+
+        if(CollectionUtils.isNotEmpty(entity.getClassifications())) {
+            for (AtlasClassification c : entity.getClassifications()) {
+                if (typesDef.hasClassificationDef(c.getTypeName())) {
+                    continue;
+                }
+
+                AtlasClassificationDef cd = typeRegistry.getClassificationDefByName(c.getTypeName());
+
+                typesDef.getClassificationDefs().add(cd);
+                result.incrementMeticsCounter("Classification");
+            }
+        }
+    }
+
+    private void addTypesAsNeeded(String typeName, ExportContext context) {
+        AtlasExportResult result   = context.result;
+        AtlasTypesDef     typesDef = result.getData().getTypesDef();
+
+        if(!typesDef.hasEntityDef(typeName)) {
+            AtlasEntityDef typeDefinition = typeRegistry.getEntityDefByName(typeName);
+
+            typesDef.getEntityDefs().add(typeDefinition);
+            result.incrementMeticsCounter("Type(s)");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/webapp/src/main/java/org/apache/atlas/web/resources/ZipSink.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/ZipSink.java b/webapp/src/main/java/org/apache/atlas/web/resources/ZipSink.java
new file mode 100644
index 0000000..4c1ca6c
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/ZipSink.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.web.resources;
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.impexp.AtlasExportResult;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.type.AtlasType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+public class ZipSink {
+    private static final Logger LOG = LoggerFactory.getLogger(ZipSink.class);
+
+    private ZipOutputStream zipOutputStream;
+    private ByteArrayOutputStream byteArrayOutputStream;
+
+    public ZipSink() {
+        init();
+    }
+
+    private void init() {
+        byteArrayOutputStream = new ByteArrayOutputStream();
+        zipOutputStream = new ZipOutputStream(byteArrayOutputStream);
+    }
+
+    public void add(AtlasEntity entity) throws AtlasBaseException {
+        String jsonData = convertToJSON(entity);
+        saveToZip(entity.getGuid(), jsonData);
+    }
+
+    public void setResult(AtlasExportResult result) throws AtlasBaseException {
+        final String fileName = "atlas-export-info";
+        String jsonData = convertToJSON(result);
+        saveToZip(fileName, jsonData);
+    }
+
+    public void setTypesDef(AtlasTypesDef typesDef) throws AtlasBaseException {
+        final String fileName = "atlas-typesdef";
+        String jsonData = convertToJSON(typesDef);
+        saveToZip(fileName, jsonData);
+    }
+
+    public void setExportOrder(Map<String, List<String>> result) throws AtlasBaseException {
+        final String fileName = "atlas-export-order";
+        String jsonData = convertToJSON(result);
+        saveToZip(fileName, jsonData);
+    }
+
+    public void writeTo(OutputStream stream) throws IOException {
+        byteArrayOutputStream.writeTo(stream);
+    }
+
+    public void close() {
+        try {
+            if(zipOutputStream != null) {
+                zipOutputStream.close();
+                zipOutputStream = null;
+            }
+        } catch (IOException e) {
+            LOG.error("Error closing Zip file", e);
+        }
+    }
+
+    private String convertToJSON(Object entity) {
+        return AtlasType.toJson(entity);
+    }
+
+    private void saveToZip(String fileName, String jsonData) throws AtlasBaseException {
+        try {
+            addToZipStream(fileName + ".json", jsonData);
+        } catch (IOException e) {
+            throw new AtlasBaseException(String.format("Error writing file %s.", fileName), e);
+        }
+    }
+
+    private void addToZipStream(String entryName, String payload) throws IOException {
+
+        ZipEntry e = new ZipEntry(entryName);
+        zipOutputStream.putNextEntry(e);
+
+        zipOutputStream.write(payload.getBytes());
+        zipOutputStream.closeEntry();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/webapp/src/main/java/org/apache/atlas/web/util/Servlets.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/util/Servlets.java b/webapp/src/main/java/org/apache/atlas/web/util/Servlets.java
index 52dd9b5..8ff5d04 100755
--- a/webapp/src/main/java/org/apache/atlas/web/util/Servlets.java
+++ b/webapp/src/main/java/org/apache/atlas/web/util/Servlets.java
@@ -24,6 +24,7 @@ import org.apache.atlas.utils.ParamChecker;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.codehaus.jettison.json.JSONException;
@@ -37,6 +38,8 @@ import javax.ws.rs.core.Response;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.nio.charset.Charset;
 import java.util.List;
 
@@ -52,6 +55,7 @@ public final class Servlets {
     }
 
     public static final String JSON_MEDIA_TYPE = MediaType.APPLICATION_JSON + "; charset=UTF-8";
+    public static final String BINARY = MediaType.APPLICATION_OCTET_STREAM;
 
     /**
      * Returns the user of the given request.
@@ -168,4 +172,25 @@ public final class Servlets {
         ParamChecker.notNull(inputStr, "Input String cannot be null");
         return StringEscapeUtils.escapeJson(inputStr);
     }
+
+    public static String getRequestIpAddress(HttpServletRequest httpServletRequest) {
+        try {
+            InetAddress inetAddr = InetAddress.getByName(httpServletRequest.getRemoteAddr());
+
+            String ip = inetAddr.getHostAddress();
+
+            return ip;
+        } catch(UnknownHostException ex) {
+            LOG.error("Error occured when retrieving IP address", ex);
+            return "";
+        }
+    }
+
+    public static String getHostName(HttpServletRequest httpServletRequest) {
+        return httpServletRequest.getLocalName();
+    }
+
+    public static String getUserName(HttpServletRequest httpServletRequest) throws IOException {
+        return httpServletRequest.getRemoteUser();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/webapp/src/test/java/org/apache/atlas/web/resources/AdminExportTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/AdminExportTest.java b/webapp/src/test/java/org/apache/atlas/web/resources/AdminExportTest.java
new file mode 100644
index 0000000..a8f8232
--- /dev/null
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/AdminExportTest.java
@@ -0,0 +1,25 @@
+/**
+ * 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.web.resources;
+
+
+public class AdminExportTest {
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9cb56c54/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java b/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
index d73190e..c3d4b72 100644
--- a/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
@@ -48,7 +48,7 @@ public class AdminResourceTest {
 
         when(serviceState.getState()).thenReturn(ServiceState.ServiceStateValue.ACTIVE);
 
-        AdminResource adminResource = new AdminResource(serviceState, null);
+        AdminResource adminResource = new AdminResource(serviceState, null, null, null);
         Response response = adminResource.getStatus();
         assertEquals(response.getStatus(), HttpServletResponse.SC_OK);
         JSONObject entity = (JSONObject) response.getEntity();
@@ -59,7 +59,7 @@ public class AdminResourceTest {
     public void testResourceGetsValueFromServiceState() throws JSONException {
         when(serviceState.getState()).thenReturn(ServiceState.ServiceStateValue.PASSIVE);
 
-        AdminResource adminResource = new AdminResource(serviceState, null);
+        AdminResource adminResource = new AdminResource(serviceState, null, null, null);
         Response response = adminResource.getStatus();
 
         verify(serviceState).getState();