Commit 1bfda02a by Shwetha GS

ATLAS-58 Make hive hook reliable (shwethags)

parent 016e36c0
...@@ -33,7 +33,7 @@ ...@@ -33,7 +33,7 @@
<packaging>jar</packaging> <packaging>jar</packaging>
<properties> <properties>
<hive.version>1.2.0</hive.version> <hive.version>1.2.1</hive.version>
<calcite.version>0.9.2-incubating</calcite.version> <calcite.version>0.9.2-incubating</calcite.version>
</properties> </properties>
...@@ -98,6 +98,11 @@ ...@@ -98,6 +98,11 @@
<artifactId>atlas-client</artifactId> <artifactId>atlas-client</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.apache.atlas</groupId>
<artifactId>atlas-notification</artifactId>
</dependency>
<!-- to bring up atlas server for integration tests --> <!-- to bring up atlas server for integration tests -->
<dependency> <dependency>
<groupId>org.apache.atlas</groupId> <groupId>org.apache.atlas</groupId>
...@@ -149,7 +154,7 @@ ...@@ -149,7 +154,7 @@
</configuration> </configuration>
</execution> </execution>
<execution> <execution>
<id>copy</id> <id>copy-hook</id>
<phase>package</phase> <phase>package</phase>
<goals> <goals>
<goal>copy</goal> <goal>copy</goal>
...@@ -209,6 +214,16 @@ ...@@ -209,6 +214,16 @@
<version>${project.version}</version> <version>${project.version}</version>
</artifactItem> </artifactItem>
<artifactItem> <artifactItem>
<groupId>${project.groupId}</groupId>
<artifactId>atlas-notification</artifactId>
<version>${project.version}</version>
</artifactItem>
<artifactItem>
<groupId>${project.groupId}</groupId>
<artifactId>atlas-common</artifactId>
<version>${project.version}</version>
</artifactItem>
<artifactItem>
<groupId>org.scala-lang</groupId> <groupId>org.scala-lang</groupId>
<artifactId>scala-compiler</artifactId> <artifactId>scala-compiler</artifactId>
<version>${scala.version}</version> <version>${scala.version}</version>
...@@ -228,6 +243,21 @@ ...@@ -228,6 +243,21 @@
<artifactId>scalap</artifactId> <artifactId>scalap</artifactId>
<version>${scala.version}</version> <version>${scala.version}</version>
</artifactItem> </artifactItem>
<artifactItem>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-multibindings</artifactId>
<version>${guice.version}</version>
</artifactItem>
<artifactItem>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_${scala.binary.version}</artifactId>
<version>${kafka.version}</version>
</artifactItem>
<artifactItem>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>${kafka.version}</version>
</artifactItem>
</artifactItems> </artifactItems>
</configuration> </configuration>
</execution> </execution>
...@@ -253,12 +283,16 @@ ...@@ -253,12 +283,16 @@
<useTestScope>true</useTestScope> <useTestScope>true</useTestScope>
<systemProperties> <systemProperties>
<systemProperty> <systemProperty>
<name>log4j.configuration</name>
<value>atlas-log4j.xml</value>
</systemProperty>
<systemProperty>
<name>atlas.log.dir</name> <name>atlas.log.dir</name>
<value>${project.build.directory}/logs</value> <value>${project.build.directory}/logs</value>
</systemProperty> </systemProperty>
</systemProperties> </systemProperties>
<stopKey>atlas-stop</stopKey> <stopKey>atlas-stop</stopKey>
<stopPort>41001</stopPort> <stopPort>21001</stopPort>
</configuration> </configuration>
<executions> <executions>
<execution> <execution>
...@@ -302,7 +336,6 @@ ...@@ -302,7 +336,6 @@
<configuration> <configuration>
<generateProjectInfo>false</generateProjectInfo> <generateProjectInfo>false</generateProjectInfo>
<generateReports>false</generateReports> <generateReports>false</generateReports>
<skip>false</skip>
</configuration> </configuration>
</plugin> </plugin>
</plugins> </plugins>
......
...@@ -67,7 +67,7 @@ done ...@@ -67,7 +67,7 @@ done
METADATA_LOG_DIR="${METADATA_LOG_DIR:-$BASEDIR/logs}" METADATA_LOG_DIR="${METADATA_LOG_DIR:-$BASEDIR/logs}"
export METADATA_LOG_DIR export METADATA_LOG_DIR
JAVA_PROPERTIES="$METADATA_OPTS -Datlas.log.dir=$METADATA_LOG_DIR -Datlas.log.file=import-hive.log" JAVA_PROPERTIES="$METADATA_OPTS -Datlas.log.dir=$METADATA_LOG_DIR -Datlas.log.file=import-hive.log -Dlog4j.configuration=atlas-log4j.xml"
shift shift
while [[ ${1} =~ ^\-D ]]; do while [[ ${1} =~ ^\-D ]]; do
......
...@@ -18,6 +18,7 @@ ...@@ -18,6 +18,7 @@
package org.apache.atlas.hive.bridge; package org.apache.atlas.hive.bridge;
import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClient;
import org.apache.atlas.AtlasServiceException; import org.apache.atlas.AtlasServiceException;
import org.apache.atlas.hive.model.HiveDataModelGenerator; import org.apache.atlas.hive.model.HiveDataModelGenerator;
...@@ -25,18 +26,17 @@ import org.apache.atlas.hive.model.HiveDataTypes; ...@@ -25,18 +26,17 @@ import org.apache.atlas.hive.model.HiveDataTypes;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.Struct; import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.json.InstanceSerialization; import org.apache.atlas.typesystem.json.InstanceSerialization;
import org.apache.atlas.typesystem.persistence.Id; import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang.StringEscapeUtils;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.metastore.api.Index;
import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.metastore.api.Order;
import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.SerDeInfo;
import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.Hive;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Partition;
import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.metadata.Table;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
...@@ -48,7 +48,6 @@ import org.slf4j.LoggerFactory; ...@@ -48,7 +48,6 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set;
/** /**
* A Bridge Utility that imports metadata from the Hive Meta Store * A Bridge Utility that imports metadata from the Hive Meta Store
...@@ -64,21 +63,23 @@ public class HiveMetaStoreBridge { ...@@ -64,21 +63,23 @@ public class HiveMetaStoreBridge {
private static final Logger LOG = LoggerFactory.getLogger(HiveMetaStoreBridge.class); private static final Logger LOG = LoggerFactory.getLogger(HiveMetaStoreBridge.class);
private final Hive hiveClient; public final Hive hiveClient;
private final AtlasClient atlasClient; private final AtlasClient atlasClient;
public HiveMetaStoreBridge(HiveConf hiveConf) throws Exception { public HiveMetaStoreBridge(HiveConf hiveConf, Configuration atlasConf) throws Exception {
this(hiveConf, null, null); this(hiveConf, atlasConf, null, null);
} }
/** /**
* Construct a HiveMetaStoreBridge. * Construct a HiveMetaStoreBridge.
* @param hiveConf hive conf * @param hiveConf hive conf
*/ */
public HiveMetaStoreBridge(HiveConf hiveConf, String doAsUser, UserGroupInformation ugi) throws Exception { public HiveMetaStoreBridge(HiveConf hiveConf, Configuration atlasConf, String doAsUser,
UserGroupInformation ugi) throws Exception {
clusterName = hiveConf.get(HIVE_CLUSTER_NAME, DEFAULT_CLUSTER_NAME); clusterName = hiveConf.get(HIVE_CLUSTER_NAME, DEFAULT_CLUSTER_NAME);
hiveClient = Hive.get(hiveConf); hiveClient = Hive.get(hiveConf);
atlasClient = new AtlasClient(hiveConf.get(ATLAS_ENDPOINT, DEFAULT_DGI_URL), ugi, doAsUser);
atlasClient = new AtlasClient(atlasConf.getString(ATLAS_ENDPOINT, DEFAULT_DGI_URL), ugi, doAsUser);
} }
public AtlasClient getAtlasClient() { public AtlasClient getAtlasClient() {
...@@ -95,72 +96,78 @@ public class HiveMetaStoreBridge { ...@@ -95,72 +96,78 @@ public class HiveMetaStoreBridge {
for (String databaseName : databases) { for (String databaseName : databases) {
Referenceable dbReference = registerDatabase(databaseName); Referenceable dbReference = registerDatabase(databaseName);
importTables(databaseName, dbReference); importTables(dbReference, databaseName);
} }
} }
public Referenceable registerDatabase(String databaseName) throws Exception { /**
Referenceable dbRef = getDatabaseReference(databaseName, clusterName); * Creates db entity
if (dbRef == null) { * @param hiveDB
LOG.info("Importing objects from databaseName : " + databaseName); * @return
Database hiveDB = hiveClient.getDatabase(databaseName); * @throws HiveException
*/
dbRef = new Referenceable(HiveDataTypes.HIVE_DB.getName()); public Referenceable createDBInstance(Database hiveDB) throws HiveException {
dbRef.set(HiveDataModelGenerator.NAME, hiveDB.getName().toLowerCase()); LOG.info("Importing objects from databaseName : " + hiveDB.getName());
dbRef.set(HiveDataModelGenerator.CLUSTER_NAME, clusterName);
dbRef.set("description", hiveDB.getDescription()); Referenceable dbRef = new Referenceable(HiveDataTypes.HIVE_DB.getName());
dbRef.set("locationUri", hiveDB.getLocationUri()); String dbName = hiveDB.getName().toLowerCase();
dbRef.set("parameters", hiveDB.getParameters()); dbRef.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, getDBQualifiedName(clusterName, dbName));
dbRef.set("ownerName", hiveDB.getOwnerName()); dbRef.set(HiveDataModelGenerator.NAME, dbName);
if (hiveDB.getOwnerType() != null) { dbRef.set(HiveDataModelGenerator.CLUSTER_NAME, clusterName);
dbRef.set("ownerType", hiveDB.getOwnerType().getValue()); dbRef.set("description", hiveDB.getDescription());
} dbRef.set("locationUri", hiveDB.getLocationUri());
dbRef.set("parameters", hiveDB.getParameters());
dbRef.set("ownerName", hiveDB.getOwnerName());
if (hiveDB.getOwnerType() != null) {
dbRef.set("ownerType", hiveDB.getOwnerType().getValue());
}
return dbRef;
}
dbRef = createInstance(dbRef); /**
* Checks if db is already registered, else creates and registers db entity
* @param databaseName
* @return
* @throws Exception
*/
private Referenceable registerDatabase(String databaseName) throws Exception {
Referenceable dbRef = getDatabaseReference(clusterName, databaseName);
if (dbRef == null) {
Database db = hiveClient.getDatabase(databaseName);
dbRef = createDBInstance(db);
dbRef = registerInstance(dbRef);
} else { } else {
LOG.info("Database {} is already registered with id {}", databaseName, dbRef.getId().id); LOG.info("Database {} is already registered with id {}", databaseName, dbRef.getId().id);
} }
return dbRef; return dbRef;
} }
public Referenceable createInstance(Referenceable referenceable) throws Exception { /**
* Registers an entity in atlas
* @param referenceable
* @return
* @throws Exception
*/
public Referenceable registerInstance(Referenceable referenceable) throws Exception {
String typeName = referenceable.getTypeName(); String typeName = referenceable.getTypeName();
LOG.debug("creating instance of type " + typeName); LOG.debug("creating instance of type " + typeName);
String entityJSON = InstanceSerialization.toJson(referenceable, true); String entityJSON = InstanceSerialization.toJson(referenceable, true);
LOG.debug("Submitting new entity {} = {}", referenceable.getTypeName(), entityJSON); LOG.debug("Submitting new entity {} = {}", referenceable.getTypeName(), entityJSON);
JSONObject jsonObject = atlasClient.createEntity(entityJSON); JSONArray guids = atlasClient.createEntity(entityJSON);
String guid = jsonObject.getString(AtlasClient.GUID); LOG.debug("created instance for type " + typeName + ", guid: " + guids);
LOG.debug("created instance for type " + typeName + ", guid: " + guid);
return new Referenceable(guid, referenceable.getTypeName(), null); return new Referenceable(guids.getString(0), referenceable.getTypeName(), null);
}
private void importTables(String databaseName, Referenceable databaseReferenceable) throws Exception {
List<String> hiveTables = hiveClient.getAllTables(databaseName);
for (String tableName : hiveTables) {
Referenceable tableReferenceable = registerTable(databaseReferenceable, databaseName, tableName);
// Import Partitions
Referenceable sdReferenceable = getSDForTable(databaseName, tableName);
registerPartitions(databaseName, tableName, tableReferenceable, sdReferenceable);
// Import Indexes
importIndexes(databaseName, tableName, databaseReferenceable, tableReferenceable);
}
} }
/** /**
* Gets reference for the database * Gets reference to the atlas entity for the database
*
*
* @param databaseName database Name * @param databaseName database Name
* @param clusterName cluster name * @param clusterName cluster name
* @return Reference for database if exists, else null * @return Reference for database if exists, else null
* @throws Exception * @throws Exception
*/ */
private Referenceable getDatabaseReference(String databaseName, String clusterName) throws Exception { private Referenceable getDatabaseReference(String clusterName, String databaseName) throws Exception {
LOG.debug("Getting reference for database {}", databaseName); LOG.debug("Getting reference for database {}", databaseName);
String typeName = HiveDataTypes.HIVE_DB.getName(); String typeName = HiveDataTypes.HIVE_DB.getName();
...@@ -169,20 +176,6 @@ public class HiveMetaStoreBridge { ...@@ -169,20 +176,6 @@ public class HiveMetaStoreBridge {
return getEntityReferenceFromDSL(typeName, dslQuery); return getEntityReferenceFromDSL(typeName, dslQuery);
} }
public Referenceable getProcessReference(String queryStr) throws Exception {
LOG.debug("Getting reference for process with query {}", queryStr);
String typeName = HiveDataTypes.HIVE_PROCESS.getName();
//todo enable DSL
// String dslQuery = String.format("%s where queryText = \"%s\"", typeName, queryStr);
// return getEntityReferenceFromDSL(typeName, dslQuery);
String gremlinQuery =
String.format("g.V.has('__typeName', '%s').has('%s.queryText', \"%s\").toList()", typeName, typeName,
StringEscapeUtils.escapeJava(queryStr));
return getEntityReferenceFromGremlin(typeName, gremlinQuery);
}
private Referenceable getEntityReferenceFromDSL(String typeName, String dslQuery) throws Exception { private Referenceable getEntityReferenceFromDSL(String typeName, String dslQuery) throws Exception {
AtlasClient dgiClient = getAtlasClient(); AtlasClient dgiClient = getAtlasClient();
JSONArray results = dgiClient.searchByDSL(dslQuery); JSONArray results = dgiClient.searchByDSL(dslQuery);
...@@ -200,8 +193,27 @@ public class HiveMetaStoreBridge { ...@@ -200,8 +193,27 @@ public class HiveMetaStoreBridge {
} }
} }
public static String getTableName(String clusterName, String dbName, String tableName) { public static String getDBQualifiedName(String clusterName, String dbName) {
return String.format("%s.%s@%s", dbName.toLowerCase(), tableName.toLowerCase(), clusterName); return String.format("%s.%s", clusterName, dbName.toLowerCase());
}
/**
* Imports all tables for the given db
* @param databaseName
* @param databaseReferenceable
* @throws Exception
*/
private void importTables(Referenceable databaseReferenceable, String databaseName) throws Exception {
List<String> hiveTables = hiveClient.getAllTables(databaseName);
for (String tableName : hiveTables) {
Table table = hiveClient.getTable(databaseName, tableName);
Referenceable tableReferenceable = registerTable(databaseReferenceable, table);
// Import Partitions
Referenceable sdReferenceable = getSDForTable(databaseName, tableName);
registerPartitions(tableReferenceable, sdReferenceable, table);
}
} }
/** /**
...@@ -216,16 +228,78 @@ public class HiveMetaStoreBridge { ...@@ -216,16 +228,78 @@ public class HiveMetaStoreBridge {
LOG.debug("Getting reference for table {}.{}", dbName, tableName); LOG.debug("Getting reference for table {}.{}", dbName, tableName);
String typeName = HiveDataTypes.HIVE_TABLE.getName(); String typeName = HiveDataTypes.HIVE_TABLE.getName();
String entityName = getTableName(clusterName, dbName, tableName); String entityName = getTableQualifiedName(clusterName, dbName, tableName);
String dslQuery = String.format("%s as t where name = '%s'", typeName, entityName); String dslQuery = String.format("%s as t where name = '%s'", typeName, entityName);
return getEntityReferenceFromDSL(typeName, dslQuery); return getEntityReferenceFromDSL(typeName, dslQuery);
} }
public static String getTableQualifiedName(String clusterName, String dbName, String tableName) {
return String.format("%s.%s.%s", clusterName, dbName.toLowerCase(), tableName.toLowerCase());
}
public Referenceable createTableInstance(Referenceable dbReference, Table hiveTable)
throws Exception {
LOG.info("Importing objects from {}.{}", hiveTable.getDbName(), hiveTable.getTableName());
Referenceable tableRef = new Referenceable(HiveDataTypes.HIVE_TABLE.getName());
String tableQualifiedName = getTableQualifiedName(clusterName, hiveTable.getDbName(), hiveTable.getTableName());
tableRef.set(HiveDataModelGenerator.NAME, tableQualifiedName);
tableRef.set(HiveDataModelGenerator.TABLE_NAME, hiveTable.getTableName().toLowerCase());
tableRef.set("owner", hiveTable.getOwner());
tableRef.set("createTime", hiveTable.getMetadata().getProperty(hive_metastoreConstants.DDL_TIME));
tableRef.set("lastAccessTime", hiveTable.getLastAccessTime());
tableRef.set("retention", hiveTable.getRetention());
tableRef.set(HiveDataModelGenerator.COMMENT, hiveTable.getParameters().get(HiveDataModelGenerator.COMMENT));
// add reference to the database
tableRef.set(HiveDataModelGenerator.DB, dbReference);
tableRef.set("columns", getColumns(hiveTable.getCols(), tableQualifiedName));
// add reference to the StorageDescriptor
Referenceable sdReferenceable = fillStorageDescStruct(hiveTable.getSd(), tableQualifiedName, tableQualifiedName);
tableRef.set("sd", sdReferenceable);
// add reference to the Partition Keys
List<Referenceable> partKeys = getColumns(hiveTable.getPartitionKeys(), tableQualifiedName);
tableRef.set("partitionKeys", partKeys);
tableRef.set("parameters", hiveTable.getParameters());
if (hiveTable.getViewOriginalText() != null) {
tableRef.set("viewOriginalText", hiveTable.getViewOriginalText());
}
if (hiveTable.getViewExpandedText() != null) {
tableRef.set("viewExpandedText", hiveTable.getViewExpandedText());
}
tableRef.set("tableType", hiveTable.getTableType().name());
tableRef.set("temporary", hiveTable.isTemporary());
return tableRef;
}
private Referenceable registerTable(Referenceable dbReference, Table table) throws Exception {
String dbName = table.getDbName();
String tableName = table.getTableName();
LOG.info("Attempting to register table [" + tableName + "]");
Referenceable tableRef = getTableReference(dbName, tableName);
if (tableRef == null) {
tableRef = createTableInstance(dbReference, table);
tableRef = registerInstance(tableRef);
} else {
LOG.info("Table {}.{} is already registered with id {}", dbName, tableName, tableRef.getId().id);
}
return tableRef;
}
private Referenceable getEntityReferenceFromGremlin(String typeName, String gremlinQuery) private Referenceable getEntityReferenceFromGremlin(String typeName, String gremlinQuery)
throws AtlasServiceException, JSONException { throws AtlasServiceException, JSONException {
AtlasClient client = getAtlasClient(); AtlasClient client = getAtlasClient();
JSONObject response = client.searchByGremlin(gremlinQuery); JSONArray results = client.searchByGremlin(gremlinQuery);
JSONArray results = response.getJSONArray(AtlasClient.RESULTS);
if (results.length() == 0) { if (results.length() == 0) {
return null; return null;
} }
...@@ -245,7 +319,7 @@ public class HiveMetaStoreBridge { ...@@ -245,7 +319,7 @@ public class HiveMetaStoreBridge {
// dbName, clusterName); // dbName, clusterName);
String datasetType = AtlasClient.DATA_SET_SUPER_TYPE; String datasetType = AtlasClient.DATA_SET_SUPER_TYPE;
String tableEntityName = getTableName(clusterName, dbName, tableName); String tableEntityName = getTableQualifiedName(clusterName, dbName, tableName);
String gremlinQuery = String.format("g.V.has('__typeName', '%s').has('%s.values', %s).as('p')." String gremlinQuery = String.format("g.V.has('__typeName', '%s').has('%s.values', %s).as('p')."
+ "out('__%s.table').has('%s.name', '%s').back('p').toList()", typeName, typeName, valuesStr, + "out('__%s.table').has('%s.name', '%s').back('p').toList()", typeName, typeName, valuesStr,
...@@ -262,92 +336,24 @@ public class HiveMetaStoreBridge { ...@@ -262,92 +336,24 @@ public class HiveMetaStoreBridge {
AtlasClient dgiClient = getAtlasClient(); AtlasClient dgiClient = getAtlasClient();
Referenceable tableInstance = dgiClient.getEntity(tableRef.getId().id); Referenceable tableInstance = dgiClient.getEntity(tableRef.getId().id);
Id sdId = (Id) tableInstance.get("sd"); Referenceable sd = (Referenceable) tableInstance.get("sd");
return new Referenceable(sdId.id, sdId.getTypeName(), null); return new Referenceable(sd.getId().id, sd.getTypeName(), null);
}
public Referenceable registerTable(String dbName, String tableName) throws Exception {
Referenceable dbReferenceable = registerDatabase(dbName);
return registerTable(dbReferenceable, dbName, tableName);
} }
public Referenceable registerTable(Referenceable dbReference, String dbName, String tableName) throws Exception { private void registerPartitions(Referenceable tableReferenceable, Referenceable sdReferenceable,
LOG.info("Attempting to register table [" + tableName + "]"); Table table) throws Exception {
Referenceable tableRef = getTableReference(dbName, tableName); String dbName = table.getDbName();
if (tableRef == null) { String tableName = table.getTableName();
LOG.info("Importing objects from " + dbName + "." + tableName); LOG.info("Registering partitions for {}.{}", dbName, tableName);
List<Partition> tableParts = hiveClient.getPartitions(table);
Table hiveTable = hiveClient.getTable(dbName, tableName);
tableRef = new Referenceable(HiveDataTypes.HIVE_TABLE.getName());
tableRef.set(HiveDataModelGenerator.NAME,
getTableName(clusterName, hiveTable.getDbName(), hiveTable.getTableName()));
tableRef.set(HiveDataModelGenerator.TABLE_NAME, hiveTable.getTableName().toLowerCase());
tableRef.set("owner", hiveTable.getOwner());
tableRef.set("createTime", hiveTable.getMetadata().getProperty(hive_metastoreConstants.DDL_TIME));
tableRef.set("lastAccessTime", hiveTable.getLastAccessTime());
tableRef.set("retention", hiveTable.getRetention());
tableRef.set(HiveDataModelGenerator.COMMENT, hiveTable.getParameters().get(HiveDataModelGenerator.COMMENT));
// add reference to the database
tableRef.set(HiveDataModelGenerator.DB, dbReference);
List<Referenceable> colList = getColumns(hiveTable.getCols());
tableRef.set("columns", colList);
// add reference to the StorageDescriptor
StorageDescriptor storageDesc = hiveTable.getSd();
Referenceable sdReferenceable = fillStorageDescStruct(storageDesc, colList);
tableRef.set("sd", sdReferenceable);
// add reference to the Partition Keys
List<Referenceable> partKeys = getColumns(hiveTable.getPartitionKeys());
tableRef.set("partitionKeys", partKeys);
tableRef.set("parameters", hiveTable.getParameters());
if (hiveTable.getViewOriginalText() != null) {
tableRef.set("viewOriginalText", hiveTable.getViewOriginalText());
}
if (hiveTable.getViewExpandedText() != null) {
tableRef.set("viewExpandedText", hiveTable.getViewExpandedText());
}
tableRef.set("tableType", hiveTable.getTableType().name());
tableRef.set("temporary", hiveTable.isTemporary());
tableRef = createInstance(tableRef); for (Partition hivePart : tableParts) {
} else { registerPartition(tableReferenceable, sdReferenceable, hivePart);
LOG.info("Table {}.{} is already registered with id {}", dbName, tableName, tableRef.getId().id);
}
return tableRef;
}
private void registerPartitions(String db, String tableName, Referenceable tableReferenceable,
Referenceable sdReferenceable) throws Exception {
Set<Partition> tableParts = hiveClient.getAllPartitionsOf(new Table(Table.getEmptyTable(db, tableName)));
if (tableParts.size() > 0) {
for (Partition hivePart : tableParts) {
registerPartition(hivePart, tableReferenceable, sdReferenceable);
}
} }
} }
public Referenceable registerPartition(Partition partition) throws Exception { private Referenceable registerPartition(Referenceable tableReferenceable, Referenceable sdReferenceable,
String dbName = partition.getTable().getDbName(); Partition hivePart) throws Exception {
String tableName = partition.getTable().getTableName();
Referenceable tableRef = registerTable(dbName, tableName);
Referenceable sdRef = getSDForTable(dbName, tableName);
return registerPartition(partition, tableRef, sdRef);
}
private Referenceable registerPartition(Partition hivePart, Referenceable tableReferenceable,
Referenceable sdReferenceable) throws Exception {
LOG.info("Registering partition for {} with values {}", tableReferenceable, LOG.info("Registering partition for {} with values {}", tableReferenceable,
StringUtils.join(hivePart.getValues(), ",")); StringUtils.join(hivePart.getValues(), ","));
String dbName = hivePart.getTable().getDbName(); String dbName = hivePart.getTable().getDbName();
...@@ -355,22 +361,8 @@ public class HiveMetaStoreBridge { ...@@ -355,22 +361,8 @@ public class HiveMetaStoreBridge {
Referenceable partRef = getPartitionReference(dbName, tableName, hivePart.getValues()); Referenceable partRef = getPartitionReference(dbName, tableName, hivePart.getValues());
if (partRef == null) { if (partRef == null) {
partRef = new Referenceable(HiveDataTypes.HIVE_PARTITION.getName()); partRef = createPartitionReferenceable(tableReferenceable, sdReferenceable, hivePart);
partRef.set("values", hivePart.getValues()); partRef = registerInstance(partRef);
partRef.set(HiveDataModelGenerator.TABLE, tableReferenceable);
//todo fix
partRef.set("createTime", hivePart.getLastAccessTime());
partRef.set("lastAccessTime", hivePart.getLastAccessTime());
// sdStruct = fillStorageDescStruct(hivePart.getSd());
// Instead of creating copies of the sdstruct for partitions we are reusing existing
// ones will fix to identify partitions with differing schema.
partRef.set("sd", sdReferenceable);
partRef.set("parameters", hivePart.getParameters());
partRef = createInstance(partRef);
} else { } else {
LOG.info("Partition {}.{} with values {} is already registered with id {}", dbName, tableName, LOG.info("Partition {}.{} with values {} is already registered with id {}", dbName, tableName,
StringUtils.join(hivePart.getValues(), ","), partRef.getId().id); StringUtils.join(hivePart.getValues(), ","), partRef.getId().id);
...@@ -378,47 +370,38 @@ public class HiveMetaStoreBridge { ...@@ -378,47 +370,38 @@ public class HiveMetaStoreBridge {
return partRef; return partRef;
} }
private void importIndexes(String db, String table, Referenceable dbReferenceable, Referenceable tableReferenceable) public Referenceable createPartitionReferenceable(Referenceable tableReferenceable, Referenceable sdReferenceable,
throws Exception { Partition hivePart) {
List<Index> indexes = hiveClient.getIndexes(db, table, Short.MAX_VALUE); Referenceable partRef = new Referenceable(HiveDataTypes.HIVE_PARTITION.getName());
if (indexes.size() > 0) { partRef.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, getPartitionQualifiedName(hivePart));
for (Index index : indexes) { partRef.set("values", hivePart.getValues());
importIndex(index, dbReferenceable, tableReferenceable);
}
}
}
//todo should be idempotent
private void importIndex(Index index, Referenceable dbReferenceable, Referenceable tableReferenceable)
throws Exception {
LOG.info("Importing index {} for {}.{}", index.getIndexName(), dbReferenceable, tableReferenceable);
Referenceable indexRef = new Referenceable(HiveDataTypes.HIVE_INDEX.getName());
indexRef.set(HiveDataModelGenerator.NAME, index.getIndexName());
indexRef.set("indexHandlerClass", index.getIndexHandlerClass());
indexRef.set(HiveDataModelGenerator.DB, dbReferenceable);
indexRef.set("createTime", index.getCreateTime()); partRef.set(HiveDataModelGenerator.TABLE, tableReferenceable);
indexRef.set("lastAccessTime", index.getLastAccessTime());
indexRef.set("origTable", index.getOrigTableName());
indexRef.set("indexTable", index.getIndexTableName());
Referenceable sdReferenceable = fillStorageDescStruct(index.getSd(), null); //todo fix
indexRef.set("sd", sdReferenceable); partRef.set("createTime", hivePart.getLastAccessTime());
partRef.set("lastAccessTime", hivePart.getLastAccessTime());
indexRef.set("parameters", index.getParameters()); // sdStruct = fillStorageDescStruct(hivePart.getSd());
// Instead of creating copies of the sdstruct for partitions we are reusing existing
// ones will fix to identify partitions with differing schema.
partRef.set("sd", sdReferenceable);
tableReferenceable.set("deferredRebuild", index.isDeferredRebuild()); partRef.set("parameters", hivePart.getParameters());
return partRef;
}
createInstance(indexRef); private String getPartitionQualifiedName(Partition partition) {
return String.format("%s.%s.%s.%s", clusterName, partition.getTable().getDbName(),
partition.getTable().getTableName(), StringUtils.join(partition.getValues(), "/"));
} }
private Referenceable fillStorageDescStruct(StorageDescriptor storageDesc, List<Referenceable> colList) private Referenceable fillStorageDescStruct(StorageDescriptor storageDesc, String tableQualifiedName,
throws Exception { String sdQualifiedName) throws Exception {
LOG.debug("Filling storage descriptor information for " + storageDesc); LOG.debug("Filling storage descriptor information for " + storageDesc);
Referenceable sdReferenceable = new Referenceable(HiveDataTypes.HIVE_STORAGEDESC.getName()); Referenceable sdReferenceable = new Referenceable(HiveDataTypes.HIVE_STORAGEDESC.getName());
sdReferenceable.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, sdQualifiedName);
SerDeInfo serdeInfo = storageDesc.getSerdeInfo(); SerDeInfo serdeInfo = storageDesc.getSerdeInfo();
LOG.debug("serdeInfo = " + serdeInfo); LOG.debug("serdeInfo = " + serdeInfo);
...@@ -439,11 +422,7 @@ public class HiveMetaStoreBridge { ...@@ -439,11 +422,7 @@ public class HiveMetaStoreBridge {
//Use the passed column list if not null, ex: use same references for table and SD //Use the passed column list if not null, ex: use same references for table and SD
List<FieldSchema> columns = storageDesc.getCols(); List<FieldSchema> columns = storageDesc.getCols();
if (columns != null && !columns.isEmpty()) { if (columns != null && !columns.isEmpty()) {
if (colList != null) { sdReferenceable.set("cols", getColumns(columns, tableQualifiedName));
sdReferenceable.set("cols", colList);
} else {
sdReferenceable.set("cols", getColumns(columns));
}
} }
List<Struct> sortColsStruct = new ArrayList<>(); List<Struct> sortColsStruct = new ArrayList<>();
...@@ -471,19 +450,25 @@ public class HiveMetaStoreBridge { ...@@ -471,19 +450,25 @@ public class HiveMetaStoreBridge {
sdReferenceable.set("parameters", storageDesc.getParameters()); sdReferenceable.set("parameters", storageDesc.getParameters());
sdReferenceable.set("storedAsSubDirectories", storageDesc.isStoredAsSubDirectories()); sdReferenceable.set("storedAsSubDirectories", storageDesc.isStoredAsSubDirectories());
return createInstance(sdReferenceable); return sdReferenceable;
}
private String getColumnQualifiedName(String tableQualifiedName, String colName) {
return String.format("%s.%s", tableQualifiedName, colName);
} }
private List<Referenceable> getColumns(List<FieldSchema> schemaList) throws Exception { private List<Referenceable> getColumns(List<FieldSchema> schemaList, String tableQualifiedName) throws Exception {
List<Referenceable> colList = new ArrayList<>(); List<Referenceable> colList = new ArrayList<>();
for (FieldSchema fs : schemaList) { for (FieldSchema fs : schemaList) {
LOG.debug("Processing field " + fs); LOG.debug("Processing field " + fs);
Referenceable colReferenceable = new Referenceable(HiveDataTypes.HIVE_COLUMN.getName()); Referenceable colReferenceable = new Referenceable(HiveDataTypes.HIVE_COLUMN.getName());
colReferenceable.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME,
getColumnQualifiedName(tableQualifiedName, fs.getName()));
colReferenceable.set(HiveDataModelGenerator.NAME, fs.getName()); colReferenceable.set(HiveDataModelGenerator.NAME, fs.getName());
colReferenceable.set("type", fs.getType()); colReferenceable.set("type", fs.getType());
colReferenceable.set(HiveDataModelGenerator.COMMENT, fs.getComment()); colReferenceable.set(HiveDataModelGenerator.COMMENT, fs.getComment());
colList.add(createInstance(colReferenceable)); colList.add(colReferenceable);
} }
return colList; return colList;
} }
...@@ -502,16 +487,9 @@ public class HiveMetaStoreBridge { ...@@ -502,16 +487,9 @@ public class HiveMetaStoreBridge {
} }
public static void main(String[] argv) throws Exception { public static void main(String[] argv) throws Exception {
HiveMetaStoreBridge hiveMetaStoreBridge = new HiveMetaStoreBridge(new HiveConf()); Configuration atlasConf = ApplicationProperties.get(ApplicationProperties.CLIENT_PROPERTIES);
HiveMetaStoreBridge hiveMetaStoreBridge = new HiveMetaStoreBridge(new HiveConf(), atlasConf);
hiveMetaStoreBridge.registerHiveDataModel(); hiveMetaStoreBridge.registerHiveDataModel();
hiveMetaStoreBridge.importHiveMetadata(); hiveMetaStoreBridge.importHiveMetadata();
} }
public void updateTable(Referenceable tableReferenceable, Table newTable) throws AtlasServiceException {
AtlasClient client = getAtlasClient();
client.updateEntity(tableReferenceable.getId()._getId(), HiveDataModelGenerator.TABLE_NAME,
newTable.getTableName().toLowerCase());
client.updateEntity(tableReferenceable.getId()._getId(), HiveDataModelGenerator.NAME,
getTableName(clusterName, newTable.getDbName(), newTable.getTableName()));
}
} }
...@@ -20,28 +20,44 @@ package org.apache.atlas.hive.hook; ...@@ -20,28 +20,44 @@ package org.apache.atlas.hive.hook;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Guice;
import com.google.inject.Inject;
import com.google.inject.Injector;
import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasException;
import org.apache.atlas.hive.bridge.HiveMetaStoreBridge; import org.apache.atlas.hive.bridge.HiveMetaStoreBridge;
import org.apache.atlas.hive.model.HiveDataTypes; import org.apache.atlas.hive.model.HiveDataTypes;
import org.apache.atlas.notification.NotificationInterface;
import org.apache.atlas.notification.NotificationModule;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.json.InstanceSerialization;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryPlan;
import org.apache.hadoop.hive.ql.exec.ExplainTask; import org.apache.hadoop.hive.ql.exec.ExplainTask;
import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Task;
import org.apache.hadoop.hive.ql.hooks.Entity; import org.apache.hadoop.hive.ql.hooks.Entity;
import org.apache.hadoop.hive.ql.hooks.Entity.Type;
import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext; import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
import org.apache.hadoop.hive.ql.hooks.HookContext; import org.apache.hadoop.hive.ql.hooks.HookContext;
import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.ReadEntity;
import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity;
import org.apache.hadoop.hive.ql.metadata.Partition;
import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.metadata.Table;
import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.plan.HiveOperation;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.codehaus.jettison.json.JSONArray;
import org.json.JSONObject; import org.json.JSONObject;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.LinkedBlockingQueue;
...@@ -55,31 +71,48 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -55,31 +71,48 @@ public class HiveHook implements ExecuteWithHookContext {
private static final Logger LOG = LoggerFactory.getLogger(HiveHook.class); private static final Logger LOG = LoggerFactory.getLogger(HiveHook.class);
// wait time determines how long we wait before we exit the jvm on
// shutdown. Pending requests after that will not be sent.
private static final int WAIT_TIME = 3;
private static ExecutorService executor;
public static final String CONF_PREFIX = "atlas.hook.hive."; public static final String CONF_PREFIX = "atlas.hook.hive.";
private static final String MIN_THREADS = CONF_PREFIX + "minThreads"; private static final String MIN_THREADS = CONF_PREFIX + "minThreads";
private static final String MAX_THREADS = CONF_PREFIX + "maxThreads"; private static final String MAX_THREADS = CONF_PREFIX + "maxThreads";
private static final String KEEP_ALIVE_TIME = CONF_PREFIX + "keepAliveTime"; private static final String KEEP_ALIVE_TIME = CONF_PREFIX + "keepAliveTime";
public static final String CONF_SYNC = CONF_PREFIX + "synchronous"; public static final String CONF_SYNC = CONF_PREFIX + "synchronous";
public static final String HOOK_NUM_RETRIES = CONF_PREFIX + "numRetries";
// wait time determines how long we wait before we exit the jvm on
// shutdown. Pending requests after that will not be sent.
private static final int WAIT_TIME = 3;
private static ExecutorService executor;
private static final int minThreadsDefault = 5; private static final int minThreadsDefault = 5;
private static final int maxThreadsDefault = 5; private static final int maxThreadsDefault = 5;
private static final long keepAliveTimeDefault = 10; private static final long keepAliveTimeDefault = 10;
private static boolean typesRegistered = false; private static boolean typesRegistered = false;
private final Configuration atlasProperties;
class HiveEvent {
public HiveConf conf;
static { public Set<ReadEntity> inputs;
// anything shared should be initialized here and destroyed in the public Set<WriteEntity> outputs;
// shutdown hook The hook contract is weird in that it creates a
// boatload of hooks. public String user;
public UserGroupInformation ugi;
public HiveOperation operation;
public QueryPlan queryPlan;
public HookContext.HookType hookType;
public JSONObject jsonPlan;
}
@Inject
private NotificationInterface notifInterface;
public HiveHook() throws AtlasException {
atlasProperties = ApplicationProperties.get(ApplicationProperties.CLIENT_PROPERTIES);
// initialize the async facility to process hook calls. We don't // initialize the async facility to process hook calls. We don't
// want to do this inline since it adds plenty of overhead for the // want to do this inline since it adds plenty of overhead for the query.
// query.
HiveConf hiveConf = new HiveConf(); HiveConf hiveConf = new HiveConf();
int minThreads = hiveConf.getInt(MIN_THREADS, minThreadsDefault); int minThreads = hiveConf.getInt(MIN_THREADS, minThreadsDefault);
int maxThreads = hiveConf.getInt(MAX_THREADS, maxThreadsDefault); int maxThreads = hiveConf.getInt(MAX_THREADS, maxThreadsDefault);
...@@ -87,7 +120,7 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -87,7 +120,7 @@ public class HiveHook implements ExecuteWithHookContext {
executor = new ThreadPoolExecutor(minThreads, maxThreads, keepAliveTime, TimeUnit.MILLISECONDS, executor = new ThreadPoolExecutor(minThreads, maxThreads, keepAliveTime, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<Runnable>(), new LinkedBlockingQueue<Runnable>(),
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Atlas Logger %d").build()); new ThreadFactoryBuilder().setNameFormat("Atlas Logger %d").build());
try { try {
Runtime.getRuntime().addShutdownHook(new Thread() { Runtime.getRuntime().addShutdownHook(new Thread() {
...@@ -108,33 +141,16 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -108,33 +141,16 @@ public class HiveHook implements ExecuteWithHookContext {
} }
LOG.info("Created Atlas Hook"); LOG.info("Created Atlas Hook");
}
class HiveEvent {
public HiveConf conf;
public Set<ReadEntity> inputs; Injector injector = Guice.createInjector(new NotificationModule());
public Set<WriteEntity> outputs; notifInterface = injector.getInstance(NotificationInterface.class);
public String user;
public UserGroupInformation ugi;
public HiveOperation operation;
public QueryPlan queryPlan;
public HookContext.HookType hookType;
public JSONObject jsonPlan;
} }
@Override @Override
public void run(final HookContext hookContext) throws Exception { public void run(final HookContext hookContext) throws Exception {
if (executor == null) {
LOG.info("No executor running. Bail.");
return;
}
// clone to avoid concurrent access // clone to avoid concurrent access
final HiveEvent event = new HiveEvent(); final HiveEvent event = new HiveEvent();
final HiveConf conf = new HiveConf(hookContext.getConf()); final HiveConf conf = new HiveConf(hookContext.getConf());
boolean debug = conf.get(CONF_SYNC, "false").equals("true");
event.conf = conf; event.conf = conf;
event.inputs = hookContext.getInputs(); event.inputs = hookContext.getInputs();
...@@ -148,7 +164,8 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -148,7 +164,8 @@ public class HiveHook implements ExecuteWithHookContext {
event.jsonPlan = getQueryPlan(event); event.jsonPlan = getQueryPlan(event);
if (debug) { boolean sync = conf.get(CONF_SYNC, "false").equals("true");
if (sync) {
fireAndForget(event); fireAndForget(event);
} else { } else {
executor.submit(new Runnable() { executor.submit(new Runnable() {
...@@ -168,7 +185,7 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -168,7 +185,7 @@ public class HiveHook implements ExecuteWithHookContext {
assert event.hookType == HookContext.HookType.POST_EXEC_HOOK : "Non-POST_EXEC_HOOK not supported!"; assert event.hookType == HookContext.HookType.POST_EXEC_HOOK : "Non-POST_EXEC_HOOK not supported!";
LOG.info("Entered Atlas hook for hook type {} operation {}", event.hookType, event.operation); LOG.info("Entered Atlas hook for hook type {} operation {}", event.hookType, event.operation);
HiveMetaStoreBridge dgiBridge = new HiveMetaStoreBridge(event.conf, event.user, event.ugi); HiveMetaStoreBridge dgiBridge = new HiveMetaStoreBridge(event.conf, atlasProperties, event.user, event.ugi);
if (!typesRegistered) { if (!typesRegistered) {
dgiBridge.registerHiveDataModel(); dgiBridge.registerHiveDataModel();
...@@ -177,11 +194,11 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -177,11 +194,11 @@ public class HiveHook implements ExecuteWithHookContext {
switch (event.operation) { switch (event.operation) {
case CREATEDATABASE: case CREATEDATABASE:
handleCreateDB(dgiBridge, event); handleEventOutputs(dgiBridge, event, Type.DATABASE);
break; break;
case CREATETABLE: case CREATETABLE:
handleCreateTable(dgiBridge, event); handleEventOutputs(dgiBridge, event, Type.TABLE);
break; break;
case CREATETABLE_AS_SELECT: case CREATETABLE_AS_SELECT:
...@@ -211,6 +228,7 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -211,6 +228,7 @@ public class HiveHook implements ExecuteWithHookContext {
} }
} }
//todo re-write with notification
private void renameTable(HiveMetaStoreBridge dgiBridge, HiveEvent event) throws Exception { private void renameTable(HiveMetaStoreBridge dgiBridge, HiveEvent event) throws Exception {
//crappy, no easy of getting new name //crappy, no easy of getting new name
assert event.inputs != null && event.inputs.size() == 1; assert event.inputs != null && event.inputs.size() == 1;
...@@ -232,30 +250,91 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -232,30 +250,91 @@ public class HiveHook implements ExecuteWithHookContext {
LOG.warn("Failed to deduct new name for " + event.queryPlan.getQueryStr()); LOG.warn("Failed to deduct new name for " + event.queryPlan.getQueryStr());
return; return;
} }
}
private Map<Type, Referenceable> createEntities(HiveMetaStoreBridge dgiBridge, Entity entity) throws Exception {
Map<Type, Referenceable> entities = new LinkedHashMap<>();
Database db = null;
Table table = null;
Partition partition = null;
switch (entity.getType()) {
case DATABASE:
db = entity.getDatabase();
break;
case TABLE:
table = entity.getTable();
db = dgiBridge.hiveClient.getDatabase(table.getDbName());
break;
case PARTITION:
partition = entity.getPartition();
table = partition.getTable();
db = dgiBridge.hiveClient.getDatabase(table.getDbName());
break;
}
db = dgiBridge.hiveClient.getDatabase(db.getName());
Referenceable dbReferenceable = dgiBridge.createDBInstance(db);
entities.put(Type.DATABASE, dbReferenceable);
Referenceable dbReferenceable = dgiBridge.registerDatabase(oldTable.getDbName()); Referenceable tableReferenceable = null;
Referenceable tableReferenceable = if (table != null) {
dgiBridge.registerTable(dbReferenceable, oldTable.getDbName(), oldTable.getTableName()); table = dgiBridge.hiveClient.getTable(table.getDbName(), table.getTableName());
LOG.info("Updating entity name {}.{} to {}", oldTable.getDbName(), oldTable.getTableName(), tableReferenceable = dgiBridge.createTableInstance(dbReferenceable, table);
newTable.getTableName()); entities.put(Type.TABLE, tableReferenceable);
dgiBridge.updateTable(tableReferenceable, newTable); }
if (partition != null) {
Referenceable partitionReferenceable = dgiBridge.createPartitionReferenceable(tableReferenceable,
(Referenceable) tableReferenceable.get("sd"), partition);
entities.put(Type.PARTITION, partitionReferenceable);
}
return entities;
} }
private void handleCreateTable(HiveMetaStoreBridge dgiBridge, HiveEvent event) throws Exception { private void handleEventOutputs(HiveMetaStoreBridge dgiBridge, HiveEvent event, Type entityType) throws Exception {
List<Referenceable> entities = new ArrayList<>();
for (WriteEntity entity : event.outputs) { for (WriteEntity entity : event.outputs) {
if (entity.getType() == Entity.Type.TABLE) { if (entity.getType() == entityType) {
entities.addAll(createEntities(dgiBridge, entity).values());
Table table = entity.getTable();
Referenceable dbReferenceable = dgiBridge.registerDatabase(table.getDbName());
dgiBridge.registerTable(dbReferenceable, table.getDbName(), table.getTableName());
} }
} }
notifyEntity(entities);
} }
private void handleCreateDB(HiveMetaStoreBridge dgiBridge, HiveEvent event) throws Exception { private void notifyEntity(Collection<Referenceable> entities) {
for (WriteEntity entity : event.outputs) { JSONArray entitiesArray = new JSONArray();
if (entity.getType() == Entity.Type.DATABASE) { for (Referenceable entity : entities) {
dgiBridge.registerDatabase(entity.getDatabase().getName()); String entityJson = InstanceSerialization.toJson(entity, true);
entitiesArray.put(entityJson);
}
notifyEntity(entitiesArray);
}
/**
* Notify atlas of the entity through message. The entity can be a complex entity with reference to other entities.
* De-duping of entities is done on server side depending on the unique attribute on the
* @param entities
*/
private void notifyEntity(JSONArray entities) {
int maxRetries = atlasProperties.getInt(HOOK_NUM_RETRIES, 3);
String message = entities.toString();
int numRetries = 0;
while (true) {
try {
notifInterface.send(NotificationInterface.NotificationType.HOOK, message);
return;
} catch(Exception e) {
numRetries++;
if(numRetries < maxRetries) {
LOG.debug("Failed to notify atlas for entity {}. Retrying", message, e);
} else {
LOG.error("Failed to notify atlas for entity {} after {} retries. Quitting", message,
maxRetries, e);
}
} }
} }
} }
...@@ -284,50 +363,42 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -284,50 +363,42 @@ public class HiveHook implements ExecuteWithHookContext {
String queryStr = normalize(event.queryPlan.getQueryStr()); String queryStr = normalize(event.queryPlan.getQueryStr());
long queryStartTime = event.queryPlan.getQueryStartTime(); long queryStartTime = event.queryPlan.getQueryStartTime();
LOG.debug("Registering CTAS query: {}", queryStr); LOG.debug("Registering query: {}", queryStr);
Referenceable processReferenceable = dgiBridge.getProcessReference(queryStr); List<Referenceable> entities = new ArrayList<>();
if (processReferenceable == null) { Referenceable processReferenceable = new Referenceable(HiveDataTypes.HIVE_PROCESS.getName());
processReferenceable = new Referenceable(HiveDataTypes.HIVE_PROCESS.getName()); processReferenceable.set("name", queryStr);
processReferenceable.set("name", event.operation.getOperationName()); processReferenceable.set("operationType", event.operation.getOperationName());
processReferenceable.set("startTime", queryStartTime); processReferenceable.set("startTime", queryStartTime);
processReferenceable.set("userName", event.user); processReferenceable.set("userName", event.user);
List<Referenceable> source = new ArrayList<>(); List<Referenceable> source = new ArrayList<>();
for (ReadEntity readEntity : inputs) { for (ReadEntity readEntity : inputs) {
if (readEntity.getType() == Entity.Type.TABLE) { if (readEntity.getType() == Type.TABLE || readEntity.getType() == Type.PARTITION) {
Table table = readEntity.getTable(); Map<Type, Referenceable> localEntities = createEntities(dgiBridge, readEntity);
String dbName = table.getDbName(); source.add(localEntities.get(Type.TABLE));
source.add(dgiBridge.registerTable(dbName, table.getTableName())); entities.addAll(localEntities.values());
}
if (readEntity.getType() == Entity.Type.PARTITION) {
dgiBridge.registerPartition(readEntity.getPartition());
}
} }
processReferenceable.set("inputs", source); }
processReferenceable.set("inputs", source);
List<Referenceable> target = new ArrayList<>();
for (WriteEntity writeEntity : outputs) { List<Referenceable> target = new ArrayList<>();
if (writeEntity.getType() == Entity.Type.TABLE || writeEntity.getType() == Entity.Type.PARTITION) { for (WriteEntity writeEntity : outputs) {
Table table = writeEntity.getTable(); if (writeEntity.getType() == Type.TABLE || writeEntity.getType() == Type.PARTITION) {
String dbName = table.getDbName(); Map<Type, Referenceable> localEntities = createEntities(dgiBridge, writeEntity);
target.add(dgiBridge.registerTable(dbName, table.getTableName())); target.add(localEntities.get(Type.TABLE));
} entities.addAll(localEntities.values());
if (writeEntity.getType() == Entity.Type.PARTITION) {
dgiBridge.registerPartition(writeEntity.getPartition());
}
} }
processReferenceable.set("outputs", target);
processReferenceable.set("queryText", queryStr);
processReferenceable.set("queryId", queryId);
processReferenceable.set("queryPlan", event.jsonPlan.toString());
processReferenceable.set("endTime", System.currentTimeMillis());
//TODO set
processReferenceable.set("queryGraph", "queryGraph");
dgiBridge.createInstance(processReferenceable);
} else {
LOG.debug("Query {} is already registered", queryStr);
} }
processReferenceable.set("outputs", target);
processReferenceable.set("queryText", queryStr);
processReferenceable.set("queryId", queryId);
processReferenceable.set("queryPlan", event.jsonPlan.toString());
processReferenceable.set("endTime", System.currentTimeMillis());
//TODO set
processReferenceable.set("queryGraph", "queryGraph");
entities.add(processReferenceable);
notifyEntity(entities);
} }
...@@ -338,7 +409,7 @@ public class HiveHook implements ExecuteWithHookContext { ...@@ -338,7 +409,7 @@ public class HiveHook implements ExecuteWithHookContext {
List<Task<?>> rootTasks = event.queryPlan.getRootTasks(); List<Task<?>> rootTasks = event.queryPlan.getRootTasks();
return explain.getJSONPlan(null, null, rootTasks, event.queryPlan.getFetchTask(), true, false, false); return explain.getJSONPlan(null, null, rootTasks, event.queryPlan.getFetchTask(), true, false, false);
} catch (Exception e) { } catch (Exception e) {
LOG.warn("Failed to get queryplan", e); LOG.info("Failed to get queryplan", e);
return new JSONObject(); return new JSONObject();
} }
} }
......
...@@ -35,6 +35,7 @@ import org.apache.atlas.typesystem.types.StructType; ...@@ -35,6 +35,7 @@ import org.apache.atlas.typesystem.types.StructType;
import org.apache.atlas.typesystem.types.StructTypeDefinition; import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType; import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeUtils; import org.apache.atlas.typesystem.types.TypeUtils;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
...@@ -94,7 +95,6 @@ public class HiveDataModelGenerator { ...@@ -94,7 +95,6 @@ public class HiveDataModelGenerator {
createColumnClass(); createColumnClass();
createPartitionClass(); createPartitionClass();
createTableClass(); createTableClass();
createIndexClass();
createRoleClass(); createRoleClass();
// DDL/DML Process // DDL/DML Process
...@@ -177,7 +177,7 @@ public class HiveDataModelGenerator { ...@@ -177,7 +177,7 @@ public class HiveDataModelGenerator {
private void createStorageDescClass() throws AtlasException { private void createStorageDescClass() throws AtlasException {
AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{ AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{
new AttributeDefinition("cols", String.format("array<%s>", HiveDataTypes.HIVE_COLUMN.getName()), new AttributeDefinition("cols", String.format("array<%s>", HiveDataTypes.HIVE_COLUMN.getName()),
Multiplicity.COLLECTION, false, null), Multiplicity.COLLECTION, true, null),
new AttributeDefinition("location", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, new AttributeDefinition("location", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false,
null), null),
new AttributeDefinition("inputFormat", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, new AttributeDefinition("inputFormat", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false,
...@@ -188,7 +188,7 @@ public class HiveDataModelGenerator { ...@@ -188,7 +188,7 @@ public class HiveDataModelGenerator {
null), null),
new AttributeDefinition(STORAGE_NUM_BUCKETS, DataTypes.INT_TYPE.getName(), Multiplicity.OPTIONAL, false, new AttributeDefinition(STORAGE_NUM_BUCKETS, DataTypes.INT_TYPE.getName(), Multiplicity.OPTIONAL, false,
null), null),
new AttributeDefinition("serdeInfo", HiveDataTypes.HIVE_SERDE.getName(), Multiplicity.OPTIONAL, false, new AttributeDefinition("serdeInfo", HiveDataTypes.HIVE_SERDE.getName(), Multiplicity.OPTIONAL, true,
null), null),
new AttributeDefinition("bucketCols", String.format("array<%s>", DataTypes.STRING_TYPE.getName()), new AttributeDefinition("bucketCols", String.format("array<%s>", DataTypes.STRING_TYPE.getName()),
Multiplicity.OPTIONAL, false, null), Multiplicity.OPTIONAL, false, null),
...@@ -201,8 +201,8 @@ public class HiveDataModelGenerator { ...@@ -201,8 +201,8 @@ public class HiveDataModelGenerator {
Multiplicity.OPTIONAL, false, null),}; Multiplicity.OPTIONAL, false, null),};
HierarchicalTypeDefinition<ClassType> definition = HierarchicalTypeDefinition<ClassType> definition =
new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_STORAGEDESC.getName(), null, new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_STORAGEDESC.getName(),
attributeDefinitions); ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
classTypeDefinitions.put(HiveDataTypes.HIVE_STORAGEDESC.getName(), definition); classTypeDefinitions.put(HiveDataTypes.HIVE_STORAGEDESC.getName(), definition);
LOG.debug("Created definition for " + HiveDataTypes.HIVE_STORAGEDESC.getName()); LOG.debug("Created definition for " + HiveDataTypes.HIVE_STORAGEDESC.getName());
} }
...@@ -236,8 +236,8 @@ public class HiveDataModelGenerator { ...@@ -236,8 +236,8 @@ public class HiveDataModelGenerator {
false, null),}; false, null),};
HierarchicalTypeDefinition<ClassType> definition = HierarchicalTypeDefinition<ClassType> definition =
new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_DB.getName(), null, new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_DB.getName(),
attributeDefinitions); ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
classTypeDefinitions.put(HiveDataTypes.HIVE_DB.getName(), definition); classTypeDefinitions.put(HiveDataTypes.HIVE_DB.getName(), definition);
LOG.debug("Created definition for " + HiveDataTypes.HIVE_DB.getName()); LOG.debug("Created definition for " + HiveDataTypes.HIVE_DB.getName());
} }
...@@ -263,8 +263,8 @@ public class HiveDataModelGenerator { ...@@ -263,8 +263,8 @@ public class HiveDataModelGenerator {
new AttributeDefinition("type", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, null), new AttributeDefinition("type", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, null),
new AttributeDefinition(COMMENT, DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null),}; new AttributeDefinition(COMMENT, DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null),};
HierarchicalTypeDefinition<ClassType> definition = HierarchicalTypeDefinition<ClassType> definition =
new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_COLUMN.getName(), null, new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_COLUMN.getName(),
attributeDefinitions); ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
classTypeDefinitions.put(HiveDataTypes.HIVE_COLUMN.getName(), definition); classTypeDefinitions.put(HiveDataTypes.HIVE_COLUMN.getName(), definition);
LOG.debug("Created definition for " + HiveDataTypes.HIVE_COLUMN.getName()); LOG.debug("Created definition for " + HiveDataTypes.HIVE_COLUMN.getName());
} }
...@@ -278,14 +278,14 @@ public class HiveDataModelGenerator { ...@@ -278,14 +278,14 @@ public class HiveDataModelGenerator {
null), null),
new AttributeDefinition("lastAccessTime", DataTypes.LONG_TYPE.getName(), Multiplicity.OPTIONAL, false, new AttributeDefinition("lastAccessTime", DataTypes.LONG_TYPE.getName(), Multiplicity.OPTIONAL, false,
null), null),
new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.REQUIRED, false, new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.REQUIRED, true,
null), null),
new AttributeDefinition("columns", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()), new AttributeDefinition("columns", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()),
Multiplicity.OPTIONAL, true, null), Multiplicity.OPTIONAL, true, null),
new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null),}; new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null),};
HierarchicalTypeDefinition<ClassType> definition = HierarchicalTypeDefinition<ClassType> definition =
new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_PARTITION.getName(), null, new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_PARTITION.getName(),
attributeDefinitions); ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
classTypeDefinitions.put(HiveDataTypes.HIVE_PARTITION.getName(), definition); classTypeDefinitions.put(HiveDataTypes.HIVE_PARTITION.getName(), definition);
LOG.debug("Created definition for " + HiveDataTypes.HIVE_PARTITION.getName()); LOG.debug("Created definition for " + HiveDataTypes.HIVE_PARTITION.getName());
} }
...@@ -302,10 +302,10 @@ public class HiveDataModelGenerator { ...@@ -302,10 +302,10 @@ public class HiveDataModelGenerator {
null), null),
new AttributeDefinition(COMMENT, DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null), new AttributeDefinition(COMMENT, DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
new AttributeDefinition("retention", DataTypes.INT_TYPE.getName(), Multiplicity.OPTIONAL, false, null), new AttributeDefinition("retention", DataTypes.INT_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.OPTIONAL, false, new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.OPTIONAL, true,
null), null),
new AttributeDefinition("partitionKeys", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()), new AttributeDefinition("partitionKeys", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()),
Multiplicity.OPTIONAL, false, null), Multiplicity.OPTIONAL, true, null),
new AttributeDefinition("columns", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()), new AttributeDefinition("columns", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()),
Multiplicity.OPTIONAL, true, null), Multiplicity.OPTIONAL, true, null),
new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null), new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
...@@ -324,33 +324,6 @@ public class HiveDataModelGenerator { ...@@ -324,33 +324,6 @@ public class HiveDataModelGenerator {
LOG.debug("Created definition for " + HiveDataTypes.HIVE_TABLE.getName()); LOG.debug("Created definition for " + HiveDataTypes.HIVE_TABLE.getName());
} }
private void createIndexClass() throws AtlasException {
AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{
new AttributeDefinition(NAME, DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, null),
new AttributeDefinition("indexHandlerClass", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED,
false, null),
new AttributeDefinition(DB, HiveDataTypes.HIVE_DB.getName(), Multiplicity.REQUIRED, false, null),
new AttributeDefinition("createTime", DataTypes.LONG_TYPE.getName(), Multiplicity.OPTIONAL, false,
null),
new AttributeDefinition("lastAccessTime", DataTypes.LONG_TYPE.getName(), Multiplicity.OPTIONAL, false,
null),
new AttributeDefinition("origTable", HiveDataTypes.HIVE_TABLE.getName(), Multiplicity.REQUIRED, false,
null),
new AttributeDefinition("indexTable", HiveDataTypes.HIVE_TABLE.getName(), Multiplicity.OPTIONAL, false,
null),
new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.REQUIRED, false,
null),
new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
new AttributeDefinition("deferredRebuild", DataTypes.BOOLEAN_TYPE.getName(), Multiplicity.OPTIONAL,
false, null),};
HierarchicalTypeDefinition<ClassType> definition =
new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_INDEX.getName(),
ImmutableList.of(AtlasClient.DATA_SET_SUPER_TYPE), attributeDefinitions);
classTypeDefinitions.put(HiveDataTypes.HIVE_INDEX.getName(), definition);
LOG.debug("Created definition for " + HiveDataTypes.HIVE_INDEX.getName());
}
private void createRoleClass() throws AtlasException { private void createRoleClass() throws AtlasException {
AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{ AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{
new AttributeDefinition("roleName", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, new AttributeDefinition("roleName", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
...@@ -373,6 +346,8 @@ public class HiveDataModelGenerator { ...@@ -373,6 +346,8 @@ public class HiveDataModelGenerator {
new AttributeDefinition("endTime", DataTypes.LONG_TYPE.getName(), Multiplicity.REQUIRED, false, null), new AttributeDefinition("endTime", DataTypes.LONG_TYPE.getName(), Multiplicity.REQUIRED, false, null),
new AttributeDefinition("userName", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, new AttributeDefinition("userName", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
null), null),
new AttributeDefinition("operationType", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
null),
new AttributeDefinition("queryText", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, new AttributeDefinition("queryText", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
null), null),
new AttributeDefinition("queryPlan", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, new AttributeDefinition("queryPlan", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
...@@ -399,16 +374,16 @@ public class HiveDataModelGenerator { ...@@ -399,16 +374,16 @@ public class HiveDataModelGenerator {
TypesDef typesDef = hiveDataModelGenerator.getTypesDef(); TypesDef typesDef = hiveDataModelGenerator.getTypesDef();
for (EnumTypeDefinition enumType : typesDef.enumTypesAsJavaList()) { for (EnumTypeDefinition enumType : typesDef.enumTypesAsJavaList()) {
System.out.println(String.format("%s(%s) - %s", enumType.name, EnumType.class.getSimpleName(), System.out.println(String.format("%s(%s) - values %s", enumType.name, EnumType.class.getSimpleName(),
Arrays.toString(enumType.enumValues))); Arrays.toString(enumType.enumValues)));
} }
for (StructTypeDefinition structType : typesDef.structTypesAsJavaList()) { for (StructTypeDefinition structType : typesDef.structTypesAsJavaList()) {
System.out.println(String.format("%s(%s) - %s", structType.typeName, StructType.class.getSimpleName(), System.out.println(String.format("%s(%s) - attributes %s", structType.typeName, StructType.class.getSimpleName(),
Arrays.toString(structType.attributeDefinitions))); Arrays.toString(structType.attributeDefinitions)));
} }
for (HierarchicalTypeDefinition<ClassType> classType : typesDef.classTypesAsJavaList()) { for (HierarchicalTypeDefinition<ClassType> classType : typesDef.classTypesAsJavaList()) {
System.out.println(String.format("%s(%s) - %s", classType.typeName, ClassType.class.getSimpleName(), System.out.println(String.format("%s(%s) - super types [%s] - attributes %s", classType.typeName, ClassType.class.getSimpleName(),
Arrays.toString(classType.attributeDefinitions))); StringUtils.join(classType.superTypes, ","), Arrays.toString(classType.attributeDefinitions)));
} }
for (HierarchicalTypeDefinition<TraitType> traitType : typesDef.traitTypesAsJavaList()) { for (HierarchicalTypeDefinition<TraitType> traitType : typesDef.traitTypesAsJavaList()) {
System.out.println(String.format("%s(%s) - %s", traitType.typeName, TraitType.class.getSimpleName(), System.out.println(String.format("%s(%s) - %s", traitType.typeName, TraitType.class.getSimpleName(),
......
...@@ -19,11 +19,11 @@ ...@@ -19,11 +19,11 @@
package org.apache.atlas.hive.hook; package org.apache.atlas.hive.hook;
import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClient;
import org.apache.atlas.ParamChecker;
import org.apache.atlas.hive.bridge.HiveMetaStoreBridge; import org.apache.atlas.hive.bridge.HiveMetaStoreBridge;
import org.apache.atlas.hive.model.HiveDataModelGenerator; import org.apache.atlas.hive.model.HiveDataModelGenerator;
import org.apache.atlas.hive.model.HiveDataTypes; import org.apache.atlas.hive.model.HiveDataTypes;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.persistence.Id;
import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.RandomStringUtils;
import org.apache.commons.lang.StringEscapeUtils; import org.apache.commons.lang.StringEscapeUtils;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
...@@ -75,7 +75,6 @@ public class HiveHookIT { ...@@ -75,7 +75,6 @@ public class HiveHookIT {
hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, System.getProperty("user.dir") + "/target/metastore"); hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, System.getProperty("user.dir") + "/target/metastore");
hiveConf.set(HiveMetaStoreBridge.ATLAS_ENDPOINT, atlasEndpoint); hiveConf.set(HiveMetaStoreBridge.ATLAS_ENDPOINT, atlasEndpoint);
hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, "jdbc:derby:./target/metastore_db;create=true"); hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, "jdbc:derby:./target/metastore_db;create=true");
hiveConf.set(HiveHook.CONF_SYNC, "true");
hiveConf.set(HiveMetaStoreBridge.HIVE_CLUSTER_NAME, CLUSTER_NAME); hiveConf.set(HiveMetaStoreBridge.HIVE_CLUSTER_NAME, CLUSTER_NAME);
hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODE, true); //to not use hdfs hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODE, true); //to not use hdfs
hiveConf.setVar(HiveConf.ConfVars.HIVETESTMODEPREFIX, ""); hiveConf.setVar(HiveConf.ConfVars.HIVETESTMODEPREFIX, "");
...@@ -93,6 +92,7 @@ public class HiveHookIT { ...@@ -93,6 +92,7 @@ public class HiveHookIT {
String dbName = "db" + random(); String dbName = "db" + random();
runCommand("create database " + dbName + " WITH DBPROPERTIES ('p1'='v1', 'p2'='v2')"); runCommand("create database " + dbName + " WITH DBPROPERTIES ('p1'='v1', 'p2'='v2')");
String dbId = assertDatabaseIsRegistered(dbName); String dbId = assertDatabaseIsRegistered(dbName);
Referenceable definition = dgiCLient.getEntity(dbId); Referenceable definition = dgiCLient.getEntity(dbId);
Map params = (Map) definition.get("parameters"); Map params = (Map) definition.get("parameters");
Assert.assertNotNull(params); Assert.assertNotNull(params);
...@@ -145,11 +145,10 @@ public class HiveHookIT { ...@@ -145,11 +145,10 @@ public class HiveHookIT {
Referenceable tableRef = dgiCLient.getEntity(tableId); Referenceable tableRef = dgiCLient.getEntity(tableId);
Assert.assertEquals(tableRef.get("tableType"), TableType.MANAGED_TABLE.name()); Assert.assertEquals(tableRef.get("tableType"), TableType.MANAGED_TABLE.name());
Assert.assertEquals(tableRef.get(HiveDataModelGenerator.COMMENT), "table comment"); Assert.assertEquals(tableRef.get(HiveDataModelGenerator.COMMENT), "table comment");
String entityName = HiveMetaStoreBridge.getTableName(CLUSTER_NAME, DEFAULT_DB, tableName); String entityName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, DEFAULT_DB, tableName);
Assert.assertEquals(tableRef.get(HiveDataModelGenerator.NAME), entityName); Assert.assertEquals(tableRef.get(HiveDataModelGenerator.NAME), entityName);
final Id sdId = (Id) tableRef.get("sd"); final Referenceable sdRef = (Referenceable) tableRef.get("sd");
Referenceable sdRef = dgiCLient.getEntity(sdId.id);
Assert.assertEquals(sdRef.get(HiveDataModelGenerator.STORAGE_IS_STORED_AS_SUB_DIRS), false); Assert.assertEquals(sdRef.get(HiveDataModelGenerator.STORAGE_IS_STORED_AS_SUB_DIRS), false);
//Create table where database doesn't exist, will create database instance as well //Create table where database doesn't exist, will create database instance as well
...@@ -160,8 +159,7 @@ public class HiveHookIT { ...@@ -160,8 +159,7 @@ public class HiveHookIT {
LOG.debug("Searching for column {}", colName); LOG.debug("Searching for column {}", colName);
String query = String query =
String.format("%s where name = '%s'", HiveDataTypes.HIVE_COLUMN.getName(), colName.toLowerCase()); String.format("%s where name = '%s'", HiveDataTypes.HIVE_COLUMN.getName(), colName.toLowerCase());
return assertEntityIsRegistered(query, true); return assertEntityIsRegistered(query);
} }
@Test @Test
...@@ -171,8 +169,8 @@ public class HiveHookIT { ...@@ -171,8 +169,8 @@ public class HiveHookIT {
String query = "create table " + ctasTableName + " as select * from " + tableName; String query = "create table " + ctasTableName + " as select * from " + tableName;
runCommand(query); runCommand(query);
assertTableIsRegistered(DEFAULT_DB, ctasTableName);
assertProcessIsRegistered(query); assertProcessIsRegistered(query);
assertTableIsRegistered(DEFAULT_DB, ctasTableName);
} }
@Test @Test
...@@ -182,8 +180,8 @@ public class HiveHookIT { ...@@ -182,8 +180,8 @@ public class HiveHookIT {
String query = "create view " + viewName + " as select * from " + tableName; String query = "create view " + viewName + " as select * from " + tableName;
runCommand(query); runCommand(query);
assertTableIsRegistered(DEFAULT_DB, viewName);
assertProcessIsRegistered(query); assertProcessIsRegistered(query);
assertTableIsRegistered(DEFAULT_DB, viewName);
} }
@Test @Test
...@@ -257,7 +255,7 @@ public class HiveHookIT { ...@@ -257,7 +255,7 @@ public class HiveHookIT {
assertProcessIsRegistered(query); assertProcessIsRegistered(query);
} }
@Test @Test(enabled = false)
public void testAlterTable() throws Exception { public void testAlterTable() throws Exception {
String tableName = createTable(); String tableName = createTable();
String newName = tableName(); String newName = tableName();
...@@ -268,7 +266,7 @@ public class HiveHookIT { ...@@ -268,7 +266,7 @@ public class HiveHookIT {
assertTableIsNotRegistered(DEFAULT_DB, tableName); assertTableIsNotRegistered(DEFAULT_DB, tableName);
} }
@Test @Test(enabled = false)
public void testAlterView() throws Exception { public void testAlterView() throws Exception {
String tableName = createTable(); String tableName = createTable();
String viewName = tableName(); String viewName = tableName();
...@@ -292,9 +290,7 @@ public class HiveHookIT { ...@@ -292,9 +290,7 @@ public class HiveHookIT {
String gremlinQuery = String gremlinQuery =
String.format("g.V.has('__typeName', '%s').has('%s.queryText', \"%s\").toList()", typeName, typeName, String.format("g.V.has('__typeName', '%s').has('%s.queryText', \"%s\").toList()", typeName, typeName,
normalize(queryStr)); normalize(queryStr));
JSONObject response = dgiCLient.searchByGremlin(gremlinQuery); assertEntityIsRegistered(gremlinQuery);
JSONArray results = response.getJSONArray(AtlasClient.RESULTS);
Assert.assertEquals(results.length(), 1);
} }
private String normalize(String str) { private String normalize(String str) {
...@@ -304,27 +300,27 @@ public class HiveHookIT { ...@@ -304,27 +300,27 @@ public class HiveHookIT {
return StringEscapeUtils.escapeJava(str.toLowerCase()); return StringEscapeUtils.escapeJava(str.toLowerCase());
} }
private String assertTableIsRegistered(String dbName, String tableName) throws Exception { private void assertTableIsNotRegistered(String dbName, String tableName) throws Exception {
return assertTableIsRegistered(dbName, tableName, true); LOG.debug("Searching for table {}.{}", dbName, tableName);
} String query = String.format(
"%s as t where tableName = '%s', db where name = '%s' and clusterName = '%s'" + " select t",
private String assertTableIsNotRegistered(String dbName, String tableName) throws Exception { HiveDataTypes.HIVE_TABLE.getName(), tableName.toLowerCase(), dbName.toLowerCase(), CLUSTER_NAME);
return assertTableIsRegistered(dbName, tableName, false); assertEntityIsNotRegistered(query);
} }
private String assertTableIsRegistered(String dbName, String tableName, boolean registered) throws Exception { private String assertTableIsRegistered(String dbName, String tableName) throws Exception {
LOG.debug("Searching for table {}.{}", dbName, tableName); LOG.debug("Searching for table {}.{}", dbName, tableName);
String query = String.format( String query = String.format(
"%s as t where tableName = '%s', db where name = '%s' and clusterName = '%s'" + " select t", "%s as t where tableName = '%s', db where name = '%s' and clusterName = '%s'" + " select t",
HiveDataTypes.HIVE_TABLE.getName(), tableName.toLowerCase(), dbName.toLowerCase(), CLUSTER_NAME); HiveDataTypes.HIVE_TABLE.getName(), tableName.toLowerCase(), dbName.toLowerCase(), CLUSTER_NAME);
return assertEntityIsRegistered(query, registered); return assertEntityIsRegistered(query);
} }
private String assertDatabaseIsRegistered(String dbName) throws Exception { private String assertDatabaseIsRegistered(String dbName) throws Exception {
LOG.debug("Searching for database {}", dbName); LOG.debug("Searching for database {}", dbName);
String query = String.format("%s where name = '%s' and clusterName = '%s'", HiveDataTypes.HIVE_DB.getName(), String query = String.format("%s where name = '%s' and clusterName = '%s'", HiveDataTypes.HIVE_DB.getName(),
dbName.toLowerCase(), CLUSTER_NAME); dbName.toLowerCase(), CLUSTER_NAME);
return assertEntityIsRegistered(query, true); return assertEntityIsRegistered(query);
} }
private void assertPartitionIsRegistered(String dbName, String tableName, String value) throws Exception { private void assertPartitionIsRegistered(String dbName, String tableName, String value) throws Exception {
...@@ -338,27 +334,34 @@ public class HiveHookIT { ...@@ -338,27 +334,34 @@ public class HiveHookIT {
+ "out('__%s.table').has('%s.tableName', '%s').out('__%s.db').has('%s.name', '%s')" + "out('__%s.table').has('%s.tableName', '%s').out('__%s.db').has('%s.name', '%s')"
+ ".has('%s.clusterName', '%s').back('p').toList()", typeName, typeName, value, typeName, + ".has('%s.clusterName', '%s').back('p').toList()", typeName, typeName, value, typeName,
tableType, tableName.toLowerCase(), tableType, dbType, dbName.toLowerCase(), dbType, CLUSTER_NAME); tableType, tableName.toLowerCase(), tableType, dbType, dbName.toLowerCase(), dbType, CLUSTER_NAME);
JSONObject response = dgiCLient.searchByGremlin(gremlinQuery); assertEntityIsRegistered(gremlinQuery);
JSONArray results = response.getJSONArray(AtlasClient.RESULTS);
Assert.assertEquals(results.length(), 1);
} }
private String assertEntityIsRegistered(String dslQuery, boolean registered) throws Exception { private String assertEntityIsRegistered(final String query) throws Exception {
JSONArray results = dgiCLient.searchByDSL(dslQuery); waitFor(2000, new Predicate() {
if (registered) { @Override
Assert.assertEquals(results.length(), 1); public boolean evaluate() throws Exception {
JSONObject row = results.getJSONObject(0); JSONArray results = dgiCLient.search(query);
if (row.has("$id$")) { return results.length() == 1;
return row.getJSONObject("$id$").getString("id");
} else {
return row.getJSONObject("_col_0").getString("id");
} }
});
JSONArray results = dgiCLient.search(query);
JSONObject row = results.getJSONObject(0);
if (row.has("__guid")) {
return row.getString("__guid");
} else if (row.has("$id$")) {
return row.getJSONObject("$id$").getString("id");
} else { } else {
Assert.assertEquals(results.length(), 0); return row.getJSONObject("_col_0").getString("id");
return null;
} }
} }
private void assertEntityIsNotRegistered(String dslQuery) throws Exception {
JSONArray results = dgiCLient.searchByDSL(dslQuery);
Assert.assertEquals(results.length(), 0);
}
@Test @Test
public void testLineage() throws Exception { public void testLineage() throws Exception {
String table1 = createTable(false); String table1 = createTable(false);
...@@ -371,16 +374,47 @@ public class HiveHookIT { ...@@ -371,16 +374,47 @@ public class HiveHookIT {
String table1Id = assertTableIsRegistered(DEFAULT_DB, table1); String table1Id = assertTableIsRegistered(DEFAULT_DB, table1);
String table2Id = assertTableIsRegistered(db2, table2); String table2Id = assertTableIsRegistered(db2, table2);
String datasetName = HiveMetaStoreBridge.getTableName(CLUSTER_NAME, db2, table2); String datasetName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, db2, table2);
JSONObject response = dgiCLient.getInputGraph(datasetName); JSONObject response = dgiCLient.getInputGraph(datasetName);
JSONObject vertices = response.getJSONObject("values").getJSONObject("vertices"); JSONObject vertices = response.getJSONObject("values").getJSONObject("vertices");
Assert.assertTrue(vertices.has(table1Id)); Assert.assertTrue(vertices.has(table1Id));
Assert.assertTrue(vertices.has(table2Id)); Assert.assertTrue(vertices.has(table2Id));
datasetName = HiveMetaStoreBridge.getTableName(CLUSTER_NAME, DEFAULT_DB, table1); datasetName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, DEFAULT_DB, table1);
response = dgiCLient.getOutputGraph(datasetName); response = dgiCLient.getOutputGraph(datasetName);
vertices = response.getJSONObject("values").getJSONObject("vertices"); vertices = response.getJSONObject("values").getJSONObject("vertices");
Assert.assertTrue(vertices.has(table1Id)); Assert.assertTrue(vertices.has(table1Id));
Assert.assertTrue(vertices.has(table2Id)); Assert.assertTrue(vertices.has(table2Id));
} }
public interface Predicate {
/**
* Perform a predicate evaluation.
*
* @return the boolean result of the evaluation.
* @throws Exception thrown if the predicate evaluation could not evaluate.
*/
boolean evaluate() throws Exception;
}
/**
* Wait for a condition, expressed via a {@link Predicate} to become true.
*
* @param timeout maximum time in milliseconds to wait for the predicate to become true.
* @param predicate predicate waiting on.
*/
protected void waitFor(int timeout, Predicate predicate) throws Exception {
ParamChecker.notNull(predicate, "predicate");
long mustEnd = System.currentTimeMillis() + timeout;
boolean eval;
while (!(eval = predicate.evaluate()) && System.currentTimeMillis() < mustEnd) {
LOG.info("Waiting up to {} msec", mustEnd - System.currentTimeMillis());
Thread.sleep(100);
}
if (!eval) {
throw new Exception("Waiting timed out after " + timeout + " msec");
}
}
} }
...@@ -64,11 +64,6 @@ ...@@ -64,11 +64,6 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>commons-configuration</groupId>
<artifactId>commons-configuration</artifactId>
</dependency>
<dependency>
<groupId>org.testng</groupId> <groupId>org.testng</groupId>
<artifactId>testng</artifactId> <artifactId>testng</artifactId>
</dependency> </dependency>
......
...@@ -27,7 +27,7 @@ import org.apache.atlas.security.SecureClientUtils; ...@@ -27,7 +27,7 @@ import org.apache.atlas.security.SecureClientUtils;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.json.InstanceSerialization; import org.apache.atlas.typesystem.json.InstanceSerialization;
import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.Configuration;
import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONException;
...@@ -40,6 +40,7 @@ import javax.ws.rs.core.MediaType; ...@@ -40,6 +40,7 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response; import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriBuilder; import javax.ws.rs.core.UriBuilder;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.List; import java.util.List;
import static org.apache.atlas.security.SecurityProperties.TLS_ENABLED; import static org.apache.atlas.security.SecurityProperties.TLS_ENABLED;
...@@ -51,6 +52,7 @@ public class AtlasClient { ...@@ -51,6 +52,7 @@ public class AtlasClient {
private static final Logger LOG = LoggerFactory.getLogger(AtlasClient.class); private static final Logger LOG = LoggerFactory.getLogger(AtlasClient.class);
public static final String NAME = "name"; public static final String NAME = "name";
public static final String GUID = "GUID"; public static final String GUID = "GUID";
public static final String TYPE = "type";
public static final String TYPENAME = "typeName"; public static final String TYPENAME = "typeName";
public static final String DEFINITION = "definition"; public static final String DEFINITION = "definition";
...@@ -60,11 +62,12 @@ public class AtlasClient { ...@@ -60,11 +62,12 @@ public class AtlasClient {
public static final String RESULTS = "results"; public static final String RESULTS = "results";
public static final String COUNT = "count"; public static final String COUNT = "count";
public static final String ROWS = "rows"; public static final String ROWS = "rows";
public static final String DATATYPE = "dataType";
public static final String BASE_URI = "api/atlas/"; public static final String BASE_URI = "api/atlas/";
public static final String TYPES = "types"; public static final String TYPES = "types";
public static final String URI_ENTITY = "entity";
public static final String URI_ENTITIES = "entities"; public static final String URI_ENTITIES = "entities";
public static final String URI_TRAITS = "traits";
public static final String URI_SEARCH = "discovery/search"; public static final String URI_SEARCH = "discovery/search";
public static final String URI_LINEAGE = "lineage/hive/table"; public static final String URI_LINEAGE = "lineage/hive/table";
...@@ -77,11 +80,17 @@ public class AtlasClient { ...@@ -77,11 +80,17 @@ public class AtlasClient {
public static final String INFRASTRUCTURE_SUPER_TYPE = "Infrastructure"; public static final String INFRASTRUCTURE_SUPER_TYPE = "Infrastructure";
public static final String DATA_SET_SUPER_TYPE = "DataSet"; public static final String DATA_SET_SUPER_TYPE = "DataSet";
public static final String PROCESS_SUPER_TYPE = "Process"; public static final String PROCESS_SUPER_TYPE = "Process";
public static final String REFERENCEABLE_SUPER_TYPE = "Referenceable";
public static final String REFERENCEABLE_ATTRIBUTE_NAME = "qualifiedName";
public static final String JSON_MEDIA_TYPE = MediaType.APPLICATION_JSON + "; charset=UTF-8"; public static final String JSON_MEDIA_TYPE = MediaType.APPLICATION_JSON + "; charset=UTF-8";
private WebResource service; private WebResource service;
protected AtlasClient() {
//do nothing. For LocalAtlasClient
}
public AtlasClient(String baseUrl) { public AtlasClient(String baseUrl) {
this(baseUrl, null, null); this(baseUrl, null, null);
} }
...@@ -89,6 +98,8 @@ public class AtlasClient { ...@@ -89,6 +98,8 @@ public class AtlasClient {
public AtlasClient(String baseUrl, UserGroupInformation ugi, String doAsUser) { public AtlasClient(String baseUrl, UserGroupInformation ugi, String doAsUser) {
DefaultClientConfig config = new DefaultClientConfig(); DefaultClientConfig config = new DefaultClientConfig();
Configuration clientConfig = null; Configuration clientConfig = null;
int readTimeout = 60000;
int connectTimeout = 60000;
try { try {
clientConfig = getClientProperties(); clientConfig = getClientProperties();
if (clientConfig.getBoolean(TLS_ENABLED, false)) { if (clientConfig.getBoolean(TLS_ENABLED, false)) {
...@@ -97,6 +108,8 @@ public class AtlasClient { ...@@ -97,6 +108,8 @@ public class AtlasClient {
// configuration object, persist it, then subsequently pass in an empty configuration to SSLFactory // configuration object, persist it, then subsequently pass in an empty configuration to SSLFactory
SecureClientUtils.persistSSLClientConfiguration(clientConfig); SecureClientUtils.persistSSLClientConfiguration(clientConfig);
} }
readTimeout = clientConfig.getInt("atlas.client.readTimeoutMSecs", readTimeout);
connectTimeout = clientConfig.getInt("atlas.client.connectTimeoutMSecs", connectTimeout);
} catch (Exception e) { } catch (Exception e) {
LOG.info("Error processing client configuration.", e); LOG.info("Error processing client configuration.", e);
} }
...@@ -106,6 +119,8 @@ public class AtlasClient { ...@@ -106,6 +119,8 @@ public class AtlasClient {
Client client = new Client(handler, config); Client client = new Client(handler, config);
client.resource(UriBuilder.fromUri(baseUrl).build()); client.resource(UriBuilder.fromUri(baseUrl).build());
client.setReadTimeout(readTimeout);
client.setConnectTimeout(connectTimeout);
service = client.resource(UriBuilder.fromUri(baseUrl).build()); service = client.resource(UriBuilder.fromUri(baseUrl).build());
} }
...@@ -124,14 +139,14 @@ public class AtlasClient { ...@@ -124,14 +139,14 @@ public class AtlasClient {
//Entity operations //Entity operations
CREATE_ENTITY(BASE_URI + URI_ENTITIES, HttpMethod.POST), CREATE_ENTITY(BASE_URI + URI_ENTITIES, HttpMethod.POST),
GET_ENTITY(BASE_URI + URI_ENTITIES, HttpMethod.GET), GET_ENTITY(BASE_URI + URI_ENTITY, HttpMethod.GET),
UPDATE_ENTITY(BASE_URI + URI_ENTITIES, HttpMethod.PUT), UPDATE_ENTITY(BASE_URI + URI_ENTITY, HttpMethod.PUT),
LIST_ENTITY(BASE_URI + URI_ENTITIES, HttpMethod.GET), LIST_ENTITIES(BASE_URI + URI_ENTITIES, HttpMethod.GET),
//Trait operations //Trait operations
ADD_TRAITS(BASE_URI + URI_TRAITS, HttpMethod.POST), ADD_TRAITS(BASE_URI + URI_ENTITY, HttpMethod.POST),
DELETE_TRAITS(BASE_URI + URI_TRAITS, HttpMethod.DELETE), DELETE_TRAITS(BASE_URI + URI_ENTITY, HttpMethod.DELETE),
LIST_TRAITS(BASE_URI + URI_TRAITS, HttpMethod.GET), LIST_TRAITS(BASE_URI + URI_ENTITY, HttpMethod.GET),
//Search operations //Search operations
SEARCH(BASE_URI + URI_SEARCH, HttpMethod.GET), SEARCH(BASE_URI + URI_SEARCH, HttpMethod.GET),
...@@ -172,18 +187,8 @@ public class AtlasClient { ...@@ -172,18 +187,8 @@ public class AtlasClient {
} }
public List<String> listTypes() throws AtlasServiceException { public List<String> listTypes() throws AtlasServiceException {
try { final JSONObject jsonObject = callAPI(API.LIST_TYPES, null);
final JSONObject jsonObject = callAPI(API.LIST_TYPES, null); return extractResults(jsonObject);
final JSONArray list = jsonObject.getJSONArray(AtlasClient.RESULTS);
ArrayList<String> types = new ArrayList<>();
for (int index = 0; index < list.length(); index++) {
types.add(list.getString(index));
}
return types;
} catch (JSONException e) {
throw new AtlasServiceException(API.LIST_TYPES, e);
}
} }
public String getType(String typeName) throws AtlasServiceException { public String getType(String typeName) throws AtlasServiceException {
...@@ -203,18 +208,33 @@ public class AtlasClient { ...@@ -203,18 +208,33 @@ public class AtlasClient {
/** /**
* Create the given entity * Create the given entity
* @param entityAsJson entity(type instance) as json * @param entities entity(type instance) as json
* @return result json object * @return json array of guids
* @throws AtlasServiceException
*/
public JSONArray createEntity(JSONArray entities) throws AtlasServiceException {
JSONObject response = callAPI(API.CREATE_ENTITY, entities.toString());
try {
return response.getJSONArray(GUID);
} catch (JSONException e) {
throw new AtlasServiceException(API.GET_ENTITY, e);
}
}
/**
* Create the given entity
* @param entitiesAsJson entity(type instance) as json
* @return json array of guids
* @throws AtlasServiceException * @throws AtlasServiceException
*/ */
public JSONObject createEntity(String entityAsJson) throws AtlasServiceException { public JSONArray createEntity(String... entitiesAsJson) throws AtlasServiceException {
return callAPI(API.CREATE_ENTITY, entityAsJson); return createEntity(new JSONArray(Arrays.asList(entitiesAsJson)));
} }
/** /**
* Get an entity given the entity id * Get an entity given the entity id
* @param guid entity id * @param guid entity id
* @return result json object * @return result object
* @throws AtlasServiceException * @throws AtlasServiceException
*/ */
public Referenceable getEntity(String guid) throws AtlasServiceException { public Referenceable getEntity(String guid) throws AtlasServiceException {
...@@ -223,6 +243,62 @@ public class AtlasClient { ...@@ -223,6 +243,62 @@ public class AtlasClient {
String entityInstanceDefinition = jsonResponse.getString(AtlasClient.DEFINITION); String entityInstanceDefinition = jsonResponse.getString(AtlasClient.DEFINITION);
return InstanceSerialization.fromJsonReferenceable(entityInstanceDefinition, true); return InstanceSerialization.fromJsonReferenceable(entityInstanceDefinition, true);
} catch (JSONException e) { } catch (JSONException e) {
throw new AtlasServiceException(API.GET_ENTITY, e);
}
}
public static String toString(JSONArray jsonArray) throws JSONException {
ArrayList<String> resultsList = new ArrayList<>();
for (int index = 0; index < jsonArray.length(); index++) {
resultsList.add(jsonArray.getString(index));
}
return StringUtils.join(resultsList, ",");
}
/**
* Get an entity given the entity id
* @param entityType entity type name
* @param attribute qualified name of the entity
* @param value
* @return result object
* @throws AtlasServiceException
*/
public Referenceable getEntity(String entityType, String attribute, String value) throws AtlasServiceException {
WebResource resource = getResource(API.GET_ENTITY);
resource = resource.queryParam(TYPE, entityType);
resource = resource.queryParam(ATTRIBUTE_NAME, attribute);
resource = resource.queryParam(ATTRIBUTE_VALUE, value);
JSONObject jsonResponse = callAPIWithResource(API.GET_ENTITY, resource);
try {
String entityInstanceDefinition = jsonResponse.getString(AtlasClient.DEFINITION);
return InstanceSerialization.fromJsonReferenceable(entityInstanceDefinition, true);
} catch (JSONException e) {
throw new AtlasServiceException(API.GET_ENTITY, e);
}
}
/**
* List entities for a given entity type
* @param entityType
* @return
* @throws AtlasServiceException
*/
public List<String> listEntities(String entityType) throws AtlasServiceException {
WebResource resource = getResource(API.LIST_ENTITIES);
resource = resource.queryParam(TYPE, entityType);
JSONObject jsonResponse = callAPIWithResource(API.LIST_ENTITIES, resource);
return extractResults(jsonResponse);
}
private List<String> extractResults(JSONObject jsonResponse) throws AtlasServiceException {
try {
JSONArray results = jsonResponse.getJSONArray(AtlasClient.RESULTS);
ArrayList<String> resultsList = new ArrayList<>();
for (int index = 0; index < results.length(); index++) {
resultsList.add(results.getString(index));
}
return resultsList;
} catch (JSONException e) {
throw new AtlasServiceException(e); throw new AtlasServiceException(e);
} }
} }
...@@ -240,10 +316,22 @@ public class AtlasClient { ...@@ -240,10 +316,22 @@ public class AtlasClient {
return callAPIWithResource(API.UPDATE_ENTITY, resource); return callAPIWithResource(API.UPDATE_ENTITY, resource);
} }
public JSONObject searchEntity(String searchQuery) throws AtlasServiceException { /**
* Search using gremlin/dsl/full text
* @param searchQuery
* @return
* @throws AtlasServiceException
*/
public JSONArray search(String searchQuery) throws AtlasServiceException {
WebResource resource = getResource(API.SEARCH); WebResource resource = getResource(API.SEARCH);
resource = resource.queryParam(QUERY, searchQuery); resource = resource.queryParam(QUERY, searchQuery);
return callAPIWithResource(API.SEARCH, resource); JSONObject result = callAPIWithResource(API.SEARCH, resource);
try {
return result.getJSONArray(RESULTS);
} catch (JSONException e) {
throw new AtlasServiceException(e);
}
} }
/** /**
...@@ -276,7 +364,7 @@ public class AtlasClient { ...@@ -276,7 +364,7 @@ public class AtlasClient {
resource = resource.queryParam(QUERY, query); resource = resource.queryParam(QUERY, query);
JSONObject result = callAPIWithResource(API.SEARCH_DSL, resource); JSONObject result = callAPIWithResource(API.SEARCH_DSL, resource);
try { try {
return result.getJSONObject(RESULTS).getJSONArray(ROWS); return result.getJSONArray(RESULTS);
} catch (JSONException e) { } catch (JSONException e) {
throw new AtlasServiceException(e); throw new AtlasServiceException(e);
} }
...@@ -288,11 +376,16 @@ public class AtlasClient { ...@@ -288,11 +376,16 @@ public class AtlasClient {
* @return result json object * @return result json object
* @throws AtlasServiceException * @throws AtlasServiceException
*/ */
public JSONObject searchByGremlin(String gremlinQuery) throws AtlasServiceException { public JSONArray searchByGremlin(String gremlinQuery) throws AtlasServiceException {
LOG.debug("Gremlin query: " + gremlinQuery); LOG.debug("Gremlin query: " + gremlinQuery);
WebResource resource = getResource(API.SEARCH_GREMLIN); WebResource resource = getResource(API.SEARCH_GREMLIN);
resource = resource.queryParam(QUERY, gremlinQuery); resource = resource.queryParam(QUERY, gremlinQuery);
return callAPIWithResource(API.SEARCH_GREMLIN, resource); JSONObject result = callAPIWithResource(API.SEARCH_GREMLIN, resource);
try {
return result.getJSONArray(RESULTS);
} catch (JSONException e) {
throw new AtlasServiceException(e);
}
} }
/** /**
......
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns="http://maven.apache.org/POM/4.0.0"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>apache-atlas</artifactId>
<groupId>org.apache.atlas</groupId>
<version>0.6-incubating-SNAPSHOT</version>
</parent>
<artifactId>atlas-common</artifactId>
<description>Apache Atlas Common</description>
<name>Apache Atlas Common</name>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.atlas</groupId>
<artifactId>atlas-typesystem</artifactId>
</dependency>
<dependency>
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
</dependency>
</dependencies>
</project>
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.service;
import org.apache.atlas.AtlasException;
/**
* Service interface to start any background jobs
*/
public interface Service {
void start() throws AtlasException;
void stop();
}
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.service;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Set;
/**
* Utility for starting and stopping all services
*/
@Singleton
public class Services {
public static final Logger LOG = LoggerFactory.getLogger(Services.class);
private final Set<Service> services;
@Inject
public Services(Set<Service> services) {
this.services = services;
}
public void start() {
try {
for (Service service : services) {
LOG.debug("Starting service {}", service.getClass().getName());
service.start();
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
public void stop() {
for (Service service : services) {
LOG.debug("Stopping service {}", service.getClass().getName());
service.stop();
}
}
}
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
'use strict'; 'use strict';
angular.module('dgc.details').factory('DetailsResource', ['$resource', function($resource) { angular.module('dgc.details').factory('DetailsResource', ['$resource', function($resource) {
return $resource('/api/atlas/entities/:id', {}, { return $resource('/api/atlas/entity/:id', {}, {
get: { get: {
method: 'GET', method: 'GET',
transformResponse: function(data) { transformResponse: function(data) {
......
...@@ -43,7 +43,7 @@ angular.module('dgc.search').controller('SearchController', ['$scope', '$locatio ...@@ -43,7 +43,7 @@ angular.module('dgc.search').controller('SearchController', ['$scope', '$locatio
}, function searchSuccess(response) { }, function searchSuccess(response) {
$scope.resultCount = response.count; $scope.resultCount = response.count;
$scope.results = response.results; $scope.results = response.results;
$scope.resultRows = $scope.results.rows; $scope.resultRows = $scope.results;
$scope.totalItems = $scope.resultCount; $scope.totalItems = $scope.resultCount;
$scope.transformedResults = {}; $scope.transformedResults = {};
$scope.dataTransitioned = false; $scope.dataTransitioned = false;
...@@ -59,7 +59,7 @@ angular.module('dgc.search').controller('SearchController', ['$scope', '$locatio ...@@ -59,7 +59,7 @@ angular.module('dgc.search').controller('SearchController', ['$scope', '$locatio
} else { } else {
$scope.transformedResults = $scope.resultRows; $scope.transformedResults = $scope.resultRows;
} }
if ($scope.results.rows) if ($scope.results)
$scope.searchMessage = $scope.resultCount + ' results matching your search query ' + $scope.query + ' were found'; $scope.searchMessage = $scope.resultCount + ' results matching your search query ' + $scope.query + ' were found';
else else
$scope.searchMessage = '0 results matching your search query ' + $scope.query + ' were found'; $scope.searchMessage = '0 results matching your search query ' + $scope.query + ' were found';
......
...@@ -57,6 +57,35 @@ ...@@ -57,6 +57,35 @@
</python.path.l> </python.path.l>
</properties> </properties>
</profile> </profile>
<profile>
<id>dist</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<build>
<plugins>
<plugin>
<artifactId>maven-assembly-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>single</goal>
</goals>
<phase>package</phase>
<configuration>
<descriptors>
<descriptor>src/main/assemblies/standalone-package.xml</descriptor>
<descriptor>src/main/assemblies/src-package.xml</descriptor>
</descriptors>
<finalName>apache-atlas-${project.version}</finalName>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles> </profiles>
<build> <build>
...@@ -69,6 +98,7 @@ ...@@ -69,6 +98,7 @@
<executions> <executions>
<execution> <execution>
<configuration> <configuration>
<skip>${skipTests}</skip>
<executable>python</executable> <executable>python</executable>
<workingDirectory>src/test/python</workingDirectory> <workingDirectory>src/test/python</workingDirectory>
<arguments> <arguments>
...@@ -86,26 +116,6 @@ ...@@ -86,26 +116,6 @@
</execution> </execution>
</executions> </executions>
</plugin> </plugin>
<plugin>
<artifactId>maven-assembly-plugin</artifactId>
<inherited>false</inherited>
<configuration>
<descriptors>
<descriptor>src/main/assemblies/standalone-package.xml</descriptor>
<descriptor>src/main/assemblies/src-package.xml</descriptor>
</descriptors>
<finalName>apache-atlas-${project.version}</finalName>
</configuration>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins> </plugins>
</build> </build>
......
...@@ -24,7 +24,7 @@ import atlas_config as mc ...@@ -24,7 +24,7 @@ import atlas_config as mc
METADATA_LOG_OPTS="-Datlas.log.dir=%s -Datlas.log.file=application.log" METADATA_LOG_OPTS="-Datlas.log.dir=%s -Datlas.log.file=application.log"
METADATA_COMMAND_OPTS="-Datlas.home=%s" METADATA_COMMAND_OPTS="-Datlas.home=%s"
METADATA_CONFIG_OPTS="-Datlas.conf=%s" METADATA_CONFIG_OPTS="-Datlas.conf=%s"
DEFAULT_JVM_OPTS="-Xmx1024m -Dlog4j.configuration=atlas-log4j.xml" DEFAULT_JVM_OPTS="-Xmx1024m -XX:MaxPermSize=512m -Dlog4j.configuration=atlas-log4j.xml"
def main(): def main():
......
...@@ -45,9 +45,16 @@ atlas.graph.index.search.elasticsearch.client-only=false ...@@ -45,9 +45,16 @@ atlas.graph.index.search.elasticsearch.client-only=false
atlas.graph.index.search.elasticsearch.local-mode=true atlas.graph.index.search.elasticsearch.local-mode=true
atlas.graph.index.search.elasticsearch.create.sleep=2000 atlas.graph.index.search.elasticsearch.create.sleep=2000
######### Notification Configs ######### ######### Notification Configs #########
atlas.notification.embedded=true atlas.notification.embedded=true
atlas.notification.kafka.data=${sys:atlas.home}/data/kafka atlas.kafka.data=${sys:atlas.home}/data/kafka
atlas.kafka.zookeeper.connect=localhost:9026
atlas.kafka.bootstrap.servers=localhost:9027
atlas.kafka.zookeeper.session.timeout.ms=400
atlas.kafka.zookeeper.sync.time.ms=20
atlas.kafka.auto.commit.interval.ms=1000
######### Hive Lineage Configs ######### ######### Hive Lineage Configs #########
# This models reflects the base super types for Data and Process # This models reflects the base super types for Data and Process
......
...@@ -50,13 +50,6 @@ ...@@ -50,13 +50,6 @@
</fileSet> </fileSet>
<fileSet> <fileSet>
<directory>../addons/hive-bridge/src/bin</directory>
<outputDirectory>bin</outputDirectory>
<fileMode>0755</fileMode>
<directoryMode>0755</directoryMode>
</fileSet>
<fileSet>
<directory>../logs</directory> <directory>../logs</directory>
<outputDirectory>logs</outputDirectory> <outputDirectory>logs</outputDirectory>
<directoryMode>0777</directoryMode> <directoryMode>0777</directoryMode>
...@@ -85,6 +78,13 @@ ...@@ -85,6 +78,13 @@
<!-- addons/hive --> <!-- addons/hive -->
<fileSet> <fileSet>
<directory>../addons/hive-bridge/src/bin</directory>
<outputDirectory>bin</outputDirectory>
<fileMode>0755</fileMode>
<directoryMode>0755</directoryMode>
</fileSet>
<fileSet>
<directory>../addons/hive-bridge/target/dependency/bridge</directory> <directory>../addons/hive-bridge/target/dependency/bridge</directory>
<outputDirectory>bridge</outputDirectory> <outputDirectory>bridge</outputDirectory>
</fileSet> </fileSet>
...@@ -93,12 +93,6 @@ ...@@ -93,12 +93,6 @@
<directory>../addons/hive-bridge/target/dependency/hook</directory> <directory>../addons/hive-bridge/target/dependency/hook</directory>
<outputDirectory>hook</outputDirectory> <outputDirectory>hook</outputDirectory>
</fileSet> </fileSet>
<fileSet>
<directory>../addons/hive-bridge/target/site</directory>
<outputDirectory>docs/hive</outputDirectory>
</fileSet>
</fileSets> </fileSets>
<files> <files>
......
...@@ -51,13 +51,13 @@ class TestMetadata(unittest.TestCase): ...@@ -51,13 +51,13 @@ class TestMetadata(unittest.TestCase):
'org.apache.atlas.Main', 'org.apache.atlas.Main',
['-app', 'metadata_home/server/webapp/atlas'], ['-app', 'metadata_home/server/webapp/atlas'],
'metadata_home/conf:metadata_home/server/webapp/atlas/WEB-INF/classes:metadata_home/server/webapp/atlas/WEB-INF/lib\\*:metadata_home/libext\\*', 'metadata_home/conf:metadata_home/server/webapp/atlas/WEB-INF/classes:metadata_home/server/webapp/atlas/WEB-INF/lib\\*:metadata_home/libext\\*',
['-Datlas.log.dir=metadata_home/logs', '-Datlas.log.file=application.log', '-Datlas.home=metadata_home', '-Datlas.conf=metadata_home/conf', '-Xmx1024m', '-Dlog4j.configuration=atlas-log4j.xml'], 'metadata_home/logs') ['-Datlas.log.dir=metadata_home/logs', '-Datlas.log.file=application.log', '-Datlas.home=metadata_home', '-Datlas.conf=metadata_home/conf', '-Xmx1024m', '-XX:MaxPermSize=512m', '-Dlog4j.configuration=atlas-log4j.xml'], 'metadata_home/logs')
else: else:
java_mock.assert_called_with( java_mock.assert_called_with(
'org.apache.atlas.Main', 'org.apache.atlas.Main',
['-app', 'metadata_home/server/webapp/atlas'], ['-app', 'metadata_home/server/webapp/atlas'],
'metadata_home/conf:metadata_home/server/webapp/atlas/WEB-INF/classes:metadata_home/server/webapp/atlas/WEB-INF/lib/*:metadata_home/libext/*', 'metadata_home/conf:metadata_home/server/webapp/atlas/WEB-INF/classes:metadata_home/server/webapp/atlas/WEB-INF/lib/*:metadata_home/libext/*',
['-Datlas.log.dir=metadata_home/logs', '-Datlas.log.file=application.log', '-Datlas.home=metadata_home', '-Datlas.conf=metadata_home/conf', '-Xmx1024m', '-Dlog4j.configuration=atlas-log4j.xml'], 'metadata_home/logs') ['-Datlas.log.dir=metadata_home/logs', '-Datlas.log.file=application.log', '-Datlas.home=metadata_home', '-Datlas.conf=metadata_home/conf', '-Xmx1024m', '-XX:MaxPermSize=512m', '-Dlog4j.configuration=atlas-log4j.xml'], 'metadata_home/logs')
pass pass
def test_jar_java_lookups_fail(self): def test_jar_java_lookups_fail(self):
......
...@@ -40,18 +40,12 @@ ...@@ -40,18 +40,12 @@
<plugin> <plugin>
<groupId>org.apache.maven.plugins</groupId> <groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-site-plugin</artifactId> <artifactId>maven-site-plugin</artifactId>
<version>3.3</version>
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>org.apache.maven.doxia</groupId> <groupId>org.apache.maven.doxia</groupId>
<artifactId>doxia-module-twiki</artifactId> <artifactId>doxia-module-twiki</artifactId>
<version>1.3</version> <version>1.3</version>
</dependency> </dependency>
<dependency>
<groupId>org.apache.maven.wagon</groupId>
<artifactId>wagon-ssh-external</artifactId>
<version>2.6</version>
</dependency>
</dependencies> </dependencies>
<executions> <executions>
<execution> <execution>
...@@ -62,45 +56,28 @@ ...@@ -62,45 +56,28 @@
</execution> </execution>
</executions> </executions>
<configuration> <configuration>
<skip>${skipDocs}</skip> <generateProjectInfo>false</generateProjectInfo>
<reportPlugins> <generateReports>false</generateReports>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-project-info-reports-plugin</artifactId>
<version>2.3</version>
<reportSets>
<reportSet>
<reports>
<report>index</report>
<report>project-team</report>
<report>mailing-list</report>
<report>issue-tracking</report>
<report>license</report>
<report>scm</report>
</reports>
</reportSet>
</reportSets>
<configuration>
<dependencyDetailsEnabled>false</dependencyDetailsEnabled>
<dependencyLocationsEnabled>false</dependencyLocationsEnabled>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-javadoc-plugin</artifactId>
<version>2.7</version>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jxr-plugin</artifactId>
<version>2.1</version>
<configuration>
<aggregate>true</aggregate>
</configuration>
</plugin>
</reportPlugins>
</configuration> </configuration>
</plugin> </plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-project-info-reports-plugin</artifactId>
<version>2.8.1</version>
<executions>
<execution>
<goals>
<goal>project-team</goal>
<goal>mailing-list</goal>
<goal>cim</goal>
<goal>issue-tracking</goal>
<goal>license</goal>
<goal>scm</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins> </plugins>
</build> </build>
</project> </project>
...@@ -17,7 +17,7 @@ ...@@ -17,7 +17,7 @@
~ See the License for the specific language governing permissions and ~ See the License for the specific language governing permissions and
~ limitations under the License. ~ limitations under the License.
--> -->
<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" name="Metadata and Governance" <project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" name="Apache Atlas"
xmlns="http://maven.apache.org/DECORATION/1.3.0" xmlns="http://maven.apache.org/DECORATION/1.3.0"
xsi:schemaLocation="http://maven.apache.org/DECORATION/1.3.0 http://maven.apache.org/xsd/decoration-1.3.0.xsd"> xsi:schemaLocation="http://maven.apache.org/DECORATION/1.3.0 http://maven.apache.org/xsd/decoration-1.3.0.xsd">
...@@ -100,21 +100,10 @@ ...@@ -100,21 +100,10 @@
<menu name="Documentation"> <menu name="Documentation">
<!-- current points to latest release --> <!-- current points to latest release -->
<item name="current" href="./0.5.0-incubating/index.html"/> <item name="current" href="./index.html"/>
<item name="0.5-incubating" href="./0.5.0-incubating/index.html"/> <item name="0.5-incubating" href="./0.5.0-incubating/index.html"/>
</menu> </menu>
<menu name="Resources">
<item name="Overview" href="index.html"/>
<item name="Getting Started" href="./QuickStart.html"/>
<item name="Architecture" href="./Architecture.html"/>
<item name="Installation" href="./InstallationSteps.html"/>
<item name="Type System" href="./TypeSystem.html"/>
<item name="Configuration" href="./Configuration.html"/>
<item name="Security" href="./Security.html"/>
<item name="Rest API" href="./api/rest.html"/>
</menu>
<menu name="ASF"> <menu name="ASF">
<item name="How Apache Works" href="http://www.apache.org/foundation/how-it-works.html"/> <item name="How Apache Works" href="http://www.apache.org/foundation/how-it-works.html"/>
<item name="Foundation" href="http://www.apache.org/foundation/"/> <item name="Foundation" href="http://www.apache.org/foundation/"/>
......
...@@ -2,5 +2,30 @@ ...@@ -2,5 +2,30 @@
---++ Introduction ---++ Introduction
---++ Metadata High Level Architecture - Overview ---++ Atlas High Level Architecture - Overview
<img src="architecture.png" height="400" width="600" /> <img src="images/twiki/architecture.png" height="400" width="600" />
---++ Bridges
External components like hive/sqoop/storm/falcon should model their taxonomy using typesystem and register the types with Atlas. For every entity created in this external component, the corresponding entity should be registered in Atlas as well.
This is typically done in a hook which runs in the external component and is called for every entity operation. Hook generally processes the entity asynchronously using a thread pool to avoid adding latency to the main operation.
The hook can then build the entity and register the entity using Atlas REST APIs. Howerver, any failure in APIs because of network issue etc can in result entity not registered in Atlas and hence inconsistent metadata.
Atlas exposes notification interface and can be used for reliable entity registration by hook as well. The hook can send notification message containing the list of entities to be registered. Atlas service contains hook consumer that listens to these messages and registers the entities.
Available bridges are:
* [[hive/Bridge-Hive][Hive Bridge]]
---++ Notification
Notification is used for reliable entity registration from hooks and for entity/type change notifications. Atlas, by default, provides kafka integration, but its possible to provide other implementations as well. Atlas service starts embedded kafka server by default.
Atlas also provides NotificationHookConsumer that runs in Atlas Service and listens to messages from hook and registers the entities in Atlas.
<img src="images/twiki/notification.png" height="100" width="150" />
---+ Hive Atlas Bridge ---+ Hive Atlas Bridge
Hive metadata can be modelled in Atlas using its Type System. The default modelling is available in org.apache.atlas.hive.model.HiveDataModelGenerator. It defines the following types:
* hive_resource_type(EnumType) - [JAR, FILE, ARCHIVE] ---++ Hive Model
* hive_principal_type(EnumType) - [USER, ROLE, GROUP] The default hive modelling is available in org.apache.atlas.hive.model.HiveDataModelGenerator. It defines the following types:
* hive_function_type(EnumType) - [JAVA] <verbatim>
* hive_order(StructType) - [col, order] hive_object_type(EnumType) - values [GLOBAL, DATABASE, TABLE, PARTITION, COLUMN]
* hive_resourceuri(StructType) - [resourceType, uri] hive_resource_type(EnumType) - values [JAR, FILE, ARCHIVE]
* hive_serde(StructType) - [name, serializationLib, parameters] hive_principal_type(EnumType) - values [USER, ROLE, GROUP]
* hive_process(ClassType) - [name, startTime, endTime, userName, sourceTableNames, targetTableNames, queryText, queryPlan, queryId, queryGraph] hive_db(ClassType) - super types [Referenceable] - attributes [name, clusterName, description, locationUri, parameters, ownerName, ownerType]
* hive_function(ClassType) - [functionName, dbName, className, ownerName, ownerType, createTime, functionType, resourceUris] hive_order(StructType) - attributes [col, order]
* hive_type(ClassType) - [name, type1, type2, fields] hive_resourceuri(StructType) - attributes [resourceType, uri]
* hive_partition(ClassType) - [values, dbName, tableName, createTime, lastAccessTime, sd, parameters] hive_serde(StructType) - attributes [name, serializationLib, parameters]
* hive_storagedesc(ClassType) - [cols, location, inputFormat, outputFormat, compressed, numBuckets, serdeInfo, bucketCols, sortCols, parameters, storedAsSubDirectories] hive_type(ClassType) - super types [] - attributes [name, type1, type2, fields]
* hive_index(ClassType) - [indexName, indexHandlerClass, dbName, createTime, lastAccessTime, origTableName, indexTableName, sd, parameters, deferredRebuild] hive_storagedesc(ClassType) - super types [Referenceable] - attributes [cols, location, inputFormat, outputFormat, compressed, numBuckets, serdeInfo, bucketCols, sortCols, parameters, storedAsSubDirectories]
* hive_role(ClassType) - [roleName, createTime, ownerName] hive_role(ClassType) - super types [] - attributes [roleName, createTime, ownerName]
* hive_column(ClassType) - [name, type, comment] hive_column(ClassType) - super types [Referenceable] - attributes [name, type, comment]
* hive_db(ClassType) - [name, description, locationUri, parameters, ownerName, ownerType] hive_table(ClassType) - super types [DataSet] - attributes [tableName, db, owner, createTime, lastAccessTime, comment, retention, sd, partitionKeys, columns, parameters, viewOriginalText, viewExpandedText, tableType, temporary]
* hive_table(ClassType) - [name, dbName, owner, createTime, lastAccessTime, retention, sd, partitionKeys, columns, parameters, viewOriginalText, viewExpandedText, tableType, temporary] hive_partition(ClassType) - super types [Referenceable] - attributes [values, table, createTime, lastAccessTime, sd, columns, parameters]
hive_process(ClassType) - super types [Process] - attributes [startTime, endTime, userName, operationType, queryText, queryPlan, queryId, queryGraph]
</verbatim>
The entities are created and de-duped using unique qualified name. They provide namespace and can be used for querying as well:
hive_db - attribute qualifiedName - clustername.dbname
hive_table - attribute name - clustername.dbname.tablename
hive_partition - attribute qualifiedName - clustername.dbname.tablename.partitionvalues
hive_process - attribute qualifiedName - queryText
---++ Importing Hive Metadata ---++ Importing Hive Metadata
org.apache.atlas.hive.bridge.HiveMetaStoreBridge imports the hive metadata into Atlas using the typesystem defined in org.apache.atlas.hive.model.HiveDataModelGenerator. import-hive.sh command can be used to facilitate this. org.apache.atlas.hive.bridge.HiveMetaStoreBridge imports the hive metadata into Atlas using the model defined in org.apache.atlas.hive.model.HiveDataModelGenerator. import-hive.sh command can be used to facilitate this.
Set-up the following configs in hive-site.xml of your hive set-up and set environment variable HIVE_CONFIG to the Set-up the following configs in hive-site.xml of your hive set-up and set environment variable HIVE_CONFIG to the hive conf directory:
hive conf directory:
* Atlas endpoint - Add the following property with the Atlas endpoint for your set-up * Atlas endpoint - Add the following property with the Atlas endpoint for your set-up
<verbatim> <verbatim>
<property> <property>
...@@ -34,20 +41,20 @@ hive conf directory: ...@@ -34,20 +41,20 @@ hive conf directory:
</property> </property>
</verbatim> </verbatim>
Usage: <dgi package>/bin/import-hive.sh. The logs are in <dgi package>/logs/import-hive.log Usage: <atlas package>/bin/hive/import-hive.sh. The logs are in <atlas package>/logs/import-hive.log
---++ Hive Hook ---++ Hive Hook
Hive supports listeners on hive command execution using hive hooks. This is used to add/update/remove entities in Atlas using the model defined in org.apache.atlas.hive.model.HiveDataModelGenerator. Hive supports listeners on hive command execution using hive hooks. This is used to add/update/remove entities in Atlas using the model defined in org.apache.atlas.hive.model.HiveDataModelGenerator.
The hook submits the request to a thread pool executor to avoid blocking the command execution. Follow the these instructions in your hive set-up to add hive hook for Atlas: The hook submits the request to a thread pool executor to avoid blocking the command execution. The thread submits the entities as message to the notification server and atlas server reads these messages and registers the entities.
* Add org.apache.atlas.hive.hook.HiveHook as post execution hook in hive-site.xml Follow these instructions in your hive set-up to add hive hook for Atlas:
* Set-up atlas hook and atlas endpoint in hive-site.xml:
<verbatim> <verbatim>
<property> <property>
<name>hive.exec.post.hooks</name> <name>hive.exec.post.hooks</name>
<value>org.apache.atlas.hive.hook.HiveHook</value> <value>org.apache.atlas.hive.hook.HiveHook</value>
</property> </property>
</verbatim> </verbatim>
* Add the following properties in hive-ste.xml with the Atlas endpoint for your set-up
<verbatim> <verbatim>
<property> <property>
<name>atlas.rest.address</name> <name>atlas.rest.address</name>
...@@ -58,15 +65,19 @@ The hook submits the request to a thread pool executor to avoid blocking the com ...@@ -58,15 +65,19 @@ The hook submits the request to a thread pool executor to avoid blocking the com
<value>primary</value> <value>primary</value>
</property> </property>
</verbatim> </verbatim>
* Add 'export HIVE_AUX_JARS_PATH=<dgi package>/hook/hive' in hive-env.sh * Add 'export HIVE_AUX_JARS_PATH=<atlas package>/hook/hive' in hive-env.sh
* Copy <atlas package>/conf/application.properties to hive conf directory <hive package>/conf
The following properties in hive-site.xml control the thread pool details: The following properties in hive-site.xml control the thread pool and notification details:
* atlas.hook.hive.synchronous - boolean, true to run the hook synchronously. default false
* atlas.hook.hive.numRetries - number of retries for notification failure. default 3
* atlas.hook.hive.minThreads - core number of threads. default 5 * atlas.hook.hive.minThreads - core number of threads. default 5
* atlas.hook.hive.maxThreads - maximum number of threads. default 5 * atlas.hook.hive.maxThreads - maximum number of threads. default 5
* atlas.hook.hive.keepAliveTime - keep alive time in msecs. default 10 * atlas.hook.hive.keepAliveTime - keep alive time in msecs. default 10
* atlas.hook.hive.synchronous - boolean, true to run the hook synchronously. default false
Refer [[Configuration][Configuration]] for notification related configurations
---++ Limitations ---++ Limitations
* Since database name, table name and column names are case insensitive in hive, the corresponding names in entities are lowercase. So, any search APIs should use lowercase while querying on the entity names * Since database name, table name and column names are case insensitive in hive, the corresponding names in entities are lowercase. So, any search APIs should use lowercase while querying on the entity names
* Only the following hive operations are captured by hive hook currently - create database, create table, create view, CTAS, load, import, export, query, alter table rename and alter view rename * Only the following hive operations are captured by hive hook currently - create database, create table, create view, CTAS, load, import, export, query, alter table rename and alter view rename
\ No newline at end of file
...@@ -49,6 +49,26 @@ atlas.lineage.hive.process.outputs.name=outputs ...@@ -49,6 +49,26 @@ atlas.lineage.hive.process.outputs.name=outputs
atlas.lineage.hive.table.schema.query=hive_table where name=?, columns atlas.lineage.hive.table.schema.query=hive_table where name=?, columns
</verbatim> </verbatim>
---+++ Notification Configs
Refer http://kafka.apache.org/documentation.html#configuration for kafka configuration. All kafka configs should be prefixed with 'atlas.kafka.'
<verbatim>
atlas.notification.embedded=true
atlas.kafka.data=${sys:atlas.home}/data/kafka
atlas.kafka.zookeeper.connect=localhost:9026
atlas.kafka.bootstrap.servers=localhost:9027
atlas.kafka.zookeeper.session.timeout.ms=400
atlas.kafka.zookeeper.sync.time.ms=20
atlas.kafka.auto.commit.interval.ms=1000
</verbatim>
---+++ Client Configs
<verbatim>
atlas.client.readTimeoutMSecs=60000
atlas.client.connectTimeoutMSecs=60000
</verbatim>
---+++ Security Properties ---+++ Security Properties
---++++ SSL config ---++++ SSL config
......
...@@ -14,7 +14,7 @@ Once the build successfully completes, artifacts can be packaged for deployment. ...@@ -14,7 +14,7 @@ Once the build successfully completes, artifacts can be packaged for deployment.
<verbatim> <verbatim>
mvn clean package -DskipTests -DskipCheck=true mvn clean package -Pdist
</verbatim> </verbatim>
......
...@@ -8,12 +8,12 @@ instance graph below. ...@@ -8,12 +8,12 @@ instance graph below.
---+++ Example Type Definitions ---+++ Example Type Definitions
<img src="guide-class-diagram.png"/> <img src="images/twiki/guide-class-diagram.png"/>
---+++ Example Instance Graph ---+++ Example Instance Graph
<img src="guide-instance-graph.png"/> <img src="images/twiki/guide-instance-graph.png"/>
---++ Running the example ---++ Running the example
......
...@@ -5,10 +5,10 @@ ...@@ -5,10 +5,10 @@
---++ Overview ---++ Overview
---+++ Data Types Overview ---+++ Data Types Overview
<img src="data-types.png" height="400" width="600" /> <img src="images/twiki/data-types.png" height="400" width="600" />
---+++ Types Instances Overview ---+++ Types Instances Overview
<img src="types-instances.png" height="400" width="600" /> <img src="images/twiki/types-instances.png" height="400" width="600" />
---++ Details ---++ Details
......
...@@ -47,6 +47,8 @@ allows integration with the whole enterprise data ecosystem. ...@@ -47,6 +47,8 @@ allows integration with the whole enterprise data ecosystem.
* [[Search][Search]] * [[Search][Search]]
* [[security][security]] * [[security][security]]
* [[Configuration][Configuration]] * [[Configuration][Configuration]]
* Bridges
* [[Bridge-Hive][Hive Bridge]]
---++ API Documentation ---++ API Documentation
...@@ -56,4 +58,4 @@ allows integration with the whole enterprise data ecosystem. ...@@ -56,4 +58,4 @@ allows integration with the whole enterprise data ecosystem.
#LicenseInfo #LicenseInfo
---+ Licensing Information ---+ Licensing Information
Metadata (Atlas) is distributed under [[http://www.apache.org/licenses/LICENSE-2.0][Apache License 2.0]]. Atlas is distributed under [[http://www.apache.org/licenses/LICENSE-2.0][Apache License 2.0]].
...@@ -27,7 +27,7 @@ ...@@ -27,7 +27,7 @@
<version>0.6-incubating-SNAPSHOT</version> <version>0.6-incubating-SNAPSHOT</version>
</parent> </parent>
<artifactId>atlas-notification</artifactId> <artifactId>atlas-notification</artifactId>
<description>Apache Atlas Client</description> <description>Apache Atlas Notification</description>
<name>Apache Atlas Notification</name> <name>Apache Atlas Notification</name>
<packaging>jar</packaging> <packaging>jar</packaging>
...@@ -39,6 +39,11 @@ ...@@ -39,6 +39,11 @@
<dependency> <dependency>
<groupId>org.apache.atlas</groupId> <groupId>org.apache.atlas</groupId>
<artifactId>atlas-common</artifactId>
</dependency>
<dependency>
<groupId>org.apache.atlas</groupId>
<artifactId>atlas-typesystem</artifactId> <artifactId>atlas-typesystem</artifactId>
</dependency> </dependency>
...@@ -53,8 +58,8 @@ ...@@ -53,8 +58,8 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>commons-configuration</groupId> <groupId>com.google.inject.extensions</groupId>
<artifactId>commons-configuration</artifactId> <artifactId>guice-multibindings</artifactId>
</dependency> </dependency>
<dependency> <dependency>
......
...@@ -30,6 +30,7 @@ import org.apache.atlas.AtlasException; ...@@ -30,6 +30,7 @@ import org.apache.atlas.AtlasException;
import org.apache.atlas.notification.NotificationConsumer; import org.apache.atlas.notification.NotificationConsumer;
import org.apache.atlas.notification.NotificationException; import org.apache.atlas.notification.NotificationException;
import org.apache.atlas.notification.NotificationInterface; import org.apache.atlas.notification.NotificationInterface;
import org.apache.atlas.service.Service;
import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.Configuration;
import org.apache.commons.configuration.ConfigurationConverter; import org.apache.commons.configuration.ConfigurationConverter;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
...@@ -49,6 +50,9 @@ import org.slf4j.LoggerFactory; ...@@ -49,6 +50,9 @@ import org.slf4j.LoggerFactory;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.MalformedURLException;
import java.net.URISyntaxException;
import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
...@@ -57,13 +61,11 @@ import java.util.Properties; ...@@ -57,13 +61,11 @@ import java.util.Properties;
import java.util.concurrent.Future; import java.util.concurrent.Future;
@Singleton @Singleton
public class KafkaNotification extends NotificationInterface { public class KafkaNotification extends NotificationInterface implements Service {
public static final Logger LOG = LoggerFactory.getLogger(KafkaNotification.class); public static final Logger LOG = LoggerFactory.getLogger(KafkaNotification.class);
public static final String PROPERTY_PREFIX = NotificationInterface.PROPERTY_PREFIX + ".kafka"; public static final String PROPERTY_PREFIX = "atlas.kafka";
private static final int ATLAS_ZK_PORT = 9026;
private static final int ATLAS_KAFKA_PORT = 9027;
private static final String ATLAS_KAFKA_DATA = "data"; private static final String ATLAS_KAFKA_DATA = "data";
public static final String ATLAS_HOOK_TOPIC = "ATLAS_HOOK"; public static final String ATLAS_HOOK_TOPIC = "ATLAS_HOOK";
...@@ -92,9 +94,8 @@ public class KafkaNotification extends NotificationInterface { ...@@ -92,9 +94,8 @@ public class KafkaNotification extends NotificationInterface {
} }
} }
@Override public KafkaNotification(Configuration applicationProperties) throws AtlasException {
public void initialize(Configuration applicationProperties) throws AtlasException { super(applicationProperties);
super.initialize(applicationProperties);
Configuration subsetConfiguration = Configuration subsetConfiguration =
ApplicationProperties.getSubsetConfiguration(applicationProperties, PROPERTY_PREFIX); ApplicationProperties.getSubsetConfiguration(applicationProperties, PROPERTY_PREFIX);
properties = ConfigurationConverter.getProperties(subsetConfiguration); properties = ConfigurationConverter.getProperties(subsetConfiguration);
...@@ -118,42 +119,42 @@ public class KafkaNotification extends NotificationInterface { ...@@ -118,42 +119,42 @@ public class KafkaNotification extends NotificationInterface {
properties.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY, "roundrobin"); properties.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY, "roundrobin");
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "smallest"); properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "smallest");
if (isEmbedded()) {
properties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + ATLAS_KAFKA_PORT);
properties.setProperty("zookeeper.connect", "localhost:" + ATLAS_ZK_PORT);
}
//todo new APIs not available yet //todo new APIs not available yet
// consumer = new KafkaConsumer(properties); // consumer = new KafkaConsumer(properties);
// consumer.subscribe(ATLAS_HOOK_TOPIC); // consumer.subscribe(ATLAS_HOOK_TOPIC);
} }
@Override private URL getURL(String url) throws MalformedURLException {
protected void _startService() throws IOException { try {
startZk(); return new URL(url);
startKafka(); } catch(MalformedURLException e) {
return new URL("http://" + url);
}
} }
private String startZk() throws IOException { private String startZk() throws IOException, InterruptedException, URISyntaxException {
//todo read zk endpoint from config String zkValue = properties.getProperty("zookeeper.connect");
this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("0.0.0.0", ATLAS_ZK_PORT), 1024); LOG.debug("Starting zookeeper at {}", zkValue);
URL zkAddress = getURL(zkValue);
this.factory = NIOServerCnxnFactory.createFactory(
new InetSocketAddress(zkAddress.getHost(), zkAddress.getPort()), 1024);
File snapshotDir = constructDir("zk/txn"); File snapshotDir = constructDir("zk/txn");
File logDir = constructDir("zk/snap"); File logDir = constructDir("zk/snap");
try { factory.startup(new ZooKeeperServer(snapshotDir, logDir, 500));
factory.startup(new ZooKeeperServer(snapshotDir, logDir, 500));
} catch (InterruptedException e) {
throw new IOException(e);
}
return factory.getLocalAddress().getAddress().toString(); return factory.getLocalAddress().getAddress().toString();
} }
private void startKafka() { private void startKafka() throws IOException, URISyntaxException {
String kafkaValue = properties.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG);
LOG.debug("Starting kafka at {}", kafkaValue);
URL kafkaAddress = getURL(kafkaValue);
Properties brokerConfig = properties; Properties brokerConfig = properties;
brokerConfig.setProperty("broker.id", "1"); brokerConfig.setProperty("broker.id", "1");
//todo read kafka endpoint from config brokerConfig.setProperty("host.name", kafkaAddress.getHost());
brokerConfig.setProperty("host.name", "0.0.0.0"); brokerConfig.setProperty("port", String.valueOf(kafkaAddress.getPort()));
brokerConfig.setProperty("port", String.valueOf(ATLAS_KAFKA_PORT));
brokerConfig.setProperty("log.dirs", constructDir("kafka").getAbsolutePath()); brokerConfig.setProperty("log.dirs", constructDir("kafka").getAbsolutePath());
brokerConfig.setProperty("log.flush.interval.messages", String.valueOf(1)); brokerConfig.setProperty("log.flush.interval.messages", String.valueOf(1));
...@@ -162,6 +163,29 @@ public class KafkaNotification extends NotificationInterface { ...@@ -162,6 +163,29 @@ public class KafkaNotification extends NotificationInterface {
LOG.debug("Embedded kafka server started with broker config {}", brokerConfig); LOG.debug("Embedded kafka server started with broker config {}", brokerConfig);
} }
@Override
public void start() throws AtlasException {
if (isEmbedded()) {
try {
startZk();
startKafka();
} catch(Exception e) {
throw new AtlasException("Failed to start embedded kafka", e);
}
}
}
@Override
public void stop() {
if (kafkaServer != null) {
kafkaServer.shutdown();
}
if (factory != null) {
factory.shutdown();
}
}
private static class SystemTime implements Time { private static class SystemTime implements Time {
@Override @Override
public long milliseconds() { public long milliseconds() {
...@@ -192,29 +216,6 @@ public class KafkaNotification extends NotificationInterface { ...@@ -192,29 +216,6 @@ public class KafkaNotification extends NotificationInterface {
} }
@Override @Override
public void _shutdown() {
if (producer != null) {
producer.close();
}
if (consumer != null) {
consumer.close();
}
for (ConsumerConnector consumerConnector : consumerConnectors) {
consumerConnector.shutdown();
}
if (kafkaServer != null) {
kafkaServer.shutdown();
}
if (factory != null) {
factory.shutdown();
}
}
@Override
public List<NotificationConsumer> createConsumers(NotificationType type, int numConsumers) { public List<NotificationConsumer> createConsumers(NotificationType type, int numConsumers) {
String topic = topicMap.get(type); String topic = topicMap.get(type);
...@@ -261,6 +262,24 @@ public class KafkaNotification extends NotificationInterface { ...@@ -261,6 +262,24 @@ public class KafkaNotification extends NotificationInterface {
} }
} }
@Override
public void close() {
if (producer != null) {
producer.close();
producer = null;
}
if (consumer != null) {
consumer.close();
consumer = null;
}
for (ConsumerConnector consumerConnector : consumerConnectors) {
consumerConnector.shutdown();
}
consumerConnectors.clear();
}
//New API, not used now //New API, not used now
private List<String> receive(long timeout) throws NotificationException { private List<String> receive(long timeout) throws NotificationException {
Map<String, ConsumerRecords> recordsMap = consumer.poll(timeout); Map<String, ConsumerRecords> recordsMap = consumer.poll(timeout);
......
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.kafka;
import com.google.inject.Provider;
import com.google.inject.Provides;
import com.google.inject.Singleton;
import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasException;
import org.apache.commons.configuration.Configuration;
public class KafkaNotificationProvider implements Provider<KafkaNotification> {
@Override
@Provides
@Singleton
public KafkaNotification get() {
try {
Configuration applicationProperties = ApplicationProperties.get();
KafkaNotification instance = new KafkaNotification(applicationProperties);
return instance;
} catch(AtlasException e) {
throw new RuntimeException(e);
}
}
}
...@@ -18,37 +18,43 @@ ...@@ -18,37 +18,43 @@
package org.apache.atlas.notification; package org.apache.atlas.notification;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.atlas.ApplicationProperties; import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClient;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.AtlasServiceException; import org.apache.atlas.service.Service;
import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.Configuration;
import org.codehaus.jettison.json.JSONArray;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.List; import java.util.List;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
public class NotificationHookConsumer { /**
* Consumer of notifications from hooks e.g., hive hook etc
*/
@Singleton
public class NotificationHookConsumer implements Service {
private static final Logger LOG = LoggerFactory.getLogger(NotificationHookConsumer.class); private static final Logger LOG = LoggerFactory.getLogger(NotificationHookConsumer.class);
public static final String CONSUMER_THREADS_PROPERTY = "atlas.notification.hook.numthreads"; public static final String CONSUMER_THREADS_PROPERTY = "atlas.notification.hook.numthreads";
public static final String ATLAS_ENDPOINT_PROPERTY = "atlas.rest.address"; public static final String ATLAS_ENDPOINT_PROPERTY = "atlas.rest.address";
@Inject @Inject
private static NotificationInterface notificationInterface; private NotificationInterface notificationInterface;
private ExecutorService executors;
private static ExecutorService executors; private AtlasClient atlasClient;
private static AtlasClient atlasClient;
public static void start() throws AtlasException { @Override
public void start() throws AtlasException {
Configuration applicationProperties = ApplicationProperties.get(); Configuration applicationProperties = ApplicationProperties.get();
notificationInterface.initialize(applicationProperties);
String atlasEndpoint = applicationProperties.getString(ATLAS_ENDPOINT_PROPERTY, "http://localhost:21000"); String atlasEndpoint = applicationProperties.getString(ATLAS_ENDPOINT_PROPERTY, "http://localhost:21000");
atlasClient = new AtlasClient(atlasEndpoint); atlasClient = new AtlasClient(atlasEndpoint);
int numThreads = applicationProperties.getInt(CONSUMER_THREADS_PROPERTY, 2); int numThreads = applicationProperties.getInt(CONSUMER_THREADS_PROPERTY, 1);
List<NotificationConsumer> consumers = List<NotificationConsumer> consumers =
notificationInterface.createConsumers(NotificationInterface.NotificationType.HOOK, numThreads); notificationInterface.createConsumers(NotificationInterface.NotificationType.HOOK, numThreads);
executors = Executors.newFixedThreadPool(consumers.size()); executors = Executors.newFixedThreadPool(consumers.size());
...@@ -58,12 +64,20 @@ public class NotificationHookConsumer { ...@@ -58,12 +64,20 @@ public class NotificationHookConsumer {
} }
} }
public static void stop() { @Override
notificationInterface.shutdown(); public void stop() {
executors.shutdown(); //Allow for completion of outstanding work
notificationInterface.close();
try {
if (executors != null && !executors.awaitTermination(5000, TimeUnit.MILLISECONDS)) {
LOG.error("Timed out waiting for consumer threads to shut down, exiting uncleanly");
}
} catch (InterruptedException e) {
LOG.error("Failure in shutting down consumers");
}
} }
static class HookConsumer implements Runnable { class HookConsumer implements Runnable {
private final NotificationConsumer consumer; private final NotificationConsumer consumer;
public HookConsumer(NotificationConsumer consumerInterface) { public HookConsumer(NotificationConsumer consumerInterface) {
...@@ -74,12 +88,13 @@ public class NotificationHookConsumer { ...@@ -74,12 +88,13 @@ public class NotificationHookConsumer {
public void run() { public void run() {
while(consumer.hasNext()) { while(consumer.hasNext()) {
String entityJson = consumer.next(); String entityJson = consumer.next();
LOG.debug("Processing message {}", entityJson); LOG.info("Processing message {}", entityJson);
try { try {
atlasClient.createEntity(entityJson); JSONArray guids = atlasClient.createEntity(new JSONArray(entityJson));
} catch (AtlasServiceException e) { LOG.info("Create entities with guid {}", guids);
} catch (Exception e) {
//todo handle failures //todo handle failures
LOG.warn("Error handling message {}", entityJson); LOG.warn("Error handling message {}", entityJson, e);
} }
} }
} }
......
...@@ -20,7 +20,6 @@ package org.apache.atlas.notification; ...@@ -20,7 +20,6 @@ package org.apache.atlas.notification;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.Configuration;
import java.io.IOException;
import java.util.List; import java.util.List;
public abstract class NotificationInterface { public abstract class NotificationInterface {
...@@ -33,26 +32,11 @@ public abstract class NotificationInterface { ...@@ -33,26 +32,11 @@ public abstract class NotificationInterface {
HOOK, ENTITIES, TYPES HOOK, ENTITIES, TYPES
} }
/** public NotificationInterface(Configuration applicationProperties) throws AtlasException {
* Initialise
* @param applicationProperties
* @throws AtlasException
*/
public void initialize(Configuration applicationProperties) throws AtlasException {
this.embedded = applicationProperties.getBoolean(PROPERTY_EMBEDDED, false); this.embedded = applicationProperties.getBoolean(PROPERTY_EMBEDDED, false);
} }
/** /**
* Start embedded notification service on atlast server
* @throws IOException
*/
public final void startService() throws IOException {
if (embedded) {
_startService();
}
}
/**
* Is the notification service embedded in atlas server * Is the notification service embedded in atlas server
* @return * @return
*/ */
...@@ -60,18 +44,9 @@ public abstract class NotificationInterface { ...@@ -60,18 +44,9 @@ public abstract class NotificationInterface {
return embedded; return embedded;
} }
protected abstract void _startService() throws IOException;
/**
* Shutdown - close all the connections
*/
public final void shutdown() {
_shutdown();
}
protected abstract void _shutdown();
public abstract List<NotificationConsumer> createConsumers(NotificationType type, int numConsumers); public abstract List<NotificationConsumer> createConsumers(NotificationType type, int numConsumers);
public abstract void send(NotificationType type, String... messages) throws NotificationException; public abstract void send(NotificationType type, String... messages) throws NotificationException;
public abstract void close();
} }
...@@ -18,11 +18,21 @@ ...@@ -18,11 +18,21 @@
package org.apache.atlas.notification; package org.apache.atlas.notification;
import com.google.inject.AbstractModule; import com.google.inject.AbstractModule;
import com.google.inject.Singleton;
import com.google.inject.multibindings.Multibinder;
import org.apache.atlas.kafka.KafkaNotification; import org.apache.atlas.kafka.KafkaNotification;
import org.apache.atlas.kafka.KafkaNotificationProvider;
import org.apache.atlas.service.Service;
public class NotificationModule extends AbstractModule { public class NotificationModule extends AbstractModule {
@Override @Override
protected void configure() { protected void configure() {
bind(NotificationInterface.class).to(KafkaNotification.class).asEagerSingleton(); bind(NotificationInterface.class).to(KafkaNotification.class).in(Singleton.class);
bind(KafkaNotification.class).toProvider(KafkaNotificationProvider.class).in(Singleton.class);
Multibinder<Service> serviceBinder = Multibinder.newSetBinder(binder(), Service.class);
serviceBinder.addBinding().to(KafkaNotification.class);
serviceBinder.addBinding().to(NotificationHookConsumer.class);
} }
} }
...@@ -18,12 +18,10 @@ ...@@ -18,12 +18,10 @@
package org.apache.atlas.kafka; package org.apache.atlas.kafka;
import com.google.inject.Inject; import com.google.inject.Inject;
import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.notification.NotificationConsumer; import org.apache.atlas.notification.NotificationConsumer;
import org.apache.atlas.notification.NotificationInterface; import org.apache.atlas.notification.NotificationInterface;
import org.apache.atlas.notification.NotificationModule; import org.apache.atlas.notification.NotificationModule;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.RandomStringUtils;
import org.testng.Assert; import org.testng.Assert;
import org.testng.annotations.AfterClass; import org.testng.annotations.AfterClass;
...@@ -35,18 +33,15 @@ import org.testng.annotations.Test; ...@@ -35,18 +33,15 @@ import org.testng.annotations.Test;
public class KafkaNotificationTest { public class KafkaNotificationTest {
@Inject @Inject
private NotificationInterface kafka; private KafkaNotification kafka;
@BeforeClass @BeforeClass
public void setUp() throws Exception { public void setUp() throws Exception {
Configuration conf = ApplicationProperties.get(); kafka.start();
conf.setProperty(KafkaNotification.PROPERTY_PREFIX + ".data", "target/data/kafka" + random());
kafka.initialize(conf);
kafka.startService();
} }
@Test @Test
public void testSendMessage() throws AtlasException { public void testSendReceiveMessage() throws AtlasException {
String msg1 = "message" + random(); String msg1 = "message" + random();
String msg2 = "message" + random(); String msg2 = "message" + random();
kafka.send(NotificationInterface.NotificationType.HOOK, msg1, msg2); kafka.send(NotificationInterface.NotificationType.HOOK, msg1, msg2);
...@@ -63,6 +58,6 @@ public class KafkaNotificationTest { ...@@ -63,6 +58,6 @@ public class KafkaNotificationTest {
@AfterClass @AfterClass
public void teardown() throws Exception { public void teardown() throws Exception {
kafka.shutdown(); kafka.stop();
} }
} }
...@@ -392,6 +392,7 @@ ...@@ -392,6 +392,7 @@
</profile> </profile>
</profiles> </profiles>
<modules> <modules>
<module>common</module>
<module>typesystem</module> <module>typesystem</module>
<module>notification</module> <module>notification</module>
<module>client</module> <module>client</module>
...@@ -972,6 +973,12 @@ ...@@ -972,6 +973,12 @@
<dependency> <dependency>
<groupId>org.apache.atlas</groupId> <groupId>org.apache.atlas</groupId>
<artifactId>atlas-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.atlas</groupId>
<artifactId>atlas-webapp</artifactId> <artifactId>atlas-webapp</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>war</type> <type>war</type>
...@@ -1298,7 +1305,7 @@ ...@@ -1298,7 +1305,7 @@
<plugin> <plugin>
<groupId>org.apache.maven.plugins</groupId> <groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-site-plugin</artifactId> <artifactId>maven-site-plugin</artifactId>
<version>3.2</version> <version>3.3</version>
</plugin> </plugin>
<plugin> <plugin>
...@@ -1348,13 +1355,6 @@ ...@@ -1348,13 +1355,6 @@
<goal>testCompile</goal> <goal>testCompile</goal>
</goals> </goals>
</execution> </execution>
<execution>
<id>attach-scaladocs</id>
<phase>verify</phase>
<goals>
<goal>doc-jar</goal>
</goals>
</execution>
</executions> </executions>
<configuration> <configuration>
<scalaVersion>${scala.version}</scalaVersion> <scalaVersion>${scala.version}</scalaVersion>
...@@ -1428,6 +1428,9 @@ ...@@ -1428,6 +1428,9 @@
-Xmx1024m -XX:MaxPermSize=512m -Dlog4j.configuration=atlas-log4j.xml -Xmx1024m -XX:MaxPermSize=512m -Dlog4j.configuration=atlas-log4j.xml
-Djava.net.preferIPv4Stack=true -Djava.net.preferIPv4Stack=true
</argLine> </argLine>
<excludes>
<exclude>**/*Base*</exclude>
</excludes>
</configuration> </configuration>
<dependencies> <dependencies>
<dependency> <dependency>
......
...@@ -4,6 +4,7 @@ Apache Atlas Release Notes ...@@ -4,6 +4,7 @@ Apache Atlas Release Notes
--trunk - unreleased --trunk - unreleased
INCOMPATIBLE CHANGES: INCOMPATIBLE CHANGES:
ATLAS-58 Make hive hook reliable (shwethags)
ATLAS-54 Rename configs in hive hook (shwethags) ATLAS-54 Rename configs in hive hook (shwethags)
ATLAS-3 Mixed Index creation fails with Date types (suma.shivaprasad via shwethags) ATLAS-3 Mixed Index creation fails with Date types (suma.shivaprasad via shwethags)
......
...@@ -32,6 +32,7 @@ public class GraphTransactionInterceptor implements MethodInterceptor { ...@@ -32,6 +32,7 @@ public class GraphTransactionInterceptor implements MethodInterceptor {
@Inject @Inject
GraphProvider<TitanGraph> graphProvider; GraphProvider<TitanGraph> graphProvider;
@Override
public Object invoke(MethodInvocation invocation) throws Throwable { public Object invoke(MethodInvocation invocation) throws Throwable {
if (titanGraph == null) { if (titanGraph == null) {
titanGraph = graphProvider.get(); titanGraph = graphProvider.get();
......
...@@ -18,11 +18,11 @@ ...@@ -18,11 +18,11 @@
package org.apache.atlas; package org.apache.atlas;
import com.google.inject.Singleton;
import com.google.inject.matcher.Matchers; import com.google.inject.matcher.Matchers;
import com.google.inject.multibindings.Multibinder; import com.google.inject.multibindings.Multibinder;
import com.google.inject.throwingproviders.ThrowingProviderBinder; import com.google.inject.throwingproviders.ThrowingProviderBinder;
import com.thinkaurelius.titan.core.TitanGraph; import com.thinkaurelius.titan.core.TitanGraph;
import com.tinkerpop.blueprints.Graph;
import org.aopalliance.intercept.MethodInterceptor; import org.aopalliance.intercept.MethodInterceptor;
import org.apache.atlas.discovery.DiscoveryService; import org.apache.atlas.discovery.DiscoveryService;
import org.apache.atlas.discovery.HiveLineageService; import org.apache.atlas.discovery.HiveLineageService;
...@@ -38,6 +38,7 @@ import org.apache.atlas.repository.typestore.GraphBackedTypeStore; ...@@ -38,6 +38,7 @@ import org.apache.atlas.repository.typestore.GraphBackedTypeStore;
import org.apache.atlas.repository.typestore.ITypeStore; import org.apache.atlas.repository.typestore.ITypeStore;
import org.apache.atlas.services.DefaultMetadataService; import org.apache.atlas.services.DefaultMetadataService;
import org.apache.atlas.services.MetadataService; import org.apache.atlas.services.MetadataService;
import org.apache.atlas.typesystem.types.TypeSystem;
/** /**
* Guice module for Repository module. * Guice module for Repository module.
...@@ -55,6 +56,8 @@ public class RepositoryMetadataModule extends com.google.inject.AbstractModule { ...@@ -55,6 +56,8 @@ public class RepositoryMetadataModule extends com.google.inject.AbstractModule {
// bind the MetadataRepositoryService interface to an implementation // bind the MetadataRepositoryService interface to an implementation
bind(MetadataRepository.class).to(GraphBackedMetadataRepository.class).asEagerSingleton(); bind(MetadataRepository.class).to(GraphBackedMetadataRepository.class).asEagerSingleton();
bind(TypeSystem.class).in(Singleton.class);
// bind the ITypeStore interface to an implementation // bind the ITypeStore interface to an implementation
bind(ITypeStore.class).to(GraphBackedTypeStore.class).asEagerSingleton(); bind(ITypeStore.class).to(GraphBackedTypeStore.class).asEagerSingleton();
......
...@@ -21,6 +21,8 @@ package org.apache.atlas.listener; ...@@ -21,6 +21,8 @@ package org.apache.atlas.listener;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.typesystem.ITypedReferenceableInstance; import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import java.util.Collection;
/** /**
* Entity (a Typed instance) change notification listener. * Entity (a Typed instance) change notification listener.
*/ */
...@@ -29,10 +31,10 @@ public interface EntityChangeListener { ...@@ -29,10 +31,10 @@ public interface EntityChangeListener {
/** /**
* This is upon adding a new typed instance to the repository. * This is upon adding a new typed instance to the repository.
* *
* @param typedInstance a typed instance * @param typedInstances a typed instance
* @throws AtlasException * @throws AtlasException
*/ */
void onEntityAdded(ITypedReferenceableInstance typedInstance) throws AtlasException; void onEntityAdded(Collection<ITypedReferenceableInstance> typedInstances) throws AtlasException;
/** /**
* This is upon adding a new trait to a typed instance. * This is upon adding a new trait to a typed instance.
......
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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;
import org.apache.atlas.AtlasException;
import org.apache.atlas.typesystem.IReferenceableInstance;
public class EntityExistsException extends AtlasException {
public EntityExistsException(IReferenceableInstance typedInstance, Exception e) {
super("Model violation for type "+ typedInstance.getTypeName(), e);
}
public EntityExistsException(IReferenceableInstance typedInstance) {
super("Model violation for type "+ typedInstance.getTypeName());
}
}
...@@ -19,7 +19,6 @@ ...@@ -19,7 +19,6 @@
package org.apache.atlas.repository; package org.apache.atlas.repository;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.typesystem.IReferenceableInstance;
import org.apache.atlas.typesystem.ITypedReferenceableInstance; import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.ITypedStruct; import org.apache.atlas.typesystem.ITypedStruct;
import org.apache.atlas.typesystem.types.AttributeInfo; import org.apache.atlas.typesystem.types.AttributeInfo;
...@@ -76,11 +75,12 @@ public interface MetadataRepository { ...@@ -76,11 +75,12 @@ public interface MetadataRepository {
/** /**
* Creates an entity definition (instance) corresponding to a given type. * Creates an entity definition (instance) corresponding to a given type.
* *
* @param entity entity (typed instance) * @param entities entity (typed instance)
* @return a globally unique identifier * @return a globally unique identifier
* @throws RepositoryException * @throws RepositoryException
* @throws EntityExistsException
*/ */
String createEntity(IReferenceableInstance entity) throws RepositoryException; String[] createEntities(ITypedReferenceableInstance... entities) throws RepositoryException, EntityExistsException;
/** /**
* Fetch the complete definition of an entity given its GUID. * Fetch the complete definition of an entity given its GUID.
...@@ -158,4 +158,13 @@ public interface MetadataRepository { ...@@ -158,4 +158,13 @@ public interface MetadataRepository {
* @param value property value * @param value property value
*/ */
void updateEntity(String guid, String property, String value) throws RepositoryException; void updateEntity(String guid, String property, String value) throws RepositoryException;
/**
* Returns the entity for the given type and qualified name
* @param entityType
* @param attribute
* @param value
* @return entity instance
*/
ITypedReferenceableInstance getEntityDefinition(String entityType, String attribute, String value) throws AtlasException;
} }
...@@ -19,6 +19,8 @@ ...@@ -19,6 +19,8 @@
package org.apache.atlas.repository.graph; package org.apache.atlas.repository.graph;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.thinkaurelius.titan.core.SchemaViolationException;
import com.thinkaurelius.titan.core.TitanGraph; import com.thinkaurelius.titan.core.TitanGraph;
import com.thinkaurelius.titan.core.TitanProperty; import com.thinkaurelius.titan.core.TitanProperty;
import com.thinkaurelius.titan.core.TitanVertex; import com.thinkaurelius.titan.core.TitanVertex;
...@@ -29,6 +31,7 @@ import com.tinkerpop.blueprints.Vertex; ...@@ -29,6 +31,7 @@ import com.tinkerpop.blueprints.Vertex;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.GraphTransaction; import org.apache.atlas.GraphTransaction;
import org.apache.atlas.repository.Constants; import org.apache.atlas.repository.Constants;
import org.apache.atlas.repository.EntityExistsException;
import org.apache.atlas.repository.EntityNotFoundException; import org.apache.atlas.repository.EntityNotFoundException;
import org.apache.atlas.repository.MetadataRepository; import org.apache.atlas.repository.MetadataRepository;
import org.apache.atlas.repository.RepositoryException; import org.apache.atlas.repository.RepositoryException;
...@@ -37,7 +40,6 @@ import org.apache.atlas.typesystem.ITypedInstance; ...@@ -37,7 +40,6 @@ import org.apache.atlas.typesystem.ITypedInstance;
import org.apache.atlas.typesystem.ITypedReferenceableInstance; import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.ITypedStruct; import org.apache.atlas.typesystem.ITypedStruct;
import org.apache.atlas.typesystem.persistence.Id; import org.apache.atlas.typesystem.persistence.Id;
import org.apache.atlas.typesystem.persistence.MapIds;
import org.apache.atlas.typesystem.types.AttributeInfo; import org.apache.atlas.typesystem.types.AttributeInfo;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes; import org.apache.atlas.typesystem.types.DataTypes;
...@@ -62,9 +64,9 @@ import java.util.Collections; ...@@ -62,9 +64,9 @@ import java.util.Collections;
import java.util.Date; import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
/** /**
* An implementation backed by a Graph database provided * An implementation backed by a Graph database provided
...@@ -78,8 +80,6 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -78,8 +80,6 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
private static final String EDGE_LABEL_PREFIX = "__"; private static final String EDGE_LABEL_PREFIX = "__";
private final AtomicInteger ID_SEQ = new AtomicInteger(0);
private final TypedInstanceToGraphMapper instanceToGraphMapper = new TypedInstanceToGraphMapper(); private final TypedInstanceToGraphMapper instanceToGraphMapper = new TypedInstanceToGraphMapper();
private final GraphToTypedInstanceMapper graphToInstanceMapper = new GraphToTypedInstanceMapper(); private final GraphToTypedInstanceMapper graphToInstanceMapper = new GraphToTypedInstanceMapper();
...@@ -89,7 +89,6 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -89,7 +89,6 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
@Inject @Inject
public GraphBackedMetadataRepository(GraphProvider<TitanGraph> graphProvider) throws AtlasException { public GraphBackedMetadataRepository(GraphProvider<TitanGraph> graphProvider) throws AtlasException {
this.typeSystem = TypeSystem.getInstance(); this.typeSystem = TypeSystem.getInstance();
this.titanGraph = graphProvider.get(); this.titanGraph = graphProvider.get();
} }
...@@ -126,6 +125,9 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -126,6 +125,9 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
return getQualifiedName(dataType, aInfo.name); return getQualifiedName(dataType, aInfo.name);
} }
public String getFieldNameInVertex(IDataType<?> dataType, String attrName) throws AtlasException {
return getQualifiedName(dataType, attrName);
}
@Override @Override
public String getEdgeLabel(IDataType<?> dataType, AttributeInfo aInfo) { public String getEdgeLabel(IDataType<?> dataType, AttributeInfo aInfo) {
return getEdgeLabel(dataType.getName(), aInfo.name); return getEdgeLabel(dataType.getName(), aInfo.name);
...@@ -142,10 +144,13 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -142,10 +144,13 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
@Override @Override
@GraphTransaction @GraphTransaction
public String createEntity(IReferenceableInstance typedInstance) throws RepositoryException { public String[] createEntities(ITypedReferenceableInstance... entities) throws RepositoryException,
LOG.info("adding entity={}", typedInstance); EntityExistsException {
LOG.info("adding entities={}", entities);
try { try {
return instanceToGraphMapper.mapTypedInstanceToGraph(typedInstance); return instanceToGraphMapper.mapTypedInstanceToGraph(entities);
} catch (EntityExistsException e) {
throw e;
} catch (AtlasException e) { } catch (AtlasException e) {
throw new RepositoryException(e); throw new RepositoryException(e);
} }
...@@ -159,19 +164,38 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -159,19 +164,38 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
Vertex instanceVertex = getVertexForGUID(guid); Vertex instanceVertex = getVertexForGUID(guid);
try { try {
LOG.debug("Found a vertex {} for guid {}", instanceVertex, guid);
return graphToInstanceMapper.mapGraphToTypedInstance(guid, instanceVertex); return graphToInstanceMapper.mapGraphToTypedInstance(guid, instanceVertex);
} catch (AtlasException e) { } catch (AtlasException e) {
throw new RepositoryException(e); throw new RepositoryException(e);
} }
} }
@Override
@GraphTransaction
public ITypedReferenceableInstance getEntityDefinition(String entityType, String attribute, String value)
throws AtlasException {
LOG.info("Retrieving entity with type={} and {}={}", entityType, attribute, value);
IDataType type = typeSystem.getDataType(IDataType.class, entityType);
String propertyKey = getFieldNameInVertex(type, attribute);
Vertex instanceVertex = getVertexForProperty(propertyKey, value);
String guid = instanceVertex.getProperty(Constants.GUID_PROPERTY_KEY);
return graphToInstanceMapper.mapGraphToTypedInstance(guid, instanceVertex);
}
private Vertex getVertexForGUID(String guid) throws EntityNotFoundException { private Vertex getVertexForGUID(String guid) throws EntityNotFoundException {
Vertex instanceVertex = GraphHelper.findVertexByGUID(titanGraph, guid); return getVertexForProperty(Constants.GUID_PROPERTY_KEY, guid);
}
private Vertex getVertexForProperty(String propertyKey, Object value) throws EntityNotFoundException {
Vertex instanceVertex = GraphHelper.findVertex(titanGraph, propertyKey, value);
if (instanceVertex == null) { if (instanceVertex == null) {
LOG.debug("Could not find a vertex for guid={}", guid); LOG.debug("Could not find a vertex with {}={}", propertyKey, value);
throw new EntityNotFoundException("Could not find an entity in the repository for guid: " + guid); throw new EntityNotFoundException("Could not find an entity in the repository with " + propertyKey + "="
+ value);
} else {
LOG.debug("Found a vertex {} with {}={}", instanceVertex, propertyKey, value);
} }
return instanceVertex; return instanceVertex;
...@@ -244,7 +268,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -244,7 +268,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
instanceVertex, Collections.<Id, Vertex>emptyMap()); instanceVertex, Collections.<Id, Vertex>emptyMap());
// update the traits in entity once adding trait instance is successful // update the traits in entity once adding trait instance is successful
((TitanVertex) instanceVertex).addProperty(Constants.TRAIT_NAMES_PROPERTY_KEY, traitName); GraphHelper.addProperty(instanceVertex, Constants.TRAIT_NAMES_PROPERTY_KEY, traitName);
} catch (RepositoryException e) { } catch (RepositoryException e) {
throw e; throw e;
...@@ -304,7 +328,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -304,7 +328,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
// add it back again // add it back again
for (String traitName : traitNames) { for (String traitName : traitNames) {
((TitanVertex) instanceVertex).addProperty(Constants.TRAIT_NAMES_PROPERTY_KEY, traitName); GraphHelper.addProperty(instanceVertex, Constants.TRAIT_NAMES_PROPERTY_KEY, traitName);
} }
} }
...@@ -368,16 +392,18 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -368,16 +392,18 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
private final class EntityProcessor implements ObjectGraphWalker.NodeProcessor { private final class EntityProcessor implements ObjectGraphWalker.NodeProcessor {
public final Map<Id, Id> idToNewIdMap;
public final Map<Id, IReferenceableInstance> idToInstanceMap; public final Map<Id, IReferenceableInstance> idToInstanceMap;
public final Map<Id, Vertex> idToVertexMap; public final Map<Id, Vertex> idToVertexMap;
public EntityProcessor() { public EntityProcessor() {
idToNewIdMap = new HashMap<>(); idToInstanceMap = new LinkedHashMap<>();
idToInstanceMap = new HashMap<>();
idToVertexMap = new HashMap<>(); idToVertexMap = new HashMap<>();
} }
public void cleanUp() {
idToInstanceMap.clear();
}
@Override @Override
public void processNode(ObjectGraphWalker.Node nd) throws AtlasException { public void processNode(ObjectGraphWalker.Node nd) throws AtlasException {
IReferenceableInstance ref = null; IReferenceableInstance ref = null;
...@@ -394,10 +420,6 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -394,10 +420,6 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
if (id != null) { if (id != null) {
if (id.isUnassigned()) { if (id.isUnassigned()) {
if (!idToNewIdMap.containsKey(id)) {
idToNewIdMap.put(id, new Id(ID_SEQ.getAndIncrement(), 0, id.className));
}
if (ref != null) { if (ref != null) {
if (idToInstanceMap.containsKey(id)) { // Oops if (idToInstanceMap.containsKey(id)) { // Oops
throw new RepositoryException( throw new RepositoryException(
...@@ -410,42 +432,74 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -410,42 +432,74 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
} }
} }
private void createVerticesForClassTypes(List<ITypedReferenceableInstance> newInstances) throws AtlasException { private List<ITypedReferenceableInstance> createVerticesForClassType(
for (ITypedReferenceableInstance typedInstance : newInstances) { List<ITypedReferenceableInstance> typedInstances) throws AtlasException {
List<ITypedReferenceableInstance> instancesCreated = new ArrayList<>();
for (ITypedReferenceableInstance typedInstance : typedInstances) {
final Id id = typedInstance.getId(); final Id id = typedInstance.getId();
if (!idToVertexMap.containsKey(id)) { if (!idToVertexMap.containsKey(id)) {
Vertex instanceVertex; Vertex instanceVertex;
if (id.isAssigned()) { // has a GUID if (id.isAssigned()) { // has a GUID
instanceVertex = GraphHelper.findVertexByGUID(titanGraph, id.id); instanceVertex = getVertexForGUID(id.id);
} else { } else {
//Check if there is already an instance with the same unique attribute value
ClassType classType = typeSystem.getDataType(ClassType.class, typedInstance.getTypeName()); ClassType classType = typeSystem.getDataType(ClassType.class, typedInstance.getTypeName());
instanceVertex = GraphHelper instanceVertex = instanceToGraphMapper.getVertexForInstanceByUniqueAttribute(classType, typedInstance);
.createVertexWithIdentity(titanGraph, typedInstance, classType.getAllSuperTypeNames()); if (instanceVertex == null) {
instanceVertex = GraphHelper.createVertexWithIdentity(titanGraph, typedInstance,
classType.getAllSuperTypeNames());
instancesCreated.add(typedInstance);
instanceToGraphMapper.mapInstanceToVertex(id, typedInstance, instanceVertex,
classType.fieldMapping().fields, idToVertexMap, true);
}
} }
idToVertexMap.put(id, instanceVertex); idToVertexMap.put(id, instanceVertex);
} }
} }
return instancesCreated;
} }
} }
private final class TypedInstanceToGraphMapper { private final class TypedInstanceToGraphMapper {
private String mapTypedInstanceToGraph(IReferenceableInstance typedInstance) throws AtlasException { private String[] mapTypedInstanceToGraph(ITypedReferenceableInstance[] typedInstances)
throws AtlasException {
EntityProcessor entityProcessor = new EntityProcessor(); EntityProcessor entityProcessor = new EntityProcessor();
try { List<String> guids = new ArrayList<>();
LOG.debug("Walking the object graph for instance {}", typedInstance.getTypeName()); for (ITypedReferenceableInstance typedInstance : typedInstances) {
new ObjectGraphWalker(typeSystem, entityProcessor, typedInstance).walk(); try {
} catch (AtlasException me) { LOG.debug("Walking the object graph for instance {}", typedInstance.getTypeName());
throw new RepositoryException("TypeSystem error when walking the ObjectGraph", me); entityProcessor.cleanUp();
} new ObjectGraphWalker(typeSystem, entityProcessor, typedInstance).walk();
} catch (AtlasException me) {
throw new RepositoryException("TypeSystem error when walking the ObjectGraph", me);
}
List<ITypedReferenceableInstance> newTypedInstances = discoverInstances(entityProcessor);
List<ITypedReferenceableInstance> instancesCreated =
entityProcessor.createVerticesForClassType(newTypedInstances);
List<ITypedReferenceableInstance> newTypedInstances = discoverInstances(entityProcessor); for (ITypedReferenceableInstance instance : instancesCreated) {
entityProcessor.createVerticesForClassTypes(newTypedInstances); try {
String guid = addDiscoveredInstances(typedInstance, entityProcessor, newTypedInstances); //new vertex, set all the properties
addFullTextProperty(entityProcessor, newTypedInstances); addDiscoveredInstance(entityProcessor, instance);
return guid; } catch(SchemaViolationException e) {
throw new EntityExistsException(typedInstance, e);
}
}
addFullTextProperty(entityProcessor, instancesCreated);
//Return guid for
Vertex instanceVertex = entityProcessor.idToVertexMap.get(typedInstance.getId());
String guid = instanceVertex.getProperty(Constants.GUID_PROPERTY_KEY);
guids.add(guid);
}
return guids.toArray(new String[guids.size()]);
} }
private void addFullTextProperty(EntityProcessor entityProcessor, private void addFullTextProperty(EntityProcessor entityProcessor,
...@@ -455,7 +509,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -455,7 +509,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
Id id = typedInstance.getId(); Id id = typedInstance.getId();
Vertex instanceVertex = entityProcessor.idToVertexMap.get(id); Vertex instanceVertex = entityProcessor.idToVertexMap.get(id);
String fullText = getFullTextForVertex(instanceVertex, true); String fullText = getFullTextForVertex(instanceVertex, true);
addProperty(instanceVertex, Constants.ENTITY_TEXT_PROPERTY_KEY, fullText); GraphHelper.setProperty(instanceVertex, Constants.ENTITY_TEXT_PROPERTY_KEY, fullText);
} }
} }
...@@ -572,63 +626,78 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -572,63 +626,78 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
ClassType cT = typeSystem.getDataType(ClassType.class, transientInstance.getTypeName()); ClassType cT = typeSystem.getDataType(ClassType.class, transientInstance.getTypeName());
ITypedReferenceableInstance newInstance = cT.convert(transientInstance, Multiplicity.REQUIRED); ITypedReferenceableInstance newInstance = cT.convert(transientInstance, Multiplicity.REQUIRED);
newTypedInstances.add(newInstance); newTypedInstances.add(newInstance);
// Now replace old references with new Ids
MapIds mapIds = new MapIds(entityProcessor.idToNewIdMap);
new ObjectGraphWalker(typeSystem, mapIds, newTypedInstances).walk();
} catch (AtlasException me) { } catch (AtlasException me) {
throw new RepositoryException( throw new RepositoryException(
String.format("Failed to create Instance(id = %s", transientInstance.getId()), me); String.format("Failed to create Instance(id = %s", transientInstance.getId()), me);
} }
} }
return newTypedInstances; //Reverse the list to create the entities in dependency order
return Lists.reverse(newTypedInstances);
} }
private String addDiscoveredInstances(IReferenceableInstance entity, EntityProcessor entityProcessor, /**
List<ITypedReferenceableInstance> newTypedInstances) throws AtlasException { * For the given type, finds an unique attribute and checks if there is an existing instance with the same
* unique value
* @param classType
* @param instance
* @return
* @throws AtlasException
*/
Vertex getVertexForInstanceByUniqueAttribute(ClassType classType, IReferenceableInstance instance)
throws AtlasException {
for (AttributeInfo attributeInfo : classType.fieldMapping().fields.values()) {
if (attributeInfo.isUnique) {
String propertyKey = getFieldNameInVertex(classType, attributeInfo);
try {
return getVertexForProperty(propertyKey, instance.get(attributeInfo.name));
} catch(EntityNotFoundException e) {
//Its ok if there is no entity with the same unique value
}
}
}
String typedInstanceGUID = null; return null;
for (ITypedReferenceableInstance typedInstance : newTypedInstances) { // Traverse over newInstances }
LOG.debug("Adding typed instance {}", typedInstance.getTypeName());
Id id = typedInstance.getId(); private void addDiscoveredInstance(EntityProcessor entityProcessor, ITypedReferenceableInstance typedInstance)
if (id == null) { // oops throws AtlasException {
throw new RepositoryException("id cannot be null"); LOG.debug("Adding typed instance {}", typedInstance.getTypeName());
}
Vertex instanceVertex = entityProcessor.idToVertexMap.get(id); Id id = typedInstance.getId();
if (id == null) { // oops
throw new RepositoryException("id cannot be null");
}
// add the attributes for the instance Vertex instanceVertex = entityProcessor.idToVertexMap.get(id);
ClassType classType = typeSystem.getDataType(ClassType.class, typedInstance.getTypeName());
final Map<String, AttributeInfo> fields = classType.fieldMapping().fields;
mapInstanceToVertex(id, typedInstance, instanceVertex, fields, entityProcessor.idToVertexMap); // add the attributes for the instance
ClassType classType = typeSystem.getDataType(ClassType.class, typedInstance.getTypeName());
final Map<String, AttributeInfo> fields = classType.fieldMapping().fields;
for (String traitName : typedInstance.getTraits()) { mapInstanceToVertex(id, typedInstance, instanceVertex, fields, entityProcessor.idToVertexMap, false);
LOG.debug("mapping trait {}", traitName);
((TitanVertex) instanceVertex).addProperty(Constants.TRAIT_NAMES_PROPERTY_KEY, traitName);
ITypedStruct traitInstance = (ITypedStruct) typedInstance.getTrait(traitName);
// add the attributes for the trait instance for (String traitName : typedInstance.getTraits()) {
mapTraitInstanceToVertex(traitInstance, typedInstance, instanceVertex, LOG.debug("mapping trait {}", traitName);
entityProcessor.idToVertexMap); GraphHelper.addProperty(instanceVertex, Constants.TRAIT_NAMES_PROPERTY_KEY, traitName);
} ITypedStruct traitInstance = (ITypedStruct) typedInstance.getTrait(traitName);
if (typedInstance.getId() == entity.getId()) { // save the guid for return // add the attributes for the trait instance
typedInstanceGUID = instanceVertex.getProperty(Constants.GUID_PROPERTY_KEY); mapTraitInstanceToVertex(traitInstance, typedInstance, instanceVertex,
} entityProcessor.idToVertexMap);
} }
return typedInstanceGUID;
} }
private void mapInstanceToVertex(Id id, ITypedInstance typedInstance, Vertex instanceVertex, private void mapInstanceToVertex(Id id, ITypedInstance typedInstance, Vertex instanceVertex,
Map<String, AttributeInfo> fields, Map<Id, Vertex> idToVertexMap) throws AtlasException { Map<String, AttributeInfo> fields, Map<Id, Vertex> idToVertexMap, boolean mapOnlyUniqueAttributes)
throws AtlasException {
LOG.debug("Mapping instance {} of {} to vertex {}", typedInstance, typedInstance.getTypeName(), LOG.debug("Mapping instance {} of {} to vertex {}", typedInstance, typedInstance.getTypeName(),
instanceVertex); instanceVertex);
for (AttributeInfo attributeInfo : fields.values()) { for (AttributeInfo attributeInfo : fields.values()) {
if (mapOnlyUniqueAttributes && !attributeInfo.isUnique) {
continue;
}
final IDataType dataType = attributeInfo.dataType(); final IDataType dataType = attributeInfo.dataType();
mapAttributesToVertex(id, typedInstance, instanceVertex, idToVertexMap, attributeInfo, dataType); mapAttributesToVertex(id, typedInstance, instanceVertex, idToVertexMap, attributeInfo, dataType);
} }
...@@ -653,7 +722,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -653,7 +722,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
//handles both int and string for enum //handles both int and string for enum
EnumValue enumValue = EnumValue enumValue =
(EnumValue) dataType.convert(typedInstance.get(attributeInfo.name), Multiplicity.REQUIRED); (EnumValue) dataType.convert(typedInstance.get(attributeInfo.name), Multiplicity.REQUIRED);
addProperty(instanceVertex, propertyName, enumValue.value); GraphHelper.setProperty(instanceVertex, propertyName, enumValue.value);
break; break;
case ARRAY: case ARRAY:
...@@ -677,8 +746,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -677,8 +746,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
break; break;
case CLASS: case CLASS:
Id referenceId = (Id) typedInstance.get(attributeInfo.name); mapClassReferenceAsEdge(instanceVertex, idToVertexMap, edgeLabel, (ITypedReferenceableInstance) attrValue);
mapClassReferenceAsEdge(instanceVertex, idToVertexMap, edgeLabel, referenceId);
break; break;
default: default:
...@@ -707,7 +775,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -707,7 +775,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
} }
// for dereference on way out // for dereference on way out
addProperty(instanceVertex, propertyName, values); GraphHelper.setProperty(instanceVertex, propertyName, values);
} }
private void mapMapCollectionToVertex(Id id, ITypedInstance typedInstance, Vertex instanceVertex, private void mapMapCollectionToVertex(Id id, ITypedInstance typedInstance, Vertex instanceVertex,
...@@ -726,11 +794,11 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -726,11 +794,11 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
String myPropertyName = propertyName + "." + entry.getKey().toString(); String myPropertyName = propertyName + "." + entry.getKey().toString();
String value = mapCollectionEntryToVertex(id, instanceVertex, attributeInfo, idToVertexMap, elementType, String value = mapCollectionEntryToVertex(id, instanceVertex, attributeInfo, idToVertexMap, elementType,
entry.getValue(), myPropertyName); entry.getValue(), myPropertyName);
addProperty(instanceVertex, myPropertyName, value); GraphHelper.setProperty(instanceVertex, myPropertyName, value);
} }
// for dereference on way out // for dereference on way out
addProperty(instanceVertex, propertyName, new ArrayList(collection.keySet())); GraphHelper.setProperty(instanceVertex, propertyName, new ArrayList(collection.keySet()));
} }
private String mapCollectionEntryToVertex(Id id, Vertex instanceVertex, AttributeInfo attributeInfo, private String mapCollectionEntryToVertex(Id id, Vertex instanceVertex, AttributeInfo attributeInfo,
...@@ -757,8 +825,8 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -757,8 +825,8 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
return structElementEdge.getId().toString(); return structElementEdge.getId().toString();
case CLASS: case CLASS:
Id referenceId = (Id) value; return mapClassReferenceAsEdge(instanceVertex, idToVertexMap, edgeLabel,
return mapClassReferenceAsEdge(instanceVertex, idToVertexMap, edgeLabel, referenceId); (ITypedReferenceableInstance) value);
default: default:
throw new IllegalArgumentException("Unknown type category: " + elementType.getTypeCategory()); throw new IllegalArgumentException("Unknown type category: " + elementType.getTypeCategory());
...@@ -766,11 +834,12 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -766,11 +834,12 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
} }
private String mapClassReferenceAsEdge(Vertex instanceVertex, Map<Id, Vertex> idToVertexMap, String propertyKey, private String mapClassReferenceAsEdge(Vertex instanceVertex, Map<Id, Vertex> idToVertexMap, String propertyKey,
Id id) throws AtlasException { ITypedReferenceableInstance typedReference) throws AtlasException {
if (id != null) { if (typedReference != null) {
Vertex referenceVertex; Vertex referenceVertex;
Id id = typedReference instanceof Id ? (Id) typedReference : typedReference.getId();
if (id.isAssigned()) { if (id.isAssigned()) {
referenceVertex = GraphHelper.findVertexByGUID(titanGraph, id.id); referenceVertex = getVertexForGUID(id.id);
} else { } else {
referenceVertex = idToVertexMap.get(id); referenceVertex = idToVertexMap.get(id);
} }
...@@ -796,7 +865,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -796,7 +865,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
// map all the attributes to this newly created vertex // map all the attributes to this newly created vertex
mapInstanceToVertex(id, structInstance, structInstanceVertex, structInstance.fieldMapping().fields, mapInstanceToVertex(id, structInstance, structInstanceVertex, structInstance.fieldMapping().fields,
idToVertexMap); idToVertexMap, false);
return structInstanceVertex; return structInstanceVertex;
} }
...@@ -820,7 +889,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -820,7 +889,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
// map all the attributes to this newly created vertex // map all the attributes to this newly created vertex
mapInstanceToVertex(typedInstanceId, traitInstance, traitInstanceVertex, mapInstanceToVertex(typedInstanceId, traitInstance, traitInstanceVertex,
traitInstance.fieldMapping().fields, idToVertexMap); traitInstance.fieldMapping().fields, idToVertexMap, false);
// add an edge to the newly created vertex from the parent // add an edge to the newly created vertex from the parent
String relationshipLabel = getEdgeLabel(typedInstanceTypeName, traitName); String relationshipLabel = getEdgeLabel(typedInstanceTypeName, traitName);
...@@ -861,15 +930,10 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -861,15 +930,10 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
//Convert Property value to Long while persisting //Convert Property value to Long while persisting
propertyValue = dateVal.getTime(); propertyValue = dateVal.getTime();
} }
addProperty(instanceVertex, vertexPropertyName, propertyValue); GraphHelper.setProperty(instanceVertex, vertexPropertyName, propertyValue);
} }
} }
private void addProperty(Vertex vertex, String propertyName, Object value) {
LOG.debug("Setting property {} = \"{}\" to vertex {}", propertyName, value, vertex);
vertex.setProperty(propertyName, value);
}
public final class GraphToTypedInstanceMapper { public final class GraphToTypedInstanceMapper {
public ITypedReferenceableInstance mapGraphToTypedInstance(String guid, Vertex instanceVertex) public ITypedReferenceableInstance mapGraphToTypedInstance(String guid, Vertex instanceVertex)
...@@ -971,6 +1035,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -971,6 +1035,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
final String guid = referenceVertex.getProperty(Constants.GUID_PROPERTY_KEY); final String guid = referenceVertex.getProperty(Constants.GUID_PROPERTY_KEY);
LOG.debug("Found vertex {} for label {} with guid {}", referenceVertex, relationshipLabel, guid); LOG.debug("Found vertex {} for label {} with guid {}", referenceVertex, relationshipLabel, guid);
if (attributeInfo.isComposite) { if (attributeInfo.isComposite) {
//Also, when you retrieve a type's instance, you get the complete object graph of the composites
LOG.debug("Found composite, mapping vertex to instance"); LOG.debug("Found composite, mapping vertex to instance");
return mapGraphToTypedInstance(guid, referenceVertex); return mapGraphToTypedInstance(guid, referenceVertex);
} else { } else {
...@@ -1092,6 +1157,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository { ...@@ -1092,6 +1157,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
LOG.debug("Found vertex {} for label {} with guid {}", referenceVertex, relationshipLabel, LOG.debug("Found vertex {} for label {} with guid {}", referenceVertex, relationshipLabel,
guid); guid);
if (attributeInfo.isComposite) { if (attributeInfo.isComposite) {
//Also, when you retrieve a type's instance, you get the complete object graph of the composites
LOG.debug("Found composite, mapping vertex to instance"); LOG.debug("Found composite, mapping vertex to instance");
return mapGraphToTypedInstance(guid, referenceVertex); return mapGraphToTypedInstance(guid, referenceVertex);
} else { } else {
......
...@@ -46,34 +46,35 @@ public final class GraphHelper { ...@@ -46,34 +46,35 @@ public final class GraphHelper {
} }
public static Vertex createVertexWithIdentity(Graph graph, ITypedReferenceableInstance typedInstance, public static Vertex createVertexWithIdentity(Graph graph, ITypedReferenceableInstance typedInstance,
Set<String> superTypeNames) { Set<String> superTypeNames) {
final Vertex vertexWithIdentity = final Vertex vertexWithIdentity = createVertexWithoutIdentity(graph, typedInstance.getTypeName(),
createVertexWithoutIdentity(graph, typedInstance.getTypeName(), typedInstance.getId(), superTypeNames); typedInstance.getId(), superTypeNames);
// add identity // add identity
final String guid = UUID.randomUUID().toString(); final String guid = UUID.randomUUID().toString();
vertexWithIdentity.setProperty(Constants.GUID_PROPERTY_KEY, guid); setProperty(vertexWithIdentity, Constants.GUID_PROPERTY_KEY, guid);
return vertexWithIdentity; return vertexWithIdentity;
} }
public static Vertex createVertexWithoutIdentity(Graph graph, String typeName, Id typedInstanceId, public static Vertex createVertexWithoutIdentity(Graph graph, String typeName, Id typedInstanceId,
Set<String> superTypeNames) { Set<String> superTypeNames) {
LOG.debug("Creating vertex for type {} id {}", typeName, typedInstanceId._getId());
final Vertex vertexWithoutIdentity = graph.addVertex(null); final Vertex vertexWithoutIdentity = graph.addVertex(null);
// add type information // add type information
vertexWithoutIdentity.setProperty(Constants.ENTITY_TYPE_PROPERTY_KEY, typeName); setProperty(vertexWithoutIdentity, Constants.ENTITY_TYPE_PROPERTY_KEY, typeName);
// add super types // add super types
for (String superTypeName : superTypeNames) { for (String superTypeName : superTypeNames) {
((TitanVertex) vertexWithoutIdentity).addProperty(Constants.SUPER_TYPES_PROPERTY_KEY, superTypeName); addProperty(vertexWithoutIdentity, Constants.SUPER_TYPES_PROPERTY_KEY, superTypeName);
} }
// add version information // add version information
vertexWithoutIdentity.setProperty(Constants.VERSION_PROPERTY_KEY, typedInstanceId.version); setProperty(vertexWithoutIdentity, Constants.VERSION_PROPERTY_KEY, typedInstanceId.version);
// add timestamp information // add timestamp information
vertexWithoutIdentity.setProperty(Constants.TIMESTAMP_PROPERTY_KEY, System.currentTimeMillis()); setProperty(vertexWithoutIdentity, Constants.TIMESTAMP_PROPERTY_KEY, System.currentTimeMillis());
return vertexWithoutIdentity; return vertexWithoutIdentity;
} }
...@@ -84,12 +85,12 @@ public final class GraphHelper { ...@@ -84,12 +85,12 @@ public final class GraphHelper {
return titanGraph.addEdge(null, fromVertex, toVertex, edgeLabel); return titanGraph.addEdge(null, fromVertex, toVertex, edgeLabel);
} }
public static Vertex findVertexByGUID(TitanGraph titanGraph, String value) { public static Vertex findVertex(TitanGraph titanGraph, String propertyKey, Object value) {
LOG.debug("Finding vertex for key={}, value={}", Constants.GUID_PROPERTY_KEY, value); LOG.debug("Finding vertex for {}={}", propertyKey, value);
GraphQuery query = titanGraph.query().has(Constants.GUID_PROPERTY_KEY, value); GraphQuery query = titanGraph.query().has(propertyKey, value);
Iterator<Vertex> results = query.vertices().iterator(); Iterator<Vertex> results = query.vertices().iterator();
// returning one since guid should be unique // returning one since entityType, qualifiedName should be unique
return results.hasNext() ? results.next() : null; return results.hasNext() ? results.next() : null;
} }
...@@ -107,6 +108,16 @@ public final class GraphHelper { ...@@ -107,6 +108,16 @@ public final class GraphHelper {
+ edge.getVertex(Direction.IN) + "]"; + edge.getVertex(Direction.IN) + "]";
} }
public static void setProperty(Vertex vertex, String propertyName, Object value) {
LOG.debug("Setting property {} = \"{}\" to vertex {}", propertyName, value, vertex);
vertex.setProperty(propertyName, value);
}
public static void addProperty(Vertex vertex, String propertyName, Object value) {
LOG.debug("Setting property {} = \"{}\" to vertex {}", propertyName, value, vertex);
((TitanVertex)vertex).addProperty(propertyName, value);
}
/* /*
public static void dumpToLog(final Graph graph) { public static void dumpToLog(final Graph graph) {
LOG.debug("*******************Graph Dump****************************"); LOG.debug("*******************Graph Dump****************************");
......
...@@ -39,6 +39,7 @@ import org.apache.atlas.typesystem.TypesDef; ...@@ -39,6 +39,7 @@ import org.apache.atlas.typesystem.TypesDef;
import org.apache.atlas.typesystem.json.InstanceSerialization; import org.apache.atlas.typesystem.json.InstanceSerialization;
import org.apache.atlas.typesystem.json.TypesSerialization; import org.apache.atlas.typesystem.json.TypesSerialization;
import org.apache.atlas.typesystem.types.AttributeDefinition; import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.AttributeInfo;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes; import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumTypeDefinition; import org.apache.atlas.typesystem.types.EnumTypeDefinition;
...@@ -49,11 +50,14 @@ import org.apache.atlas.typesystem.types.StructTypeDefinition; ...@@ -49,11 +50,14 @@ import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType; import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeSystem; import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.atlas.typesystem.types.TypeUtils; import org.apache.atlas.typesystem.types.TypeUtils;
import org.apache.atlas.typesystem.types.ValueConversionException;
import org.apache.atlas.typesystem.types.utils.TypesUtil; import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject; import org.codehaus.jettison.json.JSONObject;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import scala.actors.threadpool.Arrays;
import javax.inject.Inject; import javax.inject.Inject;
import javax.inject.Singleton; import javax.inject.Singleton;
...@@ -107,23 +111,21 @@ public class DefaultMetadataService implements MetadataService { ...@@ -107,23 +111,21 @@ public class DefaultMetadataService implements MetadataService {
} }
private static final AttributeDefinition NAME_ATTRIBUTE = private static final AttributeDefinition NAME_ATTRIBUTE =
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE); TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE);
private static final AttributeDefinition DESCRIPTION_ATTRIBUTE = private static final AttributeDefinition DESCRIPTION_ATTRIBUTE =
TypesUtil.createOptionalAttrDef("description", DataTypes.STRING_TYPE); TypesUtil.createOptionalAttrDef("description", DataTypes.STRING_TYPE);
@InterfaceAudience.Private @InterfaceAudience.Private
private void createSuperTypes() throws AtlasException { private void createSuperTypes() throws AtlasException {
if (typeSystem.isRegistered(AtlasClient.DATA_SET_SUPER_TYPE)) {
return; // this is already registered
}
HierarchicalTypeDefinition<ClassType> infraType = TypesUtil HierarchicalTypeDefinition<ClassType> infraType = TypesUtil
.createClassTypeDef(AtlasClient.INFRASTRUCTURE_SUPER_TYPE, ImmutableList.<String>of(), NAME_ATTRIBUTE, .createClassTypeDef(AtlasClient.INFRASTRUCTURE_SUPER_TYPE, ImmutableList.<String>of(), NAME_ATTRIBUTE,
DESCRIPTION_ATTRIBUTE); DESCRIPTION_ATTRIBUTE);
createType(infraType);
HierarchicalTypeDefinition<ClassType> datasetType = TypesUtil HierarchicalTypeDefinition<ClassType> datasetType = TypesUtil
.createClassTypeDef(AtlasClient.DATA_SET_SUPER_TYPE, ImmutableList.<String>of(), NAME_ATTRIBUTE, .createClassTypeDef(AtlasClient.DATA_SET_SUPER_TYPE, ImmutableList.<String>of(), NAME_ATTRIBUTE,
DESCRIPTION_ATTRIBUTE); DESCRIPTION_ATTRIBUTE);
createType(datasetType);
HierarchicalTypeDefinition<ClassType> processType = TypesUtil HierarchicalTypeDefinition<ClassType> processType = TypesUtil
.createClassTypeDef(AtlasClient.PROCESS_SUPER_TYPE, ImmutableList.<String>of(), NAME_ATTRIBUTE, .createClassTypeDef(AtlasClient.PROCESS_SUPER_TYPE, ImmutableList.<String>of(), NAME_ATTRIBUTE,
...@@ -132,12 +134,23 @@ public class DefaultMetadataService implements MetadataService { ...@@ -132,12 +134,23 @@ public class DefaultMetadataService implements MetadataService {
Multiplicity.OPTIONAL, false, null), Multiplicity.OPTIONAL, false, null),
new AttributeDefinition("outputs", DataTypes.arrayTypeName(AtlasClient.DATA_SET_SUPER_TYPE), new AttributeDefinition("outputs", DataTypes.arrayTypeName(AtlasClient.DATA_SET_SUPER_TYPE),
Multiplicity.OPTIONAL, false, null)); Multiplicity.OPTIONAL, false, null));
createType(processType);
TypesDef typesDef = TypeUtils HierarchicalTypeDefinition<ClassType> referenceableType = TypesUtil
.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(), .createClassTypeDef(AtlasClient.REFERENCEABLE_SUPER_TYPE, ImmutableList.<String>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), TypesUtil.createUniqueRequiredAttrDef(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME,
ImmutableList.of(infraType, datasetType, processType)); DataTypes.STRING_TYPE));
createType(TypesSerialization.toJson(typesDef)); createType(referenceableType);
}
private void createType(HierarchicalTypeDefinition<ClassType> type) throws AtlasException {
if (!typeSystem.isRegistered(type.typeName)) {
TypesDef typesDef = TypeUtils
.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
ImmutableList.of(type));
createType(TypesSerialization.toJson(typesDef));
}
} }
/** /**
...@@ -150,17 +163,9 @@ public class DefaultMetadataService implements MetadataService { ...@@ -150,17 +163,9 @@ public class DefaultMetadataService implements MetadataService {
@Override @Override
public JSONObject createType(String typeDefinition) throws AtlasException { public JSONObject createType(String typeDefinition) throws AtlasException {
ParamChecker.notEmpty(typeDefinition, "type definition cannot be empty"); ParamChecker.notEmpty(typeDefinition, "type definition cannot be empty");
validateTypeDefinition(typeDefinition);
TypesDef typesDef; TypesDef typesDef = validateTypeDefinition(typeDefinition);
try {
typesDef = TypesSerialization.fromJson(typeDefinition);
if (typesDef.isEmpty()) {
throw new AtlasException("Invalid type definition");
}
} catch (Exception e) {
LOG.error("Unable to deserialize json={}", typeDefinition, e);
throw new IllegalArgumentException("Unable to deserialize json ", e);
}
try { try {
final Map<String, IDataType> typesAdded = typeSystem.defineTypes(typesDef); final Map<String, IDataType> typesAdded = typeSystem.defineTypes(typesDef);
...@@ -185,6 +190,19 @@ public class DefaultMetadataService implements MetadataService { ...@@ -185,6 +190,19 @@ public class DefaultMetadataService implements MetadataService {
} }
} }
private TypesDef validateTypeDefinition(String typeDefinition) {
try {
TypesDef typesDef = TypesSerialization.fromJson(typeDefinition);
if (typesDef.isEmpty()) {
throw new IllegalArgumentException("Invalid type definition");
}
return typesDef;
} catch (Exception e) {
LOG.error("Unable to deserialize json={}", typeDefinition, e);
throw new IllegalArgumentException("Unable to deserialize json " + typeDefinition, e);
}
}
/** /**
* Return the definition for the given type. * Return the definition for the given type.
* *
...@@ -220,36 +238,43 @@ public class DefaultMetadataService implements MetadataService { ...@@ -220,36 +238,43 @@ public class DefaultMetadataService implements MetadataService {
/** /**
* Creates an entity, instance of the type. * Creates an entity, instance of the type.
* *
* @param entityInstanceDefinition definition * @param entityInstanceDefinition json array of entity definitions
* @return guid * @return guids - json array of guids
*/ */
@Override @Override
public String createEntity(String entityInstanceDefinition) throws AtlasException { public String createEntities(String entityInstanceDefinition) throws AtlasException {
ParamChecker.notEmpty(entityInstanceDefinition, "Entity instance definition cannot be empty"); ParamChecker.notEmpty(entityInstanceDefinition, "Entity instance definition cannot be empty");
ITypedReferenceableInstance entityTypedInstance = deserializeClassInstance(entityInstanceDefinition); ITypedReferenceableInstance[] typedInstances = deserializeClassInstances(entityInstanceDefinition);
final String guid = repository.createEntity(entityTypedInstance); final String[] guids = repository.createEntities(typedInstances);
onEntityAddedToRepo(entityTypedInstance); onEntityAddedToRepo(Arrays.asList(typedInstances));
return guid; return new JSONArray(Arrays.asList(guids)).toString();
} }
private ITypedReferenceableInstance deserializeClassInstance(String entityInstanceDefinition) private ITypedReferenceableInstance[] deserializeClassInstances(String entityInstanceDefinition)
throws AtlasException { throws AtlasException {
final Referenceable entityInstance;
try { try {
entityInstance = InstanceSerialization.fromJsonReferenceable(entityInstanceDefinition, true); JSONArray referableInstances = new JSONArray(entityInstanceDefinition);
ITypedReferenceableInstance[] instances = new ITypedReferenceableInstance[referableInstances.length()];
for (int index = 0; index < referableInstances.length(); index++) {
Referenceable entityInstance =
InstanceSerialization.fromJsonReferenceable(referableInstances.getString(index), true);
final String entityTypeName = entityInstance.getTypeName();
ParamChecker.notEmpty(entityTypeName, "Entity type cannot be null");
ClassType entityType = typeSystem.getDataType(ClassType.class, entityTypeName);
ITypedReferenceableInstance typedInstrance = entityType.convert(entityInstance, Multiplicity.REQUIRED);
instances[index] = typedInstrance;
}
return instances;
} catch(ValueConversionException e) {
throw e;
} catch (Exception e) { // exception from deserializer } catch (Exception e) { // exception from deserializer
LOG.error("Unable to deserialize json={}", entityInstanceDefinition, e); LOG.error("Unable to deserialize json={}", entityInstanceDefinition, e);
throw new IllegalArgumentException("Unable to deserialize json"); throw new IllegalArgumentException("Unable to deserialize json");
} }
final String entityTypeName = entityInstance.getTypeName();
ParamChecker.notEmpty(entityTypeName, "Entity type cannot be null");
ClassType entityType = typeSystem.getDataType(ClassType.class, entityTypeName);
return entityType.convert(entityInstance, Multiplicity.REQUIRED);
} }
/** /**
...@@ -266,6 +291,29 @@ public class DefaultMetadataService implements MetadataService { ...@@ -266,6 +291,29 @@ public class DefaultMetadataService implements MetadataService {
return InstanceSerialization.toJson(instance, true); return InstanceSerialization.toJson(instance, true);
} }
@Override
public String getEntityDefinition(String entityType, String attribute, String value) throws AtlasException {
validateTypeExists(entityType);
validateUniqueAttribute(entityType, attribute);
final ITypedReferenceableInstance instance = repository.getEntityDefinition(entityType, attribute, value);
return InstanceSerialization.toJson(instance, true);
}
/**
* Validate that attribute is unique attribute
* @param entityType
* @param attributeName
*/
private void validateUniqueAttribute(String entityType, String attributeName) throws AtlasException {
ClassType type = typeSystem.getDataType(ClassType.class, entityType);
AttributeInfo attribute = type.fieldMapping().fields.get(attributeName);
if (!attribute.isUnique) {
throw new IllegalArgumentException(
String.format("%s.%s is not a unique attribute", entityType, attributeName));
}
}
/** /**
* Return the list of entity names for the given type in the repository. * Return the list of entity names for the given type in the repository.
* *
...@@ -291,9 +339,9 @@ public class DefaultMetadataService implements MetadataService { ...@@ -291,9 +339,9 @@ public class DefaultMetadataService implements MetadataService {
private void validateTypeExists(String entityType) throws AtlasException { private void validateTypeExists(String entityType) throws AtlasException {
ParamChecker.notEmpty(entityType, "entity type cannot be null"); ParamChecker.notEmpty(entityType, "entity type cannot be null");
// verify if the type exists IDataType type = typeSystem.getDataType(IDataType.class, entityType);
if (!typeSystem.isRegistered(entityType)) { if (type.getTypeCategory() != DataTypes.TypeCategory.CLASS) {
throw new TypeNotFoundException("type is not defined for : " + entityType); throw new IllegalArgumentException("type " + entityType + " not a CLASS type");
} }
} }
...@@ -401,11 +449,11 @@ public class DefaultMetadataService implements MetadataService { ...@@ -401,11 +449,11 @@ public class DefaultMetadataService implements MetadataService {
} }
} }
private void onEntityAddedToRepo(ITypedReferenceableInstance typedInstance) private void onEntityAddedToRepo(Collection<ITypedReferenceableInstance> typedInstances)
throws AtlasException { throws AtlasException {
for (EntityChangeListener listener : entityChangeListeners) { for (EntityChangeListener listener : entityChangeListeners) {
listener.onEntityAdded(typedInstance); listener.onEntityAdded(typedInstances);
} }
} }
......
...@@ -66,7 +66,7 @@ public interface MetadataService { ...@@ -66,7 +66,7 @@ public interface MetadataService {
* @param entityDefinition definition * @param entityDefinition definition
* @return guid * @return guid
*/ */
String createEntity(String entityDefinition) throws AtlasException; String createEntities(String entityDefinition) throws AtlasException;
/** /**
* Return the definition for the given guid. * Return the definition for the given guid.
...@@ -77,6 +77,16 @@ public interface MetadataService { ...@@ -77,6 +77,16 @@ public interface MetadataService {
String getEntityDefinition(String guid) throws AtlasException; String getEntityDefinition(String guid) throws AtlasException;
/** /**
* Return the definition given type and attribute. The attribute has to be unique attribute for the type
* @param entityType - type name
* @param attribute - attribute name
* @param value - attribute value
* @return
* @throws AtlasException
*/
String getEntityDefinition(String entityType, String attribute, String value) throws AtlasException;
/**
* Return the list of entity names for the given type in the repository. * Return the list of entity names for the given type in the repository.
* *
* @param entityType type * @param entityType type
......
...@@ -19,11 +19,8 @@ ...@@ -19,11 +19,8 @@
package org.apache.atlas; package org.apache.atlas;
import com.thinkaurelius.titan.core.TitanGraph; import com.thinkaurelius.titan.core.TitanGraph;
import com.thinkaurelius.titan.core.util.TitanCleanup;
import org.apache.atlas.repository.graph.GraphProvider; import org.apache.atlas.repository.graph.GraphProvider;
import org.apache.atlas.typesystem.types.TypeSystem;
import org.testng.Assert; import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.Guice; import org.testng.annotations.Guice;
import org.testng.annotations.Test; import org.testng.annotations.Test;
......
...@@ -26,20 +26,25 @@ import com.tinkerpop.blueprints.util.io.graphson.GraphSONWriter; ...@@ -26,20 +26,25 @@ import com.tinkerpop.blueprints.util.io.graphson.GraphSONWriter;
import org.apache.atlas.repository.graph.GraphHelper; import org.apache.atlas.repository.graph.GraphHelper;
import org.apache.atlas.typesystem.ITypedReferenceableInstance; import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.TypesDef;
import org.apache.atlas.typesystem.types.AttributeDefinition; import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes; import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumType;
import org.apache.atlas.typesystem.types.EnumTypeDefinition; import org.apache.atlas.typesystem.types.EnumTypeDefinition;
import org.apache.atlas.typesystem.types.EnumValue; import org.apache.atlas.typesystem.types.EnumValue;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition; import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.IDataType;
import org.apache.atlas.typesystem.types.Multiplicity; import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition; import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType; import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeSystem; import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.atlas.typesystem.types.TypeUtils;
import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.apache.commons.lang.RandomStringUtils;
import org.testng.Assert; import org.testng.Assert;
import java.io.File; import java.io.File;
import java.util.Collection;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef; import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef; import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
...@@ -52,6 +57,8 @@ import static org.apache.atlas.typesystem.types.utils.TypesUtil.createTraitTypeD ...@@ -52,6 +57,8 @@ import static org.apache.atlas.typesystem.types.utils.TypesUtil.createTraitTypeD
*/ */
public final class TestUtils { public final class TestUtils {
public static final long TEST_DATE_IN_LONG = 1418265358440L;
private TestUtils() { private TestUtils() {
} }
...@@ -92,7 +99,6 @@ public final class TestUtils { ...@@ -92,7 +99,6 @@ public final class TestUtils {
EnumTypeDefinition orgLevelEnum = EnumTypeDefinition orgLevelEnum =
new EnumTypeDefinition("OrgLevel", new EnumValue("L1", 1), new EnumValue("L2", 2)); new EnumTypeDefinition("OrgLevel", new EnumValue("L1", 1), new EnumValue("L2", 2));
ts.defineEnumType(orgLevelEnum);
StructTypeDefinition addressDetails = StructTypeDefinition addressDetails =
createStructTypeDef("Address", createRequiredAttrDef("street", DataTypes.STRING_TYPE), createStructTypeDef("Address", createRequiredAttrDef("street", DataTypes.STRING_TYPE),
...@@ -105,7 +111,7 @@ public final class TestUtils { ...@@ -105,7 +111,7 @@ public final class TestUtils {
HierarchicalTypeDefinition<ClassType> personTypeDef = createClassTypeDef("Person", ImmutableList.<String>of(), HierarchicalTypeDefinition<ClassType> personTypeDef = createClassTypeDef("Person", ImmutableList.<String>of(),
createRequiredAttrDef("name", DataTypes.STRING_TYPE), createRequiredAttrDef("name", DataTypes.STRING_TYPE),
createOptionalAttrDef("orgLevel", ts.getDataType(EnumType.class, "OrgLevel")), createOptionalAttrDef("orgLevel", "OrgLevel"),
createOptionalAttrDef("address", "Address"), createOptionalAttrDef("address", "Address"),
new AttributeDefinition("department", "Department", Multiplicity.REQUIRED, false, "employees"), new AttributeDefinition("department", "Department", Multiplicity.REQUIRED, false, "employees"),
new AttributeDefinition("manager", "Manager", Multiplicity.OPTIONAL, false, "subordinates")); new AttributeDefinition("manager", "Manager", Multiplicity.OPTIONAL, false, "subordinates"));
...@@ -118,12 +124,13 @@ public final class TestUtils { ...@@ -118,12 +124,13 @@ public final class TestUtils {
createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(), createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(),
createRequiredAttrDef("level", DataTypes.INT_TYPE)); createRequiredAttrDef("level", DataTypes.INT_TYPE));
ts.defineTypes(ImmutableList.of(addressDetails), ImmutableList.of(securityClearanceTypeDef), ts.defineTypes(ImmutableList.of(orgLevelEnum), ImmutableList.of(addressDetails),
ImmutableList.of(securityClearanceTypeDef),
ImmutableList.of(deptTypeDef, personTypeDef, managerTypeDef)); ImmutableList.of(deptTypeDef, personTypeDef, managerTypeDef));
} }
public static Referenceable createDeptEg1(TypeSystem ts) throws AtlasException { public static Referenceable createDeptEg1(TypeSystem ts) throws AtlasException {
Referenceable hrDept = new Referenceable("Department"); Referenceable hrDept = new Referenceable(ENTITY_TYPE);
Referenceable john = new Referenceable("Person"); Referenceable john = new Referenceable("Person");
Referenceable jane = new Referenceable("Manager", "SecurityClearance"); Referenceable jane = new Referenceable("Manager", "SecurityClearance");
Referenceable johnAddr = new Referenceable("Address"); Referenceable johnAddr = new Referenceable("Address");
...@@ -156,4 +163,110 @@ public final class TestUtils { ...@@ -156,4 +163,110 @@ public final class TestUtils {
return hrDept; return hrDept;
} }
public static final String ENTITY_TYPE = "Department";
public static final String DATABASE_TYPE = "hive_database";
public static final String DATABASE_NAME = "foo";
public static final String TABLE_TYPE = "hive_table";
public static final String TABLE_NAME = "bar";
public static final String CLASSIFICATION = "classification";
public static final String PII = "PII";
public static final String SUPER_TYPE_NAME = "Base";
public static TypesDef defineHiveTypes() {
HierarchicalTypeDefinition<ClassType> superTypeDefinition =
createClassTypeDef(SUPER_TYPE_NAME, ImmutableList.<String>of(),
createOptionalAttrDef("namespace", DataTypes.STRING_TYPE),
createOptionalAttrDef("cluster", DataTypes.STRING_TYPE),
createOptionalAttrDef("colo", DataTypes.STRING_TYPE));
HierarchicalTypeDefinition<ClassType> databaseTypeDefinition =
createClassTypeDef(DATABASE_TYPE, ImmutableList.of(SUPER_TYPE_NAME),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
createOptionalAttrDef("created", DataTypes.DATE_TYPE),
createRequiredAttrDef("description", DataTypes.STRING_TYPE));
StructTypeDefinition structTypeDefinition = new StructTypeDefinition("serdeType",
new AttributeDefinition[]{createRequiredAttrDef("name", DataTypes.STRING_TYPE),
createRequiredAttrDef("serde", DataTypes.STRING_TYPE)});
EnumValue values[] = {new EnumValue("MANAGED", 1), new EnumValue("EXTERNAL", 2),};
EnumTypeDefinition enumTypeDefinition = new EnumTypeDefinition("tableType", values);
HierarchicalTypeDefinition<ClassType> columnsDefinition =
createClassTypeDef("column_type", ImmutableList.<String>of(),
createRequiredAttrDef("name", DataTypes.STRING_TYPE),
createRequiredAttrDef("type", DataTypes.STRING_TYPE));
StructTypeDefinition partitionDefinition = new StructTypeDefinition("partition_type",
new AttributeDefinition[]{createRequiredAttrDef("name", DataTypes.STRING_TYPE),});
HierarchicalTypeDefinition<ClassType> tableTypeDefinition =
createClassTypeDef(TABLE_TYPE, ImmutableList.of(SUPER_TYPE_NAME),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
createRequiredAttrDef("description", DataTypes.STRING_TYPE),
createRequiredAttrDef("type", DataTypes.STRING_TYPE),
createOptionalAttrDef("created", DataTypes.DATE_TYPE),
// enum
new AttributeDefinition("tableType", "tableType", Multiplicity.REQUIRED, false, null),
// array of strings
new AttributeDefinition("columnNames",
String.format("array<%s>", DataTypes.STRING_TYPE.getName()), Multiplicity.OPTIONAL,
false, null),
// array of classes
new AttributeDefinition("columns", String.format("array<%s>", "column_type"),
Multiplicity.OPTIONAL, true, null),
// array of structs
new AttributeDefinition("partitions", String.format("array<%s>", "partition_type"),
Multiplicity.OPTIONAL, true, null),
// map of primitives
new AttributeDefinition("parametersMap",
DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(), DataTypes.STRING_TYPE.getName()),
Multiplicity.OPTIONAL, true, null),
// map of classes - todo - enable this
// new AttributeDefinition("columnsMap",
// DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(),
// "column_type"),
// Multiplicity.COLLECTION, true, null),
// map of structs todo - enable this
// new AttributeDefinition("partitionsMap",
// DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(),
// "partition_type"),
// Multiplicity.COLLECTION, true, null),
// struct reference
new AttributeDefinition("serde1", "serdeType", Multiplicity.OPTIONAL, false, null),
new AttributeDefinition("serde2", "serdeType", Multiplicity.OPTIONAL, false, null),
// class reference
new AttributeDefinition("database", DATABASE_TYPE, Multiplicity.REQUIRED, true, null));
HierarchicalTypeDefinition<TraitType> piiTypeDefinition =
createTraitTypeDef(PII, ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> classificationTypeDefinition =
createTraitTypeDef(CLASSIFICATION, ImmutableList.<String>of(),
createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
HierarchicalTypeDefinition<TraitType> fetlClassificationTypeDefinition =
createTraitTypeDef("fetl" + CLASSIFICATION, ImmutableList.of(CLASSIFICATION),
createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
return TypeUtils.getTypesDef(ImmutableList.of(enumTypeDefinition),
ImmutableList.of(structTypeDefinition, partitionDefinition),
ImmutableList.of(classificationTypeDefinition, fetlClassificationTypeDefinition, piiTypeDefinition),
ImmutableList.of(superTypeDefinition, databaseTypeDefinition, columnsDefinition, tableTypeDefinition));
}
public static Collection<IDataType> createHiveTypes(TypeSystem typeSystem) throws Exception {
if (!typeSystem.isRegistered(TABLE_TYPE)) {
TypesDef typesDef = defineHiveTypes();
return typeSystem.defineTypes(typesDef).values();
}
return null;
}
public static final String randomString() {
return RandomStringUtils.randomAlphanumeric(10);
}
} }
...@@ -85,7 +85,7 @@ public class GraphBackedDiscoveryServiceTest { ...@@ -85,7 +85,7 @@ public class GraphBackedDiscoveryServiceTest {
ClassType deptType = typeSystem.getDataType(ClassType.class, "Department"); ClassType deptType = typeSystem.getDataType(ClassType.class, "Department");
ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED); ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED);
repositoryService.createEntity(hrDept2); repositoryService.createEntities(hrDept2);
} }
private void setupSampleData() throws ScriptException { private void setupSampleData() throws ScriptException {
...@@ -303,6 +303,6 @@ public class GraphBackedDiscoveryServiceTest { ...@@ -303,6 +303,6 @@ public class GraphBackedDiscoveryServiceTest {
ClassType deptType = TypeSystem.getInstance().getDataType(ClassType.class, "D"); ClassType deptType = TypeSystem.getInstance().getDataType(ClassType.class, "D");
ITypedReferenceableInstance typedInstance = deptType.convert(instance, Multiplicity.REQUIRED); ITypedReferenceableInstance typedInstance = deptType.convert(instance, Multiplicity.REQUIRED);
repositoryService.createEntity(typedInstance); repositoryService.createEntities(typedInstance);
} }
} }
\ No newline at end of file
...@@ -54,9 +54,6 @@ import org.testng.annotations.Guice; ...@@ -54,9 +54,6 @@ import org.testng.annotations.Guice;
import org.testng.annotations.Test; import org.testng.annotations.Test;
import javax.inject.Inject; import javax.inject.Inject;
import java.io.File;
import java.io.IOException;
import java.util.HashSet;
import java.util.List; import java.util.List;
/** /**
...@@ -534,7 +531,10 @@ public class HiveLineageServiceTest { ...@@ -534,7 +531,10 @@ public class HiveLineageServiceTest {
String entityJSON = InstanceSerialization.toJson(referenceable, true); String entityJSON = InstanceSerialization.toJson(referenceable, true);
System.out.println("Submitting new entity= " + entityJSON); System.out.println("Submitting new entity= " + entityJSON);
String guid = metadataService.createEntity(entityJSON); JSONArray jsonArray = new JSONArray();
jsonArray.put(entityJSON);
String response = metadataService.createEntities(jsonArray.toString());
String guid = new JSONArray(response).getString(0);
System.out.println("created instance for type " + typeName + ", guid: " + guid); System.out.println("created instance for type " + typeName + ", guid: " + guid);
// return the reference to created instance with guid // return the reference to created instance with guid
......
...@@ -18,172 +18,13 @@ ...@@ -18,172 +18,13 @@
package org.apache.atlas.repository; package org.apache.atlas.repository;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.memory.MemRepository;
import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.HierarchicalType;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.IDataType;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructType;
import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeSystem; import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.junit.Before; import org.junit.Before;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.Date;
import java.util.Map;
public abstract class BaseTest { public abstract class BaseTest {
public static final String STRUCT_TYPE_1 = "t1";
public static final String STRUCT_TYPE_2 = "t2";
public static final String TEST_DATE = "2014-12-11T02:35:58.440Z";
public static final long TEST_DATE_IN_LONG = 1418265358440L;
protected IRepository repo;
public static Struct createStruct() throws AtlasException {
StructType structType = (StructType) TypeSystem.getInstance().getDataType(StructType.class, STRUCT_TYPE_1);
Struct s = new Struct(structType.getName());
s.set("a", 1);
s.set("b", true);
s.set("c", (byte) 1);
s.set("d", (short) 2);
s.set("e", 1);
s.set("f", 1);
s.set("g", 1L);
s.set("h", 1.0f);
s.set("i", 1.0);
s.set("j", BigInteger.valueOf(1L));
s.set("k", new BigDecimal(1));
s.set("l", new Date(1418265358440L));
s.set("m", Lists.<Integer>asList(Integer.valueOf(1), new Integer[]{Integer.valueOf(1)}));
s.set("n", Lists.<BigDecimal>asList(BigDecimal.valueOf(1.1), new BigDecimal[]{BigDecimal.valueOf(1.1)}));
Map<String, Double> hm = Maps.<String, Double>newHashMap();
hm.put("a", 1.0);
hm.put("b", 2.0);
s.set("o", hm);
return s;
}
protected final TypeSystem getTypeSystem() {
return TypeSystem.getInstance();
}
protected final IRepository getRepository() {
return repo;
}
@Before @Before
public void setup() throws Exception { public void setup() throws Exception {
TypeSystem ts = TypeSystem.getInstance(); TypeSystem ts = TypeSystem.getInstance();
ts.reset(); ts.reset();
repo = new MemRepository(ts);
StructType structType =
ts.defineStructType(STRUCT_TYPE_1, true, TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("b", DataTypes.BOOLEAN_TYPE),
TypesUtil.createOptionalAttrDef("c", DataTypes.BYTE_TYPE),
TypesUtil.createOptionalAttrDef("d", DataTypes.SHORT_TYPE),
TypesUtil.createOptionalAttrDef("e", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("f", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("g", DataTypes.LONG_TYPE),
TypesUtil.createOptionalAttrDef("h", DataTypes.FLOAT_TYPE),
TypesUtil.createOptionalAttrDef("i", DataTypes.DOUBLE_TYPE),
TypesUtil.createOptionalAttrDef("j", DataTypes.BIGINTEGER_TYPE),
TypesUtil.createOptionalAttrDef("k", DataTypes.BIGDECIMAL_TYPE),
TypesUtil.createOptionalAttrDef("l", DataTypes.DATE_TYPE),
TypesUtil.createOptionalAttrDef("m", ts.defineArrayType(DataTypes.INT_TYPE)),
TypesUtil.createOptionalAttrDef("n", ts.defineArrayType(DataTypes.BIGDECIMAL_TYPE)), TypesUtil
.createOptionalAttrDef("o",
ts.defineMapType(DataTypes.STRING_TYPE, DataTypes.DOUBLE_TYPE)));
StructType recursiveStructType =
ts.defineStructType(STRUCT_TYPE_2, true, TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("s", STRUCT_TYPE_2));
}
protected Map<String, IDataType> defineTraits(HierarchicalTypeDefinition... tDefs) throws AtlasException {
return getTypeSystem().defineTraitTypes(tDefs);
}
/*
* Class Hierarchy is:
* Department(name : String, employees : Array[Person])
* Person(name : String, department : Department, manager : Manager)
* Manager(subordinates : Array[Person]) extends Person
*
* Persons can have SecurityClearance(level : Int) clearance.
*/
protected void defineDeptEmployeeTypes(TypeSystem ts) throws AtlasException {
HierarchicalTypeDefinition<ClassType> deptTypeDef = TypesUtil
.createClassTypeDef("Department", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
new AttributeDefinition("employees", String.format("array<%s>", "Person"),
Multiplicity.COLLECTION, true, "department"));
HierarchicalTypeDefinition<ClassType> personTypeDef = TypesUtil
.createClassTypeDef("Person", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
new AttributeDefinition("department", "Department", Multiplicity.REQUIRED, false, "employees"),
new AttributeDefinition("manager", "Manager", Multiplicity.OPTIONAL, false, "subordinates"));
HierarchicalTypeDefinition<ClassType> managerTypeDef = TypesUtil
.createClassTypeDef("Manager", ImmutableList.<String>of("Person"),
new AttributeDefinition("subordinates", String.format("array<%s>", "Person"),
Multiplicity.COLLECTION, false, "manager"));
HierarchicalTypeDefinition<TraitType> securityClearanceTypeDef = TypesUtil
.createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE));
ts.defineTypes(ImmutableList.<StructTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(securityClearanceTypeDef),
ImmutableList.<HierarchicalTypeDefinition<ClassType>>of(deptTypeDef, personTypeDef, managerTypeDef));
ImmutableList<HierarchicalType> types = ImmutableList
.of(ts.getDataType(HierarchicalType.class, "SecurityClearance"),
ts.getDataType(ClassType.class, "Department"), ts.getDataType(ClassType.class, "Person"),
ts.getDataType(ClassType.class, "Manager"));
repo.defineTypes(types);
}
protected Referenceable createDeptEg1(TypeSystem ts) throws AtlasException {
Referenceable hrDept = new Referenceable("Department");
Referenceable john = new Referenceable("Person");
Referenceable jane = new Referenceable("Manager", "SecurityClearance");
hrDept.set("name", "hr");
john.set("name", "John");
john.set("department", hrDept);
jane.set("name", "Jane");
jane.set("department", hrDept);
john.set("manager", jane);
hrDept.set("employees", ImmutableList.<Referenceable>of(john, jane));
jane.set("subordinates", ImmutableList.<Referenceable>of(john));
jane.getTrait("SecurityClearance").set("level", 1);
ClassType deptType = ts.getDataType(ClassType.class, "Department");
ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED);
return hrDept;
} }
} }
...@@ -28,7 +28,6 @@ import org.apache.atlas.GraphTransaction; ...@@ -28,7 +28,6 @@ import org.apache.atlas.GraphTransaction;
import org.apache.atlas.RepositoryMetadataModule; import org.apache.atlas.RepositoryMetadataModule;
import org.apache.atlas.TestUtils; import org.apache.atlas.TestUtils;
import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService; import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService;
import org.apache.atlas.repository.BaseTest;
import org.apache.atlas.repository.Constants; import org.apache.atlas.repository.Constants;
import org.apache.atlas.repository.EntityNotFoundException; import org.apache.atlas.repository.EntityNotFoundException;
import org.apache.atlas.repository.RepositoryException; import org.apache.atlas.repository.RepositoryException;
...@@ -38,14 +37,10 @@ import org.apache.atlas.typesystem.ITypedStruct; ...@@ -38,14 +37,10 @@ import org.apache.atlas.typesystem.ITypedStruct;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.Struct; import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.persistence.Id; import org.apache.atlas.typesystem.persistence.Id;
import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes; import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumTypeDefinition;
import org.apache.atlas.typesystem.types.EnumValue;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition; import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.Multiplicity; import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType; import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeSystem; import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.atlas.typesystem.types.utils.TypesUtil; import org.apache.atlas.typesystem.types.utils.TypesUtil;
...@@ -76,15 +71,6 @@ import java.util.UUID; ...@@ -76,15 +71,6 @@ import java.util.UUID;
@Guice(modules = RepositoryMetadataModule.class) @Guice(modules = RepositoryMetadataModule.class)
public class GraphBackedMetadataRepositoryTest { public class GraphBackedMetadataRepositoryTest {
private static final String ENTITY_TYPE = "Department";
private static final String DATABASE_TYPE = "hive_database";
private static final String DATABASE_NAME = "foo";
private static final String TABLE_TYPE = "hive_table";
private static final String TABLE_NAME = "bar";
private static final String CLASSIFICATION = "classification";
private static final String PII = "PII";
private static final String SUPER_TYPE_NAME = "Base";
@Inject @Inject
private GraphProvider<TitanGraph> graphProvider; private GraphProvider<TitanGraph> graphProvider;
...@@ -105,7 +91,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -105,7 +91,7 @@ public class GraphBackedMetadataRepositoryTest {
new GraphBackedSearchIndexer(graphProvider); new GraphBackedSearchIndexer(graphProvider);
TestUtils.defineDeptEmployeeTypes(typeSystem); TestUtils.defineDeptEmployeeTypes(typeSystem);
createHiveTypes(); TestUtils.createHiveTypes(typeSystem);
} }
...@@ -132,7 +118,10 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -132,7 +118,10 @@ public class GraphBackedMetadataRepositoryTest {
ClassType deptType = typeSystem.getDataType(ClassType.class, "Department"); ClassType deptType = typeSystem.getDataType(ClassType.class, "Department");
ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED); ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED);
guid = repositoryService.createEntity(hrDept2); String[] guids = repositoryService.createEntities(hrDept2);
Assert.assertNotNull(guids);
Assert.assertEquals(guids.length, 1);
guid = guids[0];
Assert.assertNotNull(guid); Assert.assertNotNull(guid);
} }
...@@ -150,7 +139,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -150,7 +139,7 @@ public class GraphBackedMetadataRepositoryTest {
@Test @Test
public void testGetEntityList() throws Exception { public void testGetEntityList() throws Exception {
List<String> entityList = repositoryService.getEntityList(ENTITY_TYPE); List<String> entityList = repositoryService.getEntityList(TestUtils.ENTITY_TYPE);
System.out.println("entityList = " + entityList); System.out.println("entityList = " + entityList);
Assert.assertNotNull(entityList); Assert.assertNotNull(entityList);
Assert.assertEquals(entityList.size(), 1); // one department Assert.assertEquals(entityList.size(), 1); // one department
...@@ -164,33 +153,33 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -164,33 +153,33 @@ public class GraphBackedMetadataRepositoryTest {
@Test(dependsOnMethods = "testSubmitEntity") @Test(dependsOnMethods = "testSubmitEntity")
public void testGetTraitLabel() throws Exception { public void testGetTraitLabel() throws Exception {
Assert.assertEquals( Assert.assertEquals(
repositoryService.getTraitLabel(typeSystem.getDataType(ClassType.class, TABLE_TYPE), CLASSIFICATION), repositoryService.getTraitLabel(typeSystem.getDataType(ClassType.class, TestUtils.TABLE_TYPE),
TABLE_TYPE + "." + CLASSIFICATION); TestUtils.CLASSIFICATION), TestUtils.TABLE_TYPE + "." + TestUtils.CLASSIFICATION);
} }
@Test @Test
public void testCreateEntity() throws Exception { public void testCreateEntity() throws Exception {
Referenceable databaseInstance = new Referenceable(DATABASE_TYPE); Referenceable databaseInstance = new Referenceable(TestUtils.DATABASE_TYPE);
databaseInstance.set("name", DATABASE_NAME); databaseInstance.set("name", TestUtils.DATABASE_NAME);
databaseInstance.set("description", "foo database"); databaseInstance.set("description", "foo database");
databaseInstance.set("created", new Date(BaseTest.TEST_DATE_IN_LONG)); databaseInstance.set("created", new Date(TestUtils.TEST_DATE_IN_LONG));
databaseInstance.set("namespace", "colo:cluster:hive:db"); databaseInstance.set("namespace", "colo:cluster:hive:db");
databaseInstance.set("cluster", "cluster-1"); databaseInstance.set("cluster", "cluster-1");
databaseInstance.set("colo", "colo-1"); databaseInstance.set("colo", "colo-1");
System.out.println("databaseInstance = " + databaseInstance); System.out.println("databaseInstance = " + databaseInstance);
ClassType dbType = typeSystem.getDataType(ClassType.class, DATABASE_TYPE); ClassType dbType = typeSystem.getDataType(ClassType.class, TestUtils.DATABASE_TYPE);
ITypedReferenceableInstance db = dbType.convert(databaseInstance, Multiplicity.REQUIRED); ITypedReferenceableInstance db = dbType.convert(databaseInstance, Multiplicity.REQUIRED);
System.out.println("db = " + db); System.out.println("db = " + db);
String dbGUID = repositoryService.createEntity(db); String dbGUID = repositoryService.createEntities(db)[0];
System.out.println("added db = " + dbGUID); System.out.println("added db = " + dbGUID);
Referenceable dbInstance = new Referenceable(dbGUID, DATABASE_TYPE, databaseInstance.getValuesMap()); Referenceable dbInstance = new Referenceable(dbGUID, TestUtils.DATABASE_TYPE, databaseInstance.getValuesMap());
ITypedReferenceableInstance table = createHiveTableInstance(dbInstance); ITypedReferenceableInstance table = createHiveTableInstance(dbInstance);
String tableGUID = repositoryService.createEntity(table); String tableGUID = repositoryService.createEntities(table)[0];
System.out.println("added table = " + tableGUID); System.out.println("added table = " + tableGUID);
} }
...@@ -199,7 +188,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -199,7 +188,7 @@ public class GraphBackedMetadataRepositoryTest {
String guid = getGUID(); String guid = getGUID();
ITypedReferenceableInstance table = repositoryService.getEntityDefinition(guid); ITypedReferenceableInstance table = repositoryService.getEntityDefinition(guid);
Assert.assertEquals(table.getDate("created"), new Date(BaseTest.TEST_DATE_IN_LONG)); Assert.assertEquals(table.getDate("created"), new Date(TestUtils.TEST_DATE_IN_LONG));
System.out.println("*** table = " + table); System.out.println("*** table = " + table);
} }
...@@ -217,7 +206,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -217,7 +206,7 @@ public class GraphBackedMetadataRepositoryTest {
@GraphTransaction @GraphTransaction
Vertex getTableEntityVertex() { Vertex getTableEntityVertex() {
TitanGraph graph = graphProvider.get(); TitanGraph graph = graphProvider.get();
GraphQuery query = graph.query().has(Constants.ENTITY_TYPE_PROPERTY_KEY, Compare.EQUAL, TABLE_TYPE); GraphQuery query = graph.query().has(Constants.ENTITY_TYPE_PROPERTY_KEY, Compare.EQUAL, TestUtils.TABLE_TYPE);
Iterator<Vertex> results = query.vertices().iterator(); Iterator<Vertex> results = query.vertices().iterator();
// returning one since guid should be unique // returning one since guid should be unique
Vertex tableVertex = results.hasNext() ? results.next() : null; Vertex tableVertex = results.hasNext() ? results.next() : null;
...@@ -232,7 +221,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -232,7 +221,7 @@ public class GraphBackedMetadataRepositoryTest {
public void testGetTraitNames() throws Exception { public void testGetTraitNames() throws Exception {
final List<String> traitNames = repositoryService.getTraitNames(getGUID()); final List<String> traitNames = repositoryService.getTraitNames(getGUID());
Assert.assertEquals(traitNames.size(), 1); Assert.assertEquals(traitNames.size(), 1);
Assert.assertEquals(traitNames, Arrays.asList(new String[]{CLASSIFICATION})); Assert.assertEquals(traitNames, Arrays.asList(new String[]{TestUtils.CLASSIFICATION}));
} }
@Test @Test
...@@ -254,11 +243,10 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -254,11 +243,10 @@ public class GraphBackedMetadataRepositoryTest {
List<String> traitNames = repositoryService.getTraitNames(aGUID); List<String> traitNames = repositoryService.getTraitNames(aGUID);
System.out.println("traitNames = " + traitNames); System.out.println("traitNames = " + traitNames);
Assert.assertEquals(traitNames.size(), 1); Assert.assertEquals(traitNames.size(), 1);
Assert.assertTrue(traitNames.contains(CLASSIFICATION)); Assert.assertTrue(traitNames.contains(TestUtils.CLASSIFICATION));
Assert.assertFalse(traitNames.contains(PII)); Assert.assertFalse(traitNames.contains(TestUtils.PII));
HierarchicalTypeDefinition<TraitType> piiTrait = TypesUtil.createTraitTypeDef(PII, ImmutableList.<String>of()); TraitType traitType = typeSystem.getDataType(TraitType.class, TestUtils.PII);
TraitType traitType = typeSystem.defineTraitType(piiTrait);
ITypedStruct traitInstance = traitType.createInstance(); ITypedStruct traitInstance = traitType.createInstance();
repositoryService.addTrait(aGUID, traitInstance); repositoryService.addTrait(aGUID, traitInstance);
...@@ -266,8 +254,8 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -266,8 +254,8 @@ public class GraphBackedMetadataRepositoryTest {
// refresh trait names // refresh trait names
traitNames = repositoryService.getTraitNames(aGUID); traitNames = repositoryService.getTraitNames(aGUID);
Assert.assertEquals(traitNames.size(), 2); Assert.assertEquals(traitNames.size(), 2);
Assert.assertTrue(traitNames.contains(PII)); Assert.assertTrue(traitNames.contains(TestUtils.PII));
Assert.assertTrue(traitNames.contains(CLASSIFICATION)); Assert.assertTrue(traitNames.contains(TestUtils.CLASSIFICATION));
} }
@Test(dependsOnMethods = "testAddTrait") @Test(dependsOnMethods = "testAddTrait")
...@@ -305,7 +293,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -305,7 +293,7 @@ public class GraphBackedMetadataRepositoryTest {
@Test(dependsOnMethods = "testAddTrait", expectedExceptions = RepositoryException.class) @Test(dependsOnMethods = "testAddTrait", expectedExceptions = RepositoryException.class)
public void testAddTraitForBadEntity() throws Exception { public void testAddTraitForBadEntity() throws Exception {
TraitType traitType = typeSystem.getDataType(TraitType.class, PII); TraitType traitType = typeSystem.getDataType(TraitType.class, TestUtils.PII);
ITypedStruct traitInstance = traitType.createInstance(); ITypedStruct traitInstance = traitType.createInstance();
repositoryService.addTrait(UUID.randomUUID().toString(), traitInstance); repositoryService.addTrait(UUID.randomUUID().toString(), traitInstance);
...@@ -318,22 +306,22 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -318,22 +306,22 @@ public class GraphBackedMetadataRepositoryTest {
List<String> traitNames = repositoryService.getTraitNames(aGUID); List<String> traitNames = repositoryService.getTraitNames(aGUID);
Assert.assertEquals(traitNames.size(), 3); Assert.assertEquals(traitNames.size(), 3);
Assert.assertTrue(traitNames.contains(PII)); Assert.assertTrue(traitNames.contains(TestUtils.PII));
Assert.assertTrue(traitNames.contains(CLASSIFICATION)); Assert.assertTrue(traitNames.contains(TestUtils.CLASSIFICATION));
Assert.assertTrue(traitNames.contains("P_I_I")); Assert.assertTrue(traitNames.contains("P_I_I"));
repositoryService.deleteTrait(aGUID, PII); repositoryService.deleteTrait(aGUID, TestUtils.PII);
// refresh trait names // refresh trait names
traitNames = repositoryService.getTraitNames(aGUID); traitNames = repositoryService.getTraitNames(aGUID);
Assert.assertEquals(traitNames.size(), 2); Assert.assertEquals(traitNames.size(), 2);
Assert.assertTrue(traitNames.contains(CLASSIFICATION)); Assert.assertTrue(traitNames.contains(TestUtils.CLASSIFICATION));
Assert.assertFalse(traitNames.contains(PII)); Assert.assertFalse(traitNames.contains(TestUtils.PII));
} }
@Test(expectedExceptions = RepositoryException.class) @Test(expectedExceptions = RepositoryException.class)
public void testDeleteTraitForNonExistentEntity() throws Exception { public void testDeleteTraitForNonExistentEntity() throws Exception {
repositoryService.deleteTrait(UUID.randomUUID().toString(), PII); repositoryService.deleteTrait(UUID.randomUUID().toString(), TestUtils.PII);
Assert.fail(); Assert.fail();
} }
...@@ -353,14 +341,14 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -353,14 +341,14 @@ public class GraphBackedMetadataRepositoryTest {
Assert.fail(); Assert.fail();
} }
Id expected = new Id(guid, tableVertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY), TABLE_TYPE); Id expected = new Id(guid, tableVertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY), TestUtils.TABLE_TYPE);
Assert.assertEquals(repositoryService.getIdFromVertex(TABLE_TYPE, tableVertex), expected); Assert.assertEquals(repositoryService.getIdFromVertex(TestUtils.TABLE_TYPE, tableVertex), expected);
} }
@Test(dependsOnMethods = "testCreateEntity") @Test(dependsOnMethods = "testCreateEntity")
public void testGetTypeName() throws Exception { public void testGetTypeName() throws Exception {
Vertex tableVertex = getTableEntityVertex(); Vertex tableVertex = getTableEntityVertex();
Assert.assertEquals(repositoryService.getTypeName(tableVertex), TABLE_TYPE); Assert.assertEquals(repositoryService.getTypeName(tableVertex), TestUtils.TABLE_TYPE);
} }
@Test(dependsOnMethods = "testCreateEntity") @Test(dependsOnMethods = "testCreateEntity")
...@@ -392,7 +380,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -392,7 +380,7 @@ public class GraphBackedMetadataRepositoryTest {
Assert.assertEquals(type, "hive_database"); Assert.assertEquals(type, "hive_database");
String name = row.getString("name"); String name = row.getString("name");
Assert.assertEquals(name, DATABASE_NAME); Assert.assertEquals(name, TestUtils.DATABASE_NAME);
} }
} }
...@@ -484,94 +472,12 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -484,94 +472,12 @@ public class GraphBackedMetadataRepositoryTest {
Assert.assertEquals(row.get("typeName"), "Person"); Assert.assertEquals(row.get("typeName"), "Person");
} }
private void createHiveTypes() throws Exception {
HierarchicalTypeDefinition<ClassType> superTypeDefinition = TypesUtil
.createClassTypeDef(SUPER_TYPE_NAME, ImmutableList.<String>of(),
TypesUtil.createOptionalAttrDef("namespace", DataTypes.STRING_TYPE),
TypesUtil.createOptionalAttrDef("cluster", DataTypes.STRING_TYPE),
TypesUtil.createOptionalAttrDef("colo", DataTypes.STRING_TYPE));
HierarchicalTypeDefinition<ClassType> databaseTypeDefinition = TypesUtil
.createClassTypeDef(DATABASE_TYPE, ImmutableList.of(SUPER_TYPE_NAME),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createOptionalAttrDef("created", DataTypes.DATE_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE));
StructTypeDefinition structTypeDefinition = new StructTypeDefinition("serdeType",
new AttributeDefinition[]{TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("serde", DataTypes.STRING_TYPE)});
EnumValue values[] = {new EnumValue("MANAGED", 1), new EnumValue("EXTERNAL", 2),};
EnumTypeDefinition enumTypeDefinition = new EnumTypeDefinition("tableType", values);
typeSystem.defineEnumType(enumTypeDefinition);
HierarchicalTypeDefinition<ClassType> columnsDefinition = TypesUtil
.createClassTypeDef("column_type", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("type", DataTypes.STRING_TYPE));
StructTypeDefinition partitionDefinition = new StructTypeDefinition("partition_type",
new AttributeDefinition[]{TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),});
HierarchicalTypeDefinition<ClassType> tableTypeDefinition = TypesUtil
.createClassTypeDef(TABLE_TYPE, ImmutableList.of(SUPER_TYPE_NAME),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("type", DataTypes.STRING_TYPE),
TypesUtil.createOptionalAttrDef("created", DataTypes.DATE_TYPE),
// enum
new AttributeDefinition("tableType", "tableType", Multiplicity.REQUIRED, false, null),
// array of strings
new AttributeDefinition("columnNames",
String.format("array<%s>", DataTypes.STRING_TYPE.getName()), Multiplicity.COLLECTION,
false, null),
// array of classes
new AttributeDefinition("columns", String.format("array<%s>", "column_type"),
Multiplicity.COLLECTION, true, null),
// array of structs
new AttributeDefinition("partitions", String.format("array<%s>", "partition_type"),
Multiplicity.COLLECTION, true, null),
// map of primitives
new AttributeDefinition("parametersMap",
DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(), DataTypes.STRING_TYPE.getName()),
Multiplicity.COLLECTION, true, null),
// map of classes - todo - enable this
// new AttributeDefinition("columnsMap",
// DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(),
// "column_type"),
// Multiplicity.COLLECTION, true, null),
// map of structs todo - enable this
// new AttributeDefinition("partitionsMap",
// DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(),
// "partition_type"),
// Multiplicity.COLLECTION, true, null),
// struct reference
new AttributeDefinition("serde1", "serdeType", Multiplicity.REQUIRED, false, null),
new AttributeDefinition("serde2", "serdeType", Multiplicity.REQUIRED, false, null),
// class reference
new AttributeDefinition("database", DATABASE_TYPE, Multiplicity.REQUIRED, true, null));
HierarchicalTypeDefinition<TraitType> classificationTypeDefinition = TypesUtil
.createTraitTypeDef(CLASSIFICATION, ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
HierarchicalTypeDefinition<TraitType> fetlClassificationTypeDefinition = TypesUtil
.createTraitTypeDef("fetl" + CLASSIFICATION, ImmutableList.of(CLASSIFICATION),
TypesUtil.createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
typeSystem.defineTypes(ImmutableList.of(structTypeDefinition, partitionDefinition),
ImmutableList.of(classificationTypeDefinition, fetlClassificationTypeDefinition),
ImmutableList.of(superTypeDefinition, databaseTypeDefinition, columnsDefinition, tableTypeDefinition));
}
private ITypedReferenceableInstance createHiveTableInstance(Referenceable databaseInstance) throws Exception { private ITypedReferenceableInstance createHiveTableInstance(Referenceable databaseInstance) throws Exception {
Referenceable tableInstance = new Referenceable(TABLE_TYPE, CLASSIFICATION); Referenceable tableInstance = new Referenceable(TestUtils.TABLE_TYPE, TestUtils.CLASSIFICATION);
tableInstance.set("name", TABLE_NAME); tableInstance.set("name", TestUtils.TABLE_NAME);
tableInstance.set("description", "bar table"); tableInstance.set("description", "bar table");
tableInstance.set("type", "managed"); tableInstance.set("type", "managed");
tableInstance.set("created", new Date(BaseTest.TEST_DATE_IN_LONG)); tableInstance.set("created", new Date(TestUtils.TEST_DATE_IN_LONG));
tableInstance.set("tableType", 1); // enum tableInstance.set("tableType", 1); // enum
// super type // super type
...@@ -587,7 +493,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -587,7 +493,7 @@ public class GraphBackedMetadataRepositoryTest {
columnNames.add("last_name"); columnNames.add("last_name");
tableInstance.set("columnNames", columnNames); tableInstance.set("columnNames", columnNames);
Struct traitInstance = (Struct) tableInstance.getTrait(CLASSIFICATION); Struct traitInstance = (Struct) tableInstance.getTrait(TestUtils.CLASSIFICATION);
traitInstance.set("tag", "foundation_etl"); traitInstance.set("tag", "foundation_etl");
Struct serde1Instance = new Struct("serdeType"); Struct serde1Instance = new Struct("serdeType");
...@@ -633,7 +539,7 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -633,7 +539,7 @@ public class GraphBackedMetadataRepositoryTest {
parametersMap.put("some", "thing"); parametersMap.put("some", "thing");
tableInstance.set("parametersMap", parametersMap); tableInstance.set("parametersMap", parametersMap);
ClassType tableType = typeSystem.getDataType(ClassType.class, TABLE_TYPE); ClassType tableType = typeSystem.getDataType(ClassType.class, TestUtils.TABLE_TYPE);
return tableType.convert(tableInstance, Multiplicity.REQUIRED); return tableType.convert(tableInstance, Multiplicity.REQUIRED);
} }
...@@ -654,7 +560,9 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -654,7 +560,9 @@ public class GraphBackedMetadataRepositoryTest {
ClassType deptType = typeSystem.getDataType(ClassType.class, "Department"); ClassType deptType = typeSystem.getDataType(ClassType.class, "Department");
ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED); ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED);
guid = repositoryService.createEntity(hrDept2); String[] guids = repositoryService.createEntities(hrDept2);
Assert.assertNotNull(guid); Assert.assertNotNull(guids);
Assert.assertEquals(guids.length, 1);
Assert.assertNotNull(guids[0]);
} }
} }
...@@ -18,40 +18,27 @@ ...@@ -18,40 +18,27 @@
package org.apache.atlas.repository.graph; package org.apache.atlas.repository.graph;
import com.google.common.collect.ImmutableList;
import com.thinkaurelius.titan.core.TitanFactory; import com.thinkaurelius.titan.core.TitanFactory;
import com.thinkaurelius.titan.core.TitanGraph; import com.thinkaurelius.titan.core.TitanGraph;
import com.thinkaurelius.titan.core.TitanGraphQuery;
import com.thinkaurelius.titan.core.TitanIndexQuery; import com.thinkaurelius.titan.core.TitanIndexQuery;
import com.thinkaurelius.titan.core.schema.TitanGraphIndex;
import com.thinkaurelius.titan.diskstorage.BackendException; import com.thinkaurelius.titan.diskstorage.BackendException;
import com.thinkaurelius.titan.diskstorage.configuration.ReadConfiguration; import com.thinkaurelius.titan.diskstorage.configuration.ReadConfiguration;
import com.thinkaurelius.titan.diskstorage.configuration.backend.CommonsConfiguration; import com.thinkaurelius.titan.diskstorage.configuration.backend.CommonsConfiguration;
import com.thinkaurelius.titan.graphdb.configuration.GraphDatabaseConfiguration; import com.thinkaurelius.titan.graphdb.configuration.GraphDatabaseConfiguration;
import com.tinkerpop.blueprints.Compare; import com.tinkerpop.blueprints.Compare;
import com.tinkerpop.blueprints.Edge;
import com.tinkerpop.blueprints.GraphQuery; import com.tinkerpop.blueprints.GraphQuery;
import com.tinkerpop.blueprints.Predicate; import com.tinkerpop.blueprints.Predicate;
import com.tinkerpop.blueprints.Vertex; import com.tinkerpop.blueprints.Vertex;
import org.apache.atlas.GraphTransaction; import org.apache.atlas.GraphTransaction;
import org.apache.atlas.repository.BaseTest; import org.apache.atlas.TestUtils;
import org.apache.atlas.repository.Constants; import org.apache.atlas.repository.Constants;
import org.apache.atlas.typesystem.ITypedReferenceableInstance; import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.Struct; import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumType;
import org.apache.atlas.typesystem.types.EnumTypeDefinition;
import org.apache.atlas.typesystem.types.EnumValue;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.IDataType; import org.apache.atlas.typesystem.types.IDataType;
import org.apache.atlas.typesystem.types.Multiplicity; import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeSystem; import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.testng.annotations.AfterClass; import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeClass;
...@@ -60,16 +47,14 @@ import org.testng.annotations.Test; ...@@ -60,16 +47,14 @@ import org.testng.annotations.Test;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Date; import java.util.Date;
import java.util.Map;
import java.util.Random; import java.util.Random;
@Test @Test
public class GraphRepoMapperScaleTest { public class GraphRepoMapperScaleTest {
private static final String DATABASE_TYPE = "hive_database_type";
private static final String DATABASE_NAME = "foo"; private static final String DATABASE_NAME = "foo";
private static final String TABLE_TYPE = "hive_table_type";
private static final String TABLE_NAME = "bar"; private static final String TABLE_NAME = "bar";
private static final String INDEX_DIR = private static final String INDEX_DIR =
...@@ -109,7 +94,8 @@ public class GraphRepoMapperScaleTest { ...@@ -109,7 +94,8 @@ public class GraphRepoMapperScaleTest {
private GraphBackedMetadataRepository repositoryService; private GraphBackedMetadataRepository repositoryService;
private GraphBackedSearchIndexer searchIndexer; private GraphBackedSearchIndexer searchIndexer;
private TypeSystem typeSystem;
private TypeSystem typeSystem = TypeSystem.getInstance();
private String dbGUID; private String dbGUID;
@BeforeClass @BeforeClass
...@@ -119,10 +105,8 @@ public class GraphRepoMapperScaleTest { ...@@ -119,10 +105,8 @@ public class GraphRepoMapperScaleTest {
repositoryService = new GraphBackedMetadataRepository(graphProvider); repositoryService = new GraphBackedMetadataRepository(graphProvider);
searchIndexer = new GraphBackedSearchIndexer(graphProvider); searchIndexer = new GraphBackedSearchIndexer(graphProvider);
Collection<IDataType> typesAdded = TestUtils.createHiveTypes(typeSystem);
typeSystem = TypeSystem.getInstance(); searchIndexer.onAdd(typesAdded);
createHiveTypes();
} }
@AfterClass @AfterClass
...@@ -137,36 +121,36 @@ public class GraphRepoMapperScaleTest { ...@@ -137,36 +121,36 @@ public class GraphRepoMapperScaleTest {
@Test @Test
public void testSubmitEntity() throws Exception { public void testSubmitEntity() throws Exception {
Referenceable databaseInstance = new Referenceable(DATABASE_TYPE); Referenceable databaseInstance = new Referenceable(TestUtils.DATABASE_TYPE);
databaseInstance.set("name", DATABASE_NAME); databaseInstance.set("name", DATABASE_NAME);
databaseInstance.set("description", "foo database"); databaseInstance.set("description", "foo database");
// System.out.println("databaseInstance = " + databaseInstance); // System.out.println("databaseInstance = " + databaseInstance);
ClassType dbType = typeSystem.getDataType(ClassType.class, DATABASE_TYPE); ClassType dbType = typeSystem.getDataType(ClassType.class, TestUtils.DATABASE_TYPE);
ITypedReferenceableInstance db = dbType.convert(databaseInstance, Multiplicity.REQUIRED); ITypedReferenceableInstance db = dbType.convert(databaseInstance, Multiplicity.REQUIRED);
dbGUID = repositoryService.createEntity(db); dbGUID = repositoryService.createEntities(db)[0];
Referenceable dbInstance = new Referenceable(dbGUID, DATABASE_TYPE, databaseInstance.getValuesMap()); Referenceable dbInstance = new Referenceable(dbGUID, TestUtils.DATABASE_TYPE, databaseInstance.getValuesMap());
for (int index = 0; index < 1000; index++) { for (int index = 0; index < 1000; index++) {
ITypedReferenceableInstance table = createHiveTableInstance(dbInstance, index); ITypedReferenceableInstance table = createHiveTableInstance(dbInstance, index);
repositoryService.createEntity(table); repositoryService.createEntities(table);
} }
} }
@Test(dependsOnMethods = "testSubmitEntity") @Test(dependsOnMethods = "testSubmitEntity")
public void testSearchIndex() throws Exception { public void testSearchIndex() throws Exception {
searchWithOutIndex(Constants.GUID_PROPERTY_KEY, dbGUID); searchWithOutIndex(Constants.GUID_PROPERTY_KEY, dbGUID);
searchWithOutIndex(Constants.ENTITY_TYPE_PROPERTY_KEY, "hive_column_type"); searchWithOutIndex(Constants.ENTITY_TYPE_PROPERTY_KEY, "column_type");
searchWithOutIndex(Constants.ENTITY_TYPE_PROPERTY_KEY, TABLE_TYPE); searchWithOutIndex(Constants.ENTITY_TYPE_PROPERTY_KEY, TestUtils.TABLE_TYPE);
searchWithOutIndex("hive_table_type.name", "bar-999"); searchWithOutIndex("hive_table.name", "bar-999");
searchWithIndex("hive_table_type.name", "bar-999"); searchWithIndex("hive_table.name", "bar-999");
searchWithIndex("hive_table_type.created", Compare.GREATER_THAN_EQUAL, BaseTest.TEST_DATE_IN_LONG); searchWithIndex("hive_table.created", Compare.GREATER_THAN_EQUAL, TestUtils.TEST_DATE_IN_LONG);
for (int index = 500; index < 600; index++) { for (int index = 500; index < 600; index++) {
searchWithIndex("hive_table_type.name", "bar-" + index); searchWithIndex("hive_table.name", "bar-" + index);
} }
} }
...@@ -216,76 +200,14 @@ public class GraphRepoMapperScaleTest { ...@@ -216,76 +200,14 @@ public class GraphRepoMapperScaleTest {
} }
} }
private void createHiveTypes() throws Exception {
HierarchicalTypeDefinition<ClassType> databaseTypeDefinition = TypesUtil
.createClassTypeDef(DATABASE_TYPE, ImmutableList.<String>of(),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE));
StructTypeDefinition structTypeDefinition = new StructTypeDefinition("hive_serde_type",
new AttributeDefinition[]{TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("serde", DataTypes.STRING_TYPE)});
EnumValue values[] = {new EnumValue("MANAGED", 1), new EnumValue("EXTERNAL", 2),};
EnumTypeDefinition enumTypeDefinition = new EnumTypeDefinition("table_type", values);
final EnumType enumType = typeSystem.defineEnumType(enumTypeDefinition);
HierarchicalTypeDefinition<ClassType> columnsDefinition = TypesUtil
.createClassTypeDef("hive_column_type", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("type", DataTypes.STRING_TYPE));
StructTypeDefinition partitionDefinition = new StructTypeDefinition("hive_partition_type",
new AttributeDefinition[]{TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),});
HierarchicalTypeDefinition<ClassType> tableTypeDefinition = TypesUtil
.createClassTypeDef(TABLE_TYPE, ImmutableList.<String>of(),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("type", DataTypes.STRING_TYPE),
TypesUtil.createOptionalAttrDef("created", DataTypes.DATE_TYPE),
// enum
new AttributeDefinition("tableType", "table_type", Multiplicity.REQUIRED, false, null),
// array of strings
new AttributeDefinition("columnNames",
String.format("array<%s>", DataTypes.STRING_TYPE.getName()), Multiplicity.COLLECTION,
false, null),
// array of classes
new AttributeDefinition("columns", String.format("array<%s>", "hive_column_type"),
Multiplicity.COLLECTION, true, null),
// array of structs
new AttributeDefinition("partitions", String.format("array<%s>", "hive_partition_type"),
Multiplicity.COLLECTION, true, null),
// struct reference
new AttributeDefinition("serde1", "hive_serde_type", Multiplicity.REQUIRED, false, null),
new AttributeDefinition("serde2", "hive_serde_type", Multiplicity.REQUIRED, false, null),
// class reference
new AttributeDefinition("database", DATABASE_TYPE, Multiplicity.REQUIRED, true, null));
HierarchicalTypeDefinition<TraitType> classificationTypeDefinition =
TypesUtil.createTraitTypeDef("pii_type", ImmutableList.<String>of());
Map<String, IDataType> types = typeSystem
.defineTypes(ImmutableList.of(structTypeDefinition, partitionDefinition),
ImmutableList.of(classificationTypeDefinition),
ImmutableList.of(databaseTypeDefinition, columnsDefinition, tableTypeDefinition));
ArrayList<IDataType> typesAdded = new ArrayList<IDataType>();
typesAdded.add(enumType);
typesAdded.addAll(types.values());
searchIndexer.onAdd(typesAdded);
}
private ITypedReferenceableInstance createHiveTableInstance(Referenceable databaseInstance, int uberIndex) private ITypedReferenceableInstance createHiveTableInstance(Referenceable databaseInstance, int uberIndex)
throws Exception { throws Exception {
Referenceable tableInstance = new Referenceable(TABLE_TYPE, "pii_type"); Referenceable tableInstance = new Referenceable(TestUtils.TABLE_TYPE);
tableInstance.set("name", TABLE_NAME + "-" + uberIndex); tableInstance.set("name", TABLE_NAME + "-" + uberIndex);
tableInstance.set("description", "bar table" + "-" + uberIndex); tableInstance.set("description", "bar table" + "-" + uberIndex);
tableInstance.set("type", "managed"); tableInstance.set("type", "managed");
tableInstance.set("created", new Date(BaseTest.TEST_DATE_IN_LONG)); tableInstance.set("created", new Date(TestUtils.TEST_DATE_IN_LONG));
tableInstance.set("tableType", 1); // enum tableInstance.set("tableType", 1); // enum
// refer to an existing class // refer to an existing class
...@@ -296,19 +218,19 @@ public class GraphRepoMapperScaleTest { ...@@ -296,19 +218,19 @@ public class GraphRepoMapperScaleTest {
columnNames.add("last_name" + "-" + uberIndex); columnNames.add("last_name" + "-" + uberIndex);
tableInstance.set("columnNames", columnNames); tableInstance.set("columnNames", columnNames);
Struct serde1Instance = new Struct("hive_serde_type"); Struct serde1Instance = new Struct("serdeType");
serde1Instance.set("name", "serde1" + "-" + uberIndex); serde1Instance.set("name", "serde1" + "-" + uberIndex);
serde1Instance.set("serde", "serde1" + "-" + uberIndex); serde1Instance.set("serde", "serde1" + "-" + uberIndex);
tableInstance.set("serde1", serde1Instance); tableInstance.set("serde1", serde1Instance);
Struct serde2Instance = new Struct("hive_serde_type"); Struct serde2Instance = new Struct("serdeType");
serde2Instance.set("name", "serde2" + "-" + uberIndex); serde2Instance.set("name", "serde2" + "-" + uberIndex);
serde2Instance.set("serde", "serde2" + "-" + uberIndex); serde2Instance.set("serde", "serde2" + "-" + uberIndex);
tableInstance.set("serde2", serde2Instance); tableInstance.set("serde2", serde2Instance);
ArrayList<Referenceable> columns = new ArrayList<>(); ArrayList<Referenceable> columns = new ArrayList<>();
for (int index = 0; index < 5; index++) { for (int index = 0; index < 5; index++) {
Referenceable columnInstance = new Referenceable("hive_column_type"); Referenceable columnInstance = new Referenceable("column_type");
columnInstance.set("name", "column_" + "-" + uberIndex + "-" + index); columnInstance.set("name", "column_" + "-" + uberIndex + "-" + index);
columnInstance.set("type", "string"); columnInstance.set("type", "string");
columns.add(columnInstance); columns.add(columnInstance);
...@@ -317,13 +239,13 @@ public class GraphRepoMapperScaleTest { ...@@ -317,13 +239,13 @@ public class GraphRepoMapperScaleTest {
ArrayList<Struct> partitions = new ArrayList<>(); ArrayList<Struct> partitions = new ArrayList<>();
for (int index = 0; index < 5; index++) { for (int index = 0; index < 5; index++) {
Struct partitionInstance = new Struct("hive_partition_type"); Struct partitionInstance = new Struct("partition_type");
partitionInstance.set("name", "partition_" + "-" + uberIndex + "-" + index); partitionInstance.set("name", "partition_" + "-" + uberIndex + "-" + index);
partitions.add(partitionInstance); partitions.add(partitionInstance);
} }
tableInstance.set("partitions", partitions); tableInstance.set("partitions", partitions);
ClassType tableType = typeSystem.getDataType(ClassType.class, TABLE_TYPE); ClassType tableType = typeSystem.getDataType(ClassType.class, TestUtils.TABLE_TYPE);
return tableType.convert(tableInstance, Multiplicity.REQUIRED); return tableType.convert(tableInstance, Multiplicity.REQUIRED);
} }
} }
......
/**
* 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.memory;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.BaseTest;
import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.TypeSystem;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class ClassTest extends BaseTest {
@Before
public void setup() throws Exception {
super.setup();
}
@Test
public void test1() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineDeptEmployeeTypes(ts);
Referenceable hrDept = createDeptEg1(ts);
ClassType deptType = ts.getDataType(ClassType.class, "Department");
ITypedReferenceableInstance hrDept2 = deptType.convert(hrDept, Multiplicity.REQUIRED);
Assert.assertEquals(hrDept2.toString(), "{\n" +
"\tid : (type: Department, id: <unassigned>)\n" +
"\tname : \thr\n" +
"\temployees : \t[{\n" +
"\tid : (type: Person, id: <unassigned>)\n" +
"\tname : \tJohn\n" +
"\tdepartment : (type: Department, id: <unassigned>)\n" +
"\tmanager : (type: Manager, id: <unassigned>)\n" +
"}, {\n" +
"\tid : (type: Manager, id: <unassigned>)\n" +
"\tsubordinates : \t[{\n" +
"\tid : (type: Person, id: <unassigned>)\n" +
"\tname : \tJohn\n" +
"\tdepartment : (type: Department, id: <unassigned>)\n" +
"\tmanager : (type: Manager, id: <unassigned>)\n" +
"}]\n" +
"\tname : \tJane\n" +
"\tdepartment : (type: Department, id: <unassigned>)\n" +
"\tmanager : <null>\n" +
"\n" +
"\tSecurityClearance : \t{\n" +
"\t\tlevel : \t\t1\n" +
"\t}}]\n" +
"}");
}
}
/**
* 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.memory;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.BaseTest;
import org.apache.atlas.typesystem.IReferenceableInstance;
import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.ITypedStruct;
import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.json.Serialization$;
import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumType;
import org.apache.atlas.typesystem.types.EnumValue;
import org.apache.atlas.typesystem.types.HierarchicalType;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructType;
import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.Date;
import java.util.Map;
public class EnumTest extends BaseTest {
@Before
public void setup() throws Exception {
super.setup();
}
void defineEnums(TypeSystem ts) throws AtlasException {
ts.defineEnumType("HiveObjectType", new EnumValue("GLOBAL", 1), new EnumValue("DATABASE", 2),
new EnumValue("TABLE", 3), new EnumValue("PARTITION", 4), new EnumValue("COLUMN", 5));
ts.defineEnumType("PrincipalType", new EnumValue("USER", 1), new EnumValue("ROLE", 2),
new EnumValue("GROUP", 3));
ts.defineEnumType("TxnState", new EnumValue("COMMITTED", 1), new EnumValue("ABORTED", 2),
new EnumValue("OPEN", 3));
ts.defineEnumType("LockLevel", new EnumValue("DB", 1), new EnumValue("TABLE", 2),
new EnumValue("PARTITION", 3));
}
protected void fillStruct(Struct s) throws AtlasException {
s.set("a", 1);
s.set("b", true);
s.set("c", (byte) 1);
s.set("d", (short) 2);
s.set("e", 1);
s.set("f", 1);
s.set("g", 1L);
s.set("h", 1.0f);
s.set("i", 1.0);
s.set("j", BigInteger.valueOf(1L));
s.set("k", new BigDecimal(1));
s.set("l", new Date(1418265358440L));
s.set("m", Lists.asList(1, new Integer[]{1}));
s.set("n", Lists.asList(BigDecimal.valueOf(1.1), new BigDecimal[]{BigDecimal.valueOf(1.1)}));
Map<String, Double> hm = Maps.newHashMap();
hm.put("a", 1.0);
hm.put("b", 2.0);
s.set("o", hm);
s.set("enum1", "GLOBAL");
s.set("enum2", 1);
s.set("enum3", "COMMITTED");
s.set("enum4", 3);
}
protected Struct createStructWithEnum(String typeName) throws AtlasException {
Struct s = new Struct(typeName);
fillStruct(s);
return s;
}
protected Referenceable createInstanceWithEnum(String typeName) throws AtlasException {
Referenceable r = new Referenceable(typeName);
fillStruct(r);
return r;
}
protected ClassType defineClassTypeWithEnum(TypeSystem ts) throws AtlasException {
return ts.defineClassType(TypesUtil.createClassTypeDef("t4", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("b", DataTypes.BOOLEAN_TYPE),
TypesUtil.createOptionalAttrDef("c", DataTypes.BYTE_TYPE),
TypesUtil.createOptionalAttrDef("d", DataTypes.SHORT_TYPE),
TypesUtil.createOptionalAttrDef("enum1", ts.getDataType(EnumType.class, "HiveObjectType")),
TypesUtil.createOptionalAttrDef("e", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("f", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("g", DataTypes.LONG_TYPE),
TypesUtil.createOptionalAttrDef("enum2", ts.getDataType(EnumType.class, "PrincipalType")),
TypesUtil.createOptionalAttrDef("h", DataTypes.FLOAT_TYPE),
TypesUtil.createOptionalAttrDef("i", DataTypes.DOUBLE_TYPE),
TypesUtil.createOptionalAttrDef("j", DataTypes.BIGINTEGER_TYPE),
TypesUtil.createOptionalAttrDef("k", DataTypes.BIGDECIMAL_TYPE),
TypesUtil.createOptionalAttrDef("enum3", ts.getDataType(EnumType.class, "TxnState")),
TypesUtil.createOptionalAttrDef("l", DataTypes.DATE_TYPE),
TypesUtil.createOptionalAttrDef("m", ts.defineArrayType(DataTypes.INT_TYPE)),
TypesUtil.createOptionalAttrDef("n", ts.defineArrayType(DataTypes.BIGDECIMAL_TYPE)),
TypesUtil.createOptionalAttrDef("o", ts.defineMapType(DataTypes.STRING_TYPE, DataTypes.DOUBLE_TYPE)),
TypesUtil.createOptionalAttrDef("enum4", ts.getDataType(EnumType.class, "LockLevel"))));
}
@Test
public void testStruct() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineEnums(ts);
StructType structType =
ts.defineStructType("t3", true, TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("b", DataTypes.BOOLEAN_TYPE),
TypesUtil.createOptionalAttrDef("c", DataTypes.BYTE_TYPE),
TypesUtil.createOptionalAttrDef("d", DataTypes.SHORT_TYPE),
TypesUtil.createOptionalAttrDef("enum1", ts.getDataType(EnumType.class, "HiveObjectType")),
TypesUtil.createOptionalAttrDef("e", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("f", DataTypes.INT_TYPE),
TypesUtil.createOptionalAttrDef("g", DataTypes.LONG_TYPE),
TypesUtil.createOptionalAttrDef("enum2", ts.getDataType(EnumType.class, "PrincipalType")),
TypesUtil.createOptionalAttrDef("h", DataTypes.FLOAT_TYPE),
TypesUtil.createOptionalAttrDef("i", DataTypes.DOUBLE_TYPE),
TypesUtil.createOptionalAttrDef("j", DataTypes.BIGINTEGER_TYPE),
TypesUtil.createOptionalAttrDef("k", DataTypes.BIGDECIMAL_TYPE),
TypesUtil.createOptionalAttrDef("enum3", ts.getDataType(EnumType.class, "TxnState")),
TypesUtil.createOptionalAttrDef("l", DataTypes.DATE_TYPE),
TypesUtil.createOptionalAttrDef("m", ts.defineArrayType(DataTypes.INT_TYPE)),
TypesUtil.createOptionalAttrDef("n", ts.defineArrayType(DataTypes.BIGDECIMAL_TYPE)), TypesUtil
.createOptionalAttrDef("o",
ts.defineMapType(DataTypes.STRING_TYPE, DataTypes.DOUBLE_TYPE)),
TypesUtil.createOptionalAttrDef("enum4", ts.getDataType(EnumType.class, "LockLevel")));
Struct s = createStructWithEnum("t3");
ITypedStruct typedS = structType.convert(s, Multiplicity.REQUIRED);
Assert.assertEquals(typedS.toString(), "{\n" +
"\ta : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\td : \t2\n" +
"\tenum1 : \tGLOBAL\n" +
"\te : \t1\n" +
"\tf : \t1\n" +
"\tg : \t1\n" +
"\tenum2 : \tUSER\n" +
"\th : \t1.0\n" +
"\ti : \t1.0\n" +
"\tj : \t1\n" +
"\tk : \t1\n" +
"\tenum3 : \tCOMMITTED\n" +
"\tl : \t" + TEST_DATE + "\n" +
"\tm : \t[1, 1]\n" +
"\tn : \t[1.1, 1.1]\n" +
"\to : \t{a=1.0, b=2.0}\n" +
"\tenum4 : \tPARTITION\n" +
"}");
}
@Test
public void testClass() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineEnums(ts);
ClassType clsType = defineClassTypeWithEnum(ts);
IReferenceableInstance r = createInstanceWithEnum("t4");
ITypedReferenceableInstance typedR = clsType.convert(r, Multiplicity.REQUIRED);
Assert.assertEquals(typedR.toString(), "{\n" +
"\tid : (type: t4, id: <unassigned>)\n" +
"\ta : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\td : \t2\n" +
"\tenum1 : \tGLOBAL\n" +
"\te : \t1\n" +
"\tf : \t1\n" +
"\tg : \t1\n" +
"\tenum2 : \tUSER\n" +
"\th : \t1.0\n" +
"\ti : \t1.0\n" +
"\tj : \t1\n" +
"\tk : \t1\n" +
"\tenum3 : \tCOMMITTED\n" +
"\tl : \t" + TEST_DATE + "\n" +
"\tm : \t[1, 1]\n" +
"\tn : \t[1.1, 1.1]\n" +
"\to : \t{a=1.0, b=2.0}\n" +
"\tenum4 : \tPARTITION\n" +
"}");
}
@Test
public void testStorage() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineEnums(ts);
ClassType clsType = defineClassTypeWithEnum(ts);
getRepository().defineTypes(ImmutableList.of((HierarchicalType) clsType));
IReferenceableInstance r = createInstanceWithEnum("t4");
IReferenceableInstance r1 = getRepository().create(r);
ITypedReferenceableInstance r2 = getRepository().get(r1.getId());
Assert.assertEquals(r2.toString(), "{\n" +
"\tid : (type: t4, id: 1)\n" +
"\ta : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\td : \t0\n" +
"\tenum1 : \tGLOBAL\n" +
"\te : \t1\n" +
"\tf : \t1\n" +
"\tg : \t1\n" +
"\tenum2 : \tUSER\n" +
"\th : \t1.0\n" +
"\ti : \t1.0\n" +
"\tj : \t1\n" +
"\tk : \t1\n" +
"\tenum3 : \tCOMMITTED\n" +
"\tl : \t" + TEST_DATE + "\n" +
"\tm : \t[1, 1]\n" +
"\tn : \t[1.1, 1.1]\n" +
"\to : \t{a=1.0, b=2.0}\n" +
"\tenum4 : \tPARTITION\n" +
"}");
}
@Test
public void testJson() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineEnums(ts);
ClassType clsType = defineClassTypeWithEnum(ts);
getRepository().defineTypes(ImmutableList.of((HierarchicalType) clsType));
IReferenceableInstance r = createInstanceWithEnum("t4");
IReferenceableInstance r1 = getRepository().create(r);
ITypedReferenceableInstance r2 = getRepository().get(r1.getId());
String jsonStr = Serialization$.MODULE$.toJson(r2);
IReferenceableInstance r3 = Serialization$.MODULE$.fromJson(jsonStr);
Assert.assertEquals(r3.toString(), "{\n" +
"\tid : (type: t4, id: 1)\n" +
"\ta : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\td : \t0\n" +
"\tenum1 : \tGLOBAL\n" +
"\te : \t1\n" +
"\tf : \t1\n" +
"\tg : \t1\n" +
"\tenum2 : \tUSER\n" +
"\th : \t1.0\n" +
"\ti : \t1.0\n" +
"\tj : \t1\n" +
"\tk : \t1\n" +
"\tenum3 : \tCOMMITTED\n" +
"\tl : \t" + TEST_DATE + "\n" +
"\tm : \t[1, 1]\n" +
"\tn : \t[1.100000000000000088817841970012523233890533447265625, 1" +
".100000000000000088817841970012523233890533447265625]\n" +
"\to : \t{a=1.0, b=2.0}\n" +
"\tenum4 : \tPARTITION\n" +
"}");
}
}
/**
* 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.memory;
import com.google.common.collect.ImmutableList;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.BaseTest;
import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.TypesDef;
import org.apache.atlas.typesystem.json.InstanceSerialization$;
import org.apache.atlas.typesystem.json.Serialization$;
import org.apache.atlas.typesystem.json.TypesSerialization$;
import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
public class InstanceE2ETest extends BaseTest {
protected List<HierarchicalTypeDefinition> createHiveTypes(TypeSystem typeSystem) throws AtlasException {
ArrayList<HierarchicalTypeDefinition> typeDefinitions = new ArrayList<>();
HierarchicalTypeDefinition<ClassType> databaseTypeDefinition = TypesUtil
.createClassTypeDef("hive_database", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE));
typeDefinitions.add(databaseTypeDefinition);
HierarchicalTypeDefinition<ClassType> tableTypeDefinition = TypesUtil
.createClassTypeDef("hive_table", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("type", DataTypes.STRING_TYPE),
new AttributeDefinition("hive_database", "hive_database", Multiplicity.REQUIRED, false,
"hive_database"));
typeDefinitions.add(tableTypeDefinition);
HierarchicalTypeDefinition<TraitType> fetlTypeDefinition = TypesUtil
.createTraitTypeDef("hive_fetl", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE));
typeDefinitions.add(fetlTypeDefinition);
typeSystem.defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList.of(fetlTypeDefinition),
ImmutableList.of(databaseTypeDefinition, tableTypeDefinition));
return typeDefinitions;
}
protected Referenceable createHiveTableReferenceable() throws AtlasException {
Referenceable databaseInstance = new Referenceable("hive_database");
databaseInstance.set("name", "hive_database");
databaseInstance.set("description", "foo database");
Referenceable tableInstance = new Referenceable("hive_table", "hive_fetl");
tableInstance.set("name", "t1");
tableInstance.set("description", "bar table");
tableInstance.set("type", "managed");
tableInstance.set("hive_database", databaseInstance);
Struct traitInstance = (Struct) tableInstance.getTrait("hive_fetl");
traitInstance.set("level", 1);
tableInstance.set("hive_fetl", traitInstance);
return tableInstance;
}
protected ITypedReferenceableInstance createHiveTableInstance(TypeSystem typeSystem) throws AtlasException {
ClassType tableType = typeSystem.getDataType(ClassType.class, "hive_table");
return tableType.convert(createHiveTableReferenceable(), Multiplicity.REQUIRED);
}
@Test
public void testType() throws AtlasException {
TypeSystem ts = getTypeSystem();
createHiveTypes(ts);
String jsonStr = TypesSerialization$.MODULE$.toJson(ts, ImmutableList.of("hive_database", "hive_table"));
System.out.println(jsonStr);
TypesDef typesDef1 = TypesSerialization$.MODULE$.fromJson(jsonStr);
System.out.println(typesDef1);
ts.reset();
ts.defineTypes(typesDef1);
jsonStr = TypesSerialization$.MODULE$.toJson(ts, ImmutableList.of("hive_database", "hive_table"));
System.out.println(jsonStr);
}
@Test
public void testInstance() throws AtlasException {
TypeSystem ts = getTypeSystem();
createHiveTypes(ts);
ITypedReferenceableInstance i = createHiveTableInstance(getTypeSystem());
String jsonStr = Serialization$.MODULE$.toJson(i);
System.out.println(jsonStr);
i = Serialization$.MODULE$.fromJson(jsonStr);
System.out.println(i);
}
@Test
public void testInstanceSerialization() throws AtlasException {
TypeSystem ts = getTypeSystem();
createHiveTypes(ts);
Referenceable r = createHiveTableReferenceable();
String jsonStr = InstanceSerialization$.MODULE$.toJson(r, true);
Referenceable r1 = InstanceSerialization$.MODULE$.fromJsonReferenceable(jsonStr, true);
ClassType tableType = ts.getDataType(ClassType.class, "hive_table");
ITypedReferenceableInstance i = tableType.convert(r1, Multiplicity.REQUIRED);
jsonStr = Serialization$.MODULE$.toJson(i);
System.out.println(jsonStr);
i = Serialization$.MODULE$.fromJson(jsonStr);
System.out.println(i);
}
}
/**
* 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.memory;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.BaseTest;
import org.apache.atlas.repository.RepositoryException;
import org.apache.atlas.typesystem.ITypedReferenceableInstance;
import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.persistence.Id;
import org.apache.atlas.typesystem.types.TypeSystem;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class StorageTest extends BaseTest {
@Before
public void setup() throws Exception {
super.setup();
}
@Test
public void test1() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineDeptEmployeeTypes(ts);
Referenceable hrDept = createDeptEg1(ts);
ITypedReferenceableInstance hrDept2 = getRepository().create(hrDept);
ITypedReferenceableInstance hrDept3 = getRepository().get(hrDept2.getId());
Assert.assertEquals(hrDept3.toString(), "{\n" +
"\tid : (type: Department, id: 1)\n" +
"\tname : \thr\n" +
"\temployees : \t[{\n" +
"\tid : (type: Person, id: 2)\n" +
"\tname : \tJohn\n" +
"\tdepartment : (type: Department, id: 1)\n" +
"\tmanager : (type: Manager, id: 3)\n" +
"}, {\n" +
"\tid : (type: Manager, id: 3)\n" +
"\tsubordinates : \t[(type: Person, id: 2)]\n" +
"\tname : \tJane\n" +
"\tdepartment : (type: Department, id: 1)\n" +
"\tmanager : <null>\n" +
"\n" +
"\tSecurityClearance : \t{\n" +
"\t\tlevel : \t\t1\n" +
"\t}}]\n" +
"}");
}
@Test
public void testGetPerson() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineDeptEmployeeTypes(ts);
Referenceable hrDept = createDeptEg1(ts);
ITypedReferenceableInstance hrDept2 = getRepository().create(hrDept);
Id e1Id = new Id(2, 0, "Person");
ITypedReferenceableInstance e1 = getRepository().get(e1Id);
Assert.assertEquals(e1.toString(), "{\n" +
"\tid : (type: Person, id: 2)\n" +
"\tname : \tJohn\n" +
"\tdepartment : (type: Department, id: 1)\n" +
"\tmanager : (type: Manager, id: 3)\n" +
"}");
}
@Test
public void testInvalidTypeName() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineDeptEmployeeTypes(ts);
Referenceable hrDept = createDeptEg1(ts);
ITypedReferenceableInstance hrDept2 = getRepository().create(hrDept);
Id e1Id = new Id(3, 0, "Person");
try {
ITypedReferenceableInstance e1 = getRepository().get(e1Id);
} catch (RepositoryException re) {
RepositoryException me = (RepositoryException) re.getCause();
Assert.assertEquals(me.getMessage(), "Invalid Id (unknown) : (type: Person, id: 3)");
}
}
@Test
public void testGetManager() throws AtlasException {
TypeSystem ts = getTypeSystem();
defineDeptEmployeeTypes(ts);
Referenceable hrDept = createDeptEg1(ts);
ITypedReferenceableInstance hrDept2 = getRepository().create(hrDept);
Id m1Id = new Id(3, 0, "Manager");
ITypedReferenceableInstance m1 = getRepository().get(m1Id);
Assert.assertEquals(m1.toString(), "{\n" +
"\tid : (type: Manager, id: 3)\n" +
"\tsubordinates : \t[(type: Person, id: 2)]\n" +
"\tname : \tJane\n" +
"\tdepartment : (type: Department, id: 1)\n" +
"\tmanager : <null>\n" +
"\n" +
"\tSecurityClearance : \t{\n" +
"\t\tlevel : \t\t1\n" +
"\t}}");
}
}
\ No newline at end of file
/**
* 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.memory;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.BaseTest;
import org.apache.atlas.typesystem.ITypedStruct;
import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.json.InstanceSerialization$;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructType;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class StructTest extends BaseTest {
StructType structType;
StructType recursiveStructType;
@Before
public void setup() throws Exception {
super.setup();
structType = (StructType) getTypeSystem().getDataType(StructType.class, STRUCT_TYPE_1);
recursiveStructType = (StructType) getTypeSystem().getDataType(StructType.class, STRUCT_TYPE_2);
}
@Test
public void test1() throws AtlasException {
Struct s = createStruct();
ITypedStruct ts = structType.convert(s, Multiplicity.REQUIRED);
Assert.assertEquals(ts.toString(), "{\n" +
"\ta : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\td : \t2\n" +
"\te : \t1\n" +
"\tf : \t1\n" +
"\tg : \t1\n" +
"\th : \t1.0\n" +
"\ti : \t1.0\n" +
"\tj : \t1\n" +
"\tk : \t1\n" +
"\tl : \t" + TEST_DATE + "\n" +
"\tm : \t[1, 1]\n" +
"\tn : \t[1.1, 1.1]\n" +
"\to : \t{a=1.0, b=2.0}\n" +
"}");
}
@Test
public void testRecursive() throws AtlasException {
Struct s1 = new Struct(recursiveStructType.getName());
s1.set("a", 1);
Struct s2 = new Struct(recursiveStructType.getName());
s2.set("a", 1);
s2.set("s", s1);
ITypedStruct ts = recursiveStructType.convert(s2, Multiplicity.REQUIRED);
Assert.assertEquals(ts.toString(), "{\n" +
"\ta : \t1\n" +
"\ts : \t{\n" +
"\t\ta : \t\t1\n" +
"\t\ts : <null>\n" +
"\n" +
"\t}\n" +
"}");
}
@Test
public void testSerialization() throws AtlasException {
Struct s = createStruct();
String jsonStr = InstanceSerialization$.MODULE$.toJson(s, true);
Struct s1 = InstanceSerialization$.MODULE$.fromJsonStruct(jsonStr, true);
ITypedStruct ts = structType.convert(s1, Multiplicity.REQUIRED);
Assert.assertEquals(ts.toString(), "{\n" +
"\ta : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\td : \t2\n" +
"\te : \t1\n" +
"\tf : \t1\n" +
"\tg : \t1\n" +
"\th : \t1.0\n" +
"\ti : \t1.0\n" +
"\tj : \t1\n" +
"\tk : \t1\n" +
"\tl : \t" + TEST_DATE + "\n" +
"\tm : \t[1, 1]\n" +
"\tn : \t[1.100000000000000088817841970012523233890533447265625, 1" +
".100000000000000088817841970012523233890533447265625]\n" +
"\to : \t{a=1.0, b=2.0}\n" +
"}");
}
}
/**
* 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.memory;
import com.google.common.collect.ImmutableList;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.BaseTest;
import org.apache.atlas.typesystem.IStruct;
import org.apache.atlas.typesystem.ITypedStruct;
import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.TraitType;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createRequiredAttrDef;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createTraitTypeDef;
public class TraitTest extends BaseTest {
@Before
public void setup() throws Exception {
super.setup();
}
/*
* Type Hierarchy is:
* A(a,b,c,d)
* B(b) extends A
* C(c) extends A
* D(d) extends B,C
*
* - There are a total of 11 fields in an instance of D
* - an attribute that is hidden by a SubType can referenced by prefixing it with the
* complete Path.
* For e.g. the 'b' attribute in A (that is a superType for B) is hidden the 'b' attribute
* in B.
* So it is availabel by the name 'A.B.D.b'
*
* - Another way to set attributes is to cast. Casting a 'D' instance of 'B' makes the 'A.B.D
* .b' attribute
* available as 'A.B.b'. Casting one more time to an 'A' makes the 'A.B.b' attribute
* available as 'b'.
*/
@Test
public void test1() throws AtlasException {
HierarchicalTypeDefinition A = createTraitTypeDef("A", null, createRequiredAttrDef("a", DataTypes.INT_TYPE),
createOptionalAttrDef("b", DataTypes.BOOLEAN_TYPE), createOptionalAttrDef("c", DataTypes.BYTE_TYPE),
createOptionalAttrDef("d", DataTypes.SHORT_TYPE));
HierarchicalTypeDefinition B = createTraitTypeDef("B", ImmutableList.<String>of("A"),
createOptionalAttrDef("b", DataTypes.BOOLEAN_TYPE));
HierarchicalTypeDefinition C =
createTraitTypeDef("C", ImmutableList.<String>of("A"), createOptionalAttrDef("c", DataTypes.BYTE_TYPE));
HierarchicalTypeDefinition D = createTraitTypeDef("D", ImmutableList.<String>of("B", "C"),
createOptionalAttrDef("d", DataTypes.SHORT_TYPE));
defineTraits(A, B, C, D);
TraitType DType = (TraitType) getTypeSystem().getDataType(TraitType.class, "D");
Struct s1 = new Struct("D");
s1.set("d", 1);
s1.set("c", 1);
s1.set("b", true);
s1.set("a", 1);
s1.set("A.B.D.b", true);
s1.set("A.B.D.c", 2);
s1.set("A.B.D.d", 2);
s1.set("A.C.D.a", 3);
s1.set("A.C.D.b", false);
s1.set("A.C.D.c", 3);
s1.set("A.C.D.d", 3);
ITypedStruct ts = DType.convert(s1, Multiplicity.REQUIRED);
Assert.assertEquals(ts.toString(), "{\n" +
"\td : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\ta : \t1\n" +
"\tA.B.D.b : \ttrue\n" +
"\tA.B.D.c : \t2\n" +
"\tA.B.D.d : \t2\n" +
"\tA.C.D.a : \t3\n" +
"\tA.C.D.b : \tfalse\n" +
"\tA.C.D.c : \t3\n" +
"\tA.C.D.d : \t3\n" +
"}");
/*
* cast to B and set the 'b' attribute on A.
*/
TraitType BType = (TraitType) getTypeSystem().getDataType(TraitType.class, "B");
IStruct s2 = DType.castAs(ts, "B");
s2.set("A.B.b", false);
Assert.assertEquals(ts.toString(), "{\n" +
"\td : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\ta : \t1\n" +
"\tA.B.D.b : \tfalse\n" +
"\tA.B.D.c : \t2\n" +
"\tA.B.D.d : \t2\n" +
"\tA.C.D.a : \t3\n" +
"\tA.C.D.b : \tfalse\n" +
"\tA.C.D.c : \t3\n" +
"\tA.C.D.d : \t3\n" +
"}");
/*
* cast again to A and set the 'b' attribute on A.
*/
TraitType AType = (TraitType) getTypeSystem().getDataType(TraitType.class, "A");
IStruct s3 = BType.castAs(s2, "A");
s3.set("b", true);
Assert.assertEquals(ts.toString(), "{\n" +
"\td : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\ta : \t1\n" +
"\tA.B.D.b : \ttrue\n" +
"\tA.B.D.c : \t2\n" +
"\tA.B.D.d : \t2\n" +
"\tA.C.D.a : \t3\n" +
"\tA.C.D.b : \tfalse\n" +
"\tA.C.D.c : \t3\n" +
"\tA.C.D.d : \t3\n" +
"}");
}
@Test
public void testRandomOrder() throws AtlasException {
HierarchicalTypeDefinition A = createTraitTypeDef("A", null, createRequiredAttrDef("a", DataTypes.INT_TYPE),
createOptionalAttrDef("b", DataTypes.BOOLEAN_TYPE), createOptionalAttrDef("c", DataTypes.BYTE_TYPE),
createOptionalAttrDef("d", DataTypes.SHORT_TYPE));
HierarchicalTypeDefinition B = createTraitTypeDef("B", ImmutableList.<String>of("A"),
createOptionalAttrDef("b", DataTypes.BOOLEAN_TYPE));
HierarchicalTypeDefinition C =
createTraitTypeDef("C", ImmutableList.<String>of("A"), createOptionalAttrDef("c", DataTypes.BYTE_TYPE));
HierarchicalTypeDefinition D = createTraitTypeDef("D", ImmutableList.<String>of("B", "C"),
createOptionalAttrDef("d", DataTypes.SHORT_TYPE));
defineTraits(B, D, A, C);
TraitType DType = (TraitType) getTypeSystem().getDataType(TraitType.class, "D");
Struct s1 = new Struct("D");
s1.set("d", 1);
s1.set("c", 1);
s1.set("b", true);
s1.set("a", 1);
s1.set("A.B.D.b", true);
s1.set("A.B.D.c", 2);
s1.set("A.B.D.d", 2);
s1.set("A.C.D.a", 3);
s1.set("A.C.D.b", false);
s1.set("A.C.D.c", 3);
s1.set("A.C.D.d", 3);
ITypedStruct ts = DType.convert(s1, Multiplicity.REQUIRED);
Assert.assertEquals(ts.toString(), "{\n" +
"\td : \t1\n" +
"\tb : \ttrue\n" +
"\tc : \t1\n" +
"\ta : \t1\n" +
"\tA.B.D.b : \ttrue\n" +
"\tA.B.D.c : \t2\n" +
"\tA.B.D.d : \t2\n" +
"\tA.C.D.a : \t3\n" +
"\tA.C.D.b : \tfalse\n" +
"\tA.C.D.c : \t3\n" +
"\tA.C.D.d : \t3\n" +
"}");
}
}
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.service;
import com.google.inject.Inject;
import com.thinkaurelius.titan.core.TitanGraph;
import com.thinkaurelius.titan.core.util.TitanCleanup;
import org.apache.atlas.RepositoryMetadataModule;
import org.apache.atlas.TestUtils;
import org.apache.atlas.TypeNotFoundException;
import org.apache.atlas.repository.EntityNotFoundException;
import org.apache.atlas.repository.graph.GraphProvider;
import org.apache.atlas.services.MetadataService;
import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.TypesDef;
import org.apache.atlas.typesystem.json.InstanceSerialization;
import org.apache.atlas.typesystem.json.TypesSerialization;
import org.apache.commons.lang.RandomStringUtils;
import org.codehaus.jettison.json.JSONArray;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
@Guice(modules = RepositoryMetadataModule.class)
public class DefaultMetadataServiceTest {
@Inject
private MetadataService metadataService;
@Inject
private GraphProvider<TitanGraph> graphProvider;
@BeforeClass
public void setUp() throws Exception {
TypesDef typesDef = TestUtils.defineHiveTypes();
try {
metadataService.getTypeDefinition(TestUtils.TABLE_TYPE);
} catch (TypeNotFoundException e) {
metadataService.createType(TypesSerialization.toJson(typesDef));
}
}
@AfterClass
public void shudown() {
try {
//TODO - Fix failure during shutdown while using BDB
graphProvider.get().shutdown();
} catch(Exception e) {
e.printStackTrace();
}
try {
TitanCleanup.clear(graphProvider.get());
} catch(Exception e) {
e.printStackTrace();
}
}
private String createInstance(Referenceable entity) throws Exception {
String entityjson = InstanceSerialization.toJson(entity, true);
JSONArray entitiesJson = new JSONArray();
entitiesJson.put(entityjson);
String response = metadataService.createEntities(entitiesJson.toString());
return new JSONArray(response).getString(0);
}
private Referenceable createDBEntity() {
Referenceable entity = new Referenceable(TestUtils.DATABASE_TYPE);
String dbName = RandomStringUtils.randomAlphanumeric(10);
entity.set("name", dbName);
entity.set("description", "us db");
return entity;
}
@Test
public void testCreateEntityWithUniqueAttribute() throws Exception {
//name is the unique attribute
Referenceable entity = createDBEntity();
String id = createInstance(entity);
//using the same name should succeed, but not create another entity
String newId = createInstance(entity);
Assert.assertEquals(newId, id);
//Same entity, but different qualified name should succeed
entity.set("name", TestUtils.randomString());
newId = createInstance(entity);
Assert.assertNotEquals(newId, id);
}
@Test
public void testCreateEntityWithUniqueAttributeWithReference() throws Exception {
Referenceable db = createDBEntity();
String dbId = createInstance(db);
Referenceable table = new Referenceable(TestUtils.TABLE_TYPE);
table.set("name", TestUtils.randomString());
table.set("description", "random table");
table.set("type", "type");
table.set("tableType", "MANAGED");
table.set("database", db);
createInstance(table);
//table create should re-use the db instance created earlier
String tableDefinitionJson =
metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, "name", (String) table.get("name"));
Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
Referenceable actualDb = (Referenceable) tableDefinition.get("database");
Assert.assertEquals(actualDb.getId().id, dbId);
}
@Test
public void testGetEntityByUniqueAttribute() throws Exception {
Referenceable entity = createDBEntity();
createInstance(entity);
//get entity by valid qualified name
String entityJson = metadataService.getEntityDefinition(TestUtils.DATABASE_TYPE, "name",
(String) entity.get("name"));
Assert.assertNotNull(entityJson);
Referenceable referenceable = InstanceSerialization.fromJsonReferenceable(entityJson, true);
Assert.assertEquals(referenceable.get("name"), entity.get("name"));
//get entity by invalid qualified name
try {
metadataService.getEntityDefinition(TestUtils.DATABASE_TYPE, "name", "random");
Assert.fail("Expected EntityNotFoundException");
} catch (EntityNotFoundException e) {
//expected
}
//get entity by non-unique attribute
try {
metadataService.getEntityDefinition(TestUtils.DATABASE_TYPE, "description",
(String) entity.get("description"));
Assert.fail("Expected IllegalArgumentException");
} catch (IllegalArgumentException e) {
//expected
}
}
}
...@@ -19,14 +19,13 @@ ...@@ -19,14 +19,13 @@
package org.apache.atlas.query package org.apache.atlas.query
import java.io.File import java.io.File
import java.util.{Date, UUID}
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import java.util.{Date, UUID}
import javax.script.{Bindings, ScriptEngine, ScriptEngineManager} import javax.script.{Bindings, ScriptEngine, ScriptEngineManager}
import com.thinkaurelius.titan.core.TitanGraph import com.thinkaurelius.titan.core.TitanGraph
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.apache.atlas.repository.BaseTest import org.apache.atlas.TestUtils
import org.apache.atlas.typesystem.types.TypeSystem
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
...@@ -140,7 +139,7 @@ object HiveTitanSample { ...@@ -140,7 +139,7 @@ object HiveTitanSample {
created: Option[Date] = None) { created: Option[Date] = None) {
val createdDate : Date = created match { val createdDate : Date = created match {
case Some(x) => x case Some(x) => x
case None => new Date(BaseTest.TEST_DATE_IN_LONG) case None => new Date(TestUtils.TEST_DATE_IN_LONG)
} }
val sd = StorageDescriptor(inputFormat, outputFormat) val sd = StorageDescriptor(inputFormat, outputFormat)
val colDefs = columns map { c => val colDefs = columns map { c =>
......
...@@ -127,7 +127,8 @@ object QueryTestsUtils extends GraphUtils { ...@@ -127,7 +127,8 @@ object QueryTestsUtils extends GraphUtils {
def jdbcTraitDef = new HierarchicalTypeDefinition[TraitType](classOf[TraitType], "JdbcAccess", null, def jdbcTraitDef = new HierarchicalTypeDefinition[TraitType](classOf[TraitType], "JdbcAccess", null,
Array[AttributeDefinition]()) Array[AttributeDefinition]())
TypeSystem.getInstance().defineTypes(ImmutableList.of[StructTypeDefinition], TypeSystem.getInstance().defineTypes(ImmutableList.of[EnumTypeDefinition],
ImmutableList.of[StructTypeDefinition],
ImmutableList.of[HierarchicalTypeDefinition[TraitType]](dimTraitDef, piiTraitDef, ImmutableList.of[HierarchicalTypeDefinition[TraitType]](dimTraitDef, piiTraitDef,
metricTraitDef, etlTraitDef, jdbcTraitDef), metricTraitDef, etlTraitDef, jdbcTraitDef),
ImmutableList.of[HierarchicalTypeDefinition[ClassType]](dbClsDef, storageDescClsDef, columnClsDef, tblClsDef, ImmutableList.of[HierarchicalTypeDefinition[ClassType]](dbClsDef, storageDescClsDef, columnClsDef, tblClsDef,
......
...@@ -126,6 +126,11 @@ ...@@ -126,6 +126,11 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>commons-configuration</groupId>
<artifactId>commons-configuration</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId> <groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId> <artifactId>commons-lang3</artifactId>
</dependency> </dependency>
......
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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;
public class TypeExistsException extends AtlasException {
public TypeExistsException(String message) {
super(message);
}
}
...@@ -115,7 +115,11 @@ public class StructInstance implements ITypedStruct { ...@@ -115,7 +115,11 @@ public class StructInstance implements ITypedStruct {
clsType.validateId((Id) val); clsType.validateId((Id) val);
cVal = val; cVal = val;
} else { } else {
cVal = i.dataType().convert(val, i.multiplicity); try {
cVal = i.dataType().convert(val, i.multiplicity);
} catch(ValueConversionException.NullConversionException e) {
throw new ValueConversionException.NullConversionException("For field '" + attrName + "'", e);
}
} }
if (cVal == null) { if (cVal == null) {
nullFlags[nullPos] = true; nullFlags[nullPos] = true;
......
...@@ -25,6 +25,7 @@ public final class AttributeDefinition { ...@@ -25,6 +25,7 @@ public final class AttributeDefinition {
public final String name; public final String name;
public final String dataTypeName; public final String dataTypeName;
public final Multiplicity multiplicity; public final Multiplicity multiplicity;
//A composite is the one whose lifecycle is dependent on the enclosing type and is not just a reference
public final boolean isComposite; public final boolean isComposite;
public final boolean isUnique; public final boolean isUnique;
public final boolean isIndexable; public final boolean isIndexable;
......
...@@ -27,6 +27,7 @@ import java.util.Map; ...@@ -27,6 +27,7 @@ import java.util.Map;
public class AttributeInfo { public class AttributeInfo {
public final String name; public final String name;
public final Multiplicity multiplicity; public final Multiplicity multiplicity;
//A composite is the one whose lifecycle is dependent on the enclosing type and is not just a reference
public final boolean isComposite; public final boolean isComposite;
public final boolean isUnique; public final boolean isUnique;
public final boolean isIndexable; public final boolean isIndexable;
......
...@@ -55,8 +55,8 @@ public class ObjectGraphWalker { ...@@ -55,8 +55,8 @@ public class ObjectGraphWalker {
throws AtlasException { throws AtlasException {
this.typeSystem = typeSystem; this.typeSystem = typeSystem;
this.nodeProcessor = nodeProcessor; this.nodeProcessor = nodeProcessor;
queue = new LinkedList<IReferenceableInstance>(); queue = new LinkedList<>();
processedIds = new HashSet<Id>(); processedIds = new HashSet<>();
if (start != null) { if (start != null) {
visitReferenceableInstance(start); visitReferenceableInstance(start);
} }
...@@ -194,7 +194,7 @@ public class ObjectGraphWalker { ...@@ -194,7 +194,7 @@ public class ObjectGraphWalker {
} }
} }
public static interface NodeProcessor { public interface NodeProcessor {
void processNode(Node nd) throws AtlasException; void processNode(Node nd) throws AtlasException;
} }
......
...@@ -22,6 +22,7 @@ import com.google.common.collect.ArrayListMultimap; ...@@ -22,6 +22,7 @@ import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Multimap; import com.google.common.collect.Multimap;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.TypeExistsException;
import org.apache.atlas.TypeNotFoundException; import org.apache.atlas.TypeNotFoundException;
import org.apache.atlas.classification.InterfaceAudience; import org.apache.atlas.classification.InterfaceAudience;
import org.apache.atlas.typesystem.TypesDef; import org.apache.atlas.typesystem.TypesDef;
...@@ -169,9 +170,9 @@ public class TypeSystem { ...@@ -169,9 +170,9 @@ public class TypeSystem {
public StructType defineStructType(String name, boolean errorIfExists, AttributeDefinition... attrDefs) public StructType defineStructType(String name, boolean errorIfExists, AttributeDefinition... attrDefs)
throws AtlasException { throws AtlasException {
StructTypeDefinition structDef = new StructTypeDefinition(name, attrDefs); StructTypeDefinition structDef = new StructTypeDefinition(name, attrDefs);
defineTypes(ImmutableList.of(structDef), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.of(structDef),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
ImmutableList.<HierarchicalTypeDefinition<ClassType>>of()); ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
return getDataType(StructType.class, structDef.typeName); return getDataType(StructType.class, structDef.typeName);
} }
...@@ -196,56 +197,50 @@ public class TypeSystem { ...@@ -196,56 +197,50 @@ public class TypeSystem {
} }
public TraitType defineTraitType(HierarchicalTypeDefinition<TraitType> traitDef) throws AtlasException { public TraitType defineTraitType(HierarchicalTypeDefinition<TraitType> traitDef) throws AtlasException {
defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList.of(traitDef), ImmutableList.of(traitDef), ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
return getDataType(TraitType.class, traitDef.typeName); return getDataType(TraitType.class, traitDef.typeName);
} }
public ClassType defineClassType(HierarchicalTypeDefinition<ClassType> classDef) throws AtlasException { public ClassType defineClassType(HierarchicalTypeDefinition<ClassType> classDef) throws AtlasException {
defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), ImmutableList.of(classDef));
ImmutableList.of(classDef));
return getDataType(ClassType.class, classDef.typeName); return getDataType(ClassType.class, classDef.typeName);
} }
public Map<String, IDataType> defineTraitTypes(HierarchicalTypeDefinition<TraitType>... traitDefs) public Map<String, IDataType> defineTraitTypes(HierarchicalTypeDefinition<TraitType>... traitDefs)
throws AtlasException { throws AtlasException {
TransientTypeSystem transientTypes = TransientTypeSystem transientTypes =
new TransientTypeSystem(ImmutableList.<StructTypeDefinition>of(), ImmutableList.copyOf(traitDefs), new TransientTypeSystem(ImmutableList.<EnumTypeDefinition>of(),
ImmutableList.<StructTypeDefinition>of(), ImmutableList.copyOf(traitDefs),
ImmutableList.<HierarchicalTypeDefinition<ClassType>>of()); ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
return transientTypes.defineTypes(); return transientTypes.defineTypes();
} }
public Map<String, IDataType> defineClassTypes(HierarchicalTypeDefinition<ClassType>... classDefs) public Map<String, IDataType> defineClassTypes(HierarchicalTypeDefinition<ClassType>... classDefs)
throws AtlasException { throws AtlasException {
TransientTypeSystem transientTypes = new TransientTypeSystem(ImmutableList.<StructTypeDefinition>of(), TransientTypeSystem transientTypes = new TransientTypeSystem(ImmutableList.<EnumTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), ImmutableList.copyOf(classDefs)); ImmutableList.<StructTypeDefinition>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
ImmutableList.copyOf(classDefs));
return transientTypes.defineTypes(); return transientTypes.defineTypes();
} }
public Map<String, IDataType> defineTypes(TypesDef typesDef) throws AtlasException { public Map<String, IDataType> defineTypes(TypesDef typesDef) throws AtlasException {
Map<String, IDataType> typesAdded = new HashMap<>(); ImmutableList<EnumTypeDefinition> enumDefs = ImmutableList.copyOf(typesDef.enumTypesAsJavaList());
for (EnumTypeDefinition enumDef : typesDef.enumTypesAsJavaList()) {
typesAdded.put(enumDef.name, defineEnumType(enumDef));
}
ImmutableList<StructTypeDefinition> structDefs = ImmutableList.copyOf(typesDef.structTypesAsJavaList()); ImmutableList<StructTypeDefinition> structDefs = ImmutableList.copyOf(typesDef.structTypesAsJavaList());
ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs = ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs =
ImmutableList.copyOf(typesDef.traitTypesAsJavaList()); ImmutableList.copyOf(typesDef.traitTypesAsJavaList());
ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs = ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs =
ImmutableList.copyOf(typesDef.classTypesAsJavaList()); ImmutableList.copyOf(typesDef.classTypesAsJavaList());
typesAdded.putAll(defineTypes(structDefs, traitDefs, classDefs)); return defineTypes(enumDefs, structDefs, traitDefs, classDefs);
return typesAdded;
} }
public Map<String, IDataType> defineTypes(ImmutableList<StructTypeDefinition> structDefs, public Map<String, IDataType> defineTypes(ImmutableList<EnumTypeDefinition> enumDefs,
ImmutableList<StructTypeDefinition> structDefs,
ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs, ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) throws AtlasException { ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) throws AtlasException {
TransientTypeSystem transientTypes = new TransientTypeSystem(structDefs, traitDefs, classDefs); TransientTypeSystem transientTypes = new TransientTypeSystem(enumDefs, structDefs, traitDefs, classDefs);
Map<String, IDataType> definedTypes = transientTypes.defineTypes(); Map<String, IDataType> definedTypes = transientTypes.defineTypes();
// LOG.debug("Defined new types " + Arrays.toString(definedTypes.keySet().toArray(new // LOG.debug("Defined new types " + Arrays.toString(definedTypes.keySet().toArray(new
// String[definedTypes.size()]))); // String[definedTypes.size()])));
...@@ -307,6 +302,7 @@ public class TypeSystem { ...@@ -307,6 +302,7 @@ public class TypeSystem {
final ImmutableList<StructTypeDefinition> structDefs; final ImmutableList<StructTypeDefinition> structDefs;
final ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs; final ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs;
final ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs; final ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs;
private final ImmutableList<EnumTypeDefinition> enumDefs;
Map<String, StructTypeDefinition> structNameToDefMap = new HashMap<>(); Map<String, StructTypeDefinition> structNameToDefMap = new HashMap<>();
Map<String, HierarchicalTypeDefinition<TraitType>> traitNameToDefMap = new HashMap<>(); Map<String, HierarchicalTypeDefinition<TraitType>> traitNameToDefMap = new HashMap<>();
Map<String, HierarchicalTypeDefinition<ClassType>> classNameToDefMap = new HashMap<>(); Map<String, HierarchicalTypeDefinition<ClassType>> classNameToDefMap = new HashMap<>();
...@@ -318,10 +314,10 @@ public class TypeSystem { ...@@ -318,10 +314,10 @@ public class TypeSystem {
List<DataTypes.MapType> recursiveMapTypes; List<DataTypes.MapType> recursiveMapTypes;
TransientTypeSystem(ImmutableList<StructTypeDefinition> structDefs, TransientTypeSystem(ImmutableList<EnumTypeDefinition> enumDefs, ImmutableList<StructTypeDefinition> structDefs,
ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs, ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) { ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) {
this.enumDefs = enumDefs;
this.structDefs = structDefs; this.structDefs = structDefs;
this.traitDefs = traitDefs; this.traitDefs = traitDefs;
this.classDefs = classDefs; this.classDefs = classDefs;
...@@ -345,10 +341,22 @@ public class TypeSystem { ...@@ -345,10 +341,22 @@ public class TypeSystem {
* - setup shallow Type instances to facilitate recursive type graphs * - setup shallow Type instances to facilitate recursive type graphs
*/ */
private void step1() throws AtlasException { private void step1() throws AtlasException {
for (EnumTypeDefinition eDef : enumDefs) {
assert eDef.name != null;
if (types.containsKey(eDef.name)) {
throw new AtlasException(String.format("Redefinition of type %s not supported", eDef.name));
}
EnumType eT = new EnumType(this, eDef.name, eDef.enumValues);
TypeSystem.this.types.put(eDef.name, eT);
typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.ENUM, eDef.name);
transientTypes.add(eDef.name);
}
for (StructTypeDefinition sDef : structDefs) { for (StructTypeDefinition sDef : structDefs) {
assert sDef.typeName != null; assert sDef.typeName != null;
if (dataType(sDef.typeName) != null) { if (dataType(sDef.typeName) != null) {
throw new AtlasException(String.format("Cannot redefine type %s", sDef.typeName)); throw new TypeExistsException(String.format("Cannot redefine type %s", sDef.typeName));
} }
TypeSystem.this.types TypeSystem.this.types
.put(sDef.typeName, new StructType(this, sDef.typeName, sDef.attributeDefinitions.length)); .put(sDef.typeName, new StructType(this, sDef.typeName, sDef.attributeDefinitions.length));
...@@ -359,7 +367,7 @@ public class TypeSystem { ...@@ -359,7 +367,7 @@ public class TypeSystem {
for (HierarchicalTypeDefinition<TraitType> traitDef : traitDefs) { for (HierarchicalTypeDefinition<TraitType> traitDef : traitDefs) {
assert traitDef.typeName != null; assert traitDef.typeName != null;
if (types.containsKey(traitDef.typeName)) { if (types.containsKey(traitDef.typeName)) {
throw new AtlasException(String.format("Cannot redefine type %s", traitDef.typeName)); throw new TypeExistsException(String.format("Cannot redefine type %s", traitDef.typeName));
} }
TypeSystem.this.types.put(traitDef.typeName, new TraitType(this, traitDef.typeName, traitDef.superTypes, TypeSystem.this.types.put(traitDef.typeName, new TraitType(this, traitDef.typeName, traitDef.superTypes,
...@@ -371,7 +379,7 @@ public class TypeSystem { ...@@ -371,7 +379,7 @@ public class TypeSystem {
for (HierarchicalTypeDefinition<ClassType> classDef : classDefs) { for (HierarchicalTypeDefinition<ClassType> classDef : classDefs) {
assert classDef.typeName != null; assert classDef.typeName != null;
if (types.containsKey(classDef.typeName)) { if (types.containsKey(classDef.typeName)) {
throw new AtlasException(String.format("Cannot redefine type %s", classDef.typeName)); throw new TypeExistsException(String.format("Cannot redefine type %s", classDef.typeName));
} }
TypeSystem.this.types.put(classDef.typeName, new ClassType(this, classDef.typeName, classDef.superTypes, TypeSystem.this.types.put(classDef.typeName, new ClassType(this, classDef.typeName, classDef.superTypes,
...@@ -588,7 +596,8 @@ public class TypeSystem { ...@@ -588,7 +596,8 @@ public class TypeSystem {
} }
@Override @Override
public Map<String, IDataType> defineTypes(ImmutableList<StructTypeDefinition> structDefs, public Map<String, IDataType> defineTypes(ImmutableList<EnumTypeDefinition> enumDefs,
ImmutableList<StructTypeDefinition> structDefs,
ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs, ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) throws AtlasException { ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) throws AtlasException {
throw new AtlasException("Internal Error: define type called on TrasientTypeSystem"); throw new AtlasException("Internal Error: define type called on TrasientTypeSystem");
......
...@@ -43,10 +43,17 @@ public class ValueConversionException extends AtlasException { ...@@ -43,10 +43,17 @@ public class ValueConversionException extends AtlasException {
super(msg); super(msg);
} }
protected ValueConversionException(String msg, Exception e) {
super(msg, e);
}
public static class NullConversionException extends ValueConversionException { public static class NullConversionException extends ValueConversionException {
public NullConversionException(Multiplicity m) { public NullConversionException(Multiplicity m) {
super(String.format("Null value not allowed for multiplicty %s", m)); super(String.format("Null value not allowed for multiplicty %s", m));
} }
public NullConversionException(String msg, Exception e) {
super(msg, e);
}
} }
} }
...@@ -41,6 +41,7 @@ atlas.graph.index.search.elasticsearch.create.sleep=2000 ...@@ -41,6 +41,7 @@ atlas.graph.index.search.elasticsearch.create.sleep=2000
atlas.graph.index.search.solr.mode=cloud atlas.graph.index.search.solr.mode=cloud
atlas.graph.index.search.solr.zookeeper-url=${solr.zk.address} atlas.graph.index.search.solr.zookeeper-url=${solr.zk.address}
######### Hive Lineage Configs ######### ######### Hive Lineage Configs #########
# This models reflects the base super types for Data and Process # This models reflects the base super types for Data and Process
#atlas.lineage.hive.table.type.name=DataSet #atlas.lineage.hive.table.type.name=DataSet
...@@ -53,8 +54,13 @@ atlas.lineage.hive.table.schema.query.hive_table=hive_table where name='%s'\, co ...@@ -53,8 +54,13 @@ atlas.lineage.hive.table.schema.query.hive_table=hive_table where name='%s'\, co
######### Notification Configs ######### ######### Notification Configs #########
atlas.notification.embedded=true atlas.notification.embedded=true
atlas.notification.implementation=org.apache.atlas.kafka.KafkaNotification
atlas.notification.kafka.data=target/data/kafka atlas.kafka.zookeeper.connect=localhost:9026
atlas.kafka.bootstrap.servers=localhost:9027
atlas.kafka.data=target/data/kafka
atlas.kafka.zookeeper.session.timeout.ms=400
atlas.kafka.zookeeper.sync.time.ms=20
atlas.kafka.auto.commit.interval.ms=100
######### Security Properties ######### ######### Security Properties #########
......
<?xml version="1.0" encoding="UTF-8" ?>
<!--
~ 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.
-->
<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
<appender name="console" class="org.apache.log4j.ConsoleAppender">
<param name="Target" value="System.out"/>
<layout class="org.apache.log4j.PatternLayout">
<param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
</layout>
</appender>
<appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
<param name="File" value="${atlas.log.dir}/audit.log"/>
<param name="Append" value="true"/>
<param name="Threshold" value="debug"/>
<layout class="org.apache.log4j.PatternLayout">
<param name="ConversionPattern" value="%d %x %m%n"/>
</layout>
</appender>
<logger name="org.apache.atlas" additivity="false">
<level value="debug"/>
<appender-ref ref="console"/>
</logger>
<logger name="AUDIT">
<level value="info"/>
<appender-ref ref="console"/>
</logger>
<root>
<priority value="warn"/>
<appender-ref ref="console"/>
</root>
</log4j:configuration>
...@@ -236,7 +236,7 @@ trait TypeHelpers { ...@@ -236,7 +236,7 @@ trait TypeHelpers {
@throws(classOf[AtlasException]) @throws(classOf[AtlasException])
def defineClassType(ts: TypeSystem, classDef: HierarchicalTypeDefinition[ClassType]): ClassType = { def defineClassType(ts: TypeSystem, classDef: HierarchicalTypeDefinition[ClassType]): ClassType = {
ts.defineTypes(ImmutableList.of[StructTypeDefinition], ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition],
ImmutableList.of[HierarchicalTypeDefinition[TraitType]], ImmutableList.of[HierarchicalTypeDefinition[TraitType]],
ImmutableList.of[HierarchicalTypeDefinition[ClassType]](classDef)) ImmutableList.of[HierarchicalTypeDefinition[ClassType]](classDef))
return ts.getDataType(classOf[ClassType], classDef.typeName) return ts.getDataType(classOf[ClassType], classDef.typeName)
......
...@@ -25,7 +25,7 @@ public class ApplicationPropertiesTest { ...@@ -25,7 +25,7 @@ public class ApplicationPropertiesTest {
@Test @Test
public void testVariables() throws Exception { public void testVariables() throws Exception {
Configuration properties = ApplicationProperties.get(); Configuration properties = ApplicationProperties.get(ApplicationProperties.APPLICATION_PROPERTIES);
//plain property without variables //plain property without variables
Assert.assertEquals(properties.getString("atlas.service"), "atlas"); Assert.assertEquals(properties.getString("atlas.service"), "atlas");
...@@ -44,7 +44,7 @@ public class ApplicationPropertiesTest { ...@@ -44,7 +44,7 @@ public class ApplicationPropertiesTest {
@Test @Test
//variable substitutions should work with subset configuration as well //variable substitutions should work with subset configuration as well
public void testSubset() throws Exception { public void testSubset() throws Exception {
Configuration configuration = ApplicationProperties.get(); Configuration configuration = ApplicationProperties.get(ApplicationProperties.APPLICATION_PROPERTIES);
Configuration subConfiguration = configuration.subset("atlas"); Configuration subConfiguration = configuration.subset("atlas");
Assert.assertEquals(subConfiguration.getString("service"), "atlas"); Assert.assertEquals(subConfiguration.getString("service"), "atlas");
......
...@@ -28,6 +28,7 @@ import org.apache.atlas.typesystem.types.AttributeDefinition; ...@@ -28,6 +28,7 @@ import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.BaseTest; import org.apache.atlas.typesystem.types.BaseTest;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes; import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumTypeDefinition;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition; import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.Multiplicity; import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition; import org.apache.atlas.typesystem.types.StructTypeDefinition;
...@@ -79,9 +80,9 @@ public class SerializationJavaTest extends BaseTest { ...@@ -79,9 +80,9 @@ public class SerializationJavaTest extends BaseTest {
createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(), createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(),
createRequiredAttrDef("level", DataTypes.INT_TYPE)); createRequiredAttrDef("level", DataTypes.INT_TYPE));
ts.defineTypes(ImmutableList.<StructTypeDefinition>of(), ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(securityClearanceTypeDef), ImmutableList.of(securityClearanceTypeDef),
ImmutableList.<HierarchicalTypeDefinition<ClassType>>of(deptTypeDef, personTypeDef, managerTypeDef)); ImmutableList.of(deptTypeDef, personTypeDef, managerTypeDef));
Referenceable hrDept = new Referenceable("Department"); Referenceable hrDept = new Referenceable("Department");
Referenceable john = new Referenceable("Person"); Referenceable john = new Referenceable("Person");
...@@ -147,8 +148,8 @@ public class SerializationJavaTest extends BaseTest { ...@@ -147,8 +148,8 @@ public class SerializationJavaTest extends BaseTest {
createTraitTypeDef("SecurityClearance2", ImmutableList.<String>of(), createTraitTypeDef("SecurityClearance2", ImmutableList.<String>of(),
createRequiredAttrDef("level", DataTypes.INT_TYPE)); createRequiredAttrDef("level", DataTypes.INT_TYPE));
ts.defineTypes(ImmutableList.<StructTypeDefinition>of(), ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(securityClearanceTypeDef), ImmutableList.of(securityClearanceTypeDef),
ImmutableList.<HierarchicalTypeDefinition<ClassType>>of()); ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
......
...@@ -136,7 +136,8 @@ public abstract class BaseTest { ...@@ -136,7 +136,8 @@ public abstract class BaseTest {
.createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(), .createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE)); TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE));
ts.defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList.of(securityClearanceTypeDef), ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.of(securityClearanceTypeDef),
ImmutableList.of(deptTypeDef, personTypeDef, managerTypeDef)); ImmutableList.of(deptTypeDef, personTypeDef, managerTypeDef));
ImmutableList.of(ts.getDataType(HierarchicalType.class, "SecurityClearance"), ImmutableList.of(ts.getDataType(HierarchicalType.class, "SecurityClearance"),
......
...@@ -80,9 +80,10 @@ public class TypeSystemTest extends BaseTest { ...@@ -80,9 +80,10 @@ public class TypeSystemTest extends BaseTest {
HierarchicalTypeDefinition<TraitType> financeTrait = HierarchicalTypeDefinition<TraitType> financeTrait =
TypesUtil.createTraitTypeDef("Finance", ImmutableList.<String>of()); TypesUtil.createTraitTypeDef("Finance", ImmutableList.<String>of());
getTypeSystem().defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList getTypeSystem().defineTypes(ImmutableList.<EnumTypeDefinition>of(),
.of(classificationTraitDefinition, piiTrait, phiTrait, pciTrait, soxTrait, secTrait, ImmutableList.<StructTypeDefinition>of(),
financeTrait), ImmutableList.<HierarchicalTypeDefinition<ClassType>>of()); ImmutableList.of(classificationTraitDefinition, piiTrait, phiTrait, pciTrait, soxTrait, secTrait,
financeTrait), ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
final ImmutableList<String> traitsNames = getTypeSystem().getTypeNamesByCategory(DataTypes.TypeCategory.TRAIT); final ImmutableList<String> traitsNames = getTypeSystem().getTypeNamesByCategory(DataTypes.TypeCategory.TRAIT);
Assert.assertEquals(traitsNames.size(), 7); Assert.assertEquals(traitsNames.size(), 7);
...@@ -102,7 +103,6 @@ public class TypeSystemTest extends BaseTest { ...@@ -102,7 +103,6 @@ public class TypeSystemTest extends BaseTest {
String enumType = random(); String enumType = random();
EnumTypeDefinition orgLevelEnum = EnumTypeDefinition orgLevelEnum =
new EnumTypeDefinition(enumType, new EnumValue(random(), 1), new EnumValue(random(), 2)); new EnumTypeDefinition(enumType, new EnumValue(random(), 1), new EnumValue(random(), 2));
ts.defineEnumType(orgLevelEnum);
String structName = random(); String structName = random();
String attrType = random(); String attrType = random();
...@@ -117,7 +117,8 @@ public class TypeSystemTest extends BaseTest { ...@@ -117,7 +117,8 @@ public class TypeSystemTest extends BaseTest {
HierarchicalTypeDefinition<TraitType> traitType = createTraitTypeDef(traitName, ImmutableList.<String>of(), HierarchicalTypeDefinition<TraitType> traitType = createTraitTypeDef(traitName, ImmutableList.<String>of(),
createRequiredAttrDef(attrType, DataTypes.INT_TYPE)); createRequiredAttrDef(attrType, DataTypes.INT_TYPE));
ts.defineTypes(ImmutableList.of(structType), ImmutableList.of(traitType), ImmutableList.of(classType)); ts.defineTypes(ImmutableList.of(orgLevelEnum), ImmutableList.of(structType),
ImmutableList.of(traitType), ImmutableList.of(classType));
} }
@Test @Test
...@@ -127,7 +128,7 @@ public class TypeSystemTest extends BaseTest { ...@@ -127,7 +128,7 @@ public class TypeSystemTest extends BaseTest {
HierarchicalTypeDefinition<ClassType> c = TypesUtil.createClassTypeDef("C", ImmutableList.of("B")); HierarchicalTypeDefinition<ClassType> c = TypesUtil.createClassTypeDef("C", ImmutableList.of("B"));
TypeSystem ts = getTypeSystem(); TypeSystem ts = getTypeSystem();
ts.defineTypes(ImmutableList.<StructTypeDefinition>of(), ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
ImmutableList.of(a, b, c)); ImmutableList.of(a, b, c));
ClassType ac = ts.getDataType(ClassType.class, "a"); ClassType ac = ts.getDataType(ClassType.class, "a");
......
...@@ -146,7 +146,7 @@ class SerializationTest extends BaseTest { ...@@ -146,7 +146,7 @@ class SerializationTest extends BaseTest {
TypesUtil.createTraitTypeDef("SecurityClearance", ImmutableList.of[String], TypesUtil.createTraitTypeDef("SecurityClearance", ImmutableList.of[String],
TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE)) TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE))
ts.defineTypes(ImmutableList.of[StructTypeDefinition], ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition],
ImmutableList.of[HierarchicalTypeDefinition[TraitType]](securityClearanceTypeDef), ImmutableList.of[HierarchicalTypeDefinition[TraitType]](securityClearanceTypeDef),
ImmutableList.of[HierarchicalTypeDefinition[ClassType]](deptTypeDef, personTypeDef, managerTypeDef) ImmutableList.of[HierarchicalTypeDefinition[ClassType]](deptTypeDef, personTypeDef, managerTypeDef)
) )
......
...@@ -45,7 +45,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers { ...@@ -45,7 +45,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers {
optionalAttr("o", DataTypes.mapTypeName(DataTypes.STRING_TYPE, DataTypes.DOUBLE_TYPE))) optionalAttr("o", DataTypes.mapTypeName(DataTypes.STRING_TYPE, DataTypes.DOUBLE_TYPE)))
ts.defineTypes(ImmutableList.of[StructTypeDefinition](sDef), ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition](sDef),
ImmutableList.of[HierarchicalTypeDefinition[TraitType]], ImmutableList.of[HierarchicalTypeDefinition[TraitType]],
ImmutableList.of[HierarchicalTypeDefinition[ClassType]] ImmutableList.of[HierarchicalTypeDefinition[ClassType]]
) )
...@@ -123,7 +123,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers { ...@@ -123,7 +123,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers {
val securityClearanceTypeDef: HierarchicalTypeDefinition[TraitType] = createTraitTypeDef("SecurityClearance", List(), val securityClearanceTypeDef: HierarchicalTypeDefinition[TraitType] = createTraitTypeDef("SecurityClearance", List(),
requiredAttr("level", DataTypes.INT_TYPE) requiredAttr("level", DataTypes.INT_TYPE)
) )
ts.defineTypes(ImmutableList.of[StructTypeDefinition], ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition],
ImmutableList.of[HierarchicalTypeDefinition[TraitType]](securityClearanceTypeDef), ImmutableList.of[HierarchicalTypeDefinition[TraitType]](securityClearanceTypeDef),
ImmutableList.of[HierarchicalTypeDefinition[ClassType]](deptTypeDef, personTypeDef, managerTypeDef)) ImmutableList.of[HierarchicalTypeDefinition[ClassType]](deptTypeDef, personTypeDef, managerTypeDef))
...@@ -136,7 +136,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers { ...@@ -136,7 +136,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers {
typesDef1.enumTypes.foreach(ts1.defineEnumType(_)) typesDef1.enumTypes.foreach(ts1.defineEnumType(_))
ts1.defineTypes(ImmutableList.copyOf(typesDef1.structTypes.toArray), ts1.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.copyOf(typesDef1.structTypes.toArray),
ImmutableList.copyOf(typesDef1.traitTypes.toArray), ImmutableList.copyOf(typesDef1.traitTypes.toArray),
ImmutableList.copyOf(typesDef1.classTypes.toArray) ImmutableList.copyOf(typesDef1.classTypes.toArray)
) )
......
...@@ -188,6 +188,11 @@ ...@@ -188,6 +188,11 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>org.apache.atlas</groupId> <groupId>org.apache.atlas</groupId>
<artifactId>atlas-common</artifactId>
</dependency>
<dependency>
<groupId>org.apache.atlas</groupId>
<artifactId>atlas-typesystem</artifactId> <artifactId>atlas-typesystem</artifactId>
</dependency> </dependency>
...@@ -202,6 +207,11 @@ ...@@ -202,6 +207,11 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.atlas</groupId>
<artifactId>atlas-notification</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId> <artifactId>hadoop-common</artifactId>
</dependency> </dependency>
...@@ -413,6 +423,45 @@ ...@@ -413,6 +423,45 @@
</configuration> </configuration>
</plugin> </plugin>
<!-- Running unit tests in pre-integration-test phase after war is built -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemProperties>
<user.dir>${project.basedir}</user.dir>
<projectBaseDir>${project.basedir}/..</projectBaseDir>
</systemProperties>
<!--<skipTests>true</skipTests>-->
<forkMode>always</forkMode>
<redirectTestOutputToFile>true</redirectTestOutputToFile>
<argLine>-Djava.awt.headless=true -Dproject.version=${project.version}
-Dhadoop.tmp.dir=${project.build.directory}/tmp-hadoop-${user.name}
-Xmx1024m -XX:MaxPermSize=512m
</argLine>
<excludes>
<exclude>**/*Base*</exclude>
</excludes>
</configuration>
<dependencies>
<dependency>
<groupId>org.apache.maven.surefire</groupId>
<artifactId>surefire-testng</artifactId>
<version>2.18.1</version>
</dependency>
</dependencies>
<executions>
<execution>
<id>default-test</id>
<phase>pre-integration-test</phase>
<goals>
<goal>test</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin> <plugin>
<groupId>org.eclipse.jetty</groupId> <groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-maven-plugin</artifactId> <artifactId>jetty-maven-plugin</artifactId>
...@@ -435,6 +484,10 @@ ...@@ -435,6 +484,10 @@
<useTestScope>true</useTestScope> <useTestScope>true</useTestScope>
<systemProperties> <systemProperties>
<systemProperty> <systemProperty>
<name>log4j.configuration</name>
<value>atlas-log4j.xml</value>
</systemProperty>
<systemProperty>
<name>atlas.log.dir</name> <name>atlas.log.dir</name>
<value>${project.build.directory}/logs</value> <value>${project.build.directory}/logs</value>
</systemProperty> </systemProperty>
...@@ -453,7 +506,7 @@ ...@@ -453,7 +506,7 @@
</systemProperty> </systemProperty>
</systemProperties> </systemProperties>
<stopKey>atlas-stop</stopKey> <stopKey>atlas-stop</stopKey>
<stopPort>41001</stopPort> <stopPort>21001</stopPort>
<daemon>${debug.jetty.daemon}</daemon> <daemon>${debug.jetty.daemon}</daemon>
<testClassesDirectory>${project.build.directory}/../../webapp/target/test-classes/</testClassesDirectory> <testClassesDirectory>${project.build.directory}/../../webapp/target/test-classes/</testClassesDirectory>
<useTestClasspath>true</useTestClasspath> <useTestClasspath>true</useTestClasspath>
......
...@@ -101,6 +101,7 @@ public final class Main { ...@@ -101,6 +101,7 @@ public final class Main {
configuration.setProperty("atlas.enableTLS", String.valueOf(enableTLS)); configuration.setProperty("atlas.enableTLS", String.valueOf(enableTLS));
showStartupInfo(buildConfiguration, enableTLS, appPort); showStartupInfo(buildConfiguration, enableTLS, appPort);
server = EmbeddedServer.newServer(appPort, appPath, enableTLS); server = EmbeddedServer.newServer(appPort, appPath, enableTLS);
server.start(); server.start();
} }
......
...@@ -38,7 +38,6 @@ import org.apache.atlas.typesystem.types.TraitType; ...@@ -38,7 +38,6 @@ import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeUtils; import org.apache.atlas.typesystem.types.TypeUtils;
import org.apache.atlas.typesystem.types.utils.TypesUtil; import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONObject;
import java.util.List; import java.util.List;
...@@ -241,12 +240,11 @@ public class QuickStart { ...@@ -241,12 +240,11 @@ public class QuickStart {
String entityJSON = InstanceSerialization.toJson(referenceable, true); String entityJSON = InstanceSerialization.toJson(referenceable, true);
System.out.println("Submitting new entity= " + entityJSON); System.out.println("Submitting new entity= " + entityJSON);
JSONObject jsonObject = metadataServiceClient.createEntity(entityJSON); JSONArray guids = metadataServiceClient.createEntity(entityJSON);
String guid = jsonObject.getString(AtlasClient.GUID); System.out.println("created instance for type " + typeName + ", guid: " + guids);
System.out.println("created instance for type " + typeName + ", guid: " + guid);
// return the Id for created instance with guid // return the Id for created instance with guid
return new Id(guid, referenceable.getId().getVersion(), referenceable.getTypeName()); return new Id(guids.getString(0), referenceable.getId().getVersion(), referenceable.getTypeName());
} }
Id database(String name, String description, String owner, String locationUri, String... traitNames) Id database(String name, String description, String owner, String locationUri, String... traitNames)
...@@ -387,11 +385,9 @@ public class QuickStart { ...@@ -387,11 +385,9 @@ public class QuickStart {
private void search() throws Exception { private void search() throws Exception {
for (String dslQuery : getDSLQueries()) { for (String dslQuery : getDSLQueries()) {
JSONObject response = metadataServiceClient.searchEntity(dslQuery); JSONArray results = metadataServiceClient.search(dslQuery);
JSONObject results = response.getJSONObject(AtlasClient.RESULTS); if (results != null) {
if (!results.isNull("rows")) { System.out.println("query [" + dslQuery + "] returned [" + results.length() + "] rows");
JSONArray rows = results.getJSONArray("rows");
System.out.println("query [" + dslQuery + "] returned [" + rows.length() + "] rows");
} else { } else {
System.out.println("query [" + dslQuery + "] failed, results:" + results.toString()); System.out.println("query [" + dslQuery + "] failed, results:" + results.toString());
} }
......
...@@ -33,7 +33,9 @@ import org.apache.atlas.ApplicationProperties; ...@@ -33,7 +33,9 @@ import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClient;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.RepositoryMetadataModule; import org.apache.atlas.RepositoryMetadataModule;
import org.apache.atlas.notification.NotificationModule;
import org.apache.atlas.repository.graph.GraphProvider; import org.apache.atlas.repository.graph.GraphProvider;
import org.apache.atlas.service.Services;
import org.apache.atlas.web.filters.AtlasAuthenticationFilter; import org.apache.atlas.web.filters.AtlasAuthenticationFilter;
import org.apache.atlas.web.filters.AuditFilter; import org.apache.atlas.web.filters.AuditFilter;
import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.Configuration;
...@@ -64,7 +66,8 @@ public class GuiceServletConfig extends GuiceServletContextListener { ...@@ -64,7 +66,8 @@ public class GuiceServletConfig extends GuiceServletContextListener {
* .html * .html
*/ */
if (injector == null) { if (injector == null) {
injector = Guice.createInjector(new RepositoryMetadataModule(), new JerseyServletModule() { injector = Guice.createInjector(new RepositoryMetadataModule(), new NotificationModule(),
new JerseyServletModule() {
@Override @Override
protected void configureServlets() { protected void configureServlets() {
filter("/*").through(AuditFilter.class); filter("/*").through(AuditFilter.class);
...@@ -110,6 +113,14 @@ public class GuiceServletConfig extends GuiceServletContextListener { ...@@ -110,6 +113,14 @@ public class GuiceServletConfig extends GuiceServletContextListener {
// perform login operations // perform login operations
LoginProcessor loginProcessor = new LoginProcessor(); LoginProcessor loginProcessor = new LoginProcessor();
loginProcessor.login(); loginProcessor.login();
startServices();
}
protected void startServices() {
LOG.debug("Starting services");
Services services = injector.getInstance(Services.class);
services.start();
} }
/** /**
...@@ -132,6 +143,15 @@ public class GuiceServletConfig extends GuiceServletContextListener { ...@@ -132,6 +143,15 @@ public class GuiceServletConfig extends GuiceServletContextListener {
Provider<GraphProvider<TitanGraph>> graphProvider = injector.getProvider(Key.get(graphProviderType)); Provider<GraphProvider<TitanGraph>> graphProvider = injector.getProvider(Key.get(graphProviderType));
final Graph graph = graphProvider.get().get(); final Graph graph = graphProvider.get().get();
graph.shutdown(); graph.shutdown();
//stop services
stopServices();
} }
} }
protected void stopServices() {
LOG.debug("Stopping services");
Services services = injector.getInstance(Services.class);
services.stop();
}
} }
\ No newline at end of file
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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 com.google.common.base.Preconditions;
import com.google.inject.Inject;
import org.apache.atlas.AtlasClient;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.EntityExistsException;
import org.apache.atlas.services.MetadataService;
import org.apache.atlas.typesystem.types.ValueConversionException;
import org.apache.atlas.web.util.Servlets;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONObject;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.inject.Singleton;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.GET;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriBuilder;
import javax.ws.rs.core.UriInfo;
import java.net.URI;
import java.util.List;
@Path("entities")
@Singleton
public class EntitiesResource {
private static final Logger LOG = LoggerFactory.getLogger(EntitiesResource.class);
@Inject
private MetadataService metadataService;
@Context
UriInfo uriInfo;
/**
* Submits the entity definitions (instances).
* The body contains the JSONArray of entity json. The service takes care of de-duping the entities based on any
* unique attribute for the give type.
*/
@POST
@Consumes(Servlets.JSON_MEDIA_TYPE)
@Produces(Servlets.JSON_MEDIA_TYPE)
public Response submit(@Context HttpServletRequest request) {
try {
final String entities = Servlets.getRequestPayload(request);
LOG.debug("submitting entities {} ", AtlasClient.toString(new JSONArray(entities)));
final String guids = metadataService.createEntities(entities);
UriBuilder ub = uriInfo.getAbsolutePathBuilder();
URI locationURI = ub.path(guids).build();
JSONObject response = new JSONObject();
response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
response.put(AtlasClient.GUID, new JSONArray(guids));
response.put(AtlasClient.DEFINITION, metadataService.getEntityDefinition(new JSONArray(guids).getString(0)));
return Response.created(locationURI).entity(response).build();
} catch(EntityExistsException e) {
LOG.error("Unique constraint violation", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.CONFLICT));
} catch (ValueConversionException ve) {
LOG.error("Unable to persist entity instance due to a desrialization error ", ve);
throw new WebApplicationException(Servlets.getErrorResponse(ve.getCause(), Response.Status.BAD_REQUEST));
} catch (AtlasException | IllegalArgumentException e) {
LOG.error("Unable to persist entity instance", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
} catch (Throwable e) {
LOG.error("Unable to persist entity instance", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
}
}
/**
* Gets the list of entities for a given entity type.
*
* @param entityType name of a type which is unique
*/
@GET
@Produces(Servlets.JSON_MEDIA_TYPE)
public Response getEntityListByType(@QueryParam("type") String entityType) {
try {
Preconditions.checkNotNull(entityType, "Entity type cannot be null");
LOG.debug("Fetching entity list for type={} ", entityType);
final List<String> entityList = metadataService.getEntityList(entityType);
JSONObject response = new JSONObject();
response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
response.put(AtlasClient.TYPENAME, entityType);
response.put(AtlasClient.RESULTS, new JSONArray(entityList));
response.put(AtlasClient.COUNT, entityList.size());
return Response.ok(response).build();
} catch (NullPointerException e) {
LOG.error("Entity type cannot be null", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
} catch (AtlasException | IllegalArgumentException e) {
LOG.error("Unable to get entity list for type {}", entityType, e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
} catch (Throwable e) {
LOG.error("Unable to get entity list for type {}", entityType, e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
}
}
}
...@@ -25,7 +25,6 @@ import org.apache.atlas.ParamChecker; ...@@ -25,7 +25,6 @@ import org.apache.atlas.ParamChecker;
import org.apache.atlas.TypeNotFoundException; import org.apache.atlas.TypeNotFoundException;
import org.apache.atlas.repository.EntityNotFoundException; import org.apache.atlas.repository.EntityNotFoundException;
import org.apache.atlas.services.MetadataService; import org.apache.atlas.services.MetadataService;
import org.apache.atlas.typesystem.types.ValueConversionException;
import org.apache.atlas.web.util.Servlets; import org.apache.atlas.web.util.Servlets;
import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONObject; import org.codehaus.jettison.json.JSONObject;
...@@ -59,7 +58,7 @@ import java.util.List; ...@@ -59,7 +58,7 @@ import java.util.List;
* An entity is an "instance" of a Type. Entities conform to the definition * An entity is an "instance" of a Type. Entities conform to the definition
* of the Type they correspond with. * of the Type they correspond with.
*/ */
@Path("entities") @Path("entity")
@Singleton @Singleton
public class EntityResource { public class EntityResource {
...@@ -82,40 +81,6 @@ public class EntityResource { ...@@ -82,40 +81,6 @@ public class EntityResource {
this.metadataService = metadataService; this.metadataService = metadataService;
} }
/**
* Submits an entity definition (instance) corresponding to a given type.
*/
@POST
@Consumes(Servlets.JSON_MEDIA_TYPE)
@Produces(Servlets.JSON_MEDIA_TYPE)
public Response submit(@Context HttpServletRequest request) {
try {
final String entity = Servlets.getRequestPayload(request);
LOG.debug("submitting entity {} ", entity);
final String guid = metadataService.createEntity(entity);
UriBuilder ub = uriInfo.getAbsolutePathBuilder();
URI locationURI = ub.path(guid).build();
JSONObject response = new JSONObject();
response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
response.put(AtlasClient.GUID, guid);
response.put(AtlasClient.DEFINITION, metadataService.getEntityDefinition(guid));
return Response.created(locationURI).entity(response).build();
} catch (ValueConversionException ve) {
LOG.error("Unable to persist entity instance due to a desrialization error ", ve);
throw new WebApplicationException(Servlets.getErrorResponse(ve.getCause(), Response.Status.BAD_REQUEST));
} catch (AtlasException | IllegalArgumentException e) {
LOG.error("Unable to persist entity instance", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
} catch (Throwable e) {
LOG.error("Unable to persist entity instance", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
}
}
/** /**
* Fetch the complete definition of an entity given its GUID. * Fetch the complete definition of an entity given its GUID.
...@@ -159,34 +124,47 @@ public class EntityResource { ...@@ -159,34 +124,47 @@ public class EntityResource {
} }
/** /**
* Gets the list of entities for a given entity type. * Fetch the complete definition of an entity given its qualified name.
* *
* @param entityType name of a type which is unique * @param entityType
* @param attribute
* @param value
*/ */
@GET @GET
@Produces(Servlets.JSON_MEDIA_TYPE) @Produces(Servlets.JSON_MEDIA_TYPE)
public Response getEntityListByType(@QueryParam("type") String entityType) { public Response getEntityDefinitionByAttribute(@QueryParam("type") String entityType,
@QueryParam("property") String attribute,
@QueryParam("value") String value) {
try { try {
Preconditions.checkNotNull(entityType, "Entity type cannot be null"); LOG.debug("Fetching entity definition for type={}, qualified name={}", entityType, value);
ParamChecker.notEmpty(entityType, "type cannot be null");
ParamChecker.notEmpty(attribute, "attribute name cannot be null");
ParamChecker.notEmpty(value, "attribute value cannot be null");
LOG.debug("Fetching entity list for type={} ", entityType); final String entityDefinition = metadataService.getEntityDefinition(entityType, attribute, value);
final List<String> entityList = metadataService.getEntityList(entityType);
JSONObject response = new JSONObject(); JSONObject response = new JSONObject();
response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId()); response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
response.put(AtlasClient.TYPENAME, entityType);
response.put(AtlasClient.RESULTS, new JSONArray(entityList));
response.put(AtlasClient.COUNT, entityList.size());
return Response.ok(response).build(); Response.Status status = Response.Status.NOT_FOUND;
} catch (NullPointerException e) { if (entityDefinition != null) {
LOG.error("Entity type cannot be null", e); response.put(AtlasClient.DEFINITION, entityDefinition);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST)); status = Response.Status.OK;
} else {
response.put(AtlasClient.ERROR, Servlets.escapeJsonString(String.format("An entity with type={%s}, " +
"qualifiedName={%s} does not exist", entityType, value)));
}
return Response.status(status).entity(response).build();
} catch (EntityNotFoundException e) {
LOG.error("An entity with type={} and qualifiedName={} does not exist", entityType, value, e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
} catch (AtlasException | IllegalArgumentException e) { } catch (AtlasException | IllegalArgumentException e) {
LOG.error("Unable to get entity list for type {}", entityType, e); LOG.error("Bad type={}, qualifiedName={}", entityType, value, e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST)); throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
} catch (Throwable e) { } catch (Throwable e) {
LOG.error("Unable to get entity list for type {}", entityType, e); LOG.error("Unable to get instance definition for type={}, qualifiedName={}", entityType, value, e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR)); throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
} }
} }
......
...@@ -88,28 +88,16 @@ public class MetadataDiscoveryResource { ...@@ -88,28 +88,16 @@ public class MetadataDiscoveryResource {
final String jsonResultStr = discoveryService.searchByDSL(query); final String jsonResultStr = discoveryService.searchByDSL(query);
response = new DSLJSONResponseBuilder().results(jsonResultStr).query(query).build(); response = new DSLJSONResponseBuilder().results(jsonResultStr).query(query).build();
return Response.ok(response).build();
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
LOG.error("Unable to get entity list for empty query", e); LOG.error("Unable to get entity list for empty query", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST)); throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
} catch (Throwable throwable) { } catch (Throwable throwable) {
LOG.error("Unable to get entity list for query {} using dsl", query, throwable); LOG.error("Unable to get entity list for query {} using dsl", query, throwable);
return searchUsingFullText(query);
try { //fall back to full-text
final String jsonResultStr = discoveryService.searchByFullText(query);
response = new FullTextJSonResponseBuilder().results(jsonResultStr).query(query).build();
} catch (DiscoveryException | IllegalArgumentException e) {
LOG.error("Unable to get entity list for query {}", query, e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
} catch (Throwable e) {
LOG.error("Unable to get entity list for query {}", query, e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
}
} }
return Response.ok(response).build();
} }
/** /**
...@@ -267,7 +255,8 @@ public class MetadataDiscoveryResource { ...@@ -267,7 +255,8 @@ public class MetadataDiscoveryResource {
count(rowsJsonArr.length()); count(rowsJsonArr.length());
queryType(QUERY_TYPE_DSL); queryType(QUERY_TYPE_DSL);
JSONObject response = super.build(); JSONObject response = super.build();
response.put(AtlasClient.RESULTS, dslResults); response.put(AtlasClient.RESULTS, rowsJsonArr);
response.put(AtlasClient.DATATYPE, dslResults.get(AtlasClient.DATATYPE));
return response; return response;
} }
......
...@@ -21,6 +21,7 @@ package org.apache.atlas.web.resources; ...@@ -21,6 +21,7 @@ package org.apache.atlas.web.resources;
import com.sun.jersey.api.client.ClientResponse; import com.sun.jersey.api.client.ClientResponse;
import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClient;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.TypeExistsException;
import org.apache.atlas.services.MetadataService; import org.apache.atlas.services.MetadataService;
import org.apache.atlas.typesystem.types.DataTypes; import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.web.util.Servlets; import org.apache.atlas.web.util.Servlets;
...@@ -96,6 +97,9 @@ public class TypesResource { ...@@ -96,6 +97,9 @@ public class TypesResource {
response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId()); response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
response.put(AtlasClient.TYPES, typesResponse); response.put(AtlasClient.TYPES, typesResponse);
return Response.status(ClientResponse.Status.CREATED).entity(response).build(); return Response.status(ClientResponse.Status.CREATED).entity(response).build();
} catch (TypeExistsException e) {
LOG.error("Type already exists", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.CONFLICT));
} catch (AtlasException | IllegalArgumentException e) { } catch (AtlasException | IllegalArgumentException e) {
LOG.error("Unable to persist types", e); LOG.error("Unable to persist types", e);
throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST)); throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
......
...@@ -20,8 +20,6 @@ package org.apache.atlas.web.service; ...@@ -20,8 +20,6 @@ package org.apache.atlas.web.service;
import org.apache.atlas.ApplicationProperties; import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.commons.configuration.ConfigurationException;
import org.apache.commons.configuration.PropertiesConfiguration;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.alias.CredentialProvider; import org.apache.hadoop.security.alias.CredentialProvider;
import org.apache.hadoop.security.alias.CredentialProviderFactory; import org.apache.hadoop.security.alias.CredentialProviderFactory;
......
...@@ -50,7 +50,6 @@ ...@@ -50,7 +50,6 @@
<appender-ref ref="FILE"/> <appender-ref ref="FILE"/>
</logger> </logger>
<logger name="AUDIT"> <logger name="AUDIT">
<level value="info"/> <level value="info"/>
<appender-ref ref="AUDIT"/> <appender-ref ref="AUDIT"/>
......
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.notification;
import com.google.inject.Inject;
import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.json.InstanceSerialization;
import org.apache.atlas.web.resources.BaseResourceIT;
import org.codehaus.jettison.json.JSONArray;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
@Guice(modules = NotificationModule.class)
public class NotificationHookConsumerIT extends BaseResourceIT{
@Inject
private NotificationInterface kafka;
private String dbName;
@BeforeClass
public void setUp() throws Exception {
super.setUp();
createTypeDefinitions();
}
@AfterClass
public void teardown() throws Exception {
kafka.close();
}
private void sendHookMessage(Referenceable entity) throws NotificationException {
String entityJson = InstanceSerialization.toJson(entity, true);
JSONArray jsonArray = new JSONArray();
jsonArray.put(entityJson);
kafka.send(NotificationInterface.NotificationType.HOOK, jsonArray.toString());
}
@Test
public void testConsumeHookMessage() throws Exception {
Referenceable entity = new Referenceable(DATABASE_TYPE);
dbName = "db" + randomString();
entity.set("name", dbName);
entity.set("description", randomString());
sendHookMessage(entity);
waitFor(1000, new Predicate() {
@Override
public boolean evaluate() throws Exception {
JSONArray results =
serviceClient.searchByDSL(String.format("%s where name='%s'", DATABASE_TYPE, dbName));
return results.length() == 1;
}
});
}
@Test (dependsOnMethods = "testConsumeHookMessage")
public void testEnityDeduping() throws Exception {
// Referenceable db = serviceClient.getEntity(DATABASE_TYPE, "name", dbName);
Referenceable db = new Referenceable(DATABASE_TYPE);
db.set("name", dbName);
db.set("description", randomString());
Referenceable table = new Referenceable(HIVE_TABLE_TYPE);
final String tableName = randomString();
table.set("name", tableName);
table.set("db", db);
sendHookMessage(table);
waitFor(1000, new Predicate() {
@Override
public boolean evaluate() throws Exception {
JSONArray results =
serviceClient.searchByDSL(String.format("%s where name='%s'", HIVE_TABLE_TYPE, tableName));
return results.length() == 1;
}
});
JSONArray results =
serviceClient.searchByDSL(String.format("%s where name='%s'", DATABASE_TYPE, dbName));
Assert.assertEquals(results.length(), 1);
}
}
...@@ -21,8 +21,10 @@ import com.google.inject.Provider; ...@@ -21,8 +21,10 @@ import com.google.inject.Provider;
import com.google.inject.TypeLiteral; import com.google.inject.TypeLiteral;
import com.thinkaurelius.titan.core.TitanGraph; import com.thinkaurelius.titan.core.TitanGraph;
import com.thinkaurelius.titan.core.util.TitanCleanup; import com.thinkaurelius.titan.core.util.TitanCleanup;
import com.tinkerpop.blueprints.Graph; import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.graph.GraphProvider; import org.apache.atlas.repository.graph.GraphProvider;
import org.apache.commons.configuration.Configuration;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
...@@ -31,6 +33,7 @@ import javax.servlet.ServletContextEvent; ...@@ -31,6 +33,7 @@ import javax.servlet.ServletContextEvent;
public class TestGuiceServletConfig extends GuiceServletConfig { public class TestGuiceServletConfig extends GuiceServletConfig {
private static final Logger LOG = LoggerFactory.getLogger(TestGuiceServletConfig.class); private static final Logger LOG = LoggerFactory.getLogger(TestGuiceServletConfig.class);
private boolean servicesEnabled;
@Override @Override
public void contextInitialized(ServletContextEvent servletContextEvent) { public void contextInitialized(ServletContextEvent servletContextEvent) {
...@@ -40,6 +43,8 @@ public class TestGuiceServletConfig extends GuiceServletConfig { ...@@ -40,6 +43,8 @@ public class TestGuiceServletConfig extends GuiceServletConfig {
@Override @Override
public void contextDestroyed(ServletContextEvent servletContextEvent) { public void contextDestroyed(ServletContextEvent servletContextEvent) {
super.contextDestroyed(servletContextEvent);
if(injector != null) { if(injector != null) {
TypeLiteral<GraphProvider<TitanGraph>> graphProviderType = new TypeLiteral<GraphProvider<TitanGraph>>() {}; TypeLiteral<GraphProvider<TitanGraph>> graphProviderType = new TypeLiteral<GraphProvider<TitanGraph>>() {};
Provider<GraphProvider<TitanGraph>> graphProvider = injector.getProvider(Key.get(graphProviderType)); Provider<GraphProvider<TitanGraph>> graphProvider = injector.getProvider(Key.get(graphProviderType));
...@@ -47,11 +52,30 @@ public class TestGuiceServletConfig extends GuiceServletConfig { ...@@ -47,11 +52,30 @@ public class TestGuiceServletConfig extends GuiceServletConfig {
LOG.info("Clearing graph store"); LOG.info("Clearing graph store");
try { try {
graph.shutdown();
TitanCleanup.clear(graph); TitanCleanup.clear(graph);
} catch (Exception e) { } catch (Exception e) {
LOG.warn("Clearing graph store failed ", e); LOG.warn("Clearing graph store failed ", e);
} }
} }
} }
@Override
protected void startServices() {
try {
Configuration conf = ApplicationProperties.get();
servicesEnabled = conf.getBoolean("atlas.services.enabled", true);
if (servicesEnabled) {
super.startServices();
}
} catch (AtlasException e) {
throw new RuntimeException(e);
}
}
@Override
protected void stopServices() {
if (servicesEnabled) {
super.stopServices();
}
}
} }
...@@ -18,26 +18,45 @@ ...@@ -18,26 +18,45 @@
package org.apache.atlas.web.resources; package org.apache.atlas.web.resources;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.sun.jersey.api.client.Client; import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientResponse; import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.WebResource; import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.api.client.config.DefaultClientConfig; import com.sun.jersey.api.client.config.DefaultClientConfig;
import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClient;
import org.apache.atlas.ParamChecker;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.Struct;
import org.apache.atlas.typesystem.TypesDef; import org.apache.atlas.typesystem.TypesDef;
import org.apache.atlas.typesystem.json.InstanceSerialization; import org.apache.atlas.typesystem.json.InstanceSerialization;
import org.apache.atlas.typesystem.json.TypesSerialization; import org.apache.atlas.typesystem.json.TypesSerialization;
import org.apache.atlas.typesystem.persistence.Id; import org.apache.atlas.typesystem.persistence.Id;
import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumTypeDefinition;
import org.apache.atlas.typesystem.types.EnumValue;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition; import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.IDataType;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeUtils;
import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.apache.atlas.web.util.Servlets; import org.apache.atlas.web.util.Servlets;
import org.apache.commons.lang.RandomStringUtils;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONObject; import org.codehaus.jettison.json.JSONObject;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert; import org.testng.Assert;
import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeClass;
import javax.ws.rs.HttpMethod; import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.Response; import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriBuilder; import javax.ws.rs.core.UriBuilder;
import java.util.List;
/** /**
* Base class for integration tests. * Base class for integration tests.
...@@ -48,6 +67,7 @@ public abstract class BaseResourceIT { ...@@ -48,6 +67,7 @@ public abstract class BaseResourceIT {
protected WebResource service; protected WebResource service;
protected AtlasClient serviceClient; protected AtlasClient serviceClient;
public static String baseUrl = "http://localhost:21000/"; public static String baseUrl = "http://localhost:21000/";
public static final Logger LOG = LoggerFactory.getLogger(BaseResourceIT.class);
@BeforeClass @BeforeClass
public void setUp() throws Exception { public void setUp() throws Exception {
...@@ -89,11 +109,174 @@ public abstract class BaseResourceIT { ...@@ -89,11 +109,174 @@ public abstract class BaseResourceIT {
String entityJSON = InstanceSerialization.toJson(referenceable, true); String entityJSON = InstanceSerialization.toJson(referenceable, true);
System.out.println("Submitting new entity= " + entityJSON); System.out.println("Submitting new entity= " + entityJSON);
JSONObject jsonObject = serviceClient.createEntity(entityJSON); JSONArray guids = serviceClient.createEntity(entityJSON);
String guid = jsonObject.getString(AtlasClient.GUID); System.out.println("created instance for type " + typeName + ", guid: " + guids);
System.out.println("created instance for type " + typeName + ", guid: " + guid);
// return the reference to created instance with guid // return the reference to created instance with guid
return new Id(guid, 0, referenceable.getTypeName()); return new Id(guids.getString(0), 0, referenceable.getTypeName());
}
protected static final String DATABASE_TYPE = "hive_db";
protected static final String HIVE_TABLE_TYPE = "hive_table";
protected static final String COLUMN_TYPE = "hive_column";
protected static final String HIVE_PROCESS_TYPE = "hive_process";
protected void createTypeDefinitions() throws Exception {
HierarchicalTypeDefinition<ClassType> dbClsDef = TypesUtil
.createClassTypeDef(DATABASE_TYPE, null,
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE),
attrDef("locationUri", DataTypes.STRING_TYPE),
attrDef("owner", DataTypes.STRING_TYPE), attrDef("createTime", DataTypes.INT_TYPE));
HierarchicalTypeDefinition<ClassType> columnClsDef = TypesUtil
.createClassTypeDef(COLUMN_TYPE, null, attrDef("name", DataTypes.STRING_TYPE),
attrDef("dataType", DataTypes.STRING_TYPE), attrDef("comment", DataTypes.STRING_TYPE));
StructTypeDefinition structTypeDefinition = new StructTypeDefinition("serdeType",
new AttributeDefinition[]{TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("serde", DataTypes.STRING_TYPE)});
EnumValue values[] = {new EnumValue("MANAGED", 1), new EnumValue("EXTERNAL", 2),};
EnumTypeDefinition enumTypeDefinition = new EnumTypeDefinition("tableType", values);
HierarchicalTypeDefinition<ClassType> tblClsDef = TypesUtil
.createClassTypeDef(HIVE_TABLE_TYPE, ImmutableList.of("DataSet"),
attrDef("owner", DataTypes.STRING_TYPE), attrDef("createTime", DataTypes.LONG_TYPE),
attrDef("lastAccessTime", DataTypes.DATE_TYPE),
attrDef("temporary", DataTypes.BOOLEAN_TYPE),
new AttributeDefinition("db", DATABASE_TYPE, Multiplicity.REQUIRED, true, null),
new AttributeDefinition("columns", DataTypes.arrayTypeName(COLUMN_TYPE),
Multiplicity.OPTIONAL, true, null),
new AttributeDefinition("tableType", "tableType", Multiplicity.OPTIONAL, false, null),
new AttributeDefinition("serde1", "serdeType", Multiplicity.OPTIONAL, false, null),
new AttributeDefinition("serde2", "serdeType", Multiplicity.OPTIONAL, false, null));
HierarchicalTypeDefinition<ClassType> loadProcessClsDef = TypesUtil
.createClassTypeDef(HIVE_PROCESS_TYPE, ImmutableList.of("Process"),
attrDef("userName", DataTypes.STRING_TYPE), attrDef("startTime", DataTypes.INT_TYPE),
attrDef("endTime", DataTypes.LONG_TYPE),
attrDef("queryText", DataTypes.STRING_TYPE, Multiplicity.REQUIRED),
attrDef("queryPlan", DataTypes.STRING_TYPE, Multiplicity.REQUIRED),
attrDef("queryId", DataTypes.STRING_TYPE, Multiplicity.REQUIRED),
attrDef("queryGraph", DataTypes.STRING_TYPE, Multiplicity.REQUIRED));
HierarchicalTypeDefinition<TraitType> classificationTrait = TypesUtil
.createTraitTypeDef("classification", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
HierarchicalTypeDefinition<TraitType> piiTrait =
TypesUtil.createTraitTypeDef("pii", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> phiTrait =
TypesUtil.createTraitTypeDef("phi", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> pciTrait =
TypesUtil.createTraitTypeDef("pci", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> soxTrait =
TypesUtil.createTraitTypeDef("sox", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> secTrait =
TypesUtil.createTraitTypeDef("sec", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> financeTrait =
TypesUtil.createTraitTypeDef("finance", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> dimTraitDef = TypesUtil.createTraitTypeDef("Dimension", null);
HierarchicalTypeDefinition<TraitType> factTraitDef = TypesUtil.createTraitTypeDef("Fact", null);
HierarchicalTypeDefinition<TraitType> metricTraitDef = TypesUtil.createTraitTypeDef("Metric", null);
HierarchicalTypeDefinition<TraitType> etlTraitDef = TypesUtil.createTraitTypeDef("ETL", null);
TypesDef typesDef = TypeUtils.getTypesDef(ImmutableList.of(enumTypeDefinition),
ImmutableList.of(structTypeDefinition),
ImmutableList.of(classificationTrait, piiTrait, phiTrait, pciTrait, soxTrait, secTrait, financeTrait,
dimTraitDef, factTraitDef, metricTraitDef, etlTraitDef),
ImmutableList.of(dbClsDef, columnClsDef, tblClsDef, loadProcessClsDef));
createType(typesDef);
}
AttributeDefinition attrDef(String name, IDataType dT) {
return attrDef(name, dT, Multiplicity.OPTIONAL, false, null);
}
AttributeDefinition attrDef(String name, IDataType dT, Multiplicity m) {
return attrDef(name, dT, m, false, null);
}
AttributeDefinition attrDef(String name, IDataType dT, Multiplicity m, boolean isComposite,
String reverseAttributeName) {
Preconditions.checkNotNull(name);
Preconditions.checkNotNull(dT);
return new AttributeDefinition(name, dT.getName(), m, isComposite, reverseAttributeName);
}
protected String randomString() {
return RandomStringUtils.randomAlphanumeric(10);
}
protected Referenceable createHiveTableInstance(String dbName, String tableName) throws Exception {
Referenceable databaseInstance = new Referenceable(DATABASE_TYPE);
databaseInstance.set("name", dbName);
databaseInstance.set("description", "foo database");
Referenceable tableInstance =
new Referenceable(HIVE_TABLE_TYPE, "classification", "pii", "phi", "pci", "sox", "sec", "finance");
tableInstance.set("name", tableName);
tableInstance.set("db", databaseInstance);
tableInstance.set("description", "bar table");
tableInstance.set("lastAccessTime", "2014-07-11T08:00:00.000Z");
tableInstance.set("type", "managed");
tableInstance.set("level", 2);
tableInstance.set("tableType", 1); // enum
tableInstance.set("compressed", false);
Struct traitInstance = (Struct) tableInstance.getTrait("classification");
traitInstance.set("tag", "foundation_etl");
Struct serde1Instance = new Struct("serdeType");
serde1Instance.set("name", "serde1");
serde1Instance.set("serde", "serde1");
tableInstance.set("serde1", serde1Instance);
Struct serde2Instance = new Struct("serdeType");
serde2Instance.set("name", "serde2");
serde2Instance.set("serde", "serde2");
tableInstance.set("serde2", serde2Instance);
List<String> traits = tableInstance.getTraits();
Assert.assertEquals(traits.size(), 7);
return tableInstance;
}
public interface Predicate {
/**
* Perform a predicate evaluation.
*
* @return the boolean result of the evaluation.
* @throws Exception thrown if the predicate evaluation could not evaluate.
*/
boolean evaluate() throws Exception;
}
/**
* Wait for a condition, expressed via a {@link Predicate} to become true.
*
* @param timeout maximum time in milliseconds to wait for the predicate to become true.
* @param predicate predicate waiting on.
*/
protected void waitFor(int timeout, Predicate predicate) throws Exception {
ParamChecker.notNull(predicate, "predicate");
long mustEnd = System.currentTimeMillis() + timeout;
boolean eval;
while (!(eval = predicate.evaluate()) && System.currentTimeMillis() < mustEnd) {
LOG.info("Waiting up to {} msec", mustEnd - System.currentTimeMillis());
Thread.sleep(100);
}
if (!eval) {
throw new Exception("Waiting timed out after " + timeout + " msec");
}
} }
} }
...@@ -32,13 +32,10 @@ import org.apache.atlas.typesystem.json.InstanceSerialization$; ...@@ -32,13 +32,10 @@ import org.apache.atlas.typesystem.json.InstanceSerialization$;
import org.apache.atlas.typesystem.json.TypesSerialization; import org.apache.atlas.typesystem.json.TypesSerialization;
import org.apache.atlas.typesystem.json.TypesSerialization$; import org.apache.atlas.typesystem.json.TypesSerialization$;
import org.apache.atlas.typesystem.persistence.Id; import org.apache.atlas.typesystem.persistence.Id;
import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType; import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes; import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumTypeDefinition; import org.apache.atlas.typesystem.types.EnumTypeDefinition;
import org.apache.atlas.typesystem.types.EnumValue;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition; import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition; import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType; import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeUtils; import org.apache.atlas.typesystem.types.TypeUtils;
...@@ -66,10 +63,8 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -66,10 +63,8 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
private static final Logger LOG = LoggerFactory.getLogger(EntityJerseyResourceIT.class); private static final Logger LOG = LoggerFactory.getLogger(EntityJerseyResourceIT.class);
private static final String DATABASE_TYPE = "hive_database"; private final String DATABASE_NAME = "db" + randomString();
private static final String DATABASE_NAME = "foo"; private final String TABLE_NAME = "table" + randomString();
private static final String TABLE_TYPE = "hive_table_type";
private static final String TABLE_NAME = "bar";
private static final String TRAITS = "traits"; private static final String TRAITS = "traits";
private Referenceable tableInstance; private Referenceable tableInstance;
...@@ -80,12 +75,12 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -80,12 +75,12 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
public void setUp() throws Exception { public void setUp() throws Exception {
super.setUp(); super.setUp();
createHiveTypes(); createTypeDefinitions();
} }
@Test @Test
public void testSubmitEntity() throws Exception { public void testSubmitEntity() throws Exception {
tableInstance = createHiveTableInstance(); tableInstance = createHiveTableInstance(DATABASE_NAME, TABLE_NAME);
tableId = createInstance(tableInstance); tableId = createInstance(tableInstance);
final String guid = tableId._getId(); final String guid = tableId._getId();
...@@ -116,80 +111,25 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -116,80 +111,25 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
} }
@Test @Test
public void testUniqueAttribute() throws Exception { public void testGetEntityByAttribute() throws Exception {
//create type Referenceable databaseInstance = new Referenceable(DATABASE_TYPE);
String typeName = "type" + randomString(); String dbName = randomString();
HierarchicalTypeDefinition<ClassType> typeDefinition = TypesUtil databaseInstance.set("name", dbName);
.createClassTypeDef(typeName, ImmutableList.<String>of(), databaseInstance.set("description", "foo database");
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE)); createInstance(databaseInstance);
TypesDef typesDef = TypeUtils
.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
ImmutableList.of(typeDefinition));
createType(typesDef);
//create entity
String name = "name" + randomString();
Referenceable referenceable = new Referenceable(typeName);
referenceable.set("name", name);
createInstance(referenceable);
//create entity with same name again - should fail
try {
createInstance(referenceable);
Assert.fail("Expected exception");
} catch(Exception e) {
//expected exception
}
//create another type with same attribute - should allow
typeName = "type" + randomString();
typeDefinition = TypesUtil
.createClassTypeDef(typeName, ImmutableList.<String>of(),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE));
typesDef = TypeUtils
.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
ImmutableList.of(typeDefinition));
createType(typesDef);
referenceable = new Referenceable(typeName); //get entity by attribute
referenceable.set("name", name); Referenceable referenceable = serviceClient.getEntity(DATABASE_TYPE, "name", dbName);
createInstance(referenceable); Assert.assertEquals(referenceable.getTypeName(), DATABASE_TYPE);
Assert.assertEquals(referenceable.get("name"), dbName);
} }
@Test @Test
public void testSubmitEntityWithBadDateFormat() throws Exception { public void testSubmitEntityWithBadDateFormat() throws Exception {
try { try {
Referenceable databaseInstance = new Referenceable(DATABASE_TYPE); Referenceable tableInstance = createHiveTableInstance("db" + randomString(), "table" + randomString());
databaseInstance.set("name", DATABASE_NAME); tableInstance.set("lastAccessTime", "2014-07-11");
databaseInstance.set("description", "foo database");
Referenceable tableInstance =
new Referenceable(TABLE_TYPE, "classification", "pii", "phi", "pci", "sox", "sec", "finance");
tableInstance.set("name", TABLE_NAME);
tableInstance.set("description", "bar table");
tableInstance.set("date", "2014-07-11");
tableInstance.set("type", "managed");
tableInstance.set("level", 2);
tableInstance.set("tableType", 1); // enum
tableInstance.set("database", databaseInstance);
tableInstance.set("compressed", false);
Struct traitInstance = (Struct) tableInstance.getTrait("classification");
traitInstance.set("tag", "foundation_etl");
Struct serde1Instance = new Struct("serdeType");
serde1Instance.set("name", "serde1");
serde1Instance.set("serde", "serde1");
tableInstance.set("serde1", serde1Instance);
Struct serde2Instance = new Struct("serdeType");
serde2Instance.set("name", "serde2");
serde2Instance.set("serde", "serde2");
tableInstance.set("serde2", serde2Instance);
tableId = createInstance(tableInstance); tableId = createInstance(tableInstance);
Assert.fail("Was expecting an exception here "); Assert.fail("Was expecting an exception here ");
} catch (AtlasServiceException e) { } catch (AtlasServiceException e) {
...@@ -216,13 +156,14 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -216,13 +156,14 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
Assert.assertEquals(clientResponse.getStatus(), Response.Status.BAD_REQUEST.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.BAD_REQUEST.getStatusCode());
//non-string property, update //non-string property, update
clientResponse = addProperty(guid, "level", "4"); String currentTime = String.valueOf(System.currentTimeMillis());
clientResponse = addProperty(guid, "createTime", currentTime);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode());
entityRef = getEntityDefinition(getEntityDefinition(guid)); entityRef = getEntityDefinition(getEntityDefinition(guid));
Assert.assertNotNull(entityRef); Assert.assertNotNull(entityRef);
tableInstance.set("level", 4); tableInstance.set("createTime", currentTime);
} }
@Test(dependsOnMethods = "testSubmitEntity", expectedExceptions = IllegalArgumentException.class) @Test(dependsOnMethods = "testSubmitEntity", expectedExceptions = IllegalArgumentException.class)
...@@ -245,7 +186,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -245,7 +186,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
public void testAddReferenceProperty() throws Exception { public void testAddReferenceProperty() throws Exception {
//Create new db instance //Create new db instance
Referenceable databaseInstance = new Referenceable(DATABASE_TYPE); Referenceable databaseInstance = new Referenceable(DATABASE_TYPE);
databaseInstance.set("name", "newdb"); databaseInstance.set("name", randomString());
databaseInstance.set("description", "new database"); databaseInstance.set("description", "new database");
Id dbInstance = createInstance(databaseInstance); Id dbInstance = createInstance(databaseInstance);
...@@ -253,7 +194,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -253,7 +194,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
//Add reference property //Add reference property
final String guid = tableId._getId(); final String guid = tableId._getId();
ClientResponse clientResponse = addProperty(guid, "database", dbId); ClientResponse clientResponse = addProperty(guid, "db", dbId);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode());
} }
...@@ -276,14 +217,14 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -276,14 +217,14 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
} }
private ClientResponse addProperty(String guid, String property, String value) { private ClientResponse addProperty(String guid, String property, String value) {
WebResource resource = service.path("api/atlas/entities").path(guid); WebResource resource = service.path("api/atlas/entity").path(guid);
return resource.queryParam("property", property).queryParam("value", value).accept(Servlets.JSON_MEDIA_TYPE) return resource.queryParam("property", property).queryParam("value", value).accept(Servlets.JSON_MEDIA_TYPE)
.type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.PUT, ClientResponse.class); .type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.PUT, ClientResponse.class);
} }
private ClientResponse getEntityDefinition(String guid) { private ClientResponse getEntityDefinition(String guid) {
WebResource resource = service.path("api/atlas/entities").path(guid); WebResource resource = service.path("api/atlas/entity").path(guid);
return resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE) return resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.GET, ClientResponse.class); .method(HttpMethod.GET, ClientResponse.class);
} }
...@@ -299,7 +240,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -299,7 +240,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
@Test @Test
public void testGetInvalidEntityDefinition() throws Exception { public void testGetInvalidEntityDefinition() throws Exception {
WebResource resource = service.path("api/atlas/entities").path("blah"); WebResource resource = service.path("api/atlas/entity").path("blah");
ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE) ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.GET, ClientResponse.class); .method(HttpMethod.GET, ClientResponse.class);
...@@ -315,20 +256,9 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -315,20 +256,9 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
@Test(dependsOnMethods = "testSubmitEntity") @Test(dependsOnMethods = "testSubmitEntity")
public void testGetEntityList() throws Exception { public void testGetEntityList() throws Exception {
ClientResponse clientResponse = List<String> entities = serviceClient.listEntities(HIVE_TABLE_TYPE);
service.path("api/atlas/entities").queryParam("type", TABLE_TYPE).accept(Servlets.JSON_MEDIA_TYPE) Assert.assertNotNull(entities);
.type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.GET, ClientResponse.class); Assert.assertTrue(entities.contains(tableId._getId()));
Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode());
String responseAsString = clientResponse.getEntity(String.class);
Assert.assertNotNull(responseAsString);
JSONObject response = new JSONObject(responseAsString);
Assert.assertNotNull(response.get(AtlasClient.REQUEST_ID));
final JSONArray list = response.getJSONArray(AtlasClient.RESULTS);
Assert.assertNotNull(list);
Assert.assertEquals(list.length(), 1);
} }
@Test @Test
...@@ -349,10 +279,10 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -349,10 +279,10 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
@Test @Test
public void testGetEntityListForNoInstances() throws Exception { public void testGetEntityListForNoInstances() throws Exception {
addNewType(); String typeName = addNewType();
ClientResponse clientResponse = ClientResponse clientResponse =
service.path("api/atlas/entities").queryParam("type", "test").accept(Servlets.JSON_MEDIA_TYPE) service.path("api/atlas/entities").queryParam("type", typeName).accept(Servlets.JSON_MEDIA_TYPE)
.type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.GET, ClientResponse.class); .type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.GET, ClientResponse.class);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode());
...@@ -366,21 +296,23 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -366,21 +296,23 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
Assert.assertEquals(list.length(), 0); Assert.assertEquals(list.length(), 0);
} }
private void addNewType() throws Exception { private String addNewType() throws Exception {
String typeName = "test" + randomString();
HierarchicalTypeDefinition<ClassType> testTypeDefinition = TypesUtil HierarchicalTypeDefinition<ClassType> testTypeDefinition = TypesUtil
.createClassTypeDef("test", ImmutableList.<String>of(), .createClassTypeDef(typeName, ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE), TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE)); TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE));
String typesAsJSON = TypesSerialization.toJson(testTypeDefinition); String typesAsJSON = TypesSerialization.toJson(testTypeDefinition, false);
createType(typesAsJSON); createType(typesAsJSON);
return typeName;
} }
@Test(dependsOnMethods = "testSubmitEntity") @Test(dependsOnMethods = "testSubmitEntity")
public void testGetTraitNames() throws Exception { public void testGetTraitNames() throws Exception {
final String guid = tableId._getId(); final String guid = tableId._getId();
ClientResponse clientResponse = ClientResponse clientResponse =
service.path("api/atlas/entities").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE) service.path("api/atlas/entity").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE)
.type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.GET, ClientResponse.class); .type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.GET, ClientResponse.class);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode());
...@@ -410,7 +342,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -410,7 +342,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
final String guid = tableId._getId(); final String guid = tableId._getId();
ClientResponse clientResponse = ClientResponse clientResponse =
service.path("api/atlas/entities").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE) service.path("api/atlas/entity").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE)
.type(Servlets.JSON_MEDIA_TYPE) .type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON); .method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.CREATED.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.CREATED.getStatusCode());
...@@ -433,7 +365,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -433,7 +365,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
final String guid = tableId._getId(); final String guid = tableId._getId();
ClientResponse clientResponse = ClientResponse clientResponse =
service.path("api/atlas/entities").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE) service.path("api/atlas/entity").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE)
.type(Servlets.JSON_MEDIA_TYPE) .type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON); .method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
...@@ -456,7 +388,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -456,7 +388,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
final String guid = tableId._getId(); final String guid = tableId._getId();
ClientResponse clientResponse = ClientResponse clientResponse =
service.path("api/atlas/entities").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE) service.path("api/atlas/entity").path(guid).path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE)
.type(Servlets.JSON_MEDIA_TYPE) .type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON); .method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.CREATED.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.CREATED.getStatusCode());
...@@ -497,7 +429,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -497,7 +429,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
LOG.debug("traitInstanceAsJSON = " + traitInstanceAsJSON); LOG.debug("traitInstanceAsJSON = " + traitInstanceAsJSON);
ClientResponse clientResponse = ClientResponse clientResponse =
service.path("api/atlas/entities").path("random").path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE) service.path("api/atlas/entity").path("random").path(TRAITS).accept(Servlets.JSON_MEDIA_TYPE)
.type(Servlets.JSON_MEDIA_TYPE) .type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON); .method(HttpMethod.POST, ClientResponse.class, traitInstanceAsJSON);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
...@@ -507,7 +439,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -507,7 +439,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
public void testDeleteTrait() throws Exception { public void testDeleteTrait() throws Exception {
final String guid = tableId._getId(); final String guid = tableId._getId();
ClientResponse clientResponse = service.path("api/atlas/entities").path(guid).path(TRAITS).path(traitName) ClientResponse clientResponse = service.path("api/atlas/entity").path(guid).path(TRAITS).path(traitName)
.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE) .accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.DELETE, ClientResponse.class); .method(HttpMethod.DELETE, ClientResponse.class);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode());
...@@ -525,7 +457,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -525,7 +457,7 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
public void testDeleteTraitNonExistent() throws Exception { public void testDeleteTraitNonExistent() throws Exception {
final String traitName = "blah_trait"; final String traitName = "blah_trait";
ClientResponse clientResponse = service.path("api/atlas/entities").path("random").path(TRAITS).path(traitName) ClientResponse clientResponse = service.path("api/atlas/entity").path("random").path(TRAITS).path(traitName)
.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE) .accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.DELETE, ClientResponse.class); .method(HttpMethod.DELETE, ClientResponse.class);
Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
...@@ -544,10 +476,6 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -544,10 +476,6 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
return RandomStringUtils.random(10); return RandomStringUtils.random(10);
} }
private String randomString() {
return RandomStringUtils.randomAlphanumeric(10);
}
@Test @Test
public void testUTF8() throws Exception { public void testUTF8() throws Exception {
String classType = random(); String classType = random();
...@@ -572,90 +500,4 @@ public class EntityJerseyResourceIT extends BaseResourceIT { ...@@ -572,90 +500,4 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
Referenceable getReferenceable = InstanceSerialization.fromJsonReferenceable(definition, true); Referenceable getReferenceable = InstanceSerialization.fromJsonReferenceable(definition, true);
Assert.assertEquals(getReferenceable.get(attrName), attrValue); Assert.assertEquals(getReferenceable.get(attrName), attrValue);
} }
private void createHiveTypes() throws Exception {
HierarchicalTypeDefinition<ClassType> databaseTypeDefinition = TypesUtil
.createClassTypeDef(DATABASE_TYPE, ImmutableList.<String>of(),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE));
StructTypeDefinition structTypeDefinition = new StructTypeDefinition("serdeType",
new AttributeDefinition[]{TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("serde", DataTypes.STRING_TYPE)});
EnumValue values[] = {new EnumValue("MANAGED", 1), new EnumValue("EXTERNAL", 2),};
EnumTypeDefinition enumTypeDefinition = new EnumTypeDefinition("tableType", values);
HierarchicalTypeDefinition<ClassType> tableTypeDefinition = TypesUtil
.createClassTypeDef(TABLE_TYPE, ImmutableList.<String>of(),
TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
TypesUtil.createOptionalAttrDef("description", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("type", DataTypes.STRING_TYPE),
TypesUtil.createRequiredAttrDef("date", DataTypes.DATE_TYPE),
TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE),
new AttributeDefinition("tableType", "tableType", Multiplicity.REQUIRED, false, null),
new AttributeDefinition("serde1", "serdeType", Multiplicity.REQUIRED, false, null),
new AttributeDefinition("serde2", "serdeType", Multiplicity.REQUIRED, false, null),
new AttributeDefinition("database", DATABASE_TYPE, Multiplicity.REQUIRED, true, null),
new AttributeDefinition("compressed", DataTypes.BOOLEAN_TYPE.getName(), Multiplicity.OPTIONAL,
true, null));
HierarchicalTypeDefinition<TraitType> classificationTraitDefinition = TypesUtil
.createTraitTypeDef("classification", ImmutableList.<String>of(),
TypesUtil.createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
HierarchicalTypeDefinition<TraitType> piiTrait =
TypesUtil.createTraitTypeDef("pii", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> phiTrait =
TypesUtil.createTraitTypeDef("phi", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> pciTrait =
TypesUtil.createTraitTypeDef("pci", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> soxTrait =
TypesUtil.createTraitTypeDef("sox", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> secTrait =
TypesUtil.createTraitTypeDef("sec", ImmutableList.<String>of());
HierarchicalTypeDefinition<TraitType> financeTrait =
TypesUtil.createTraitTypeDef("finance", ImmutableList.<String>of());
TypesDef typesDef = TypeUtils
.getTypesDef(ImmutableList.of(enumTypeDefinition), ImmutableList.of(structTypeDefinition), ImmutableList
.of(classificationTraitDefinition, piiTrait, phiTrait, pciTrait, soxTrait, secTrait,
financeTrait), ImmutableList.of(databaseTypeDefinition, tableTypeDefinition));
createType(typesDef);
}
private Referenceable createHiveTableInstance() throws Exception {
Referenceable databaseInstance = new Referenceable(DATABASE_TYPE);
databaseInstance.set("name", DATABASE_NAME);
databaseInstance.set("description", "foo database");
Referenceable tableInstance =
new Referenceable(TABLE_TYPE, "classification", "pii", "phi", "pci", "sox", "sec", "finance");
tableInstance.set("name", TABLE_NAME);
tableInstance.set("description", "bar table");
tableInstance.set("date", "2014-07-11T08:00:00.000Z");
tableInstance.set("type", "managed");
tableInstance.set("level", 2);
tableInstance.set("tableType", 1); // enum
tableInstance.set("database", databaseInstance);
tableInstance.set("compressed", false);
Struct traitInstance = (Struct) tableInstance.getTrait("classification");
traitInstance.set("tag", "foundation_etl");
Struct serde1Instance = new Struct("serdeType");
serde1Instance.set("name", "serde1");
serde1Instance.set("serde", "serde1");
tableInstance.set("serde1", serde1Instance);
Struct serde2Instance = new Struct("serdeType");
serde2Instance.set("name", "serde2");
serde2Instance.set("serde", "serde2");
tableInstance.set("serde2", serde2Instance);
List<String> traits = tableInstance.getTraits();
Assert.assertEquals(traits.size(), 7);
return tableInstance;
}
} }
...@@ -18,25 +18,12 @@ ...@@ -18,25 +18,12 @@
package org.apache.atlas.web.resources; package org.apache.atlas.web.resources;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.sun.jersey.api.client.ClientResponse; import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.WebResource; import com.sun.jersey.api.client.WebResource;
import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClient;
import org.apache.atlas.typesystem.Referenceable; import org.apache.atlas.typesystem.Referenceable;
import org.apache.atlas.typesystem.TypesDef;
import org.apache.atlas.typesystem.persistence.Id; import org.apache.atlas.typesystem.persistence.Id;
import org.apache.atlas.typesystem.types.AttributeDefinition;
import org.apache.atlas.typesystem.types.ClassType;
import org.apache.atlas.typesystem.types.DataTypes;
import org.apache.atlas.typesystem.types.EnumTypeDefinition;
import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
import org.apache.atlas.typesystem.types.IDataType;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.StructTypeDefinition;
import org.apache.atlas.typesystem.types.TraitType;
import org.apache.atlas.typesystem.types.TypeUtils;
import org.apache.atlas.typesystem.types.utils.TypesUtil;
import org.apache.atlas.web.util.Servlets; import org.apache.atlas.web.util.Servlets;
import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONObject; import org.codehaus.jettison.json.JSONObject;
...@@ -54,18 +41,20 @@ import java.util.List; ...@@ -54,18 +41,20 @@ import java.util.List;
public class HiveLineageJerseyResourceIT extends BaseResourceIT { public class HiveLineageJerseyResourceIT extends BaseResourceIT {
private static final String BASE_URI = "api/atlas/lineage/hive/table/"; private static final String BASE_URI = "api/atlas/lineage/hive/table/";
private String salesFactTable;
private String salesMonthlyTable;
@BeforeClass @BeforeClass
public void setUp() throws Exception { public void setUp() throws Exception {
super.setUp(); super.setUp();
setUpTypes(); createTypeDefinitions();
setupInstances(); setupInstances();
} }
@Test @Test
public void testInputsGraph() throws Exception { public void testInputsGraph() throws Exception {
WebResource resource = service.path(BASE_URI).path("sales_fact_monthly_mv").path("inputs").path("graph"); WebResource resource = service.path(BASE_URI).path(salesMonthlyTable).path("inputs").path("graph");
ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE) ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.GET, ClientResponse.class); .method(HttpMethod.GET, ClientResponse.class);
...@@ -93,7 +82,7 @@ public class HiveLineageJerseyResourceIT extends BaseResourceIT { ...@@ -93,7 +82,7 @@ public class HiveLineageJerseyResourceIT extends BaseResourceIT {
@Test @Test
public void testOutputsGraph() throws Exception { public void testOutputsGraph() throws Exception {
WebResource resource = service.path(BASE_URI).path("sales_fact").path("outputs").path("graph"); WebResource resource = service.path(BASE_URI).path(salesFactTable).path("outputs").path("graph");
ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE) ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.GET, ClientResponse.class); .method(HttpMethod.GET, ClientResponse.class);
...@@ -121,7 +110,7 @@ public class HiveLineageJerseyResourceIT extends BaseResourceIT { ...@@ -121,7 +110,7 @@ public class HiveLineageJerseyResourceIT extends BaseResourceIT {
@Test @Test
public void testSchema() throws Exception { public void testSchema() throws Exception {
WebResource resource = service.path(BASE_URI).path("sales_fact").path("schema"); WebResource resource = service.path(BASE_URI).path(salesFactTable).path("schema");
ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE) ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
.method(HttpMethod.GET, ClientResponse.class); .method(HttpMethod.GET, ClientResponse.class);
...@@ -167,105 +156,41 @@ public class HiveLineageJerseyResourceIT extends BaseResourceIT { ...@@ -167,105 +156,41 @@ public class HiveLineageJerseyResourceIT extends BaseResourceIT {
Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode()); Assert.assertEquals(clientResponse.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
} }
private void setUpTypes() throws Exception {
TypesDef typesDef = createTypeDefinitions();
createType(typesDef);
}
private static final String DATABASE_TYPE = "hive_db";
private static final String HIVE_TABLE_TYPE = "hive_table";
private static final String COLUMN_TYPE = "hive_column";
private static final String HIVE_PROCESS_TYPE = "hive_process";
private TypesDef createTypeDefinitions() {
HierarchicalTypeDefinition<ClassType> dbClsDef = TypesUtil
.createClassTypeDef(DATABASE_TYPE, null, attrDef("name", DataTypes.STRING_TYPE),
attrDef("description", DataTypes.STRING_TYPE), attrDef("locationUri", DataTypes.STRING_TYPE),
attrDef("owner", DataTypes.STRING_TYPE), attrDef("createTime", DataTypes.INT_TYPE));
HierarchicalTypeDefinition<ClassType> columnClsDef = TypesUtil
.createClassTypeDef(COLUMN_TYPE, null, attrDef("name", DataTypes.STRING_TYPE),
attrDef("dataType", DataTypes.STRING_TYPE), attrDef("comment", DataTypes.STRING_TYPE));
HierarchicalTypeDefinition<ClassType> tblClsDef = TypesUtil
.createClassTypeDef(HIVE_TABLE_TYPE, ImmutableList.of("DataSet"),
attrDef("owner", DataTypes.STRING_TYPE), attrDef("createTime", DataTypes.INT_TYPE),
attrDef("lastAccessTime", DataTypes.INT_TYPE), attrDef("tableType", DataTypes.STRING_TYPE),
attrDef("temporary", DataTypes.BOOLEAN_TYPE),
new AttributeDefinition("db", DATABASE_TYPE, Multiplicity.REQUIRED, false, null),
new AttributeDefinition("columns", DataTypes.arrayTypeName(COLUMN_TYPE),
Multiplicity.COLLECTION, true, null));
HierarchicalTypeDefinition<ClassType> loadProcessClsDef = TypesUtil
.createClassTypeDef(HIVE_PROCESS_TYPE, ImmutableList.of("Process"),
attrDef("userName", DataTypes.STRING_TYPE), attrDef("startTime", DataTypes.INT_TYPE),
attrDef("endTime", DataTypes.INT_TYPE),
attrDef("queryText", DataTypes.STRING_TYPE, Multiplicity.REQUIRED),
attrDef("queryPlan", DataTypes.STRING_TYPE, Multiplicity.REQUIRED),
attrDef("queryId", DataTypes.STRING_TYPE, Multiplicity.REQUIRED),
attrDef("queryGraph", DataTypes.STRING_TYPE, Multiplicity.REQUIRED));
HierarchicalTypeDefinition<TraitType> dimTraitDef = TypesUtil.createTraitTypeDef("Dimension", null);
HierarchicalTypeDefinition<TraitType> factTraitDef = TypesUtil.createTraitTypeDef("Fact", null);
HierarchicalTypeDefinition<TraitType> metricTraitDef = TypesUtil.createTraitTypeDef("Metric", null);
HierarchicalTypeDefinition<TraitType> etlTraitDef = TypesUtil.createTraitTypeDef("ETL", null);
HierarchicalTypeDefinition<TraitType> piiTraitDef = TypesUtil.createTraitTypeDef("PII", null);
return TypeUtils.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
ImmutableList.of(dimTraitDef, factTraitDef, metricTraitDef, etlTraitDef, piiTraitDef),
ImmutableList.of(dbClsDef, columnClsDef, tblClsDef, loadProcessClsDef));
}
AttributeDefinition attrDef(String name, IDataType dT) {
return attrDef(name, dT, Multiplicity.OPTIONAL, false, null);
}
AttributeDefinition attrDef(String name, IDataType dT, Multiplicity m) {
return attrDef(name, dT, m, false, null);
}
AttributeDefinition attrDef(String name, IDataType dT, Multiplicity m, boolean isComposite,
String reverseAttributeName) {
Preconditions.checkNotNull(name);
Preconditions.checkNotNull(dT);
return new AttributeDefinition(name, dT.getName(), m, isComposite, reverseAttributeName);
}
private void setupInstances() throws Exception { private void setupInstances() throws Exception {
Id salesDB = database("Sales", "Sales Database", "John ETL", "hdfs://host:8000/apps/warehouse/sales"); Id salesDB = database("Sales" + randomString(), "Sales Database", "John ETL",
"hdfs://host:8000/apps/warehouse/sales");
List<Referenceable> salesFactColumns = ImmutableList List<Referenceable> salesFactColumns = ImmutableList
.of(column("time_id", "int", "time id"), column("product_id", "int", "product id"), .of(column("time_id", "int", "time id"), column("product_id", "int", "product id"),
column("customer_id", "int", "customer id", "PII"), column("customer_id", "int", "customer id", "pii"),
column("sales", "double", "product id", "Metric")); column("sales", "double", "product id", "Metric"));
Id salesFact = table("sales_fact", "sales fact table", salesDB, "Joe", "Managed", salesFactColumns, "Fact"); salesFactTable = "sales_fact" + randomString();
Id salesFact = table(salesFactTable, "sales fact table", salesDB, "Joe", "MANAGED", salesFactColumns, "Fact");
List<Referenceable> timeDimColumns = ImmutableList List<Referenceable> timeDimColumns = ImmutableList
.of(column("time_id", "int", "time id"), column("dayOfYear", "int", "day Of Year"), .of(column("time_id", "int", "time id"), column("dayOfYear", "int", "day Of Year"),
column("weekDay", "int", "week Day")); column("weekDay", "int", "week Day"));
Id timeDim = Id timeDim =
table("time_dim", "time dimension table", salesDB, "John Doe", "External", timeDimColumns, "Dimension"); table("time_dim" + randomString(), "time dimension table", salesDB, "John Doe", "EXTERNAL",
timeDimColumns, "Dimension");
Id reportingDB = Id reportingDB =
database("Reporting", "reporting database", "Jane BI", "hdfs://host:8000/apps/warehouse/reporting"); database("Reporting" + randomString(), "reporting database", "Jane BI",
"hdfs://host:8000/apps/warehouse/reporting");
Id salesFactDaily = Id salesFactDaily =
table("sales_fact_daily_mv", "sales fact daily materialized view", reportingDB, "Joe BI", "Managed", table("sales_fact_daily_mv" + randomString(), "sales fact daily materialized view", reportingDB,
salesFactColumns, "Metric"); "Joe BI", "MANAGED", salesFactColumns, "Metric");
loadProcess("loadSalesDaily", "John ETL", ImmutableList.of(salesFact, timeDim), loadProcess("loadSalesDaily", "John ETL", ImmutableList.of(salesFact, timeDim),
ImmutableList.of(salesFactDaily), "create table as select ", "plan", "id", "graph", "ETL"); ImmutableList.of(salesFactDaily), "create table as select ", "plan", "id", "graph", "ETL");
salesMonthlyTable = "sales_fact_monthly_mv" + randomString();
Id salesFactMonthly = Id salesFactMonthly =
table("sales_fact_monthly_mv", "sales fact monthly materialized view", reportingDB, "Jane BI", table(salesMonthlyTable, "sales fact monthly materialized view", reportingDB, "Jane BI",
"Managed", salesFactColumns, "Metric"); "MANAGED", salesFactColumns, "Metric");
loadProcess("loadSalesMonthly", "John ETL", ImmutableList.of(salesFactDaily), loadProcess("loadSalesMonthly", "John ETL", ImmutableList.of(salesFactDaily),
ImmutableList.of(salesFactMonthly), "create table as select ", "plan", "id", "graph", "ETL"); ImmutableList.of(salesFactMonthly), "create table as select ", "plan", "id", "graph", "ETL");
......
...@@ -76,11 +76,9 @@ public class MetadataDiscoveryJerseyResourceIT extends BaseResourceIT { ...@@ -76,11 +76,9 @@ public class MetadataDiscoveryJerseyResourceIT extends BaseResourceIT {
Assert.assertEquals(response.getString("query"), dslQuery); Assert.assertEquals(response.getString("query"), dslQuery);
Assert.assertEquals(response.getString("queryType"), "dsl"); Assert.assertEquals(response.getString("queryType"), "dsl");
JSONObject results = response.getJSONObject(AtlasClient.RESULTS); JSONArray results = response.getJSONArray(AtlasClient.RESULTS);
Assert.assertNotNull(results); Assert.assertNotNull(results);
Assert.assertEquals(results.length(), 1);
JSONArray rows = results.getJSONArray(AtlasClient.ROWS);
Assert.assertEquals(rows.length(), 1);
int numRows = response.getInt(AtlasClient.COUNT); int numRows = response.getInt(AtlasClient.COUNT);
Assert.assertEquals(numRows, 1); Assert.assertEquals(numRows, 1);
......
...@@ -113,6 +113,7 @@ public class BaseSecurityTest { ...@@ -113,6 +113,7 @@ public class BaseSecurityTest {
protected PropertiesConfiguration getSSLConfiguration(String providerUrl) { protected PropertiesConfiguration getSSLConfiguration(String providerUrl) {
String projectBaseDirectory = System.getProperty("projectBaseDir"); String projectBaseDirectory = System.getProperty("projectBaseDir");
final PropertiesConfiguration configuration = new PropertiesConfiguration(); final PropertiesConfiguration configuration = new PropertiesConfiguration();
configuration.setProperty("atlas.services.enabled", false);
configuration.setProperty(TLS_ENABLED, true); configuration.setProperty(TLS_ENABLED, true);
configuration.setProperty(TRUSTSTORE_FILE_KEY, projectBaseDirectory + "/webapp/target/atlas.keystore"); configuration.setProperty(TRUSTSTORE_FILE_KEY, projectBaseDirectory + "/webapp/target/atlas.keystore");
configuration.setProperty(KEYSTORE_FILE_KEY, projectBaseDirectory + "/webapp/target/atlas.keystore"); configuration.setProperty(KEYSTORE_FILE_KEY, projectBaseDirectory + "/webapp/target/atlas.keystore");
......
...@@ -40,7 +40,7 @@ import static org.apache.atlas.security.SecurityProperties.TLS_ENABLED; ...@@ -40,7 +40,7 @@ import static org.apache.atlas.security.SecurityProperties.TLS_ENABLED;
* kerberos user for the invocation. Need a separate use case since the Jersey layer cached the URL connection handler, * kerberos user for the invocation. Need a separate use case since the Jersey layer cached the URL connection handler,
* which indirectly caches the kerberos delegation token. * which indirectly caches the kerberos delegation token.
*/ */
public class NegativeSSLAndKerberosIT extends BaseSSLAndKerberosTest { public class NegativeSSLAndKerberosTest extends BaseSSLAndKerberosTest {
private TestSecureEmbeddedServer secureEmbeddedServer; private TestSecureEmbeddedServer secureEmbeddedServer;
private String originalConf; private String originalConf;
...@@ -65,7 +65,7 @@ public class NegativeSSLAndKerberosIT extends BaseSSLAndKerberosTest { ...@@ -65,7 +65,7 @@ public class NegativeSSLAndKerberosIT extends BaseSSLAndKerberosTest {
String confLocation = System.getProperty("atlas.conf"); String confLocation = System.getProperty("atlas.conf");
URL url; URL url;
if (confLocation == null) { if (confLocation == null) {
url = NegativeSSLAndKerberosIT.class.getResource("/application.properties"); url = NegativeSSLAndKerberosTest.class.getResource("/application.properties");
} else { } else {
url = new File(confLocation, "application.properties").toURI().toURL(); url = new File(confLocation, "application.properties").toURI().toURL();
} }
......
...@@ -45,7 +45,7 @@ import java.security.PrivilegedExceptionAction; ...@@ -45,7 +45,7 @@ import java.security.PrivilegedExceptionAction;
import static org.apache.atlas.security.SecurityProperties.TLS_ENABLED; import static org.apache.atlas.security.SecurityProperties.TLS_ENABLED;
public class SSLAndKerberosIT extends BaseSSLAndKerberosTest { public class SSLAndKerberosTest extends BaseSSLAndKerberosTest {
public static final String TEST_USER_JAAS_SECTION = "TestUser"; public static final String TEST_USER_JAAS_SECTION = "TestUser";
public static final String TESTUSER = "testuser"; public static final String TESTUSER = "testuser";
public static final String TESTPASS = "testpass"; public static final String TESTPASS = "testpass";
...@@ -74,7 +74,7 @@ public class SSLAndKerberosIT extends BaseSSLAndKerberosTest { ...@@ -74,7 +74,7 @@ public class SSLAndKerberosIT extends BaseSSLAndKerberosTest {
String confLocation = System.getProperty("atlas.conf"); String confLocation = System.getProperty("atlas.conf");
URL url; URL url;
if (confLocation == null) { if (confLocation == null) {
url = SSLAndKerberosIT.class.getResource("/application.properties"); url = SSLAndKerberosTest.class.getResource("/application.properties");
} else { } else {
url = new File(confLocation, "application.properties").toURI().toURL(); url = new File(confLocation, "application.properties").toURI().toURL();
} }
......
...@@ -41,7 +41,7 @@ import static org.apache.atlas.security.SecurityProperties.KEYSTORE_PASSWORD_KEY ...@@ -41,7 +41,7 @@ import static org.apache.atlas.security.SecurityProperties.KEYSTORE_PASSWORD_KEY
import static org.apache.atlas.security.SecurityProperties.SERVER_CERT_PASSWORD_KEY; import static org.apache.atlas.security.SecurityProperties.SERVER_CERT_PASSWORD_KEY;
import static org.apache.atlas.security.SecurityProperties.TRUSTSTORE_PASSWORD_KEY; import static org.apache.atlas.security.SecurityProperties.TRUSTSTORE_PASSWORD_KEY;
public class SSLIT extends BaseSSLAndKerberosTest { public class SSLTest extends BaseSSLAndKerberosTest {
private AtlasClient dgiCLient; private AtlasClient dgiCLient;
private Path jksPath; private Path jksPath;
private String providerUrl; private String providerUrl;
......
/* /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one
* contributor license agreements. See the NOTICE file distributed with * or more contributor license agreements. See the NOTICE file
* this work for additional information regarding copyright ownership. * distributed with this work for additional information
* The ASF licenses this file to You under the Apache License, Version 2.0 * regarding copyright ownership. The ASF licenses this file
* (the "License"); you may not use this file except in compliance with * to you under the Apache License, Version 2.0 (the
* the License. You may obtain a copy of the License at * "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 * <p/>
* * http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
...@@ -27,12 +28,13 @@ import java.net.URL; ...@@ -27,12 +28,13 @@ import java.net.URL;
import static org.apache.atlas.security.SecurityProperties.CERT_STORES_CREDENTIAL_PROVIDER_PATH; import static org.apache.atlas.security.SecurityProperties.CERT_STORES_CREDENTIAL_PROVIDER_PATH;
public class SecureEmbeddedServerIT extends SecureEmbeddedServerITBase { public class SecureEmbeddedServerTest extends SecureEmbeddedServerTestBase {
@Test @Test
public void testServerConfiguredUsingCredentialProvider() throws Exception { public void testServerConfiguredUsingCredentialProvider() throws Exception {
// setup the configuration // setup the configuration
final PropertiesConfiguration configuration = new PropertiesConfiguration(); final PropertiesConfiguration configuration = new PropertiesConfiguration();
configuration.setProperty(CERT_STORES_CREDENTIAL_PROVIDER_PATH, providerUrl); configuration.setProperty(CERT_STORES_CREDENTIAL_PROVIDER_PATH, providerUrl);
configuration.setProperty("atlas.notification.embedded", "false");
// setup the credential provider // setup the credential provider
setupCredentials(); setupCredentials();
...@@ -58,6 +60,6 @@ public class SecureEmbeddedServerIT extends SecureEmbeddedServerITBase { ...@@ -58,6 +60,6 @@ public class SecureEmbeddedServerIT extends SecureEmbeddedServerITBase {
} finally { } finally {
secureEmbeddedServer.server.stop(); secureEmbeddedServer.server.stop();
} }
} }
} }
...@@ -45,12 +45,16 @@ import java.io.File; ...@@ -45,12 +45,16 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
import static org.apache.atlas.security.SecurityProperties.*; import static org.apache.atlas.security.SecurityProperties.CERT_STORES_CREDENTIAL_PROVIDER_PATH;
import static org.apache.atlas.security.SecurityProperties.DEFAULT_KEYSTORE_FILE_LOCATION;
import static org.apache.atlas.security.SecurityProperties.KEYSTORE_PASSWORD_KEY;
import static org.apache.atlas.security.SecurityProperties.SERVER_CERT_PASSWORD_KEY;
import static org.apache.atlas.security.SecurityProperties.TRUSTSTORE_PASSWORD_KEY;
/** /**
* Secure Test class for jersey resources. * Secure Test class for jersey resources.
*/ */
public class SecureEmbeddedServerITBase { public class SecureEmbeddedServerTestBase {
private SecureEmbeddedServer secureEmbeddedServer; private SecureEmbeddedServer secureEmbeddedServer;
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment