Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 32580200C24 for ; Thu, 9 Feb 2017 02:49:51 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 3119B160B67; Thu, 9 Feb 2017 01:49:51 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 8C92C160B49 for ; Thu, 9 Feb 2017 02:49:49 +0100 (CET) Received: (qmail 75219 invoked by uid 500); 9 Feb 2017 01:49:48 -0000 Mailing-List: contact commits-help@atlas.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@atlas.incubator.apache.org Delivered-To: mailing list commits@atlas.incubator.apache.org Received: (qmail 75209 invoked by uid 99); 9 Feb 2017 01:49:48 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 09 Feb 2017 01:49:48 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id ED807C702F for ; Thu, 9 Feb 2017 01:49:47 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -6.219 X-Spam-Level: X-Spam-Status: No, score=-6.219 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-2.999] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id 1-ZFS-RiiiLK for ; Thu, 9 Feb 2017 01:49:41 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id A76255F5D3 for ; Thu, 9 Feb 2017 01:49:39 +0000 (UTC) Received: (qmail 72562 invoked by uid 99); 9 Feb 2017 01:48:11 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 09 Feb 2017 01:48:11 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E8B32DFBAD; Thu, 9 Feb 2017 01:48:10 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: madhan@apache.org To: commits@atlas.incubator.apache.org Message-Id: <10e7b76f77844e6f879fbb932fbe442d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: incubator-atlas git commit: ATLAS-1541: renamed GraphEntityMapper to EntityGraphRetriever; added methods to retrieve an entity from the given vertex Date: Thu, 9 Feb 2017 01:48:10 +0000 (UTC) archived-at: Thu, 09 Feb 2017 01:49:51 -0000 Repository: incubator-atlas Updated Branches: refs/heads/master db18c8240 -> a67e89625 ATLAS-1541: renamed GraphEntityMapper to EntityGraphRetriever; added methods to retrieve an entity from the given vertex Project: http://git-wip-us.apache.org/repos/asf/incubator-atlas/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-atlas/commit/a67e8962 Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/a67e8962 Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/a67e8962 Branch: refs/heads/master Commit: a67e896253e3af4631cfbe40c77e14221f06f278 Parents: db18c82 Author: Madhan Neethiraj Authored: Wed Feb 8 14:56:40 2017 -0800 Committer: Madhan Neethiraj Committed: Wed Feb 8 17:48:04 2017 -0800 ---------------------------------------------------------------------- .../store/graph/v1/AtlasEntityStoreV1.java | 13 +- .../store/graph/v1/EntityGraphRetriever.java | 410 +++++++++++++++++++ .../store/graph/v1/GraphEntityMapper.java | 400 ------------------ .../store/graph/v1/AtlasEntityStoreV1Test.java | 4 +- 4 files changed, 418 insertions(+), 409 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/a67e8962/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 a95ae8e..365f3cd 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 @@ -55,18 +55,15 @@ import java.util.Map; public class AtlasEntityStoreV1 implements AtlasEntityStore { protected AtlasTypeRegistry typeRegistry; - protected final GraphHelper graphHelper = GraphHelper.getInstance(); private final EntityGraphMapper graphMapper; - private final GraphEntityMapper entityMapper; private final AtlasGraph graph; private static final Logger LOG = LoggerFactory.getLogger(AtlasEntityStoreV1.class); @Inject - public AtlasEntityStoreV1(EntityGraphMapper vertexMapper, GraphEntityMapper entityMapper) { + public AtlasEntityStoreV1(EntityGraphMapper vertexMapper) { this.graphMapper = vertexMapper; - this.entityMapper = entityMapper; this.graph = AtlasGraphProvider.getGraphInstance(); } @@ -81,7 +78,9 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore { LOG.debug("Retrieving entity with guid={}", guid); } - return entityMapper.toAtlasEntity(guid, true); + EntityGraphRetriever entityRetriever = new EntityGraphRetriever(typeRegistry); + + return entityRetriever.toAtlasEntityWithExtInfo(guid); } @Override @@ -112,7 +111,9 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore { String guid = GraphHelper.getGuid(entityVertex); - return entityMapper.toAtlasEntity(guid, true); + EntityGraphRetriever entityRetriever = new EntityGraphRetriever(typeRegistry); + + return entityRetriever.toAtlasEntityWithExtInfo(guid); } @Override http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/a67e8962/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 new file mode 100644 index 0000000..e3c1f5c --- /dev/null +++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java @@ -0,0 +1,410 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.atlas.repository.store.graph.v1; + +import org.apache.atlas.AtlasErrorCode; +import org.apache.atlas.AtlasException; +import org.apache.atlas.exception.AtlasBaseException; +import org.apache.atlas.model.instance.AtlasClassification; +import org.apache.atlas.model.instance.AtlasEntity; +import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityExtInfo; +import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo; +import org.apache.atlas.model.instance.AtlasObjectId; +import org.apache.atlas.model.instance.AtlasStruct; +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef; +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.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; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BIGDECIMAL; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BIGINTEGER; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BOOLEAN; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BYTE; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_DATE; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_DOUBLE; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_FLOAT; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_INT; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_LONG; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_SHORT; +import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_STRING; +import static org.apache.atlas.repository.graph.GraphHelper.EDGE_LABEL_PREFIX; + + +public final class EntityGraphRetriever { + private static final Logger LOG = LoggerFactory.getLogger(EntityGraphRetriever.class); + + private static final GraphHelper graphHelper = GraphHelper.getInstance(); + + private final AtlasTypeRegistry typeRegistry; + + public EntityGraphRetriever(AtlasTypeRegistry typeRegistry) { + this.typeRegistry = typeRegistry; + } + + public AtlasEntity toAtlasEntity(String guid) throws AtlasBaseException { + return toAtlasEntity(getEntityVertex(guid)); + } + + public AtlasEntity toAtlasEntity(AtlasVertex entityVertex) throws AtlasBaseException { + return mapVertexToAtlasEntity(entityVertex, null); + } + + public AtlasEntityWithExtInfo toAtlasEntityWithExtInfo(String guid) throws AtlasBaseException { + return toAtlasEntityWithExtInfo(getEntityVertex(guid)); + } + + public AtlasEntityWithExtInfo toAtlasEntityWithExtInfo(AtlasVertex entityVertex) throws AtlasBaseException { + AtlasEntityExtInfo entityExtInfo = new AtlasEntityExtInfo(); + AtlasEntity entity = mapVertexToAtlasEntity(entityVertex, entityExtInfo); + AtlasEntityWithExtInfo ret = new AtlasEntityWithExtInfo(entity, entityExtInfo); + + ret.compact(); + + return ret; + } + + private AtlasVertex getEntityVertex(String guid) throws AtlasBaseException { + try { + return graphHelper.getVertexForGUID(guid); + } catch (AtlasException excp) { + throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid); + } + } + + private AtlasEntity mapVertexToAtlasEntity(AtlasVertex entityVertex, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { + String guid = GraphHelper.getGuid(entityVertex); + AtlasEntity entity = entityExtInfo != null ? entityExtInfo.getEntity(guid) : null; + + if (entity == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Mapping graph vertex to atlas entity for guid {}", guid); + } + + entity = new AtlasEntity(); + + if (entityExtInfo != null) { + entityExtInfo.addReferredEntity(guid, entity); + } + + mapSystemAttributes(entityVertex, entity); + + mapAttributes(entityVertex, entity, entityExtInfo); + + mapClassifications(entityVertex, entity, entityExtInfo); + } + + return entity; + } + + private AtlasEntity mapSystemAttributes(AtlasVertex entityVertex, AtlasEntity entity) { + if (LOG.isDebugEnabled()) { + LOG.debug("Mapping system attributes for type {}", entity.getTypeName()); + } + + entity.setGuid(GraphHelper.getGuid(entityVertex)); + entity.setTypeName(GraphHelper.getTypeName(entityVertex)); + entity.setStatus(GraphHelper.getStatus(entityVertex)); + entity.setVersion(GraphHelper.getVersion(entityVertex).longValue()); + + entity.setCreatedBy(GraphHelper.getCreatedByAsString(entityVertex)); + entity.setUpdatedBy(GraphHelper.getModifiedByAsString(entityVertex)); + + entity.setCreateTime(new Date(GraphHelper.getCreatedTime(entityVertex))); + entity.setUpdateTime(new Date(GraphHelper.getModifiedTime(entityVertex))); + + return entity; + } + + private void mapAttributes(AtlasVertex entityVertex, AtlasStruct struct, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { + AtlasType objType = typeRegistry.getType(struct.getTypeName()); + + if (!(objType instanceof AtlasStructType)) { + throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, struct.getTypeName()); + } + + AtlasStructType structType = (AtlasStructType) objType; + + for (AtlasAttribute attribute : structType.getAllAttributes().values()) { + Object attrValue = mapVertexToAttribute(entityVertex, attribute, entityExtInfo); + + struct.setAttribute(attribute.getName(), attrValue); + } + } + + private void mapClassifications(AtlasVertex entityVertex, AtlasEntity entity, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { + List classificationNames = GraphHelper.getTraitNames(entityVertex); + + if (CollectionUtils.isNotEmpty(classificationNames)) { + List classifications = new ArrayList<>(classificationNames.size()); + + for (String classficationName : classificationNames) { + if (LOG.isDebugEnabled()) { + LOG.debug("mapping classification {} to atlas entity", classficationName); + } + + Iterable edges = entityVertex.getEdges(AtlasEdgeDirection.OUT, classficationName); + AtlasEdge edge = (edges != null && edges.iterator().hasNext()) ? edges.iterator().next() : null; + + if (edge != null) { + AtlasClassification classification = new AtlasClassification(classficationName); + + mapAttributes(edge.getInVertex(), classification, entityExtInfo); + + classifications.add(classification); + } + } + + entity.setClassifications(classifications); + } + } + + private Object mapVertexToAttribute(AtlasVertex entityVertex, AtlasAttribute attribute, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { + Object ret = null; + AtlasType attrType = attribute.getAttributeType(); + String vertexPropertyName = attribute.getQualifiedName(); + String edgeLabel = EDGE_LABEL_PREFIX + vertexPropertyName; + boolean isOwnedAttribute = attribute.isOwnedRef(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Mapping vertex {} to atlas entity {}.{}", entityVertex, attribute.getDefinedInDef().getName(), attribute.getName()); + } + + switch (attrType.getTypeCategory()) { + case PRIMITIVE: + ret = mapVertexToPrimitive(entityVertex, vertexPropertyName, attribute.getAttributeDef()); + break; + case ENUM: + ret = GraphHelper.getProperty(entityVertex, vertexPropertyName); + break; + case STRUCT: + ret = mapVertexToStruct(entityVertex, edgeLabel, null, entityExtInfo); + break; + case ENTITY: + ret = mapVertexToObjectId(entityVertex, edgeLabel, null, entityExtInfo, isOwnedAttribute); + break; + case ARRAY: + ret = mapVertexToArray(entityVertex, (AtlasArrayType) attrType, vertexPropertyName, entityExtInfo, isOwnedAttribute); + break; + case MAP: + ret = mapVertexToMap(entityVertex, (AtlasMapType) attrType, vertexPropertyName, entityExtInfo, isOwnedAttribute); + break; + case CLASSIFICATION: + // do nothing + break; + } + + return ret; + } + + private Map mapVertexToMap(AtlasVertex entityVertex, AtlasMapType atlasMapType, final String propertyName, + AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { + List mapKeys = GraphHelper.getListProperty(entityVertex, propertyName); + + if (CollectionUtils.isEmpty(mapKeys)) { + return null; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Mapping map attribute {} for vertex {}", atlasMapType.getTypeName(), entityVertex); + } + + Map ret = new HashMap<>(mapKeys.size()); + AtlasType mapValueType = atlasMapType.getValueType(); + + for (String mapKey : mapKeys) { + final String keyPropertyName = propertyName + "." + mapKey; + final String edgeLabel = EDGE_LABEL_PREFIX + keyPropertyName; + final Object keyValue = GraphHelper.getMapValueProperty(mapValueType, entityVertex, keyPropertyName); + + Object mapValue = mapVertexToCollectionEntry(entityVertex, mapValueType, keyValue, edgeLabel, entityExtInfo, isOwnedAttribute); + if (mapValue != null) { + ret.put(mapKey, mapValue); + } + } + + return ret; + } + + private List mapVertexToArray(AtlasVertex entityVertex, AtlasArrayType arrayType, String propertyName, + AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { + AtlasType arrayElementType = arrayType.getElementType(); + List arrayElements = GraphHelper.getArrayElementsProperty(arrayElementType, entityVertex, propertyName); + + if (CollectionUtils.isEmpty(arrayElements)) { + return null; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Mapping array attribute {} for vertex {}", arrayElementType.getTypeName(), entityVertex); + } + + List arrValues = new ArrayList(arrayElements.size()); + String edgeLabel = EDGE_LABEL_PREFIX + propertyName; + + for (Object element : arrayElements) { + Object arrValue = mapVertexToCollectionEntry(entityVertex, arrayElementType, element, + edgeLabel, entityExtInfo, isOwnedAttribute); + + if (arrValue != null) { + arrValues.add(arrValue); + } + } + + return arrValues; + } + + private Object mapVertexToCollectionEntry(AtlasVertex entityVertex, AtlasType arrayElement, Object value, String edgeLabel, + AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { + Object ret = null; + + switch (arrayElement.getTypeCategory()) { + case PRIMITIVE: + case ENUM: + ret = value; + break; + + case ARRAY: + case MAP: + case CLASSIFICATION: + break; + + case STRUCT: + ret = mapVertexToStruct(entityVertex, edgeLabel, (AtlasEdge) value, entityExtInfo); + break; + + case ENTITY: + ret = mapVertexToObjectId(entityVertex, edgeLabel, (AtlasEdge) value, entityExtInfo, isOwnedAttribute); + break; + + default: + break; + } + + return ret; + } + + private Object mapVertexToPrimitive(AtlasVertex entityVertex, final String vertexPropertyName, AtlasAttributeDef attrDef) { + Object ret = null; + + if (GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Object.class) == null) { + return null; + } + + switch (attrDef.getTypeName().toLowerCase()) { + case ATLAS_TYPE_STRING: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, String.class); + break; + case ATLAS_TYPE_SHORT: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Short.class); + break; + case ATLAS_TYPE_INT: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Integer.class); + break; + case ATLAS_TYPE_BIGINTEGER: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, BigInteger.class); + break; + case ATLAS_TYPE_BOOLEAN: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Boolean.class); + break; + case ATLAS_TYPE_BYTE: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Byte.class); + break; + case ATLAS_TYPE_LONG: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Long.class); + break; + case ATLAS_TYPE_FLOAT: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Float.class); + break; + case ATLAS_TYPE_DOUBLE: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Double.class); + break; + case ATLAS_TYPE_BIGDECIMAL: + ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, BigDecimal.class); + break; + case ATLAS_TYPE_DATE: + ret = new Date(GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Long.class)); + break; + default: + break; + } + + return ret; + } + + private AtlasObjectId mapVertexToObjectId(AtlasVertex entityVertex, String edgeLabel, AtlasEdge edge, + AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { + AtlasObjectId ret = null; + + if (edge == null) { + edge = graphHelper.getEdgeForLabel(entityVertex, edgeLabel); + } + + if (GraphHelper.elementExists(edge)) { + final AtlasVertex referenceVertex = edge.getInVertex(); + + if (referenceVertex != null) { + if (entityExtInfo != null && isOwnedAttribute) { + AtlasEntity entity = mapVertexToAtlasEntity(referenceVertex, entityExtInfo); + + if (entity != null) { + ret = entity.getAtlasObjectId(); + } + } else { + ret = new AtlasObjectId(GraphHelper.getTypeName(referenceVertex), GraphHelper.getGuid(referenceVertex)); + } + } + } + + return ret; + } + + private AtlasStruct mapVertexToStruct(AtlasVertex entityVertex, String edgeLabel, AtlasEdge edge, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { + AtlasStruct ret = null; + + if (edge == null) { + edge = graphHelper.getEdgeForLabel(entityVertex, edgeLabel); + } + + if (GraphHelper.elementExists(edge)) { + final AtlasVertex referenceVertex = edge.getInVertex(); + ret = new AtlasStruct(GraphHelper.getTypeName(referenceVertex)); + + mapAttributes(referenceVertex, ret, entityExtInfo); + } + + return ret; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/a67e8962/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/GraphEntityMapper.java ---------------------------------------------------------------------- diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/GraphEntityMapper.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/GraphEntityMapper.java deleted file mode 100644 index a0d8940..0000000 --- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/GraphEntityMapper.java +++ /dev/null @@ -1,400 +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.v1; - -import com.google.inject.Inject; -import com.google.inject.Singleton; -import org.apache.atlas.AtlasErrorCode; -import org.apache.atlas.AtlasException; -import org.apache.atlas.exception.AtlasBaseException; -import org.apache.atlas.model.instance.AtlasClassification; -import org.apache.atlas.model.instance.AtlasEntity; -import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityExtInfo; -import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo; -import org.apache.atlas.model.instance.AtlasObjectId; -import org.apache.atlas.model.instance.AtlasStruct; -import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef; -import org.apache.atlas.repository.Constants; -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.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; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BIGDECIMAL; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BIGINTEGER; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BOOLEAN; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BYTE; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_DATE; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_DOUBLE; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_FLOAT; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_INT; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_LONG; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_SHORT; -import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_STRING; -import static org.apache.atlas.repository.graph.GraphHelper.EDGE_LABEL_PREFIX; - -@Singleton -public final class GraphEntityMapper { - private static final Logger LOG = LoggerFactory.getLogger(GraphEntityMapper.class); - private static final GraphHelper graphHelper = GraphHelper.getInstance(); - - @Inject - protected AtlasTypeRegistry typeRegistry; - - public AtlasEntityWithExtInfo toAtlasEntity(String guid, boolean includeReferences) throws AtlasBaseException { - AtlasEntityExtInfo entityExtInfo = includeReferences ? new AtlasEntityExtInfo() : null; - AtlasEntity entity = mapVertexToAtlasEntity(guid, entityExtInfo); - AtlasEntityWithExtInfo ret = new AtlasEntityWithExtInfo(entity, entityExtInfo); - - ret.compact(); - - return ret; - } - - private AtlasEntity mapVertexToAtlasEntity(String guid, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { - AtlasEntity entity = entityExtInfo != null ? entityExtInfo.getEntity(guid) : null; - - if (entity != null) { - return entity; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Mapping graph vertex to atlas entity for guid {}", guid); - } - - try { - AtlasVertex entityVertex = graphHelper.getVertexForGUID(guid); - entity = new AtlasEntity(); - - if (entityExtInfo != null) { - entityExtInfo.addReferredEntity(guid, entity); - } - - mapSystemAttributes(entityVertex, entity); - - mapAttributes(entityVertex, entity, entityExtInfo); - - mapClassifications(entityVertex, entity, entityExtInfo); - - } catch (AtlasException e) { - throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid); - } - - return entity; - } - - private AtlasEntity mapSystemAttributes(AtlasVertex entityVertex, AtlasEntity entity) { - if (LOG.isDebugEnabled()) { - LOG.debug("Mapping system attributes for type {}", entity.getTypeName()); - } - - entity.setGuid(GraphHelper.getGuid(entityVertex)); - entity.setTypeName(GraphHelper.getSingleValuedProperty(entityVertex, Constants.ENTITY_TYPE_PROPERTY_KEY, String.class)); - entity.setStatus(GraphHelper.getStatus(entityVertex)); - entity.setVersion(GraphHelper.getVersion(entityVertex).longValue()); - - entity.setCreatedBy(GraphHelper.getCreatedByAsString(entityVertex)); - entity.setUpdatedBy(GraphHelper.getModifiedByAsString(entityVertex)); - - entity.setCreateTime(new Date(GraphHelper.getCreatedTime(entityVertex))); - entity.setUpdateTime(new Date(GraphHelper.getModifiedTime(entityVertex))); - - return entity; - } - - private void mapAttributes(AtlasVertex entityVertex, AtlasStruct struct, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { - AtlasType objType = typeRegistry.getType(struct.getTypeName()); - - if (!(objType instanceof AtlasStructType)) { - throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, struct.getTypeName()); - } - - AtlasStructType structType = (AtlasStructType) objType; - Collection attributes = structType.getAllAttributes().values(); - - if (CollectionUtils.isNotEmpty(attributes)) { - for (AtlasAttribute attribute : attributes) { - Object attrValue = mapVertexToAttribute(entityVertex, attribute, entityExtInfo); - - struct.setAttribute(attribute.getName(), attrValue); - } - } - } - - private void mapClassifications(AtlasVertex entityVertex, AtlasEntity entity, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { - List classficationNames = GraphHelper.getTraitNames(entityVertex); - - if (CollectionUtils.isNotEmpty(classficationNames)) { - List classifications = new ArrayList<>(); - - for (String classficationName : classficationNames) { - if (LOG.isDebugEnabled()) { - LOG.debug("mapping classification {} to atlas entity", classficationName); - } - - Iterable edges = entityVertex.getEdges(AtlasEdgeDirection.OUT, classficationName); - - AtlasEdge edge = (edges != null && edges.iterator().hasNext()) ? edges.iterator().next() : null; - - if (edge != null) { - AtlasClassification classification = new AtlasClassification(classficationName); - - mapAttributes(edge.getInVertex(), classification, entityExtInfo); - - classifications.add(classification); - } - } - - entity.setClassifications(classifications); - } - } - - private Object mapVertexToAttribute(AtlasVertex entityVertex, AtlasAttribute attribute, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { - Object ret = null; - AtlasType attrType = attribute.getAttributeType(); - String vertexPropertyName = attribute.getQualifiedName(); - String edgeLabel = EDGE_LABEL_PREFIX + vertexPropertyName; - boolean isOwnedAttribute = attribute.isOwnedRef(); - - if (LOG.isDebugEnabled()) { - LOG.debug("Mapping vertex {} to atlas entity {}.{}", entityVertex, attribute.getDefinedInDef().getName(), attribute.getName()); - } - - switch (attrType.getTypeCategory()) { - case PRIMITIVE: - ret = mapVertexToPrimitive(entityVertex, vertexPropertyName, attribute.getAttributeDef()); - break; - case ENUM: - ret = GraphHelper.getProperty(entityVertex, vertexPropertyName); - break; - case STRUCT: - ret = mapVertexToStruct(entityVertex, edgeLabel, null, entityExtInfo); - break; - case ENTITY: - ret = mapVertexToObjectId(entityVertex, edgeLabel, null, entityExtInfo, isOwnedAttribute); - break; - case ARRAY: - ret = mapVertexToArray(entityVertex, (AtlasArrayType) attrType, vertexPropertyName, entityExtInfo, isOwnedAttribute); - break; - case MAP: - ret = mapVertexToMap(entityVertex, (AtlasMapType) attrType, vertexPropertyName, entityExtInfo, isOwnedAttribute); - break; - case CLASSIFICATION: - // do nothing - break; - } - - return ret; - } - - private Map mapVertexToMap(AtlasVertex entityVertex, AtlasMapType atlasMapType, final String propertyName, - AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { - Map ret = new HashMap<>(); - List mapKeys = GraphHelper.getListProperty(entityVertex, propertyName); - AtlasType mapValueType = atlasMapType.getValueType(); - - if (LOG.isDebugEnabled()) { - LOG.debug("Mapping map attribute {} for vertex {}", atlasMapType.getTypeName(), entityVertex); - } - - if (CollectionUtils.isEmpty(mapKeys)) { - return null; - } - - for (String mapKey : mapKeys) { - final String keyPropertyName = propertyName + "." + mapKey; - final String edgeLabel = EDGE_LABEL_PREFIX + keyPropertyName; - final Object keyValue = GraphHelper.getMapValueProperty(mapValueType, entityVertex, keyPropertyName); - - Object mapValue = mapVertexToCollectionEntry(entityVertex, mapValueType, keyValue, edgeLabel, entityExtInfo, isOwnedAttribute); - if (mapValue != null) { - ret.put(mapKey, mapValue); - } - } - - return ret; - } - - private List mapVertexToArray(AtlasVertex entityVertex, AtlasArrayType arrayType, String propertyName, - AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { - AtlasType arrayElementType = arrayType.getElementType(); - List arrayElements = GraphHelper.getArrayElementsProperty(arrayElementType, entityVertex, propertyName); - List arrValues = new ArrayList(); - String edgeLabel = EDGE_LABEL_PREFIX + propertyName; - - if (LOG.isDebugEnabled()) { - LOG.debug("Mapping array attribute {} for vertex {}", arrayElementType.getTypeName(), entityVertex); - } - - if (CollectionUtils.isEmpty(arrayElements)) { - return null; - } - - for (Object element : arrayElements) { - Object arrValue = mapVertexToCollectionEntry(entityVertex, arrayElementType, element, - edgeLabel, entityExtInfo, isOwnedAttribute); - - arrValues.add(arrValue); - } - - return arrValues; - } - - private Object mapVertexToCollectionEntry(AtlasVertex entityVertex, AtlasType arrayElement, Object value, String edgeLabel, - AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { - Object ret = null; - - switch (arrayElement.getTypeCategory()) { - case PRIMITIVE: - case ENUM: - ret = value; - break; - - case ARRAY: - case MAP: - case CLASSIFICATION: - break; - - case STRUCT: - ret = mapVertexToStruct(entityVertex, edgeLabel, (AtlasEdge) value, entityExtInfo); - break; - - case ENTITY: - ret = mapVertexToObjectId(entityVertex, edgeLabel, (AtlasEdge) value, entityExtInfo, isOwnedAttribute); - break; - - default: - break; - } - - return ret; - } - - private Object mapVertexToPrimitive(AtlasVertex entityVertex, final String vertexPropertyName, AtlasAttributeDef attrDef) { - Object ret = null; - - if (GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Object.class) == null) { - return null; - } - - switch (attrDef.getTypeName().toLowerCase()) { - case ATLAS_TYPE_STRING: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, String.class); - break; - case ATLAS_TYPE_SHORT: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Short.class); - break; - case ATLAS_TYPE_INT: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Integer.class); - break; - case ATLAS_TYPE_BIGINTEGER: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, BigInteger.class); - break; - case ATLAS_TYPE_BOOLEAN: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Boolean.class); - break; - case ATLAS_TYPE_BYTE: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Byte.class); - break; - case ATLAS_TYPE_LONG: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Long.class); - break; - case ATLAS_TYPE_FLOAT: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Float.class); - break; - case ATLAS_TYPE_DOUBLE: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Double.class); - break; - case ATLAS_TYPE_BIGDECIMAL: - ret = GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, BigDecimal.class); - break; - case ATLAS_TYPE_DATE: - ret = new Date(GraphHelper.getSingleValuedProperty(entityVertex, vertexPropertyName, Long.class)); - break; - default: - break; - } - - return ret; - } - - private AtlasObjectId mapVertexToObjectId(AtlasVertex entityVertex, final String edgeLabel, final AtlasEdge optionalEdge, - AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException { - AtlasObjectId ret = new AtlasObjectId(); - AtlasEdge edge; - - if (optionalEdge == null) { - edge = graphHelper.getEdgeForLabel(entityVertex, edgeLabel); - } else { - edge = optionalEdge; - } - - if (GraphHelper.elementExists(edge)) { - final AtlasVertex referenceVertex = edge.getInVertex(); - final String guid = GraphHelper.getSingleValuedProperty(referenceVertex, Constants.GUID_PROPERTY_KEY, String.class); - final String typeName = GraphHelper.getTypeName(referenceVertex); - - if (entityExtInfo != null && isOwnedAttribute) { - mapVertexToAtlasEntity(guid, entityExtInfo); - } - - ret = new AtlasObjectId(typeName, guid); - } - - return ret; - } - - private AtlasStruct mapVertexToStruct(AtlasVertex entityVertex, final String edgeLabel, final AtlasEdge optionalEdge, - AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException { - AtlasStruct ret = null; - AtlasEdge edge; - - if (optionalEdge == null) { - edge = graphHelper.getEdgeForLabel(entityVertex, edgeLabel); - } else { - edge = optionalEdge; - } - - if (GraphHelper.elementExists(edge)) { - final AtlasVertex referenceVertex = edge.getInVertex(); - ret = new AtlasStruct(GraphHelper.getSingleValuedProperty(referenceVertex, Constants.ENTITY_TYPE_PROPERTY_KEY, String.class)); - - mapAttributes(referenceVertex, ret, entityExtInfo); - } - - return ret; - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/a67e8962/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java ---------------------------------------------------------------------- diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java index 197e46a..ed1e494 100644 --- a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java +++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java @@ -69,8 +69,6 @@ import java.util.List; import java.util.Map; import static org.apache.atlas.TestUtils.COLUMNS_ATTR_NAME; -import static org.apache.atlas.TestUtils.COLUMN_TYPE; -import static org.apache.atlas.TestUtils.NAME; import static org.apache.atlas.TestUtils.TABLE_TYPE; import static org.apache.atlas.TestUtils.randomString; import static org.testng.Assert.assertEquals; @@ -133,7 +131,7 @@ public class AtlasEntityStoreV1Test { MapVertexMapper mapVertexMapper = new MapVertexMapper(deleteHandler); - entityStore = new AtlasEntityStoreV1(new EntityGraphMapper(arrVertexMapper, mapVertexMapper, deleteHandler), new GraphEntityMapper()); + entityStore = new AtlasEntityStoreV1(new EntityGraphMapper(arrVertexMapper, mapVertexMapper, deleteHandler)); entityStore.init(typeRegistry); RequestContextV1.clear();