Commit 7660c9b2 by Shwetha GS

ATLAS-128 DSL - Add support for comparisions on list type (suma.shivaprasad via shwethags)

parent 539f2431
...@@ -329,12 +329,16 @@ public class HiveHookIT { ...@@ -329,12 +329,16 @@ public class HiveHookIT {
String tableType = HiveDataTypes.HIVE_TABLE.getName(); String tableType = HiveDataTypes.HIVE_TABLE.getName();
LOG.debug("Searching for partition of {}.{} with values {}", dbName, tableName, value); LOG.debug("Searching for partition of {}.{} with values {}", dbName, tableName, value);
//todo replace with DSL /* 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.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);
assertEntityIsRegistered(gremlinQuery); */
String dslQuery = String.format("%s as p where values = ['%s'], table where tableName = '%s', "
+ "db where name = '%s' and clusterName = '%s' select p", typeName, value,
tableName.toLowerCase(), dbName.toLowerCase(), CLUSTER_NAME);
assertEntityIsRegistered(dslQuery);
} }
private String assertEntityIsRegistered(final String query) throws Exception { private String assertEntityIsRegistered(final String query) throws Exception {
......
...@@ -9,6 +9,7 @@ ATLAS-54 Rename configs in hive hook (shwethags) ...@@ -9,6 +9,7 @@ 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)
ALL CHANGES: ALL CHANGES:
ATLAS-128 DSL - Add support for comparisions on list type (suma.shivaprasad via shwethags)
ATLAS-168 Atlas UI - Max column in hive 4 (darshankumar89 via shwethags) ATLAS-168 Atlas UI - Max column in hive 4 (darshankumar89 via shwethags)
ATLAS-155 Images do not show up on the dashboard (darshankumar89 via shwethags) ATLAS-155 Images do not show up on the dashboard (darshankumar89 via shwethags)
ATLAS-134 Some defects found when reviewing the source code (ltfxyz via shwethags) ATLAS-134 Some defects found when reviewing the source code (ltfxyz via shwethags)
......
...@@ -18,6 +18,8 @@ ...@@ -18,6 +18,8 @@
package org.apache.atlas.discovery.graph; package org.apache.atlas.discovery.graph;
import com.google.common.collect.ImmutableCollection;
import com.google.common.collect.ImmutableList;
import com.thinkaurelius.titan.core.TitanVertex; import com.thinkaurelius.titan.core.TitanVertex;
import org.apache.atlas.AtlasException; import org.apache.atlas.AtlasException;
import org.apache.atlas.query.Expressions; import org.apache.atlas.query.Expressions;
...@@ -32,6 +34,7 @@ import org.apache.atlas.typesystem.ITypedReferenceableInstance; ...@@ -32,6 +34,7 @@ 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.types.AttributeInfo; import org.apache.atlas.typesystem.types.AttributeInfo;
import org.apache.atlas.typesystem.types.DataTypes;
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.StructType; import org.apache.atlas.typesystem.types.StructType;
...@@ -40,6 +43,7 @@ import org.apache.atlas.typesystem.types.TypeSystem; ...@@ -40,6 +43,7 @@ import org.apache.atlas.typesystem.types.TypeSystem;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List; import java.util.List;
/** /**
...@@ -106,11 +110,18 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi ...@@ -106,11 +110,18 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
case PRIMITIVE: case PRIMITIVE:
case ENUM: case ENUM:
return dataType.convert(value, Multiplicity.OPTIONAL); return dataType.convert(value, Multiplicity.OPTIONAL);
case ARRAY: case ARRAY:
// todo DataTypes.ArrayType arrType = (DataTypes.ArrayType) dataType;
break; IDataType<?> elemType = arrType.getElemType();
ImmutableCollection.Builder result = ImmutableList.builder();
List list = (List) value;
for(Object listElement : list) {
Object collectionEntry = constructCollectionEntry(elemType, listElement);
if(collectionEntry != null) {
result.add(collectionEntry);
}
}
return (U)result.build();
case MAP: case MAP:
// todo // todo
break; break;
...@@ -161,6 +172,25 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi ...@@ -161,6 +172,25 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
return null; return null;
} }
public <U> U constructCollectionEntry(IDataType<U> elementType, Object value) throws AtlasException {
switch (elementType.getTypeCategory()) {
case PRIMITIVE:
case ENUM:
return constructInstance(elementType, value);
//The array values in case of STRUCT, CLASS contain the edgeId if the outgoing edge which links to the STRUCT, CLASS vertex referenced
case STRUCT:
case CLASS:
String edgeId = (String) value;
return (U) metadataRepository.getGraphToInstanceMapper().getReferredEntity(edgeId, elementType);
case ARRAY:
case MAP:
case TRAIT:
return null;
default:
throw new UnsupportedOperationException("Load for type " + elementType + " in collections is not supported");
}
}
@Override @Override
public String edgeLabel(TypeUtils.FieldInfo fInfo) { public String edgeLabel(TypeUtils.FieldInfo fInfo) {
return fInfo.reverseDataType() == null ? edgeLabel(fInfo.dataType(), fInfo.attrInfo()) : return fInfo.reverseDataType() == null ? edgeLabel(fInfo.dataType(), fInfo.attrInfo()) :
......
...@@ -152,7 +152,7 @@ public interface MetadataRepository { ...@@ -152,7 +152,7 @@ public interface MetadataRepository {
void deleteTrait(String guid, String traitNameToBeDeleted) throws RepositoryException; void deleteTrait(String guid, String traitNameToBeDeleted) throws RepositoryException;
/** /**
* Adds the property to the entity that corresponds to the GUID * Adds/Updates the property to/in the entity that corresponds to the GUID
* @param guid entity id * @param guid entity id
* @param property property name * @param property property name
* @param value property value * @param value property value
......
...@@ -48,9 +48,9 @@ public class TitanGraphProvider implements GraphProvider<TitanGraph> { ...@@ -48,9 +48,9 @@ public class TitanGraphProvider implements GraphProvider<TitanGraph> {
*/ */
private static final String GRAPH_PREFIX = "atlas.graph"; private static final String GRAPH_PREFIX = "atlas.graph";
private static TitanGraph graphInstance; private static volatile TitanGraph graphInstance;
private static Configuration getConfiguration() throws AtlasException { public static Configuration getConfiguration() throws AtlasException {
Configuration configProperties = ApplicationProperties.get(); Configuration configProperties = ApplicationProperties.get();
return ApplicationProperties.getSubsetConfiguration(configProperties, GRAPH_PREFIX); return ApplicationProperties.getSubsetConfiguration(configProperties, GRAPH_PREFIX);
} }
...@@ -84,10 +84,7 @@ public class TitanGraphProvider implements GraphProvider<TitanGraph> { ...@@ -84,10 +84,7 @@ public class TitanGraphProvider implements GraphProvider<TitanGraph> {
} }
} }
@Override public static TitanGraph getGraphInstance() {
@Singleton
@Provides
public TitanGraph get() {
if(graphInstance == null) { if(graphInstance == null) {
synchronized (TitanGraphProvider.class) { synchronized (TitanGraphProvider.class) {
if(graphInstance == null) { if(graphInstance == null) {
...@@ -104,4 +101,11 @@ public class TitanGraphProvider implements GraphProvider<TitanGraph> { ...@@ -104,4 +101,11 @@ public class TitanGraphProvider implements GraphProvider<TitanGraph> {
} }
return graphInstance; return graphInstance;
} }
@Override
@Singleton
@Provides
public TitanGraph get() {
return getGraphInstance();
}
} }
...@@ -18,7 +18,11 @@ ...@@ -18,7 +18,11 @@
package org.apache.atlas.query package org.apache.atlas.query
import java.util
import com.google.common.collect.ImmutableCollection
import org.apache.atlas.AtlasException import org.apache.atlas.AtlasException
import org.apache.atlas.typesystem.ITypedInstance
import org.apache.atlas.typesystem.types.DataTypes.{ArrayType, PrimitiveType, TypeCategory} import org.apache.atlas.typesystem.types.DataTypes.{ArrayType, PrimitiveType, TypeCategory}
import org.apache.atlas.typesystem.types._ import org.apache.atlas.typesystem.types._
...@@ -468,6 +472,22 @@ object Expressions { ...@@ -468,6 +472,22 @@ object Expressions {
} }
} }
import scala.collection.JavaConversions._
case class ListLiteral[_](dataType: ArrayType, rawValue: List[Expressions.Literal[_]]) extends Expression with LeafNode {
val lc : java.util.List[Expressions.Literal[_]] = rawValue
val value = if (rawValue != null) dataType.convert(lc, Multiplicity.REQUIRED)
override def toString = value match {
case l: Seq[_]
=> l.mkString("[",",","]")
case c: ImmutableCollection[_] =>
c.asList.mkString("[",",","]")
case x =>
x.toString
}
}
def literal[T](typ: PrimitiveType[T], rawValue: Any) = new Literal[T](typ, rawValue) def literal[T](typ: PrimitiveType[T], rawValue: Any) = new Literal[T](typ, rawValue)
def boolean(rawValue: Any) = literal(DataTypes.BOOLEAN_TYPE, rawValue) def boolean(rawValue: Any) = literal(DataTypes.BOOLEAN_TYPE, rawValue)
...@@ -492,6 +512,12 @@ object Expressions { ...@@ -492,6 +512,12 @@ object Expressions {
def date(rawValue: Any) = literal(DataTypes.DATE_TYPE, rawValue) def date(rawValue: Any) = literal(DataTypes.DATE_TYPE, rawValue)
def list[_ <: PrimitiveType[_]](listElements: List[Expressions.Literal[_]]) = {
listLiteral(TypeSystem.getInstance().defineArrayType(listElements.head.dataType), listElements)
}
def listLiteral[_ <: PrimitiveType[_]](typ: ArrayType, rawValue: List[Expressions.Literal[_]]) = new ListLiteral(typ, rawValue)
case class ArithmeticExpression(symbol: String, case class ArithmeticExpression(symbol: String,
left: Expression, left: Expression,
right: Expression) right: Expression)
...@@ -601,7 +627,9 @@ object Expressions { ...@@ -601,7 +627,9 @@ object Expressions {
s"datatype. Can not resolve due to unresolved children") s"datatype. Can not resolve due to unresolved children")
} }
if(left.dataType == DataTypes.DATE_TYPE) { if(left.dataType.getName.startsWith(DataTypes.ARRAY_TYPE_PREFIX)) {
left.dataType;
} else if(left.dataType == DataTypes.DATE_TYPE) {
DataTypes.DATE_TYPE DataTypes.DATE_TYPE
} else if (left.dataType != DataTypes.STRING_TYPE || right.dataType != DataTypes.STRING_TYPE) { } else if (left.dataType != DataTypes.STRING_TYPE || right.dataType != DataTypes.STRING_TYPE) {
TypeUtils.combinedType(left.dataType, right.dataType) TypeUtils.combinedType(left.dataType, right.dataType)
...@@ -651,7 +679,6 @@ object Expressions { ...@@ -651,7 +679,6 @@ object Expressions {
val GEN_COL_ALIAS_PREFIX = "_col" val GEN_COL_ALIAS_PREFIX = "_col"
case class SelectExpression(child: Expression, selectList: List[Expression]) extends Expression { case class SelectExpression(child: Expression, selectList: List[Expression]) extends Expression {
val children = List(child) ::: selectList val children = List(child) ::: selectList
lazy val selectListWithAlias = selectList.zipWithIndex map { lazy val selectListWithAlias = selectList.zipWithIndex map {
case (s: AliasExpression, _) => s case (s: AliasExpression, _) => s
......
...@@ -18,18 +18,23 @@ ...@@ -18,18 +18,23 @@
package org.apache.atlas.query package org.apache.atlas.query
import java.util
import java.util.Date import java.util.Date
import com.thinkaurelius.titan.core.TitanVertex import com.thinkaurelius.titan.core.TitanVertex
import com.tinkerpop.blueprints.Direction import com.tinkerpop.blueprints.{Vertex, Direction}
import org.apache.atlas.AtlasException
import org.apache.atlas.query.Expressions.{ComparisonExpression, ExpressionException} import org.apache.atlas.query.Expressions.{ComparisonExpression, ExpressionException}
import org.apache.atlas.query.TypeUtils.FieldInfo import org.apache.atlas.query.TypeUtils.FieldInfo
import org.apache.atlas.repository.graph.GraphBackedMetadataRepository
import org.apache.atlas.typesystem.persistence.Id import org.apache.atlas.typesystem.persistence.Id
import org.apache.atlas.typesystem.types.DataTypes._ import org.apache.atlas.typesystem.types.DataTypes._
import org.apache.atlas.typesystem.types._ import org.apache.atlas.typesystem.types._
import org.apache.atlas.typesystem.{ITypedInstance, ITypedReferenceableInstance} import org.apache.atlas.typesystem.{ITypedInstance, ITypedReferenceableInstance}
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
/** /**
* Represents the Bridge between the QueryProcessor and the Graph Persistence scheme used. * Represents the Bridge between the QueryProcessor and the Graph Persistence scheme used.
...@@ -186,13 +191,15 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies { ...@@ -186,13 +191,15 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies {
val superTypeAttributeName = "superTypeNames" val superTypeAttributeName = "superTypeNames"
val idAttributeName = "guid" val idAttributeName = "guid"
def edgeLabel(dataType: IDataType[_], aInfo: AttributeInfo) = s"${dataType.getName}.${aInfo.name}" def edgeLabel(dataType: IDataType[_], aInfo: AttributeInfo) = s"__${dataType.getName}.${aInfo.name}"
def edgeLabel(propertyName: String) = s"__${propertyName}"
val fieldPrefixInSelect = "it" val fieldPrefixInSelect = "it"
def traitLabel(cls: IDataType[_], traitName: String) = s"${cls.getName}.$traitName" def traitLabel(cls: IDataType[_], traitName: String) = s"${cls.getName}.$traitName"
def fieldNameInVertex(dataType: IDataType[_], aInfo: AttributeInfo) = aInfo.name def fieldNameInVertex(dataType: IDataType[_], aInfo: AttributeInfo) = GraphBackedMetadataRepository.getQualifiedName(dataType, aInfo.name)
def getIdFromVertex(dataTypeNm: String, v: TitanVertex): Id = def getIdFromVertex(dataTypeNm: String, v: TitanVertex): Id =
new Id(v.getId.toString, 0, dataTypeNm) new Id(v.getId.toString, 0, dataTypeNm)
...@@ -209,6 +216,8 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies { ...@@ -209,6 +216,8 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies {
def constructInstance[U](dataType: IDataType[U], v: AnyRef): U = { def constructInstance[U](dataType: IDataType[U], v: AnyRef): U = {
dataType.getTypeCategory match { dataType.getTypeCategory match {
case DataTypes.TypeCategory.PRIMITIVE => dataType.convert(v, Multiplicity.OPTIONAL) case DataTypes.TypeCategory.PRIMITIVE => dataType.convert(v, Multiplicity.OPTIONAL)
case DataTypes.TypeCategory.ARRAY =>
dataType.convert(v, Multiplicity.OPTIONAL)
case DataTypes.TypeCategory.STRUCT case DataTypes.TypeCategory.STRUCT
if dataType.getName == TypeSystem.getInstance().getIdType.getName => { if dataType.getName == TypeSystem.getInstance().getIdType.getName => {
val sType = dataType.asInstanceOf[StructType] val sType = dataType.asInstanceOf[StructType]
...@@ -278,7 +287,7 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies { ...@@ -278,7 +287,7 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies {
case DataTypes.TypeCategory.PRIMITIVE => loadPrimitiveAttribute(dataType, aInfo, i, v) case DataTypes.TypeCategory.PRIMITIVE => loadPrimitiveAttribute(dataType, aInfo, i, v)
case DataTypes.TypeCategory.ENUM => loadEnumAttribute(dataType, aInfo, i, v) case DataTypes.TypeCategory.ENUM => loadEnumAttribute(dataType, aInfo, i, v)
case DataTypes.TypeCategory.ARRAY => case DataTypes.TypeCategory.ARRAY =>
throw new UnsupportedOperationException(s"load for ${aInfo.dataType()} not supported") loadArrayAttribute(dataType, aInfo, i, v)
case DataTypes.TypeCategory.MAP => case DataTypes.TypeCategory.MAP =>
throw new UnsupportedOperationException(s"load for ${aInfo.dataType()} not supported") throw new UnsupportedOperationException(s"load for ${aInfo.dataType()} not supported")
case DataTypes.TypeCategory.STRUCT => loadStructAttribute(dataType, aInfo, i, v) case DataTypes.TypeCategory.STRUCT => loadStructAttribute(dataType, aInfo, i, v)
...@@ -314,9 +323,26 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies { ...@@ -314,9 +323,26 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies {
} }
} }
private def loadStructAttribute(dataType: IDataType[_], aInfo: AttributeInfo,
private def loadArrayAttribute[T](dataType: IDataType[_], aInfo: AttributeInfo,
i: ITypedInstance, v: TitanVertex): Unit = { i: ITypedInstance, v: TitanVertex): Unit = {
val eLabel = edgeLabel(FieldInfo(dataType, aInfo, null)) import scala.collection.JavaConversions._
val list: java.util.List[_] = v.getProperty(aInfo.name)
val arrayType: DataTypes.ArrayType = aInfo.dataType.asInstanceOf[ArrayType]
var values = new util.ArrayList[Any]
list.foreach( listElement =>
values += mapVertexToCollectionEntry(v, aInfo, arrayType.getElemType, i, listElement)
)
i.set(aInfo.name, values)
}
private def loadStructAttribute(dataType: IDataType[_], aInfo: AttributeInfo,
i: ITypedInstance, v: TitanVertex, edgeLbl: Option[String] = None): Unit = {
val eLabel = edgeLbl match {
case Some(x) => x
case None => edgeLabel(FieldInfo(dataType, aInfo, null))
}
val edges = v.getEdges(Direction.OUT, eLabel) val edges = v.getEdges(Direction.OUT, eLabel)
val sVertex = edges.iterator().next().getVertex(Direction.IN).asInstanceOf[TitanVertex] val sVertex = edges.iterator().next().getVertex(Direction.IN).asInstanceOf[TitanVertex]
if (aInfo.dataType().getTypeCategory == DataTypes.TypeCategory.STRUCT) { if (aInfo.dataType().getTypeCategory == DataTypes.TypeCategory.STRUCT) {
...@@ -329,5 +355,22 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies { ...@@ -329,5 +355,22 @@ object GraphPersistenceStrategy1 extends GraphPersistenceStrategies {
i.set(aInfo.name, cInstance) i.set(aInfo.name, cInstance)
} }
} }
private def mapVertexToCollectionEntry(instanceVertex: TitanVertex, attributeInfo: AttributeInfo, elementType: IDataType[_], i: ITypedInstance, value: Any): Any = {
elementType.getTypeCategory match {
case DataTypes.TypeCategory.PRIMITIVE => value
case DataTypes.TypeCategory.ENUM => value
case DataTypes.TypeCategory.STRUCT =>
throw new UnsupportedOperationException(s"load for ${attributeInfo.dataType()} not supported")
case DataTypes.TypeCategory.TRAIT =>
throw new UnsupportedOperationException(s"load for ${attributeInfo.dataType()} not supported")
case DataTypes.TypeCategory.CLASS => //loadStructAttribute(elementType, attributeInfo, i, v)
throw new UnsupportedOperationException(s"load for ${attributeInfo.dataType()} not supported")
case _ =>
throw new UnsupportedOperationException(s"load for ${attributeInfo.dataType()} not supported")
}
}
} }
...@@ -77,6 +77,7 @@ trait SelectExpressionHandling { ...@@ -77,6 +77,7 @@ trait SelectExpressionHandling {
val l = ArrayBuffer[String]() val l = ArrayBuffer[String]()
e.traverseUp { e.traverseUp {
case BackReference(alias, _, _) => l += alias case BackReference(alias, _, _) => l += alias
case ClassExpression(clsName) => l += clsName
} }
l.toSet.toList l.toSet.toList
} }
...@@ -140,14 +141,19 @@ class GremlinTranslator(expr: Expression, ...@@ -140,14 +141,19 @@ class GremlinTranslator(expr: Expression,
} }
val validateComparisonForm: PartialFunction[Expression, Unit] = { val validateComparisonForm: PartialFunction[Expression, Unit] = {
case c@ComparisonExpression(_, left, right) => case c@ComparisonExpression(op, left, right) =>
if (!left.isInstanceOf[FieldExpression]) { if (!left.isInstanceOf[FieldExpression]) {
throw new GremlinTranslationException(c, s"lhs of comparison is not a field") throw new GremlinTranslationException(c, s"lhs of comparison is not a field")
} }
if (!right.isInstanceOf[Literal[_]]) { if (!right.isInstanceOf[Literal[_]] && !right.isInstanceOf[ListLiteral[_]]) {
throw new GremlinTranslationException(c, throw new GremlinTranslationException(c,
s"rhs of comparison is not a literal") s"rhs of comparison is not a literal")
} }
if(right.isInstanceOf[ListLiteral[_]] && (!op.equals("=") && !op.equals("!="))) {
throw new GremlinTranslationException(c,
s"operation not supported with list literal")
}
() ()
} }
...@@ -201,7 +207,8 @@ class GremlinTranslator(expr: Expression, ...@@ -201,7 +207,8 @@ class GremlinTranslator(expr: Expression,
typeTestExpression(clsName) typeTestExpression(clsName)
case TraitExpression(clsName) => case TraitExpression(clsName) =>
typeTestExpression(clsName) typeTestExpression(clsName)
case fe@FieldExpression(fieldName, fInfo, child) if fe.dataType.getTypeCategory == TypeCategory.PRIMITIVE => { case fe@FieldExpression(fieldName, fInfo, child)
if fe.dataType.getTypeCategory == TypeCategory.PRIMITIVE || fe.dataType.getTypeCategory == TypeCategory.ARRAY => {
val fN = "\"" + gPersistenceBehavior.fieldNameInVertex(fInfo.dataType, fInfo.attrInfo) + "\"" val fN = "\"" + gPersistenceBehavior.fieldNameInVertex(fInfo.dataType, fInfo.attrInfo) + "\""
child match { child match {
case Some(e) => s"${genQuery(e, inSelect)}.$fN" case Some(e) => s"${genQuery(e, inSelect)}.$fN"
...@@ -218,8 +225,7 @@ class GremlinTranslator(expr: Expression, ...@@ -218,8 +225,7 @@ class GremlinTranslator(expr: Expression,
case None => step case None => step
} }
} }
case fe@FieldExpression(fieldName, fInfo, child) case fe@FieldExpression(fieldName, fInfo, child) if fInfo.traitName != null => {
if fInfo.traitName != null => {
val direction = gPersistenceBehavior.instanceToTraitEdgeDirection val direction = gPersistenceBehavior.instanceToTraitEdgeDirection
val edgeLbl = gPersistenceBehavior.edgeLabel(fInfo) val edgeLbl = gPersistenceBehavior.edgeLabel(fInfo)
val step = s"""$direction("$edgeLbl")""" val step = s"""$direction("$edgeLbl")"""
...@@ -294,12 +300,20 @@ class GremlinTranslator(expr: Expression, ...@@ -294,12 +300,20 @@ class GremlinTranslator(expr: Expression,
s"""out("${gPersistenceBehavior.traitLabel(clsExp.dataType, traitName)}")""" s"""out("${gPersistenceBehavior.traitLabel(clsExp.dataType, traitName)}")"""
case isTraitUnaryExpression(traitName, child) => case isTraitUnaryExpression(traitName, child) =>
s"""out("${gPersistenceBehavior.traitLabel(child.dataType, traitName)}")""" s"""out("${gPersistenceBehavior.traitLabel(child.dataType, traitName)}")"""
case hasFieldLeafExpression(fieldName, Some(clsExp)) => case hasFieldLeafExpression(fieldName, clsExp) => clsExp match {
s"""has("$fieldName")""" case None => s"""has("$fieldName")"""
case Some(x) =>
x match {
case c: ClassExpression =>
s"""has("${x.asInstanceOf[ClassExpression].clsName}.$fieldName")"""
case default => s"""has("$fieldName")"""
}
}
case hasFieldUnaryExpression(fieldName, child) => case hasFieldUnaryExpression(fieldName, child) =>
s"""${genQuery(child, inSelect)}.has("$fieldName")""" s"""${genQuery(child, inSelect)}.has("$fieldName")"""
case ArithmeticExpression(symb, left, right) => s"${genQuery(left, inSelect)} $symb ${genQuery(right, inSelect)}" case ArithmeticExpression(symb, left, right) => s"${genQuery(left, inSelect)} $symb ${genQuery(right, inSelect)}"
case l: Literal[_] => l.toString case l: Literal[_] => l.toString
case list: ListLiteral[_] => list.toString
case in@TraitInstanceExpression(child) => { case in@TraitInstanceExpression(child) => {
val direction = gPersistenceBehavior.traitToInstanceEdgeDirection val direction = gPersistenceBehavior.traitToInstanceEdgeDirection
s"${genQuery(child, inSelect)}.$direction()" s"${genQuery(child, inSelect)}.$direction()"
...@@ -356,6 +370,7 @@ class GremlinTranslator(expr: Expression, ...@@ -356,6 +370,7 @@ class GremlinTranslator(expr: Expression,
} }
/* /*
* TODO
* Translation Issues: * Translation Issues:
* 1. back references in filters. For e.g. testBackreference: 'DB as db Table where (db.name = "Reporting")' * 1. back references in filters. For e.g. testBackreference: 'DB as db Table where (db.name = "Reporting")'
* this is translated to: * this is translated to:
......
...@@ -34,6 +34,8 @@ trait QueryKeywords { ...@@ -34,6 +34,8 @@ trait QueryKeywords {
protected implicit def asParser(k: Keyword): Parser[String] = k.str protected implicit def asParser(k: Keyword): Parser[String] = k.str
protected val LIST_LPAREN = Keyword("[")
protected val LIST_RPAREN = Keyword("]")
protected val LPAREN = Keyword("(") protected val LPAREN = Keyword("(")
protected val RPAREN = Keyword(")") protected val RPAREN = Keyword(")")
protected val EQ = Keyword("=") protected val EQ = Keyword("=")
...@@ -222,8 +224,11 @@ class QueryParser extends StandardTokenParsers with QueryKeywords with Expressio ...@@ -222,8 +224,11 @@ class QueryParser extends StandardTokenParsers with QueryKeywords with Expressio
def multiERight = (STAR | DIV) ~ atomE ^^ { case op ~ r => (op, r)} def multiERight = (STAR | DIV) ~ atomE ^^ { case op ~ r => (op, r)}
def atomE = literal | identifier | LPAREN ~> expr <~ RPAREN | listLiteral
def atomE = literal | identifier | LPAREN ~> expr <~ RPAREN def listLiteral = LIST_LPAREN ~ rep1sep(literal, COMMA) ~ LIST_RPAREN ^^ {
case lp ~ le ~ rp => list(le)
}
def identifier = rep1sep(ident, DOT) ^^ { l => l match { def identifier = rep1sep(ident, DOT) ^^ { l => l match {
case h :: Nil => id(h) case h :: Nil => id(h)
......
...@@ -112,7 +112,7 @@ object FieldValidator extends PartialFunction[Expression, Expression] { ...@@ -112,7 +112,7 @@ object FieldValidator extends PartialFunction[Expression, Expression] {
case fe@FieldExpression(fNm, fInfo, Some(child)) if isSrc(child) => case fe@FieldExpression(fNm, fInfo, Some(child)) if isSrc(child) =>
throw new ExpressionException(fe, s"srcType of field doesn't match input type") throw new ExpressionException(fe, s"srcType of field doesn't match input type")
case hasFieldUnaryExpression(fNm, child) if child.dataType == srcDataType => case hasFieldUnaryExpression(fNm, child) if child.dataType == srcDataType =>
hasFieldLeafExpression(fNm) hasFieldLeafExpression(fNm, Some(child))
case hF@hasFieldUnaryExpression(fNm, child) if isSrc(child) => case hF@hasFieldUnaryExpression(fNm, child) if isSrc(child) =>
throw new ExpressionException(hF, s"srcType of field doesn't match input type") throw new ExpressionException(hF, s"srcType of field doesn't match input type")
case isTraitUnaryExpression(fNm, child) if child.dataType == srcDataType => case isTraitUnaryExpression(fNm, child) if child.dataType == srcDataType =>
......
...@@ -101,12 +101,12 @@ public class GraphBackedMetadataRepositoryTest { ...@@ -101,12 +101,12 @@ public class GraphBackedMetadataRepositoryTest {
try { try {
//TODO - Fix failure during shutdown while using BDB //TODO - Fix failure during shutdown while using BDB
graphProvider.get().shutdown(); graphProvider.get().shutdown();
} catch(Exception e) { } catch (Exception e) {
e.printStackTrace(); e.printStackTrace();
} }
try { try {
TitanCleanup.clear(graphProvider.get()); TitanCleanup.clear(graphProvider.get());
} catch(Exception e) { } catch (Exception e) {
e.printStackTrace(); e.printStackTrace();
} }
} }
......
...@@ -40,6 +40,7 @@ import org.apache.atlas.typesystem.types.IDataType; ...@@ -40,6 +40,7 @@ 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.TypeSystem; import org.apache.atlas.typesystem.types.TypeSystem;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.testng.Assert;
import org.testng.annotations.AfterClass; import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test; import org.testng.annotations.Test;
...@@ -147,7 +148,7 @@ public class GraphRepoMapperScaleTest { ...@@ -147,7 +148,7 @@ public class GraphRepoMapperScaleTest {
searchWithOutIndex("hive_table.name", "bar-999"); searchWithOutIndex("hive_table.name", "bar-999");
searchWithIndex("hive_table.name", "bar-999"); searchWithIndex("hive_table.name", "bar-999");
searchWithIndex("hive_table.created", Compare.GREATER_THAN_EQUAL, TestUtils.TEST_DATE_IN_LONG); searchWithIndex("hive_table.created", Compare.GREATER_THAN_EQUAL, TestUtils.TEST_DATE_IN_LONG, 1000);
for (int index = 500; index < 600; index++) { for (int index = 500; index < 600; index++) {
searchWithIndex("hive_table.name", "bar-" + index); searchWithIndex("hive_table.name", "bar-" + index);
...@@ -185,7 +186,7 @@ public class GraphRepoMapperScaleTest { ...@@ -185,7 +186,7 @@ public class GraphRepoMapperScaleTest {
} }
} }
private void searchWithIndex(String key, Predicate searchPredicate, Object value) { private void searchWithIndex(String key, Predicate searchPredicate, Object value, int expectedResults) {
TitanGraph graph = graphProvider.get(); TitanGraph graph = graphProvider.get();
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
int count = 0; int count = 0;
...@@ -197,6 +198,7 @@ public class GraphRepoMapperScaleTest { ...@@ -197,6 +198,7 @@ public class GraphRepoMapperScaleTest {
} finally { } finally {
System.out.println("Search on [" + key + "=" + value + "] returned results: " + count + ", took " + ( System.out.println("Search on [" + key + "=" + value + "] returned results: " + count + ", took " + (
System.currentTimeMillis() - start) + " ms"); System.currentTimeMillis() - start) + " ms");
Assert.assertEquals(count, expectedResults);
} }
} }
......
...@@ -19,7 +19,10 @@ ...@@ -19,7 +19,10 @@
package org.apache.atlas.query package org.apache.atlas.query
import com.thinkaurelius.titan.core.TitanGraph import com.thinkaurelius.titan.core.TitanGraph
import com.thinkaurelius.titan.core.util.TitanCleanup
import org.apache.atlas.discovery.graph.DefaultGraphPersistenceStrategy
import org.apache.atlas.query.Expressions._ import org.apache.atlas.query.Expressions._
import org.apache.atlas.repository.graph.{TitanGraphProvider, GraphBackedMetadataRepository}
import org.apache.atlas.typesystem.types.TypeSystem import org.apache.atlas.typesystem.types.TypeSystem
import org.junit.runner.RunWith import org.junit.runner.RunWith
import org.scalatest._ import org.scalatest._
...@@ -29,15 +32,25 @@ import org.scalatest.junit.JUnitRunner ...@@ -29,15 +32,25 @@ import org.scalatest.junit.JUnitRunner
class GremlinTest2 extends FunSuite with BeforeAndAfterAll with BaseGremlinTest { class GremlinTest2 extends FunSuite with BeforeAndAfterAll with BaseGremlinTest {
var g: TitanGraph = null var g: TitanGraph = null
var gProvider:TitanGraphProvider = null;
var gp:GraphPersistenceStrategies = null;
override def beforeAll() { override def beforeAll() {
TypeSystem.getInstance().reset() TypeSystem.getInstance().reset()
QueryTestsUtils.setupTypes QueryTestsUtils.setupTypes
g = QueryTestsUtils.setupTestGraph gProvider = new TitanGraphProvider();
gp = new DefaultGraphPersistenceStrategy(new GraphBackedMetadataRepository(gProvider))
g = QueryTestsUtils.setupTestGraph(gProvider)
} }
override def afterAll() { override def afterAll() {
g.shutdown() g.shutdown()
try {
TitanCleanup.clear(g);
} catch {
case ex: Exception =>
print("Could not clear the graph ", ex);
}
} }
test("testTraitSelect") { test("testTraitSelect") {
......
...@@ -27,9 +27,7 @@ class ParserTest extends BaseTest { ...@@ -27,9 +27,7 @@ class ParserTest extends BaseTest {
@Before @Before
override def setup { override def setup {
super.setup super.setup
QueryTestsUtils.setupTypes QueryTestsUtils.setupTypes
} }
@Test def testFrom: Unit = { @Test def testFrom: Unit = {
...@@ -98,4 +96,13 @@ class ParserTest extends BaseTest { ...@@ -98,4 +96,13 @@ class ParserTest extends BaseTest {
) )
} }
@Test def testList: Unit = {
val p = new QueryParser
println(p(
"Partition as p where values = ['2015-01-01']," +
" table where name = 'tableoq8ty'," +
" db where name = 'default' and clusterName = 'test'").right.get.toString
)
}
} }
...@@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList ...@@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList
import com.thinkaurelius.titan.core.{TitanFactory, TitanGraph} import com.thinkaurelius.titan.core.{TitanFactory, TitanGraph}
import com.tinkerpop.blueprints.Vertex import com.tinkerpop.blueprints.Vertex
import com.typesafe.config.{Config, ConfigFactory} import com.typesafe.config.{Config, ConfigFactory}
import org.apache.atlas.repository.graph.TitanGraphProvider
import org.apache.atlas.typesystem.types._ import org.apache.atlas.typesystem.types._
import org.apache.commons.configuration.{Configuration, ConfigurationException, MapConfiguration} import org.apache.commons.configuration.{Configuration, ConfigurationException, MapConfiguration}
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils
...@@ -49,9 +50,9 @@ trait GraphUtils { ...@@ -49,9 +50,9 @@ trait GraphUtils {
} }
def titanGraph(conf: Config) = { def titanGraph(conf: Configuration) = {
try { try {
val g = TitanFactory.open(getConfiguration(conf)) val g = TitanFactory.open(conf)
val mgmt = g.getManagementSystem val mgmt = g.getManagementSystem
val typname = mgmt.makePropertyKey("typeName").dataType(classOf[String]).make() val typname = mgmt.makePropertyKey("typeName").dataType(classOf[String]).make()
mgmt.buildIndex("byTypeName", classOf[Vertex]).addKey(typname).buildCompositeIndex() mgmt.buildIndex("byTypeName", classOf[Vertex]).addKey(typname).buildCompositeIndex()
...@@ -79,30 +80,44 @@ object QueryTestsUtils extends GraphUtils { ...@@ -79,30 +80,44 @@ object QueryTestsUtils extends GraphUtils {
Array( Array(
attrDef("name", DataTypes.STRING_TYPE), attrDef("name", DataTypes.STRING_TYPE),
attrDef("owner", DataTypes.STRING_TYPE), attrDef("owner", DataTypes.STRING_TYPE),
attrDef("createTime", DataTypes.INT_TYPE) attrDef("createTime", DataTypes.INT_TYPE),
attrDef("clusterName", DataTypes.STRING_TYPE)
)) ))
def storageDescClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "StorageDesc", null, def hiveOrderDef = new StructTypeDefinition("HiveOrder",
Array(
attrDef("col", DataTypes.STRING_TYPE),
attrDef("order", DataTypes.INT_TYPE)
))
def storageDescClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "StorageDescriptor", null,
Array( Array(
attrDef("inputFormat", DataTypes.STRING_TYPE), attrDef("inputFormat", DataTypes.STRING_TYPE),
attrDef("outputFormat", DataTypes.STRING_TYPE) attrDef("outputFormat", DataTypes.STRING_TYPE),
new AttributeDefinition("sortCols", DataTypes.arrayTypeName("HiveOrder"), Multiplicity.REQUIRED, false, null)
)) ))
def columnClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "Column", null, def columnClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "Column", null,
Array( Array(
attrDef("name", DataTypes.STRING_TYPE), attrDef("name", DataTypes.STRING_TYPE),
attrDef("dataType", DataTypes.STRING_TYPE), attrDef("dataType", DataTypes.STRING_TYPE),
new AttributeDefinition("sd", "StorageDesc", Multiplicity.REQUIRED, false, null) new AttributeDefinition("sd", "StorageDescriptor", Multiplicity.REQUIRED, false, null)
)) ))
def tblClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "Table", null, def tblClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "Table", null,
Array( Array(
attrDef("name", DataTypes.STRING_TYPE), attrDef("name", DataTypes.STRING_TYPE),
new AttributeDefinition("db", "DB", Multiplicity.REQUIRED, false, null), new AttributeDefinition("db", "DB", Multiplicity.REQUIRED, false, null),
new AttributeDefinition("sd", "StorageDesc", Multiplicity.REQUIRED, false, null), new AttributeDefinition("sd", "StorageDescriptor", Multiplicity.REQUIRED, false, null),
attrDef("created", DataTypes.DATE_TYPE) attrDef("created", DataTypes.DATE_TYPE)
)) ))
def partitionClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "Partition", null,
Array(
new AttributeDefinition("values", DataTypes.arrayTypeName(DataTypes.STRING_TYPE.getName), Multiplicity.REQUIRED, false, null),
new AttributeDefinition("table", "Table", Multiplicity.REQUIRED, false, null)
))
def loadProcessClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "LoadProcess", null, def loadProcessClsDef = new HierarchicalTypeDefinition[ClassType](classOf[ClassType], "LoadProcess", null,
Array( Array(
attrDef("name", DataTypes.STRING_TYPE), attrDef("name", DataTypes.STRING_TYPE),
...@@ -128,19 +143,17 @@ object QueryTestsUtils extends GraphUtils { ...@@ -128,19 +143,17 @@ object QueryTestsUtils extends GraphUtils {
Array[AttributeDefinition]()) Array[AttributeDefinition]())
TypeSystem.getInstance().defineTypes(ImmutableList.of[EnumTypeDefinition], TypeSystem.getInstance().defineTypes(ImmutableList.of[EnumTypeDefinition],
ImmutableList.of[StructTypeDefinition], ImmutableList.of[StructTypeDefinition](hiveOrderDef),
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,
loadProcessClsDef, viewClsDef)) partitionClsDef, loadProcessClsDef, viewClsDef))
() ()
} }
def setupTestGraph: TitanGraph = { def setupTestGraph(gp: TitanGraphProvider): TitanGraph = {
var conf = ConfigFactory.load() val g = gp.get
conf = conf.getConfig("graphRepo")
val g = titanGraph(conf)
val manager: ScriptEngineManager = new ScriptEngineManager val manager: ScriptEngineManager = new ScriptEngineManager
val engine: ScriptEngine = manager.getEngineByName("gremlin-groovy") val engine: ScriptEngine = manager.getEngineByName("gremlin-groovy")
val bindings: Bindings = engine.createBindings val bindings: Bindings = engine.createBindings
......
...@@ -48,9 +48,9 @@ public class DataTypes { ...@@ -48,9 +48,9 @@ public class DataTypes {
public static BigDecimalType BIGDECIMAL_TYPE = new BigDecimalType(); public static BigDecimalType BIGDECIMAL_TYPE = new BigDecimalType();
public static DateType DATE_TYPE = new DateType(); public static DateType DATE_TYPE = new DateType();
public static StringType STRING_TYPE = new StringType(); public static StringType STRING_TYPE = new StringType();
static String ARRAY_TYPE_PREFIX = "array<"; public static String ARRAY_TYPE_PREFIX = "array<";
static String ARRAY_TYPE_SUFFIX = ">"; static String ARRAY_TYPE_SUFFIX = ">";
static String MAP_TYPE_PREFIX = "map<"; public static String MAP_TYPE_PREFIX = "map<";
static String MAP_TYPE_SUFFIX = ">"; static String MAP_TYPE_SUFFIX = ">";
public static String arrayTypeName(String elemTypeName) { public static String arrayTypeName(String elemTypeName) {
......
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