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 08685200BAA for ; Thu, 13 Oct 2016 01:21:41 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0735E160AEE; Wed, 12 Oct 2016 23:21:41 +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 57231160ACA for ; Thu, 13 Oct 2016 01:21:39 +0200 (CEST) Received: (qmail 76715 invoked by uid 500); 12 Oct 2016 23:21:38 -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 76706 invoked by uid 99); 12 Oct 2016 23:21:38 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 12 Oct 2016 23:21:38 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 10C1FC0A04 for ; Wed, 12 Oct 2016 23:21:38 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-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 (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id jtcz99bpcXfC for ; Wed, 12 Oct 2016 23:21:35 +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 50B405F1EE for ; Wed, 12 Oct 2016 23:21:33 +0000 (UTC) Received: (qmail 76644 invoked by uid 99); 12 Oct 2016 23:21:32 -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; Wed, 12 Oct 2016 23:21:32 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 630A5DFF8C; Wed, 12 Oct 2016 23:21:32 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sumasai@apache.org To: commits@atlas.incubator.apache.org Date: Wed, 12 Oct 2016 23:21:32 -0000 Message-Id: <039d956dfe9d4de4a78be696db8c7ce3@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/3] incubator-atlas git commit: ATLAS-1223 Type REST API v2 implementation (apoorvnaik via sumasai) archived-at: Wed, 12 Oct 2016 23:21:41 -0000 Repository: incubator-atlas Updated Branches: refs/heads/master 0d01f356c -> f8fe0945c http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/f8fe0945/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java ---------------------------------------------------------------------- diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java new file mode 100644 index 0000000..9726530 --- /dev/null +++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasTypeDefGraphStoreV1.java @@ -0,0 +1,307 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.atlas.repository.store.graph.v1; + +import com.google.common.base.Preconditions; + +import org.apache.atlas.exception.AtlasBaseException; +import org.apache.atlas.model.typedef.AtlasBaseTypeDef; +import org.apache.atlas.model.typedef.AtlasClassificationDef; +import org.apache.atlas.model.typedef.AtlasEntityDef; +import org.apache.atlas.model.typedef.AtlasEnumDef; +import org.apache.atlas.model.typedef.AtlasStructDef; +import org.apache.atlas.repository.Constants; +import org.apache.atlas.repository.graph.AtlasGraphProvider; +import org.apache.atlas.repository.graphdb.AtlasEdge; +import org.apache.atlas.repository.graphdb.AtlasEdgeDirection; +import org.apache.atlas.repository.graphdb.AtlasGraph; +import org.apache.atlas.repository.graphdb.AtlasVertex; +import org.apache.atlas.repository.store.graph.AtlasTypeDefGraphStore; +import org.apache.atlas.typesystem.types.DataTypes.TypeCategory; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Date; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; + +import static org.apache.atlas.repository.Constants.TYPE_CATEGORY_PROPERTY_KEY; +import static org.apache.atlas.repository.Constants.VERTEX_TYPE_PROPERTY_KEY; +import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.VERTEX_TYPE; + + +/** + * Graph persistence store for TypeDef - v1 + */ +public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore { + private static final Logger LOG = LoggerFactory.getLogger(AtlasTypeDefGraphStoreV1.class); + + protected final AtlasGraph atlasGraph = AtlasGraphProvider.getGraphInstance(); + + public AtlasTypeDefGraphStoreV1() { + super(); + + enumDefStore = new AtlasEnumDefStoreV1(this); + structDefStore = new AtlasStructDefStoreV1(this); + classificationDefStore = new AtlasClassificationDefStoreV1(this); + entityDefStore = new AtlasEntityDefStoreV1(this); + } + + @Override + public void init() { + + } + + public AtlasGraph getAtlasGraph() { return atlasGraph; } + + public AtlasVertex findTypeVertexByName(String typeName) { + Iterator results = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE) + .has(Constants.TYPENAME_PROPERTY_KEY, typeName) + .vertices().iterator(); + + AtlasVertex ret = (results != null && results.hasNext()) ? (AtlasVertex) results.next() : null; + + return ret; + } + + public AtlasVertex findTypeVertexByNameAndCategory(String typeName, TypeCategory category) { + Iterator results = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE) + .has(Constants.TYPENAME_PROPERTY_KEY, typeName) + .has(TYPE_CATEGORY_PROPERTY_KEY, category) + .vertices().iterator(); + + AtlasVertex ret = (results != null && results.hasNext()) ? (AtlasVertex) results.next() : null; + + return ret; + } + + public AtlasVertex findTypeVertexByGuid(String typeGuid) { + Iterator vertices = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE) + .has(Constants.GUID_PROPERTY_KEY, typeGuid) + .vertices().iterator(); + + AtlasVertex ret = (vertices != null && vertices.hasNext()) ? vertices.next() : null; + + return ret; + } + + public AtlasVertex findTypeVertexByGuidAndCategory(String typeGuid, TypeCategory category) { + Iterator vertices = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE) + .has(Constants.GUID_PROPERTY_KEY, typeGuid) + .has(TYPE_CATEGORY_PROPERTY_KEY, category) + .vertices().iterator(); + + AtlasVertex ret = (vertices != null && vertices.hasNext()) ? vertices.next() : null; + + return ret; + } + + public Iterator findTypeVerticesByCategory(TypeCategory category) { + Iterator ret = atlasGraph.query().has(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE) + .has(TYPE_CATEGORY_PROPERTY_KEY, category) + .vertices().iterator(); + + return ret; + } + + public AtlasVertex createTypeVertex(AtlasBaseTypeDef typeDef) { + // Validate all the required checks + Preconditions.checkArgument(StringUtils.isNotBlank(typeDef.getName()), "Type name can't be null/empty"); + Preconditions.checkArgument(StringUtils.isNotBlank(typeDef.getTypeVersion()), "Type version can't be null/empty"); + Preconditions.checkArgument(typeDef.getVersion() != null, "Version can't be null"); + + AtlasVertex ret = atlasGraph.addVertex(); + + if (StringUtils.isBlank(typeDef.getTypeVersion())) { + typeDef.setTypeVersion("1.0"); + } + + if (StringUtils.isBlank(typeDef.getGuid())) { + typeDef.setGuid(UUID.randomUUID().toString()); + } + + if (typeDef.getCreateTime() == null) { + typeDef.setCreateTime(new Date()); + } + + if (typeDef.getUpdateTime() == null) { + typeDef.setUpdateTime(new Date()); + } + + ret.setProperty(VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE); // Mark as type vertex + ret.setProperty(TYPE_CATEGORY_PROPERTY_KEY, getTypeCategory(typeDef)); + + ret.setProperty(Constants.TYPENAME_PROPERTY_KEY, typeDef.getName()); + ret.setProperty(Constants.TYPEDESCRIPTION_PROPERTY_KEY, + StringUtils.isNotBlank(typeDef.getDescription()) ? typeDef.getDescription() : typeDef.getName()); + ret.setProperty(Constants.TYPEVERSION_PROPERTY_KEY, typeDef.getTypeVersion()); + ret.setProperty(Constants.GUID_PROPERTY_KEY, typeDef.getGuid()); + ret.setProperty(Constants.TIMESTAMP_PROPERTY_KEY, typeDef.getCreateTime().getTime()); + ret.setProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY, typeDef.getUpdateTime().getTime()); + ret.setProperty(Constants.VERSION_PROPERTY_KEY, typeDef.getVersion()); + + return ret; + } + + public void deleteTypeVertex(AtlasVertex vertex) throws AtlasBaseException { + Iterator inEdges = vertex.getEdges(AtlasEdgeDirection.IN).iterator(); + + if (inEdges.hasNext()) { + throw new AtlasBaseException("has references"); + } + + Iterable edges = vertex.getEdges(AtlasEdgeDirection.OUT); + + for (AtlasEdge edge : edges) { + atlasGraph.removeEdge(edge); + } + + atlasGraph.removeVertex(vertex); + } + + public void vertexToTypeDef(AtlasVertex vertex, AtlasBaseTypeDef typeDef) { + String name = vertex.getProperty(Constants.TYPENAME_PROPERTY_KEY, String.class); + String description = vertex.getProperty(Constants.TYPEDESCRIPTION_PROPERTY_KEY, String.class); + String typeVersion = vertex.getProperty(Constants.TYPEVERSION_PROPERTY_KEY, String.class); + String guid = vertex.getProperty(Constants.GUID_PROPERTY_KEY, String.class); + Long createTime = vertex.getProperty(Constants.TIMESTAMP_PROPERTY_KEY, Long.class); + Long updateTime = vertex.getProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY, Long.class); + Long version = vertex.getProperty(Constants.VERSION_PROPERTY_KEY, Long.class); + + typeDef.setName(name); + typeDef.setDescription(description); + typeDef.setTypeVersion(typeVersion); + typeDef.setGuid(guid); + + if (createTime != null) { + typeDef.setCreateTime(new Date(createTime)); + } + + if (updateTime != null) { + typeDef.setUpdateTime(new Date(updateTime)); + } + + if (version != null) { + typeDef.setVersion(version); + } + } + + public boolean isTypeVertex(AtlasVertex vertex) { + String vertexType = vertex.getProperty(Constants.VERTEX_TYPE_PROPERTY_KEY, String.class); + + boolean ret = VERTEX_TYPE.equals(vertexType); + + return ret; + } + + public boolean isTypeVertex(AtlasVertex vertex, TypeCategory category) { + boolean ret = false; + + if (isTypeVertex(vertex)) { + TypeCategory vertexCategory = vertex.getProperty(Constants.TYPE_CATEGORY_PROPERTY_KEY, TypeCategory.class); + + ret = category.equals(vertexCategory); + } + + return ret; + } + + public boolean isTypeVertex(AtlasVertex vertex, TypeCategory[] categories) { + boolean ret = false; + + if (isTypeVertex(vertex)) { + TypeCategory vertexCategory = vertex.getProperty(TYPE_CATEGORY_PROPERTY_KEY, TypeCategory.class); + + for (TypeCategory category : categories) { + if (category.equals(vertexCategory)) { + ret = true; + + break; + } + } + } + + return ret; + } + + public AtlasEdge getOrCreateEdge(AtlasVertex outVertex, AtlasVertex inVertex, String edgeLabel) { + AtlasEdge ret = null; + Iterable edges = outVertex.getEdges(AtlasEdgeDirection.OUT, edgeLabel); + + for (AtlasEdge edge : edges) { + if (edge.getInVertex().getId().equals(inVertex.getId())) { + ret = edge; + break; + } + } + + if (ret == null) { + ret = addEdge(outVertex, inVertex, edgeLabel); + } + + return ret; + } + + public AtlasEdge addEdge(AtlasVertex outVertex, AtlasVertex inVertex, String edgeLabel) { + AtlasEdge ret = atlasGraph.addEdge(outVertex, inVertex, edgeLabel); + + return ret; + } + + public void createSuperTypeEdges(AtlasVertex vertex, Set superTypes) { + if (CollectionUtils.isNotEmpty(superTypes)) { + for (String superType : superTypes) { + AtlasVertex superTypeVertex = findTypeVertexByNameAndCategory(superType, TypeCategory.CLASS); + + getOrCreateEdge(vertex, superTypeVertex, AtlasGraphUtilsV1.SUPERTYPE_EDGE_LABEL); + } + } + // TODO: remove any other superType edges, if any exists + } + + public Set getSuperTypeNames(AtlasVertex vertex) { + Set ret = new HashSet<>(); + Iterable edges = vertex.getEdges(AtlasEdgeDirection.OUT, AtlasGraphUtilsV1.SUPERTYPE_EDGE_LABEL); + + for (AtlasEdge edge : edges) { + ret.add(edge.getInVertex().getProperty(Constants.TYPENAME_PROPERTY_KEY, String.class)); + } + + return ret; + } + + private TypeCategory getTypeCategory(AtlasBaseTypeDef typeDef) { + TypeCategory ret = null; + + if (typeDef instanceof AtlasEntityDef) { + ret = TypeCategory.CLASS; + } else if (typeDef instanceof AtlasClassificationDef) { + ret = TypeCategory.TRAIT; + } else if (typeDef instanceof AtlasStructDef) { + ret = TypeCategory.STRUCT; + } else if (typeDef instanceof AtlasEnumDef) { + ret = TypeCategory.ENUM; + } + + return ret; + } +} http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/f8fe0945/repository/src/main/java/org/apache/atlas/repository/util/FilterUtil.java ---------------------------------------------------------------------- diff --git a/repository/src/main/java/org/apache/atlas/repository/util/FilterUtil.java b/repository/src/main/java/org/apache/atlas/repository/util/FilterUtil.java new file mode 100644 index 0000000..08d09f7 --- /dev/null +++ b/repository/src/main/java/org/apache/atlas/repository/util/FilterUtil.java @@ -0,0 +1,105 @@ +/** + * 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.util; + +import org.apache.atlas.model.SearchFilter; +import org.apache.atlas.model.typedef.AtlasBaseTypeDef; +import org.apache.atlas.model.typedef.AtlasClassificationDef; +import org.apache.atlas.model.typedef.AtlasEntityDef; +import org.apache.atlas.model.typedef.AtlasEnumDef; +import org.apache.atlas.model.typedef.AtlasStructDef; +import org.apache.commons.collections.Predicate; +import org.apache.commons.collections.PredicateUtils; +import org.apache.commons.collections.functors.NotPredicate; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.List; + +public class FilterUtil { + public static Predicate getPredicateFromSearchFilter(SearchFilter searchFilter) { + List predicates = new ArrayList<>(); + final String type = searchFilter.getParam(SearchFilter.PARAM_TYPE); + final String supertype = searchFilter.getParam(SearchFilter.PARAM_SUPERTYPE); + final String notSupertype = searchFilter.getParam(SearchFilter.PARAM_NOT_SUPERTYPE); + + // Add filter for the type/category + if (StringUtils.isNotBlank(type)) { + predicates.add(getTypePredicate(type)); + } + + // Add filter for the supertype + if (StringUtils.isNotBlank(supertype)) { + predicates.add(getSuperTypePredicate(supertype)); + } + + // Add filter for the supertype negation + if (StringUtils.isNotBlank(notSupertype)) { + predicates.add(new NotPredicate(getSuperTypePredicate(notSupertype))); + } + + return PredicateUtils.allPredicate(predicates); + } + + private static Predicate getSuperTypePredicate(final String supertype) { + return new Predicate() { + private boolean isClassificationDef(Object o) { + return o instanceof AtlasClassificationDef; + } + + private boolean isEntityDef(Object o) { + return o instanceof AtlasEntityDef; + } + + @Override + public boolean evaluate(Object o) { + return (isClassificationDef(o) && + ((AtlasClassificationDef) o).getSuperTypes().contains(supertype))|| + (isEntityDef(o) && + ((AtlasEntityDef)o).getSuperTypes().contains(supertype)); + } + }; + } + + private static Predicate getTypePredicate(final String type) { + return new Predicate() { + @Override + public boolean evaluate(Object o) { + if (o instanceof AtlasBaseTypeDef) { + switch (type.toUpperCase()) { + case "CLASS": + case "ENTITY": + return o instanceof AtlasEntityDef; + case "TRAIT": + case "CLASSIFICATION": + return o instanceof AtlasClassificationDef; + case "STRUCT": + return o instanceof AtlasStructDef; + case "ENUM": + return o instanceof AtlasEnumDef; + default: + // This shouldn't have happened + return false; + } + } else { + return false; + } + } + }; + } +} http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/f8fe0945/repository/src/main/java/org/apache/atlas/util/TypeDefSorter.java ---------------------------------------------------------------------- diff --git a/repository/src/main/java/org/apache/atlas/util/TypeDefSorter.java b/repository/src/main/java/org/apache/atlas/util/TypeDefSorter.java new file mode 100644 index 0000000..21e6d3f --- /dev/null +++ b/repository/src/main/java/org/apache/atlas/util/TypeDefSorter.java @@ -0,0 +1,81 @@ +/** + * 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.util; + +import com.google.common.collect.ImmutableSet; + +import org.apache.atlas.model.typedef.AtlasClassificationDef; +import org.apache.atlas.model.typedef.AtlasEntityDef; +import org.apache.atlas.model.typedef.AtlasStructDef; +import org.apache.atlas.typesystem.types.HierarchicalType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class TypeDefSorter { + private static final Logger LOG = LoggerFactory.getLogger(TypeDefSorter.class); + + public static List sortTypes(List types) { + Map typesByName = new HashMap<>(); + for (T type : types) { + typesByName.put(type.getName(), type); + } + List result = new ArrayList<>(types.size()); + Set processed = new HashSet<>(); + for (T type : types) { + addToResult(type, result, processed, typesByName); + } + return result; + } + + private static void addToResult(T type, List result, + Set processed, + Map typesByName) { + if (processed.contains(type)) { + return; + } + processed.add(type); + Set superTypeNames = new HashSet<>(); + try { + AtlasClassificationDef classificationDef = AtlasClassificationDef.class.cast(type); + superTypeNames.addAll(classificationDef.getSuperTypes()); + } catch (ClassCastException ex) { + LOG.warn("Casting to ClassificationDef failed"); + } + try { + AtlasEntityDef entityDef = AtlasEntityDef.class.cast(type); + superTypeNames.addAll(entityDef.getSuperTypes()); + } catch (ClassCastException ex) { + LOG.warn("Casting to AtlasEntityDef failed"); + } + for (String superTypeName : superTypeNames) { + // Recursively add any supertypes first to the result. + T superType = typesByName.get(superTypeName); + if (superType != null) { + addToResult(superType, result, processed, typesByName); + } + } + result.add(type); + } +} http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/f8fe0945/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java ---------------------------------------------------------------------- diff --git a/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java b/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java index 429df65..728bec3 100644 --- a/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java +++ b/webapp/src/main/java/org/apache/atlas/web/rest/TypesREST.java @@ -17,16 +17,9 @@ */ package org.apache.atlas.web.rest; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.DELETE; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.PUT; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; +import com.google.inject.Inject; +import org.apache.atlas.exception.AtlasBaseException; import org.apache.atlas.model.SearchFilter; import org.apache.atlas.model.typedef.AtlasClassificationDef; import org.apache.atlas.model.typedef.AtlasClassificationDef.AtlasClassificationDefs; @@ -36,24 +29,61 @@ import org.apache.atlas.model.typedef.AtlasEnumDef; import org.apache.atlas.model.typedef.AtlasEnumDef.AtlasEnumDefs; import org.apache.atlas.model.typedef.AtlasStructDef; import org.apache.atlas.model.typedef.AtlasStructDef.AtlasStructDefs; +import org.apache.atlas.model.typedef.AtlasTypesDef; +import org.apache.atlas.store.AtlasTypeDefStore; +import org.apache.atlas.type.AtlasTypeRegistry; import org.apache.atlas.web.util.Servlets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Set; -@Path("types") +import javax.inject.Singleton; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.Response; + + +@Path("v2/types") +@Singleton public class TypesREST { private static final Logger LOG = LoggerFactory.getLogger(TypesREST.class); + private AtlasTypeDefStore typeDefStore; + + @Context + private HttpServletRequest httpServletRequest; + + @Inject + public TypesREST(AtlasTypeDefStore typeDefStore, AtlasTypeRegistry atlasTypeRegistry) { + LOG.info("new TypesREST"); + this.typeDefStore = typeDefStore; + } + + /******* EnumDef REST calls *******/ + @POST @Path("/enumdef") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasEnumDef createEnumDef(AtlasEnumDef enumDef) throws Exception { AtlasEnumDef ret = null; - // TODO: ret = store.createEnumDef() - - return ret; + try { + ret = typeDefStore.createEnumDef(enumDef); + return ret; + } catch (AtlasBaseException ex) { + throw new WebApplicationException(Servlets.getErrorResponse(ex, Response.Status.BAD_REQUEST)); + } } @GET @@ -62,7 +92,7 @@ public class TypesREST { public AtlasEnumDef getEnumDefByName(@PathParam("name") String name) throws Exception { AtlasEnumDef ret = null; - // TODO: ret = store.getEnumDefByName(name) + ret = typeDefStore.getEnumDefByName(name); return ret; } @@ -73,29 +103,31 @@ public class TypesREST { public AtlasEnumDef getEnumDefByGuid(@PathParam("guid") String guid) throws Exception { AtlasEnumDef ret = null; - // TODO: ret = store.getEnumDefByGuid(guid) + ret = typeDefStore.getEnumDefByGuid(guid); return ret; } @PUT @Path("/enumdef/name/{name}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasEnumDef updateEnumDefByName(@PathParam("name") String name, AtlasEnumDef enumDef) throws Exception { AtlasEnumDef ret = null; - // TODO: ret = store.updateEnumDefByName(name, enumDef) + ret = typeDefStore.updateEnumDefByName(name, enumDef); return ret; } @PUT @Path("/enumdef/guid/{guid}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasEnumDef updateEnumDefByGuid(@PathParam("guid") String guid, AtlasEnumDef enumDef) throws Exception { AtlasEnumDef ret = null; - // TODO: ret = store.updateEnumDefByGuid(guid, enumDef) + ret = typeDefStore.updateEnumDefByGuid(guid, enumDef); return ret; } @@ -104,38 +136,46 @@ public class TypesREST { @Path("/enumdef/name/{name}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteEnumDefByName(@PathParam("name") String name) throws Exception { - // TODO: store.deleteEnumDefByName(name) + typeDefStore.deleteEnumDefByName(name); } @DELETE @Path("/enumdef/guid/{guid}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteEnumDefByGuid(@PathParam("guid") String guid) throws Exception { - // TODO: store.deleteEnumDefByGuid(guid) + typeDefStore.deleteEnumDefByGuid(guid); } @GET @Path("/enumdef") @Produces(Servlets.JSON_MEDIA_TYPE) - public AtlasEnumDefs searchEnumDefs(@Context HttpServletRequest request) throws Exception { + public AtlasEnumDefs searchEnumDefs() throws Exception { AtlasEnumDefs ret = null; - // TODO: SearchFilter filter = getSearchFilter(request); - // TODO: ret = store.searchEnumDefs(filter); + SearchFilter filter = getSearchFilter(); + + ret = typeDefStore.searchEnumDefs(filter); return ret; } + /******* StructDef REST calls *******/ + @POST @Path("/structdef") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasStructDef createStructDef(AtlasStructDef structDef) throws Exception { AtlasStructDef ret = null; - // TODO: ret = store.createStructDef() + try { + ret = typeDefStore.createStructDef(structDef); + return ret; + } catch (AtlasBaseException ex) { + throw new WebApplicationException(Servlets.getErrorResponse(ex, Response.Status.BAD_REQUEST)); + } - return ret; } @GET @@ -144,7 +184,7 @@ public class TypesREST { public AtlasStructDef getStructDefByName(@PathParam("name") String name) throws Exception { AtlasStructDef ret = null; - // TODO: ret = store.getStructDefByName(name) + ret = typeDefStore.getStructDefByName(name); return ret; } @@ -155,29 +195,31 @@ public class TypesREST { public AtlasStructDef getStructDefByGuid(@PathParam("guid") String guid) throws Exception { AtlasStructDef ret = null; - // TODO: ret = store.getStructDefByGuid(guid) + ret = typeDefStore.getStructDefByGuid(guid); return ret; } @PUT @Path("/structdef/name/{name}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasStructDef updateStructDefByName(@PathParam("name") String name, AtlasStructDef structDef) throws Exception { AtlasStructDef ret = null; - // TODO: ret = store.updateStructDefByName(name, structDef) + ret = typeDefStore.updateStructDefByName(name, structDef); return ret; } @PUT @Path("/structdef/guid/{guid}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasStructDef updateStructDefByGuid(@PathParam("guid") String guid, AtlasStructDef structDef) throws Exception { AtlasStructDef ret = null; - // TODO: ret = store.updateStructDefByGuid(guid, structDef) + ret = typeDefStore.updateStructDefByGuid(guid, structDef); return ret; } @@ -186,38 +228,43 @@ public class TypesREST { @Path("/structdef/name/{name}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteStructDefByName(@PathParam("name") String name) throws Exception { - // TODO: store.deleteStructDefByName(name) + typeDefStore.deleteStructDefByName(name); } @DELETE @Path("/structdef/guid/{guid}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteStructDefByGuid(@PathParam("guid") String guid) throws Exception { - // TODO: store.deleteStructDefByGuid(guid) + typeDefStore.deleteStructDefByGuid(guid); } @GET @Path("/structdef") @Produces(Servlets.JSON_MEDIA_TYPE) - public AtlasStructDefs searchStructDefs(@Context HttpServletRequest request) throws Exception { + public AtlasStructDefs searchStructDefs() throws Exception { AtlasStructDefs ret = null; - // TODO: SearchFilter filter = getSearchFilter(request); - // TODO: ret = store.searchStructDefs(filter); + SearchFilter filter = getSearchFilter(); + ret = typeDefStore.searchStructDefs(filter); return ret; } + /******* ClassificationDef REST calls *******/ @POST @Path("/classificationdef") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasClassificationDef createClassificationDef(AtlasClassificationDef classificationDef) throws Exception { AtlasClassificationDef ret = null; - // TODO: ret = store.createClassificationDef() - - return ret; + try { + ret = typeDefStore.createClassificationDef(classificationDef); + return ret; + } catch (AtlasBaseException ex) { + throw new WebApplicationException(Servlets.getErrorResponse(ex, Response.Status.BAD_REQUEST)); + } } @GET @@ -226,7 +273,7 @@ public class TypesREST { public AtlasClassificationDef getClassificationDefByName(@PathParam("name") String name) throws Exception { AtlasClassificationDef ret = null; - // TODO: ret = store.getClassificationDefByName(name) + ret = typeDefStore.getClassificationDefByName(name); return ret; } @@ -237,29 +284,31 @@ public class TypesREST { public AtlasClassificationDef getClassificationDefByGuid(@PathParam("guid") String guid) throws Exception { AtlasClassificationDef ret = null; - // TODO: ret = store.getClassificationDefByGuid(guid) + ret = typeDefStore.getClassificationDefByGuid(guid); return ret; } @PUT @Path("/classificationdef/name/{name}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasClassificationDef updateClassificationDefByName(@PathParam("name") String name, AtlasClassificationDef classificationDef) throws Exception { AtlasClassificationDef ret = null; - // TODO: ret = store.updateClassificationDefByName(name, classificationDef) + ret = typeDefStore.updateClassificationDefByName(name, classificationDef); return ret; } @PUT @Path("/classificationdef/guid/{guid}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasClassificationDef updateClassificationDefByGuid(@PathParam("guid") String guid, AtlasClassificationDef classificationDef) throws Exception { AtlasClassificationDef ret = null; - // TODO: ret = store.updateClassificationDefByGuid(guid, classificationDef) + ret = typeDefStore.updateClassificationDefByGuid(guid, classificationDef); return ret; } @@ -268,38 +317,44 @@ public class TypesREST { @Path("/classificationdef/name/{name}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteClassificationDefByName(@PathParam("name") String name) throws Exception { - // TODO: store.deleteClassificationDefByName(name) + typeDefStore.deleteClassificationDefByName(name); } @DELETE @Path("/classificationdef/guid/{guid}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteClassificationDefByGuid(@PathParam("guid") String guid) throws Exception { - // TODO: store.deleteClassificationDefByGuid(guid) + typeDefStore.deleteClassificationDefByGuid(guid); } @GET @Path("/classificationdef") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) - public AtlasClassificationDefs searchClassificationDefs(SearchFilter filter) throws Exception { + public AtlasClassificationDefs searchClassificationDefs() throws Exception { AtlasClassificationDefs ret = null; - // TODO: SearchFilter filter = getSearchFilter(request); - // TODO: ret = store.searchClassificationDefs(filter); + SearchFilter filter = getSearchFilter(); + ret = typeDefStore.searchClassificationDefs(filter); return ret; } + /******* EntityDef REST calls *******/ @POST @Path("/entitydef") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasEntityDef createEntityDef(AtlasEntityDef entityDef) throws Exception { AtlasEntityDef ret = null; - // TODO: ret = store.createEntityDef() - - return ret; + try { + ret = typeDefStore.createEntityDefs(entityDef); + return ret; + } catch (AtlasBaseException ex) { + throw new WebApplicationException(Servlets.getErrorResponse(ex, Response.Status.BAD_REQUEST)); + } } @GET @@ -308,7 +363,7 @@ public class TypesREST { public AtlasEntityDef getEntityDefByName(@PathParam("name") String name) throws Exception { AtlasEntityDef ret = null; - // TODO: ret = store.getEntityDefByName(name) + ret = typeDefStore.getEntityDefByName(name); return ret; } @@ -316,32 +371,34 @@ public class TypesREST { @GET @Path("/entitydef/guid/{guid}") @Produces(Servlets.JSON_MEDIA_TYPE) - public AtlasEntityDef getEntityDefByIdByGuid(@PathParam("guid") String guid) throws Exception { + public AtlasEntityDef getEntityDefByGuid(@PathParam("guid") String guid) throws Exception { AtlasEntityDef ret = null; - // TODO: ret = store.getEntityDefByGuid(guid) + ret = typeDefStore.getEntityDefByGuid(guid); return ret; } @PUT @Path("/entitydef/name/{name}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasEntityDef updateEntityDefByName(@PathParam("name") String name, AtlasEntityDef entityDef) throws Exception { AtlasEntityDef ret = null; - // TODO: ret = store.updateEntityDefByName(name, entityDef) + ret = typeDefStore.updateEntityDefByName(name, entityDef); return ret; } @PUT @Path("/entitydef/guid/{guid}") + @Consumes(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE) public AtlasEntityDef updateEntityDefByGuid(@PathParam("guid") String guid, AtlasEntityDef entityDef) throws Exception { AtlasEntityDef ret = null; - // TODO: ret = store.updateEntityDefByGuid(guid, entityDef) + ret = typeDefStore.updateEntityDefByGuid(guid, entityDef); return ret; } @@ -350,25 +407,109 @@ public class TypesREST { @Path("/entitydef/name/{name}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteEntityDef(@PathParam("name") String name) throws Exception { - // TODO: store.deleteEntityDefByName(name) + typeDefStore.deleteEntityDefByName(name); } @DELETE @Path("/entitydef/guid/{guid}") @Produces(Servlets.JSON_MEDIA_TYPE) public void deleteEntityDefByGuid(@PathParam("guid") String guid) throws Exception { - // TODO: store.deleteEntityDefByGuid(guid) + typeDefStore.deleteEntityDefByGuid(guid); } @GET @Path("/entitydef") @Produces(Servlets.JSON_MEDIA_TYPE) - public AtlasEntityDefs searchEntityDefs(SearchFilter filter) throws Exception { + public AtlasEntityDefs searchEntityDefs() throws Exception { AtlasEntityDefs ret = null; - // TODO: SearchFilter filter = getSearchFilter(request); - // TODO: ret = store.searchEntityDefs(filter); + SearchFilter filter = getSearchFilter(); + ret = typeDefStore.searchEntityDefs(filter); + + return ret; + } + + /******************************************************************/ + /** Bulk API operations **/ + /******************************************************************/ + + + /** + * Bulk retrieval API for retrieving all type definitions in Atlas + * @return A composite wrapper object with lists of all type definitions + * @throws Exception + */ + @GET + @Path("/typedefs") + @Produces(Servlets.JSON_MEDIA_TYPE) + public AtlasTypesDef getAllTypeDefs() throws Exception { + SearchFilter searchFilter = getSearchFilter(); + + AtlasTypesDef typesDef = null; + + try { + typesDef = typeDefStore.searchTypesDef(searchFilter); + } catch (AtlasBaseException ex) { + throw new WebApplicationException(Servlets.getErrorResponse(ex, Response.Status.NOT_FOUND)); + } + + return typesDef; + } + + /** + * Bulk create APIs for all atlas type definitions, only new definitions will be created. + * Any changes to the existing definitions will be discarded + * @param typesDef A composite wrapper object with corresponding lists of the type definition + * @return A composite wrapper object with lists of type definitions that were successfully + * created + * @throws Exception + */ + @POST + @Path("/typedefs") + @Consumes(Servlets.JSON_MEDIA_TYPE) + @Produces(Servlets.JSON_MEDIA_TYPE) + public AtlasTypesDef createAtlasTypeDefs(final AtlasTypesDef typesDef) throws Exception { + AtlasTypesDef ret = null; + try { + ret = typeDefStore.createTypesDef(typesDef); + } catch (AtlasBaseException ex) { + throw new WebApplicationException(Servlets.getErrorResponse(ex, Response.Status.BAD_REQUEST)); + } + return ret; + } + + /** + * Bulk update API for all types, changes detected in the type definitions would be persisted + * @param typesDef A composite object that captures all type definition changes + * @return A composite object with lists of type definitions that were updated + * @throws Exception + */ + @PUT + @Path("/typedefs") + @Consumes(Servlets.JSON_MEDIA_TYPE) + @Produces(Servlets.JSON_MEDIA_TYPE) + public AtlasTypesDef updateAtlasTypeDefs(final AtlasTypesDef typesDef) throws Exception { + AtlasTypesDef ret = null; + + try { + ret = typeDefStore.updateTypesDef(typesDef); + } catch (AtlasBaseException ex) { + throw new WebApplicationException(Servlets.getErrorResponse(ex, Response.Status.NOT_MODIFIED)); + } return ret; } -} + + /** + * Populate a SearchFilter on the basis of the Query Parameters + * @return + */ + private SearchFilter getSearchFilter() { + SearchFilter ret = new SearchFilter(); + Set keySet = httpServletRequest.getParameterMap().keySet(); + for (String key : keySet) { + ret.setParam(String.valueOf(key), String.valueOf(httpServletRequest.getParameter(key))); + } + + return ret; + }} http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/f8fe0945/webapp/src/main/java/org/apache/atlas/web/rest/module/RestModule.java ---------------------------------------------------------------------- diff --git a/webapp/src/main/java/org/apache/atlas/web/rest/module/RestModule.java b/webapp/src/main/java/org/apache/atlas/web/rest/module/RestModule.java new file mode 100644 index 0000000..4d9f854 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/web/rest/module/RestModule.java @@ -0,0 +1,33 @@ +/** + * 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.rest.module; + +import com.google.inject.AbstractModule; + +import org.apache.atlas.type.AtlasTypeRegistry; + +/** + * @author anaik + */ + +public class RestModule extends AbstractModule { + @Override + protected void configure() { + bind(AtlasTypeRegistry.class).to(AtlasTypeRegistry.class).asEagerSingleton(); + } +} http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/f8fe0945/webapp/src/main/webapp/WEB-INF/web.xml ---------------------------------------------------------------------- diff --git a/webapp/src/main/webapp/WEB-INF/web.xml b/webapp/src/main/webapp/WEB-INF/web.xml index e2a85e9..c551814 100755 --- a/webapp/src/main/webapp/WEB-INF/web.xml +++ b/webapp/src/main/webapp/WEB-INF/web.xml @@ -27,7 +27,7 @@ guice.packages - org.apache.atlas.web.resources,org.apache.atlas.web.params + org.apache.atlas.web.resources,org.apache.atlas.web.params,org.apache.atlas.web.rest