You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@s2graph.apache.org by st...@apache.org on 2016/12/01 08:30:11 UTC
[01/10] incubator-s2graph git commit: [S2GRAPH-131]: Add actual
implementation on interfaces from TinkerPop3 structure package. - Change
core.Edge/Vertex/Graph to core.S2Edge/S2Vertex/S2Graph. - Implement base
interfaces for tinkerpop3 structure packag
Repository: incubator-s2graph
Updated Branches:
refs/heads/master f74c224ac -> 247b2cb9d
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala b/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
index 2d34c7a..1a85dba 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
@@ -54,7 +54,7 @@ object RestHandler {
* Public API, only return Future.successful or Future.failed
* Don't throw exception
*/
-class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
+class RestHandler(graph: S2Graph)(implicit ec: ExecutionContext) {
import RestHandler._
val requestParser = new RequestParser(graph)
@@ -172,7 +172,7 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
}
def getEdgesAsync(jsonQuery: JsValue, impIdOpt: Option[String] = None)
- (post: (Graph, QueryOption, StepResult) => JsValue): Future[JsValue] = {
+ (post: (S2Graph, QueryOption, StepResult) => JsValue): Future[JsValue] = {
def query(obj: JsValue): Future[JsValue] = {
(obj \ "queries").asOpt[JsValue] match {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/Storage.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/Storage.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/Storage.scala
index efe7a3d..59b7518 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/Storage.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/Storage.scala
@@ -39,10 +39,10 @@ import com.typesafe.config.{Config, ConfigFactory}
import org.apache.hadoop.hbase.util.Bytes
-abstract class Storage[Q, R](val graph: Graph,
+abstract class Storage[Q, R](val graph: S2Graph,
val config: Config)(implicit ec: ExecutionContext) {
import HBaseType._
- import Graph._
+ import S2Graph._
val BackoffTimeout = graph.BackoffTimeout
val MaxRetryNum = graph.MaxRetryNum
@@ -100,7 +100,7 @@ abstract class Storage[Q, R](val graph: Graph,
* @param vertex: vertex to serialize
* @return serializer implementation
*/
- def vertexSerializer(vertex: Vertex): Serializable[Vertex] = new VertexSerializable(vertex)
+ def vertexSerializer(vertex: S2Vertex): Serializable[S2Vertex] = new VertexSerializable(vertex)
/**
* create deserializer that can parse stored CanSKeyValue into snapshotEdge.
@@ -122,7 +122,7 @@ abstract class Storage[Q, R](val graph: Graph,
snapshotEdgeDeserializers.get(schemaVer).getOrElse(throw new RuntimeException(s"not supported version: ${schemaVer}"))
/** create deserializer that can parse stored CanSKeyValue into indexEdge. */
- val indexEdgeDeserializers: Map[String, Deserializable[Edge]] = Map(
+ val indexEdgeDeserializers: Map[String, Deserializable[S2Edge]] = Map(
VERSION1 -> new IndexEdgeDeserializable(graph),
VERSION2 -> new IndexEdgeDeserializable(graph),
VERSION3 -> new IndexEdgeDeserializable(graph),
@@ -133,7 +133,7 @@ abstract class Storage[Q, R](val graph: Graph,
indexEdgeDeserializers.get(schemaVer).getOrElse(throw new RuntimeException(s"not supported version: ${schemaVer}"))
/** create deserializer that can parser stored CanSKeyValue into vertex. */
- val vertexDeserializer: Deserializable[Vertex] = new VertexDeserializable(graph)
+ val vertexDeserializer: Deserializable[S2Vertex] = new VertexDeserializable(graph)
/**
@@ -195,7 +195,7 @@ abstract class Storage[Q, R](val graph: Graph,
* @param queryRequest
* @return
*/
- protected def buildRequest(queryRequest: QueryRequest, edge: Edge): Q
+ protected def buildRequest(queryRequest: QueryRequest, edge: S2Edge): Q
/**
* fetch IndexEdges for given queryParam in queryRequest.
@@ -242,7 +242,7 @@ abstract class Storage[Q, R](val graph: Graph,
* @param withWait
* @return
*/
- def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]]
+ def incrementCounts(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]]
/**
* this method need to be called when client shutdown. this is responsible to cleanUp the resources
@@ -276,9 +276,9 @@ abstract class Storage[Q, R](val graph: Graph,
/** Public Interface */
- def getVertices(vertices: Seq[Vertex]): Future[Seq[Vertex]] = {
+ def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
def fromResult(kvs: Seq[SKeyValue],
- version: String): Option[Vertex] = {
+ version: String): Option[S2Vertex] = {
if (kvs.isEmpty) None
else vertexDeserializer.fromKeyValues(None, kvs, version, None)
// .map(S2Vertex(graph, _))
@@ -286,7 +286,7 @@ abstract class Storage[Q, R](val graph: Graph,
val futures = vertices.map { vertex =>
val queryParam = QueryParam.Empty
- val q = Query.toQuery(Seq(vertex), queryParam)
+ val q = Query.toQuery(Seq(vertex), Seq(queryParam))
val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
fetchVertexKeyValues(queryRequest).map { kvs =>
fromResult(kvs, vertex.serviceColumn.schemaVersion)
@@ -297,7 +297,7 @@ abstract class Storage[Q, R](val graph: Graph,
Future.sequence(futures).map { result => result.toList.flatten }
}
- def mutateStrongEdges(_edges: Seq[Edge], withWait: Boolean): Future[Seq[Boolean]] = {
+ def mutateStrongEdges(_edges: Seq[S2Edge], withWait: Boolean): Future[Seq[Boolean]] = {
val edgeWithIdxs = _edges.zipWithIndex
val grouped = edgeWithIdxs.groupBy { case (edge, idx) =>
@@ -332,7 +332,7 @@ abstract class Storage[Q, R](val graph: Graph,
}
}
- def mutateVertex(vertex: Vertex, withWait: Boolean): Future[Boolean] = {
+ def mutateVertex(vertex: S2Vertex, withWait: Boolean): Future[Boolean] = {
if (vertex.op == GraphUtil.operations("delete")) {
writeToStorage(vertex.hbaseZkAddr,
vertexSerializer(vertex).toKeyValues.map(_.copy(operation = SKeyValue.Delete)), withWait)
@@ -344,14 +344,14 @@ abstract class Storage[Q, R](val graph: Graph,
}
}
- def mutateVertices(vertices: Seq[Vertex],
+ def mutateVertices(vertices: Seq[S2Vertex],
withWait: Boolean = false): Future[Seq[Boolean]] = {
val futures = vertices.map { vertex => mutateVertex(vertex, withWait) }
Future.sequence(futures)
}
- def mutateEdgesInner(edges: Seq[Edge],
+ def mutateEdgesInner(edges: Seq[S2Edge],
checkConsistency: Boolean,
withWait: Boolean): Future[Boolean] = {
assert(edges.nonEmpty)
@@ -360,7 +360,7 @@ abstract class Storage[Q, R](val graph: Graph,
val zkQuorum = edges.head.innerLabel.hbaseZkAddr
val futures = edges.map { edge =>
- val (_, edgeUpdate) = Edge.buildOperation(None, Seq(edge))
+ val (_, edgeUpdate) = S2Edge.buildOperation(None, Seq(edge))
val mutations =
indexedEdgeMutations(edgeUpdate) ++ snapshotEdgeMutations(edgeUpdate) ++ increments(edgeUpdate)
@@ -382,7 +382,7 @@ abstract class Storage[Q, R](val graph: Graph,
Random.nextInt(Math.min(BackoffTimeout, slot * Math.pow(2, tryNum)).toInt)
}
- def retry(tryNum: Int)(edges: Seq[Edge], statusCode: Byte, fetchedSnapshotEdgeOpt: Option[Edge]): Future[Boolean] = {
+ def retry(tryNum: Int)(edges: Seq[S2Edge], statusCode: Byte, fetchedSnapshotEdgeOpt: Option[S2Edge]): Future[Boolean] = {
if (tryNum >= MaxRetryNum) {
edges.foreach { edge =>
logger.error(s"commit failed after $MaxRetryNum\n${edge.toLogString}")
@@ -442,9 +442,9 @@ abstract class Storage[Q, R](val graph: Graph,
}
}
- protected def commitUpdate(edges: Seq[Edge],
+ protected def commitUpdate(edges: Seq[S2Edge],
statusCode: Byte,
- fetchedSnapshotEdgeOpt: Option[Edge]): Future[Boolean] = {
+ fetchedSnapshotEdgeOpt: Option[S2Edge]): Future[Boolean] = {
// Future.failed(new PartialFailureException(edges.head, 0, "ahahah"))
assert(edges.nonEmpty)
// assert(statusCode == 0 || fetchedSnapshotEdgeOpt.isDefined)
@@ -460,7 +460,7 @@ abstract class Storage[Q, R](val graph: Graph,
* lock = (squashedEdge, pendingE)
* releaseLock = (edgeMutate.newSnapshotEdge, None)
*/
- val (squashedEdge, edgeMutate) = Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
+ val (squashedEdge, edgeMutate) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
assert(edgeMutate.newSnapshotEdge.isDefined)
@@ -482,7 +482,7 @@ abstract class Storage[Q, R](val graph: Graph,
* lock = (snapshotEdge, pendingE)
* releaseLock = (edgeMutate.newSnapshotEdge, None)
*/
- val (squashedEdge, edgeMutate) = Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
+ val (squashedEdge, edgeMutate) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
if (edgeMutate.newSnapshotEdge.isEmpty) {
logger.debug(s"drop this requests: \n${edges.map(_.toLogString).mkString("\n")}")
Future.successful(true)
@@ -508,8 +508,8 @@ abstract class Storage[Q, R](val graph: Graph,
*/
logger.debug(s"${pendingEdge.toLogString} has been expired.")
val (squashedEdge, edgeMutate) =
- if (pendingEdge.ts == snapshotEdge.ts) Edge.buildOperation(None, pendingEdge +: edges)
- else Edge.buildOperation(fetchedSnapshotEdgeOpt, pendingEdge +: edges)
+ if (pendingEdge.ts == snapshotEdge.ts) S2Edge.buildOperation(None, pendingEdge +: edges)
+ else S2Edge.buildOperation(fetchedSnapshotEdgeOpt, pendingEdge +: edges)
val lockTs = Option(System.currentTimeMillis())
val newPendingEdge = squashedEdge.copy(statusCode = 1, lockTs = lockTs, version = snapshotEdge.version + 1)
@@ -524,7 +524,7 @@ abstract class Storage[Q, R](val graph: Graph,
* this can't be proceed so retry from re-fetch.
* throw EX
*/
- val (squashedEdge, _) = Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
+ val (squashedEdge, _) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
Future.failed(new PartialFailureException(squashedEdge, 0, s"others[${pendingEdge.ts}] is mutating. me[${squashedEdge.ts}]"))
}
}
@@ -551,7 +551,7 @@ abstract class Storage[Q, R](val graph: Graph,
val _edges =
if (fetchedSnapshotEdgeOpt.isDefined && fetchedSnapshotEdgeOpt.get.pendingEdgeOpt.isDefined) fetchedSnapshotEdgeOpt.get.pendingEdgeOpt.get +: edges
else edges
- val (squashedEdge, edgeMutate) = Edge.buildOperation(fetchedSnapshotEdgeOpt, _edges)
+ val (squashedEdge, edgeMutate) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, _edges)
val newVersion = fetchedSnapshotEdgeOpt.map(_.version).getOrElse(squashedEdge.ts) + 2
val releaseLockSnapshotEdge = edgeMutate.newSnapshotEdge match {
case None => squashedEdge.toSnapshotEdge.copy(statusCode = 0, pendingEdgeOpt = None, version = newVersion)
@@ -575,8 +575,8 @@ abstract class Storage[Q, R](val graph: Graph,
* @return
*/
protected def commitProcess(statusCode: Byte,
- squashedEdge: Edge,
- fetchedSnapshotEdgeOpt:Option[Edge],
+ squashedEdge: S2Edge,
+ fetchedSnapshotEdgeOpt:Option[S2Edge],
lockSnapshotEdge: SnapshotEdge,
releaseLockSnapshotEdge: SnapshotEdge,
edgeMutate: EdgeMutate): Future[Boolean] = {
@@ -588,7 +588,7 @@ abstract class Storage[Q, R](val graph: Graph,
} yield lockReleased
}
- case class PartialFailureException(edge: Edge, statusCode: Byte, failReason: String) extends NoStackException(failReason)
+ case class PartialFailureException(edge: S2Edge, statusCode: Byte, failReason: String) extends NoStackException(failReason)
protected def debug(ret: Boolean, phase: String, snapshotEdge: SnapshotEdge) = {
val msg = Seq(s"[$ret] [$phase]", s"${snapshotEdge.toLogString()}").mkString("\n")
@@ -611,8 +611,8 @@ abstract class Storage[Q, R](val graph: Graph,
* @return
*/
protected def acquireLock(statusCode: Byte,
- squashedEdge: Edge,
- fetchedSnapshotEdgeOpt: Option[Edge],
+ squashedEdge: S2Edge,
+ fetchedSnapshotEdgeOpt: Option[S2Edge],
lockEdge: SnapshotEdge): Future[Boolean] = {
if (statusCode >= 1) {
logger.debug(s"skip acquireLock: [$statusCode]\n${squashedEdge.toLogString}")
@@ -663,7 +663,7 @@ abstract class Storage[Q, R](val graph: Graph,
*/
protected def releaseLock(predicate: Boolean,
statusCode: Byte,
- squashedEdge: Edge,
+ squashedEdge: S2Edge,
releaseLockEdge: SnapshotEdge): Future[Boolean] = {
if (!predicate) {
Future.failed(new PartialFailureException(squashedEdge, 3, "predicate failed."))
@@ -708,7 +708,7 @@ abstract class Storage[Q, R](val graph: Graph,
*/
protected def commitIndexEdgeMutations(predicate: Boolean,
statusCode: Byte,
- squashedEdge: Edge,
+ squashedEdge: S2Edge,
edgeMutate: EdgeMutate): Future[Boolean] = {
if (!predicate) Future.failed(new PartialFailureException(squashedEdge, 1, "predicate failed."))
else {
@@ -742,7 +742,7 @@ abstract class Storage[Q, R](val graph: Graph,
*/
protected def commitIndexEdgeDegreeMutations(predicate: Boolean,
statusCode: Byte,
- squashedEdge: Edge,
+ squashedEdge: S2Edge,
edgeMutate: EdgeMutate): Future[Boolean] = {
def _write(kvs: Seq[SKeyValue], withWait: Boolean): Future[Boolean] = {
@@ -772,8 +772,8 @@ abstract class Storage[Q, R](val graph: Graph,
/** end of methods for consistency */
- def mutateLog(snapshotEdgeOpt: Option[Edge], edges: Seq[Edge],
- newEdge: Edge, edgeMutate: EdgeMutate) =
+ def mutateLog(snapshotEdgeOpt: Option[S2Edge], edges: Seq[S2Edge],
+ newEdge: S2Edge, edgeMutate: EdgeMutate) =
Seq("----------------------------------------------",
s"SnapshotEdge: ${snapshotEdgeOpt.map(_.toLogString)}",
s"requestEdges: ${edges.map(_.toLogString).mkString("\n")}",
@@ -796,17 +796,17 @@ abstract class Storage[Q, R](val graph: Graph,
val edge = edgeWithScore.edge
val score = edgeWithScore.score
- val edgeSnapshot = edge.copyEdge(propsWithTs = Edge.propsToState(edge.updatePropsWithTs()))
+ val edgeSnapshot = edge.copyEdge(propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()))
val reversedSnapshotEdgeMutations = snapshotEdgeSerializer(edgeSnapshot.toSnapshotEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put))
- val edgeForward = edge.copyEdge(propsWithTs = Edge.propsToState(edge.updatePropsWithTs()))
+ val edgeForward = edge.copyEdge(propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()))
val forwardIndexedEdgeMutations = edgeForward.edgesWithIndex.flatMap { indexEdge =>
indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete)) ++
buildIncrementsAsync(indexEdge, -1L)
}
/** reverted direction */
- val edgeRevert = edge.copyEdge(propsWithTs = Edge.propsToState(edge.updatePropsWithTs()))
+ val edgeRevert = edge.copyEdge(propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()))
val reversedIndexedEdgesMutations = edgeRevert.duplicateEdge.edgesWithIndex.flatMap { indexEdge =>
indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete)) ++
buildIncrementsAsync(indexEdge, -1L)
@@ -829,8 +829,8 @@ abstract class Storage[Q, R](val graph: Graph,
/** Parsing Logic: parse from kv from Storage into Edge */
def toEdge[K: CanSKeyValue](kv: K,
queryRequest: QueryRequest,
- cacheElementOpt: Option[Edge],
- parentEdges: Seq[EdgeWithScore]): Option[Edge] = {
+ cacheElementOpt: Option[S2Edge],
+ parentEdges: Seq[EdgeWithScore]): Option[S2Edge] = {
logger.debug(s"toEdge: $kv")
try {
@@ -851,7 +851,7 @@ abstract class Storage[Q, R](val graph: Graph,
queryRequest: QueryRequest,
cacheElementOpt: Option[SnapshotEdge] = None,
isInnerCall: Boolean,
- parentEdges: Seq[EdgeWithScore]): Option[Edge] = {
+ parentEdges: Seq[EdgeWithScore]): Option[S2Edge] = {
// logger.debug(s"SnapshottoEdge: $kv")
val queryParam = queryRequest.queryParam
val schemaVer = queryParam.label.schemaVersion
@@ -959,7 +959,7 @@ abstract class Storage[Q, R](val graph: Graph,
/** End Of Parse Logic */
- protected def toRequestEdge(queryRequest: QueryRequest, parentEdges: Seq[EdgeWithScore]): Edge = {
+ protected def toRequestEdge(queryRequest: QueryRequest, parentEdges: Seq[EdgeWithScore]): S2Edge = {
val srcVertex = queryRequest.vertex
val queryParam = queryRequest.queryParam
val tgtVertexIdOpt = queryParam.tgtVertexInnerIdOpt
@@ -986,7 +986,7 @@ abstract class Storage[Q, R](val graph: Graph,
}
}
- protected def fetchSnapshotEdgeInner(edge: Edge): Future[(QueryParam, Option[Edge], Option[SKeyValue])] = {
+ protected def fetchSnapshotEdgeInner(edge: S2Edge): Future[(QueryParam, Option[S2Edge], Option[SKeyValue])] = {
/** TODO: Fix this. currently fetchSnapshotEdge should not use future cache
* so use empty cacheKey.
* */
@@ -994,7 +994,7 @@ abstract class Storage[Q, R](val graph: Graph,
direction = GraphUtil.fromDirection(edge.labelWithDir.dir),
tgtVertexIdOpt = Option(edge.tgtVertex.innerIdVal),
cacheTTLInMillis = -1)
- val q = Query.toQuery(Seq(edge.srcVertex), queryParam)
+ val q = Query.toQuery(Seq(edge.srcVertex), Seq(queryParam))
val queryRequest = QueryRequest(q, 0, edge.srcVertex, queryParam)
// val q = Query.toQuery(Seq(edge.srcVertex), queryParam)
@@ -1097,15 +1097,15 @@ abstract class Storage[Q, R](val graph: Graph,
}
//TODO: ServiceColumn do not have durability property yet.
- def buildDeleteBelongsToId(vertex: Vertex): Seq[SKeyValue] = {
+ def buildDeleteBelongsToId(vertex: S2Vertex): Seq[SKeyValue] = {
val kvs = vertexSerializer(vertex).toKeyValues
val kv = kvs.head
vertex.belongLabelIds.map { id =>
- kv.copy(qualifier = Bytes.toBytes(Vertex.toPropKey(id)), operation = SKeyValue.Delete)
+ kv.copy(qualifier = Bytes.toBytes(S2Vertex.toPropKey(id)), operation = SKeyValue.Delete)
}
}
- def buildVertexPutsAsync(edge: Edge): Seq[SKeyValue] = {
+ def buildVertexPutsAsync(edge: S2Edge): Seq[SKeyValue] = {
val storeVertex = edge.innerLabel.extraOptions.get("storeVertex").map(_.as[Boolean]).getOrElse(false)
if (storeVertex) {
@@ -1118,7 +1118,7 @@ abstract class Storage[Q, R](val graph: Graph,
}
}
- def buildDegreePuts(edge: Edge, degreeVal: Long): Seq[SKeyValue] = {
+ def buildDegreePuts(edge: S2Edge, degreeVal: Long): Seq[SKeyValue] = {
edge.property(LabelMeta.degree.name, degreeVal, edge.ts)
val kvs = edge.edgesWithIndexValid.flatMap { indexEdge =>
indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
@@ -1127,7 +1127,7 @@ abstract class Storage[Q, R](val graph: Graph,
kvs
}
- def buildPutsAll(vertex: Vertex): Seq[SKeyValue] = {
+ def buildPutsAll(vertex: S2Vertex): Seq[SKeyValue] = {
vertex.op match {
case d: Byte if d == GraphUtil.operations("delete") => vertexSerializer(vertex).toKeyValues.map(_.copy(operation = SKeyValue.Delete))
case _ => vertexSerializer(vertex).toKeyValues.map(_.copy(operation = SKeyValue.Put))
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorage.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorage.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorage.scala
index b0287d5..93b2454 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorage.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorage.scala
@@ -89,7 +89,7 @@ object AsynchbaseStorage {
}
-class AsynchbaseStorage(override val graph: Graph,
+class AsynchbaseStorage(override val graph: S2Graph,
override val config: Config)(implicit ec: ExecutionContext)
extends Storage[AsyncRPC, Deferred[StepResult]](graph, config) {
@@ -242,7 +242,7 @@ class AsynchbaseStorage(override val graph: Graph,
* @param queryRequest
* @return Scanner or GetRequest with proper setup with StartKey, EndKey, RangeFilter.
*/
- override def buildRequest(queryRequest: QueryRequest, edge: Edge): AsyncRPC = {
+ override def buildRequest(queryRequest: QueryRequest, edge: S2Edge): AsyncRPC = {
import Serializable._
val queryParam = queryRequest.queryParam
val label = queryParam.label
@@ -424,7 +424,7 @@ class AsynchbaseStorage(override val graph: Graph,
* @param withWait
* @return
*/
- override def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]] = {
+ override def incrementCounts(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]] = {
val _client = client(withWait)
val defers: Seq[Deferred[(Boolean, Long, Long)]] = for {
@@ -517,9 +517,9 @@ class AsynchbaseStorage(override val graph: Graph,
/** Asynchbase implementation override default getVertices to use future Cache */
- override def getVertices(vertices: Seq[Vertex]): Future[Seq[Vertex]] = {
+ override def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
def fromResult(kvs: Seq[SKeyValue],
- version: String): Option[Vertex] = {
+ version: String): Option[S2Vertex] = {
if (kvs.isEmpty) None
else vertexDeserializer.fromKeyValues(None, kvs, version, None)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeDeserializable.scala
index e11a5f6..5549f4e 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeDeserializable.scala
@@ -29,10 +29,10 @@ import org.apache.s2graph.core._
import scala.collection.immutable
object IndexEdgeDeserializable{
- def getNewInstance(graph: Graph) = new IndexEdgeDeserializable(graph)
+ def getNewInstance(graph: S2Graph) = new IndexEdgeDeserializable(graph)
}
-class IndexEdgeDeserializable(graph: Graph,
- bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[Edge] {
+class IndexEdgeDeserializable(graph: S2Graph,
+ bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[S2Edge] {
import StorageDeserializable._
type QualifierRaw = (Array[(LabelMeta, InnerValLike)], VertexId, Byte, Boolean, Int)
@@ -41,7 +41,7 @@ class IndexEdgeDeserializable(graph: Graph,
override def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
_kvs: Seq[T],
schemaVer: String,
- cacheElementOpt: Option[Edge]): Edge = {
+ cacheElementOpt: Option[S2Edge]): S2Edge = {
assert(_kvs.size == 1)
@@ -64,7 +64,7 @@ class IndexEdgeDeserializable(graph: Graph,
val srcVertex = graph.newVertex(srcVertexId, version)
//TODO:
val edge = graph.newEdge(srcVertex, null,
- label, labelWithDir.dir, GraphUtil.defaultOpByte, version, Edge.EmptyState)
+ label, labelWithDir.dir, GraphUtil.defaultOpByte, version, S2Edge.EmptyState)
var tsVal = version
if (pos == kv.row.length) {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
index 60f7d80..706d8cb 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
@@ -26,8 +26,8 @@ import org.apache.s2graph.core.types._
import org.apache.s2graph.core._
import scala.collection.immutable
-class IndexEdgeDeserializable(graph: Graph,
- bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[Edge] {
+class IndexEdgeDeserializable(graph: S2Graph,
+ bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[S2Edge] {
import StorageDeserializable._
type QualifierRaw = (Array[(LabelMeta, InnerValLike)], VertexId, Byte, Boolean, Int)
@@ -68,7 +68,7 @@ class IndexEdgeDeserializable(graph: Graph,
override def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
_kvs: Seq[T],
schemaVer: String,
- cacheElementOpt: Option[Edge]): Edge = {
+ cacheElementOpt: Option[S2Edge]): S2Edge = {
assert(_kvs.size == 1)
// val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
@@ -85,7 +85,7 @@ class IndexEdgeDeserializable(graph: Graph,
val srcVertex = graph.newVertex(srcVertexId, version)
//TODO:
val edge = graph.newEdge(srcVertex, null,
- label, labelWithDir.dir, GraphUtil.defaultOpByte, version, Edge.EmptyState)
+ label, labelWithDir.dir, GraphUtil.defaultOpByte, version, S2Edge.EmptyState)
var tsVal = version
val (idxPropsRaw, tgtVertexIdRaw, op, tgtVertexIdInQualifier, _) =
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeDeserializable.scala
index 6c1906e..f4802c0 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeDeserializable.scala
@@ -24,9 +24,9 @@ import org.apache.s2graph.core.mysqls.{ServiceColumn, Label, LabelIndex, LabelMe
import org.apache.s2graph.core.storage.StorageDeserializable._
import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable, SKeyValue, StorageDeserializable}
import org.apache.s2graph.core.types.{HBaseType, LabelWithDirection, SourceAndTargetVertexIdPair, SourceVertexId}
-import org.apache.s2graph.core.{Graph, Edge, SnapshotEdge, Vertex}
+import org.apache.s2graph.core.{S2Graph, S2Edge, SnapshotEdge, S2Vertex}
-class SnapshotEdgeDeserializable(graph: Graph) extends Deserializable[SnapshotEdge] {
+class SnapshotEdgeDeserializable(graph: S2Graph) extends Deserializable[SnapshotEdge] {
def statusCodeWithOp(byte: Byte): (Byte, Byte) = {
val statusCode = byte >> 4
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeDeserializable.scala
index 64b2e31..d3dec1e 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeDeserializable.scala
@@ -24,9 +24,9 @@ import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
import org.apache.s2graph.core.storage.StorageDeserializable._
import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable, StorageDeserializable}
import org.apache.s2graph.core.types.TargetVertexId
-import org.apache.s2graph.core.{Graph, Edge, SnapshotEdge, Vertex}
+import org.apache.s2graph.core.{S2Graph, S2Edge, SnapshotEdge, S2Vertex}
-class SnapshotEdgeDeserializable(graph: Graph) extends Deserializable[SnapshotEdge] {
+class SnapshotEdgeDeserializable(graph: S2Graph) extends Deserializable[SnapshotEdge] {
def statusCodeWithOp(byte: Byte): (Byte, Byte) = {
val statusCode = byte >> 4
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexDeserializable.scala
index 6e2311f..ee93505 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexDeserializable.scala
@@ -19,20 +19,20 @@
package org.apache.s2graph.core.storage.serde.vertex
-import org.apache.s2graph.core.mysqls.Label
+import org.apache.s2graph.core.mysqls.{ColumnMeta, Label}
import org.apache.s2graph.core.storage.StorageDeserializable._
import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable}
import org.apache.s2graph.core.types.{InnerVal, InnerValLike, VertexId}
-import org.apache.s2graph.core.{Graph, QueryParam, Vertex}
+import org.apache.s2graph.core.{S2Graph, QueryParam, S2Vertex}
import scala.collection.mutable.ListBuffer
-class VertexDeserializable(graph: Graph,
- bytesToInt: (Array[Byte], Int) => Int = bytesToInt) extends Deserializable[Vertex] {
+class VertexDeserializable(graph: S2Graph,
+ bytesToInt: (Array[Byte], Int) => Int = bytesToInt) extends Deserializable[S2Vertex] {
def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
_kvs: Seq[T],
version: String,
- cacheElementOpt: Option[Vertex]): Vertex = {
+ cacheElementOpt: Option[S2Vertex]): S2Vertex = {
val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
@@ -40,7 +40,7 @@ class VertexDeserializable(graph: Graph,
val (vertexId, _) = VertexId.fromBytes(kv.row, 0, kv.row.length, version)
var maxTs = Long.MinValue
- val propsMap = new collection.mutable.HashMap[Int, InnerValLike]
+ val propsMap = new collection.mutable.HashMap[ColumnMeta, InnerValLike]
val belongLabelIds = new ListBuffer[Int]
for {
@@ -53,15 +53,18 @@ class VertexDeserializable(graph: Graph,
val ts = kv.timestamp
if (ts > maxTs) maxTs = ts
- if (Vertex.isLabelId(propKey)) {
- belongLabelIds += Vertex.toLabelId(propKey)
+ if (S2Vertex.isLabelId(propKey)) {
+ belongLabelIds += S2Vertex.toLabelId(propKey)
} else {
val v = kv.value
val (value, _) = InnerVal.fromBytes(v, 0, v.length, version)
- propsMap += (propKey -> value)
+ val columnMeta = vertexId.column.metasMap(propKey)
+ propsMap += (columnMeta -> value)
}
}
assert(maxTs != Long.MinValue)
- graph.newVertex(vertexId, maxTs, propsMap.toMap, belongLabelIds = belongLabelIds)
+ val vertex = graph.newVertex(vertexId, maxTs, S2Vertex.EmptyProps, belongLabelIds = belongLabelIds)
+ S2Vertex.fillPropsWithTs(vertex, propsMap.toMap)
+ vertex
}
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexSerializable.scala
index 77bbb87..1dbcd00 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexSerializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexSerializable.scala
@@ -19,12 +19,12 @@
package org.apache.s2graph.core.storage.serde.vertex
-import org.apache.s2graph.core.Vertex
+import org.apache.s2graph.core.S2Vertex
import org.apache.s2graph.core.storage.StorageSerializable._
import org.apache.s2graph.core.storage.{SKeyValue, Serializable}
-import org.apache.s2graph.core.utils.logger
+import scala.collection.JavaConverters._
-case class VertexSerializable(vertex: Vertex, intToBytes: Int => Array[Byte] = intToBytes) extends Serializable[Vertex] {
+case class VertexSerializable(vertex: S2Vertex, intToBytes: Int => Array[Byte] = intToBytes) extends Serializable[S2Vertex] {
override val table = vertex.hbaseTableName.getBytes
override val ts = vertex.ts
@@ -38,8 +38,11 @@ case class VertexSerializable(vertex: Vertex, intToBytes: Int => Array[Byte] = i
/** vertex override toKeyValues since vertex expect to produce multiple sKeyValues */
override def toKeyValues: Seq[SKeyValue] = {
val row = toRowKey
- val base = for ((k, v) <- vertex.props ++ vertex.defaultProps) yield intToBytes(k) -> v.bytes
- val belongsTo = vertex.belongLabelIds.map { labelId => intToBytes(Vertex.toPropKey(labelId)) -> Array.empty[Byte] }
+ val base = for ((k, v) <- vertex.props.asScala ++ vertex.defaultProps.asScala) yield {
+ val columnMeta = v.columnMeta
+ intToBytes(columnMeta.seq) -> v.innerVal.bytes
+ }
+ val belongsTo = vertex.belongLabelIds.map { labelId => intToBytes(S2Vertex.toPropKey(labelId)) -> Array.empty[Byte] }
(base ++ belongsTo).map { case (qualifier, value) =>
SKeyValue(vertex.hbaseTableName.getBytes, row, cf, qualifier, value, vertex.ts)
} toSeq
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala b/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
index d90cf8e..ea7aa41 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
@@ -268,6 +268,23 @@ trait CanInnerValLike[A] {
object CanInnerValLike {
implicit val encodingVer = "v2"
+ def castValue(element: Any, classType: String): Any = {
+ import InnerVal._
+ element match {
+ case bd: BigDecimal =>
+ classType match {
+ case DOUBLE => bd.doubleValue()
+ case FLOAT => bd.floatValue()
+ case LONG => bd.longValue()
+ case INT | "int" => bd.intValue()
+ case SHORT => bd.shortValue()
+ case BYTE => bd.byteValue()
+ case _ => throw new RuntimeException(s"not supported data type: $element, $classType")
+ }
+ case _ => element
+// throw new RuntimeException(s"not supported data type: $element, ${element.getClass.getCanonicalName}, $classType")
+ }
+ }
def validate(element: Any, classType: String): Boolean = {
import InnerVal._
classType match {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/types/VertexId.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/types/VertexId.scala b/s2core/src/main/scala/org/apache/s2graph/core/types/VertexId.scala
index a949f3e..eb2d42a 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/types/VertexId.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/types/VertexId.scala
@@ -51,7 +51,7 @@ object VertexId extends HBaseDeserializable {
}
}
-class VertexId protected (val column: ServiceColumn, val innerId: InnerValLike) extends HBaseSerializable {
+class VertexId (val column: ServiceColumn, val innerId: InnerValLike) extends HBaseSerializable {
val storeHash: Boolean = true
val storeColId: Boolean = true
val colId = column.id.get
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala
deleted file mode 100644
index 55b796d..0000000
--- a/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.s2graph.core
-
-import org.apache.s2graph.core.JSONParser._
-import org.apache.s2graph.core.mysqls.{ServiceColumn, LabelMeta}
-import org.apache.s2graph.core.types.{InnerVal, InnerValLikeWithTs, VertexId}
-import org.apache.s2graph.core.utils.logger
-import org.scalatest.FunSuite
-import play.api.libs.json.{JsObject, Json}
-
-class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
- import Edge._
- initTests()
-
- val testLabelMeta1 = LabelMeta(Option(-1), labelV2.id.get, "is_blocked", 1.toByte, "true", "boolean")
- val testLabelMeta3 = LabelMeta(Option(-1), labelV2.id.get, "time", 3.toByte, "-1", "long")
-
-// test("toLogString") {
-// val testServiceName = serviceNameV2
-// val testLabelName = labelNameV2
-// val bulkQueries = List(
-// ("1445240543366", "update", "{\"is_blocked\":true}"),
-// ("1445240543362", "insert", "{\"is_hidden\":false}"),
-// ("1445240543364", "insert", "{\"is_hidden\":false,\"weight\":10}"),
-// ("1445240543363", "delete", "{}"),
-// ("1445240543365", "update", "{\"time\":1, \"weight\":-10}"))
-//
-// val (srcId, tgtId, labelName) = ("1", "2", testLabelName)
-//
-// val bulkEdge = (for ((ts, op, props) <- bulkQueries) yield {
-// val properties = fromJsonToProperties(Json.parse(props).as[JsObject])
-// Edge.toEdge(srcId, tgtId, labelName, "out", properties, ts.toLong, op).toLogString
-// }).mkString("\n")
-//
-// val attachedProps = "\"from\":\"1\",\"to\":\"2\",\"label\":\"" + testLabelName +
-// "\",\"service\":\"" + testServiceName + "\""
-// val expected = Seq(
-// Seq("1445240543366", "update", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_blocked\":true}"),
-// Seq("1445240543362", "insert", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_hidden\":false}"),
-// Seq("1445240543364", "insert", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_hidden\":false,\"weight\":10}"),
-// Seq("1445240543363", "delete", "e", "1", "2", testLabelName, "{" + attachedProps + "}"),
-// Seq("1445240543365", "update", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"time\":1,\"weight\":-10}")
-// ).map(_.mkString("\t")).mkString("\n")
-//
-// assert(bulkEdge === expected)
-// }
-
- test("buildOperation") {
- val schemaVersion = "v2"
- val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
- val srcVertex = graph.newVertex(vertexId)
- val tgtVertex = srcVertex
-
- val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
-
- val snapshotEdge = None
- val propsWithTs = Map(timestampProp)
- val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
-
- val newVersion = 0L
-
- val newPropsWithTs = Map(
- timestampProp,
- testLabelMeta1 -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
- )
-
- val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
- logger.info(edgeMutate.toLogString)
-
- assert(edgeMutate.newSnapshotEdge.isDefined)
- assert(edgeMutate.edgesToInsert.nonEmpty)
- assert(edgeMutate.edgesToDelete.isEmpty)
- }
-
- test("buildMutation: snapshotEdge: None with newProps") {
- val schemaVersion = "v2"
- val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
- val srcVertex = graph.newVertex(vertexId)
- val tgtVertex = srcVertex
-
- val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
-
- val snapshotEdge = None
- val propsWithTs = Map(timestampProp)
- val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
-
- val newVersion = 0L
-
- val newPropsWithTs = Map(
- timestampProp,
- testLabelMeta1 -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
- )
-
- val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
- logger.info(edgeMutate.toLogString)
-
- assert(edgeMutate.newSnapshotEdge.isDefined)
- assert(edgeMutate.edgesToInsert.nonEmpty)
- assert(edgeMutate.edgesToDelete.isEmpty)
- }
-
- test("buildMutation: oldPropsWithTs == newPropsWithTs, Drop all requests") {
- val schemaVersion = "v2"
- val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
- val srcVertex = graph.newVertex(vertexId)
- val tgtVertex = srcVertex
-
- val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
-
- val snapshotEdge = None
- val propsWithTs = Map(timestampProp)
- val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
-
- val newVersion = 0L
-
- val newPropsWithTs = propsWithTs
-
- val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
- logger.info(edgeMutate.toLogString)
-
- assert(edgeMutate.newSnapshotEdge.isEmpty)
- assert(edgeMutate.edgesToInsert.isEmpty)
- assert(edgeMutate.edgesToDelete.isEmpty)
- }
-
- test("buildMutation: All props older than snapshotEdge's LastDeletedAt") {
- val schemaVersion = "v2"
- val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
- val srcVertex = graph.newVertex(vertexId)
- val tgtVertex = srcVertex
-
- val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
- val oldPropsWithTs = Map(
- timestampProp,
- LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
- )
-
- val propsWithTs = Map(
- timestampProp,
- testLabelMeta3 -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 2),
- LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
- )
-
- val _snapshotEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, op = GraphUtil.operations("delete"), propsWithTs = propsWithTs)
-
- val snapshotEdge = Option(_snapshotEdge)
-
-
- val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
-
- val newVersion = 0L
- val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
- logger.info(edgeMutate.toLogString)
-
- assert(edgeMutate.newSnapshotEdge.nonEmpty)
- assert(edgeMutate.edgesToInsert.isEmpty)
- assert(edgeMutate.edgesToDelete.isEmpty)
- }
-
- test("buildMutation: All props newer than snapshotEdge's LastDeletedAt") {
- val schemaVersion = "v2"
- val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
- val srcVertex = graph.newVertex(vertexId)
- val tgtVertex = srcVertex
-
- val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
- val oldPropsWithTs = Map(
- timestampProp,
- LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
- )
-
- val propsWithTs = Map(
- timestampProp,
- testLabelMeta3 -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 4),
- LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
- )
-
- val _snapshotEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, op = GraphUtil.operations("delete"), propsWithTs = propsWithTs)
-
- val snapshotEdge = Option(_snapshotEdge)
-
- val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
-
- val newVersion = 0L
- val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
- logger.info(edgeMutate.toLogString)
-
- assert(edgeMutate.newSnapshotEdge.nonEmpty)
- assert(edgeMutate.edgesToInsert.nonEmpty)
- assert(edgeMutate.edgesToDelete.isEmpty)
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
index cc08b62..d280570 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
@@ -34,14 +34,14 @@ trait IntegrateCommon extends FunSuite with Matchers with BeforeAndAfterAll {
import TestUtil._
- var graph: Graph = _
+ var graph: S2Graph = _
var parser: RequestParser = _
var management: Management = _
var config: Config = _
override def beforeAll = {
config = ConfigFactory.load()
- graph = new Graph(config)(ExecutionContext.Implicits.global)
+ graph = new S2Graph(config)(ExecutionContext.Implicits.global)
management = new Management(graph)
parser = new RequestParser(graph)
initTestData()
@@ -92,7 +92,7 @@ trait IntegrateCommon extends FunSuite with Matchers with BeforeAndAfterAll {
}
}
- val vertexPropsKeys = List("age" -> "int", "im" -> "string")
+ val vertexPropsKeys = List("age" -> "integer", "im" -> "string")
vertexPropsKeys.map { case (key, keyType) =>
Management.addVertexProp(testServiceName, testColumnName, key, keyType)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala
new file mode 100644
index 0000000..3eb04ff
--- /dev/null
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.s2graph.core.Integrate.tinkerpop
+
+import org.apache.s2graph.core.mysqls.Label
+import org.apache.s2graph.core.utils.logger
+import org.apache.s2graph.core.{S2Graph, TestCommonWithModels, S2Vertex}
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal
+import org.apache.tinkerpop.gremlin.structure.{Edge, Vertex, T}
+import org.scalatest.{FunSuite, Matchers}
+
+
+class S2GraphTest extends FunSuite with Matchers with TestCommonWithModels {
+
+ import scala.collection.JavaConversions._
+ import scala.concurrent.ExecutionContext.Implicits.global
+
+ initTests()
+
+ val g = new S2Graph(config)
+
+ def printEdges(edges: Seq[Edge]): Unit = {
+ edges.foreach { edge =>
+ logger.debug(s"[FetchedEdge]: $edge")
+ }
+ }
+
+ import scala.language.implicitConversions
+
+ def newVertexId(id: Any, label: Label = labelV2) = g.newVertexId(label.srcService, label.srcColumn, id)
+
+ def addVertex(id: AnyRef, label: Label = labelV2) =
+ g.addVertex(T.label, label.srcService.serviceName + S2Vertex.VertexLabelDelimiter + label.srcColumnName,
+ T.id, id).asInstanceOf[S2Vertex]
+
+ val srcId = Long.box(20)
+ val range = (100 until 110)
+ testData(srcId, range)
+
+ // val testProps = Seq(
+ // Prop("affinity_score", "0.0", DOUBLE),
+ // Prop("is_blocked", "false", BOOLEAN),
+ // Prop("time", "0", INT),
+ // Prop("weight", "0", INT),
+ // Prop("is_hidden", "true", BOOLEAN),
+ // Prop("phone_number", "xxx-xxx-xxxx", STRING),
+ // Prop("score", "0.1", FLOAT),
+ // Prop("age", "10", INT)
+ // )
+ def testData(srcId: AnyRef, range: Range, label: Label = labelV2) = {
+ val src = addVertex(srcId)
+
+ for {
+ i <- range
+ } {
+ val tgt = addVertex(Int.box(i))
+
+ src.addEdge(labelV2.label, tgt,
+ "age", Int.box(10),
+ "affinity_score", Double.box(0.1),
+ "is_blocked", Boolean.box(true),
+ "ts", Long.box(i))
+ }
+ }
+
+ test("test traversal.") {
+ val vertices = g.traversal().V(newVertexId(srcId)).out(labelV2.label).toSeq
+
+ vertices.size should be(range.size)
+ range.reverse.zip(vertices).foreach { case (tgtId, vertex) =>
+ val vertexId = g.newVertexId(labelV2.tgtService, labelV2.tgtColumn, tgtId)
+ val expectedId = g.newVertex(vertexId)
+ vertex.asInstanceOf[S2Vertex].innerId should be(expectedId.innerId)
+ }
+ }
+
+ test("test traversal. limit 1") {
+ val vertexIdParams = Seq(newVertexId(srcId))
+ val t: GraphTraversal[Vertex, Double] = g.traversal().V(vertexIdParams: _*).outE(labelV2.label).limit(1).values("affinity_score")
+ for {
+ affinityScore <- t
+ } {
+ logger.debug(s"$affinityScore")
+ affinityScore should be (0.1)
+ }
+ }
+ test("test traversal. 3") {
+
+ val l = label
+
+ val srcA = addVertex(Long.box(1), l)
+ val srcB = addVertex(Long.box(2), l)
+ val srcC = addVertex(Long.box(3), l)
+
+ val tgtA = addVertex(Long.box(101), l)
+ val tgtC = addVertex(Long.box(103), l)
+
+ srcA.addEdge(l.label, tgtA)
+ srcA.addEdge(l.label, tgtC)
+ tgtC.addEdge(l.label, srcB)
+ tgtA.addEdge(l.label, srcC)
+
+ val vertexIdParams = Seq(srcA.id)
+ val vertices = g.traversal().V(vertexIdParams: _*).out(l.label).out(l.label).toSeq
+ vertices.size should be(2)
+ vertices.foreach { v =>
+ val vertex = v.asInstanceOf[S2Vertex]
+ // TODO: we have too many id. this is ugly and confusing so fix me.
+ vertex.id.innerId == srcB.id.innerId || vertex.id.innerId == srcC.id.innerId should be(true)
+ logger.debug(s"[Vertex]: $v")
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/test/scala/org/apache/s2graph/core/S2EdgeTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/S2EdgeTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/S2EdgeTest.scala
new file mode 100644
index 0000000..94883c9
--- /dev/null
+++ b/s2core/src/test/scala/org/apache/s2graph/core/S2EdgeTest.scala
@@ -0,0 +1,210 @@
+/*
+ * 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.s2graph.core
+
+import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core.mysqls.{ServiceColumn, LabelMeta}
+import org.apache.s2graph.core.types.{InnerVal, InnerValLikeWithTs, VertexId}
+import org.apache.s2graph.core.utils.logger
+import org.scalatest.FunSuite
+import play.api.libs.json.{JsObject, Json}
+
+class S2EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
+ import S2Edge._
+ initTests()
+
+ val testLabelMeta1 = LabelMeta(Option(-1), labelV2.id.get, "is_blocked", 1.toByte, "true", "boolean")
+ val testLabelMeta3 = LabelMeta(Option(-1), labelV2.id.get, "time", 3.toByte, "-1", "long")
+
+// test("toLogString") {
+// val testServiceName = serviceNameV2
+// val testLabelName = labelNameV2
+// val bulkQueries = List(
+// ("1445240543366", "update", "{\"is_blocked\":true}"),
+// ("1445240543362", "insert", "{\"is_hidden\":false}"),
+// ("1445240543364", "insert", "{\"is_hidden\":false,\"weight\":10}"),
+// ("1445240543363", "delete", "{}"),
+// ("1445240543365", "update", "{\"time\":1, \"weight\":-10}"))
+//
+// val (srcId, tgtId, labelName) = ("1", "2", testLabelName)
+//
+// val bulkEdge = (for ((ts, op, props) <- bulkQueries) yield {
+// val properties = fromJsonToProperties(Json.parse(props).as[JsObject])
+// Edge.toEdge(srcId, tgtId, labelName, "out", properties, ts.toLong, op).toLogString
+// }).mkString("\n")
+//
+// val attachedProps = "\"from\":\"1\",\"to\":\"2\",\"label\":\"" + testLabelName +
+// "\",\"service\":\"" + testServiceName + "\""
+// val expected = Seq(
+// Seq("1445240543366", "update", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_blocked\":true}"),
+// Seq("1445240543362", "insert", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_hidden\":false}"),
+// Seq("1445240543364", "insert", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_hidden\":false,\"weight\":10}"),
+// Seq("1445240543363", "delete", "e", "1", "2", testLabelName, "{" + attachedProps + "}"),
+// Seq("1445240543365", "update", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"time\":1,\"weight\":-10}")
+// ).map(_.mkString("\t")).mkString("\n")
+//
+// assert(bulkEdge === expected)
+// }
+
+ test("buildOperation") {
+ val schemaVersion = "v2"
+ val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
+ val srcVertex = graph.newVertex(vertexId)
+ val tgtVertex = srcVertex
+
+ val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+
+ val snapshotEdge = None
+ val propsWithTs = Map(timestampProp)
+ val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
+
+ val newVersion = 0L
+
+ val newPropsWithTs = Map(
+ timestampProp,
+ testLabelMeta1 -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
+ )
+
+ val edgeMutate = S2Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
+ logger.info(edgeMutate.toLogString)
+
+ assert(edgeMutate.newSnapshotEdge.isDefined)
+ assert(edgeMutate.edgesToInsert.nonEmpty)
+ assert(edgeMutate.edgesToDelete.isEmpty)
+ }
+
+ test("buildMutation: snapshotEdge: None with newProps") {
+ val schemaVersion = "v2"
+ val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
+ val srcVertex = graph.newVertex(vertexId)
+ val tgtVertex = srcVertex
+
+ val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+
+ val snapshotEdge = None
+ val propsWithTs = Map(timestampProp)
+ val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
+
+ val newVersion = 0L
+
+ val newPropsWithTs = Map(
+ timestampProp,
+ testLabelMeta1 -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
+ )
+
+ val edgeMutate = S2Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
+ logger.info(edgeMutate.toLogString)
+
+ assert(edgeMutate.newSnapshotEdge.isDefined)
+ assert(edgeMutate.edgesToInsert.nonEmpty)
+ assert(edgeMutate.edgesToDelete.isEmpty)
+ }
+
+ test("buildMutation: oldPropsWithTs == newPropsWithTs, Drop all requests") {
+ val schemaVersion = "v2"
+ val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
+ val srcVertex = graph.newVertex(vertexId)
+ val tgtVertex = srcVertex
+
+ val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+
+ val snapshotEdge = None
+ val propsWithTs = Map(timestampProp)
+ val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
+
+ val newVersion = 0L
+
+ val newPropsWithTs = propsWithTs
+
+ val edgeMutate = S2Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
+ logger.info(edgeMutate.toLogString)
+
+ assert(edgeMutate.newSnapshotEdge.isEmpty)
+ assert(edgeMutate.edgesToInsert.isEmpty)
+ assert(edgeMutate.edgesToDelete.isEmpty)
+ }
+
+ test("buildMutation: All props older than snapshotEdge's LastDeletedAt") {
+ val schemaVersion = "v2"
+ val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
+ val srcVertex = graph.newVertex(vertexId)
+ val tgtVertex = srcVertex
+
+ val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+ val oldPropsWithTs = Map(
+ timestampProp,
+ LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
+ )
+
+ val propsWithTs = Map(
+ timestampProp,
+ testLabelMeta3 -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 2),
+ LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
+ )
+
+ val _snapshotEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, op = GraphUtil.operations("delete"), propsWithTs = propsWithTs)
+
+ val snapshotEdge = Option(_snapshotEdge)
+
+
+ val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
+
+ val newVersion = 0L
+ val edgeMutate = S2Edge.buildMutation(snapshotEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
+ logger.info(edgeMutate.toLogString)
+
+ assert(edgeMutate.newSnapshotEdge.nonEmpty)
+ assert(edgeMutate.edgesToInsert.isEmpty)
+ assert(edgeMutate.edgesToDelete.isEmpty)
+ }
+
+ test("buildMutation: All props newer than snapshotEdge's LastDeletedAt") {
+ val schemaVersion = "v2"
+ val vertexId = VertexId(ServiceColumn.Default, InnerVal.withStr("dummy", schemaVersion))
+ val srcVertex = graph.newVertex(vertexId)
+ val tgtVertex = srcVertex
+
+ val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+ val oldPropsWithTs = Map(
+ timestampProp,
+ LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
+ )
+
+ val propsWithTs = Map(
+ timestampProp,
+ testLabelMeta3 -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 4),
+ LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
+ )
+
+ val _snapshotEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, op = GraphUtil.operations("delete"), propsWithTs = propsWithTs)
+
+ val snapshotEdge = Option(_snapshotEdge)
+
+ val requestEdge = graph.newEdge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
+
+ val newVersion = 0L
+ val edgeMutate = S2Edge.buildMutation(snapshotEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
+ logger.info(edgeMutate.toLogString)
+
+ assert(edgeMutate.newSnapshotEdge.nonEmpty)
+ assert(edgeMutate.edgesToInsert.nonEmpty)
+ assert(edgeMutate.edgesToDelete.isEmpty)
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala b/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
index 12eae77..1997354 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
@@ -32,13 +32,13 @@ trait TestCommonWithModels {
import InnerVal._
import types.HBaseType._
- var graph: Graph = _
+ var graph: S2Graph = _
var config: Config = _
var management: Management = _
def initTests() = {
config = ConfigFactory.load()
- graph = new Graph(config)(ExecutionContext.Implicits.global)
+ graph = new S2Graph(config)(ExecutionContext.Implicits.global)
management = new Management(graph)
implicit val session = AutoSession
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/test/scala/org/apache/s2graph/core/benchmark/BenchmarkCommon.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/benchmark/BenchmarkCommon.scala b/s2core/src/test/scala/org/apache/s2graph/core/benchmark/BenchmarkCommon.scala
index 9e220cf..30011b1 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/benchmark/BenchmarkCommon.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/benchmark/BenchmarkCommon.scala
@@ -20,7 +20,7 @@
package org.apache.s2graph.core.benchmark
import com.typesafe.config.{ConfigFactory, Config}
-import org.apache.s2graph.core.{Management, Graph}
+import org.apache.s2graph.core.{Management, S2Graph$}
import org.specs2.mutable.Specification
import scalikejdbc.AutoSession
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
index cb6090e..25dd0e4 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
@@ -37,7 +37,7 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
val ts = System.currentTimeMillis()
val dummyTs = LabelMeta.timestamp -> InnerValLikeWithTs.withLong(ts, ts, label.schemaVersion)
- def validate(label: Label)(edge: Edge)(sql: String)(expected: Boolean) = {
+ def validate(label: Label)(edge: S2Edge)(sql: String)(expected: Boolean) = {
def debug(whereOpt: Try[Where]) = {
println("==================")
println(s"$whereOpt")
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2counter_core/src/main/scala/org/apache/s2graph/counter/helper/CounterAdmin.scala
----------------------------------------------------------------------
diff --git a/s2counter_core/src/main/scala/org/apache/s2graph/counter/helper/CounterAdmin.scala b/s2counter_core/src/main/scala/org/apache/s2graph/counter/helper/CounterAdmin.scala
index e530e65..17ecc87 100644
--- a/s2counter_core/src/main/scala/org/apache/s2graph/counter/helper/CounterAdmin.scala
+++ b/s2counter_core/src/main/scala/org/apache/s2graph/counter/helper/CounterAdmin.scala
@@ -21,7 +21,7 @@ package org.apache.s2graph.counter.helper
import com.typesafe.config.Config
import org.apache
-import org.apache.s2graph.core.Graph
+import org.apache.s2graph.core.S2Graph
import org.apache.s2graph.core.mysqls.Label
import org.apache.s2graph.counter
import org.apache.s2graph.counter.config.S2CounterConfig
@@ -37,7 +37,7 @@ class CounterAdmin(config: Config) {
val s2config = new S2CounterConfig(config)
val counterModel = new CounterModel(config)
val graphOp = new GraphOperation(config)
- val s2graph = new Graph(config)(scala.concurrent.ExecutionContext.global)
+ val s2graph = new S2Graph(config)(scala.concurrent.ExecutionContext.global)
val storageManagement = new org.apache.s2graph.core.Management(s2graph)
def setupCounterOnGraph(): Unit = {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2counter_core/src/test/scala/org/apache/s2graph/counter/core/RankingCounterSpec.scala
----------------------------------------------------------------------
diff --git a/s2counter_core/src/test/scala/org/apache/s2graph/counter/core/RankingCounterSpec.scala b/s2counter_core/src/test/scala/org/apache/s2graph/counter/core/RankingCounterSpec.scala
index c1d25f3..62174ad 100644
--- a/s2counter_core/src/test/scala/org/apache/s2graph/counter/core/RankingCounterSpec.scala
+++ b/s2counter_core/src/test/scala/org/apache/s2graph/counter/core/RankingCounterSpec.scala
@@ -21,7 +21,7 @@ package org.apache.s2graph.counter.core
import com.typesafe.config.ConfigFactory
import org.apache.s2graph.core.mysqls.Label
-import org.apache.s2graph.core.{Graph, Management}
+import org.apache.s2graph.core.{S2Graph$, Management}
import org.apache.s2graph.counter.config.S2CounterConfig
import org.apache.s2graph.counter.core.TimedQualifier.IntervalUnit
import org.apache.s2graph.counter.core.v2.{GraphOperation, RankingStorageGraph}
@@ -87,7 +87,7 @@ class RankingCounterSpec extends Specification with BeforeAfterAll {
}
val graphOp = new GraphOperation(config)
- val graph = new Graph(config)(scala.concurrent.ExecutionContext.global)
+ val graph = new S2Graph(config)(scala.concurrent.ExecutionContext.global)
val management = new Management(graph)
management.createService(service, s2config.HBASE_ZOOKEEPER_QUORUM, s"${service}_dev", 1, None, "gz")
val strJs =
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2counter_loader/src/main/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctions.scala
----------------------------------------------------------------------
diff --git a/s2counter_loader/src/main/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctions.scala b/s2counter_loader/src/main/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctions.scala
index a659ed3..37779dc 100644
--- a/s2counter_loader/src/main/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctions.scala
+++ b/s2counter_loader/src/main/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctions.scala
@@ -19,7 +19,7 @@
package org.apache.s2graph.counter.loader.core
-import org.apache.s2graph.core.{Edge, Graph, GraphUtil}
+import org.apache.s2graph.core.{S2Edge, S2Graph, GraphUtil}
import org.apache.s2graph.counter.loader.config.StreamingConfig
import org.apache.s2graph.counter.models.CounterModel
import org.apache.s2graph.spark.config.S2ConfigFactory
@@ -32,12 +32,12 @@ object CounterEtlFunctions extends Logging {
lazy val preFetchSize = StreamingConfig.PROFILE_PREFETCH_SIZE
lazy val config = S2ConfigFactory.config
lazy val counterModel = new CounterModel(config)
- lazy val graph = new Graph(config)(scala.concurrent.ExecutionContext.Implicits.global)
+ lazy val graph = new S2Graph(config)(scala.concurrent.ExecutionContext.Implicits.global)
- def logToEdge(line: String): Option[Edge] = {
+ def logToEdge(line: String): Option[S2Edge] = {
for {
- elem <- graph.toGraphElement(line) if elem.isInstanceOf[Edge]
- edge <- Some(elem.asInstanceOf[Edge]).filter { x =>
+ elem <- graph.toGraphElement(line) if elem.isInstanceOf[S2Edge]
+ edge <- Some(elem.asInstanceOf[S2Edge]).filter { x =>
filterOps.contains(x.op)
}
} yield {
@@ -50,8 +50,8 @@ object CounterEtlFunctions extends Logging {
* 1427082276804 insert edge 19073318 52453027_93524145648511699 story_user_ch_doc_view {"doc_type" : "l", "channel_subscribing" : "y", "view_from" : "feed"}
*/
for {
- elem <- graph.toGraphElement(line) if elem.isInstanceOf[Edge]
- edge <- Some(elem.asInstanceOf[Edge]).filter { x =>
+ elem <- graph.toGraphElement(line) if elem.isInstanceOf[S2Edge]
+ edge <- Some(elem.asInstanceOf[S2Edge]).filter { x =>
filterOps.contains(x.op)
}
} yield {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2counter_loader/src/test/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctionsSpec.scala
----------------------------------------------------------------------
diff --git a/s2counter_loader/src/test/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctionsSpec.scala b/s2counter_loader/src/test/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctionsSpec.scala
index 8ae1de3..6985758 100644
--- a/s2counter_loader/src/test/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctionsSpec.scala
+++ b/s2counter_loader/src/test/scala/org/apache/s2graph/counter/loader/core/CounterEtlFunctionsSpec.scala
@@ -22,7 +22,7 @@ package org.apache.s2graph.counter.loader.core
import com.typesafe.config.ConfigFactory
import org.apache.s2graph.core.mysqls.{Label, Service}
import org.apache.s2graph.core.types.HBaseType
-import org.apache.s2graph.core.{Graph, Management}
+import org.apache.s2graph.core.{S2Graph$, Management}
import org.apache.s2graph.counter.models.DBModel
import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers}
@@ -33,7 +33,7 @@ class CounterEtlFunctionsSpec extends FlatSpec with BeforeAndAfterAll with Match
val cluster = config.getString("hbase.zookeeper.quorum")
DBModel.initialize(config)
- val graph = new Graph(config)(global)
+ val graph = new S2Graph(config)(global)
val management = new Management(graph)
override def beforeAll: Unit = {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
----------------------------------------------------------------------
diff --git a/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala b/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
index a6f8f5c..a47fda7 100644
--- a/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
+++ b/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
@@ -39,7 +39,7 @@ import org.apache.s2graph.core.rest.RestHandler
import org.apache.s2graph.core.rest.RestHandler.{CanLookup, HandlerResult}
import org.apache.s2graph.core.utils.Extensions._
import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{Graph, PostProcess}
+import org.apache.s2graph.core.{S2Graph, PostProcess}
import play.api.libs.json._
import scala.collection.mutable
@@ -217,7 +217,7 @@ object NettyServer extends App {
val maxBodySize = Try(config.getInt("max.body.size")).recover { case _ => 65536 * 2 }.get
// init s2graph with config
- val s2graph = new Graph(config)(ec)
+ val s2graph = new S2Graph(config)(ec)
val rest = new RestHandler(s2graph)(ec)
val deployInfo = Try(Source.fromFile("./release_info").mkString("")).recover { case _ => "release info not found\n" }.get
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2rest_play/app/org/apache/s2graph/rest/play/Bootstrap.scala
----------------------------------------------------------------------
diff --git a/s2rest_play/app/org/apache/s2graph/rest/play/Bootstrap.scala b/s2rest_play/app/org/apache/s2graph/rest/play/Bootstrap.scala
index 692ab1e..e5fc75d 100644
--- a/s2rest_play/app/org/apache/s2graph/rest/play/Bootstrap.scala
+++ b/s2rest_play/app/org/apache/s2graph/rest/play/Bootstrap.scala
@@ -23,7 +23,7 @@ import java.util.concurrent.Executors
import org.apache.s2graph.core.rest.{RequestParser, RestHandler}
import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{ExceptionHandler, Graph, Management}
+import org.apache.s2graph.core.{ExceptionHandler, S2Graph, Management}
import org.apache.s2graph.rest.play.actors.QueueActor
import org.apache.s2graph.rest.play.config.Config
import org.apache.s2graph.rest.play.controllers.ApplicationController
@@ -36,7 +36,7 @@ import scala.io.Source
import scala.util.Try
object Global extends WithFilters(new GzipFilter()) {
- var s2graph: Graph = _
+ var s2graph: S2Graph = _
var storageManagement: Management = _
var s2parser: RequestParser = _
var s2rest: RestHandler = _
@@ -50,7 +50,7 @@ object Global extends WithFilters(new GzipFilter()) {
val config = Config.conf.underlying
// init s2graph with config
- s2graph = new Graph(config)(ec)
+ s2graph = new S2Graph(config)(ec)
storageManagement = new Management(s2graph)
s2parser = new RequestParser(s2graph)
s2rest = new RestHandler(s2graph)(ec)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2rest_play/app/org/apache/s2graph/rest/play/actors/QueueActor.scala
----------------------------------------------------------------------
diff --git a/s2rest_play/app/org/apache/s2graph/rest/play/actors/QueueActor.scala b/s2rest_play/app/org/apache/s2graph/rest/play/actors/QueueActor.scala
index d46d8d2..75ffbc3 100644
--- a/s2rest_play/app/org/apache/s2graph/rest/play/actors/QueueActor.scala
+++ b/s2rest_play/app/org/apache/s2graph/rest/play/actors/QueueActor.scala
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit
import akka.actor._
import org.apache.s2graph.core.ExceptionHandler._
import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{ExceptionHandler, Graph, GraphElement}
+import org.apache.s2graph.core.{ExceptionHandler, S2Graph, GraphElement}
import org.apache.s2graph.rest.play.actors.Protocol.FlushAll
import org.apache.s2graph.rest.play.config.Config
import play.api.Play.current
@@ -46,7 +46,7 @@ object QueueActor {
var router: ActorRef = _
// Akka.system.actorOf(props(), name = "queueActor")
- def init(s2: Graph, walLogHandler: ExceptionHandler) = {
+ def init(s2: S2Graph, walLogHandler: ExceptionHandler) = {
router = Akka.system.actorOf(props(s2, walLogHandler))
}
@@ -56,10 +56,10 @@ object QueueActor {
Thread.sleep(Config.ASYNC_HBASE_CLIENT_FLUSH_INTERVAL * 2)
}
- def props(s2: Graph, walLogHandler: ExceptionHandler): Props = Props(classOf[QueueActor], s2, walLogHandler)
+ def props(s2: S2Graph, walLogHandler: ExceptionHandler): Props = Props(classOf[QueueActor], s2, walLogHandler)
}
-class QueueActor(s2: Graph, walLogHandler: ExceptionHandler) extends Actor with ActorLogging {
+class QueueActor(s2: S2Graph, walLogHandler: ExceptionHandler) extends Actor with ActorLogging {
import Protocol._
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2rest_play/app/org/apache/s2graph/rest/play/controllers/EdgeController.scala
----------------------------------------------------------------------
diff --git a/s2rest_play/app/org/apache/s2graph/rest/play/controllers/EdgeController.scala b/s2rest_play/app/org/apache/s2graph/rest/play/controllers/EdgeController.scala
index b1635fb..aed8ced 100644
--- a/s2rest_play/app/org/apache/s2graph/rest/play/controllers/EdgeController.scala
+++ b/s2rest_play/app/org/apache/s2graph/rest/play/controllers/EdgeController.scala
@@ -38,7 +38,7 @@ object EdgeController extends Controller {
import ApplicationController._
import play.api.libs.concurrent.Execution.Implicits._
- private val s2: Graph = org.apache.s2graph.rest.play.Global.s2graph
+ private val s2: S2Graph = org.apache.s2graph.rest.play.Global.s2graph
private val requestParser: RequestParser = org.apache.s2graph.rest.play.Global.s2parser
private val walLogHandler: ExceptionHandler = org.apache.s2graph.rest.play.Global.wallLogHandler
@@ -51,9 +51,9 @@ object EdgeController extends Controller {
val kafkaTopic = toKafkaTopic(graphElem.isAsync)
graphElem match {
- case v: Vertex =>
+ case v: S2Vertex =>
enqueue(kafkaTopic, graphElem, tsv)
- case e: Edge =>
+ case e: S2Edge =>
e.innerLabel.extraOptions.get("walLog") match {
case None =>
enqueue(kafkaTopic, e, tsv)
@@ -73,7 +73,7 @@ object EdgeController extends Controller {
}
}
- private def toDeleteAllFailMessages(srcVertices: Seq[Vertex], labels: Seq[Label], dir: Int, ts: Long ) = {
+ private def toDeleteAllFailMessages(srcVertices: Seq[S2Vertex], labels: Seq[Label], dir: Int, ts: Long ) = {
for {
vertex <- srcVertices
id = vertex.id.toString
@@ -92,7 +92,7 @@ object EdgeController extends Controller {
val result = s2.mutateElements(elements.map(_._1), true)
result onComplete { results =>
results.get.zip(elements).map {
- case (false, (e: Edge, tsv: String)) =>
+ case (false, (e: S2Edge, tsv: String)) =>
val kafkaMessages = if(e.op == GraphUtil.operations("deleteAll")){
toDeleteAllFailMessages(Seq(e.srcVertex), Seq(e.innerLabel), e.labelWithDir.dir, e.ts)
} else{
@@ -267,7 +267,7 @@ object EdgeController extends Controller {
}
def deleteEach(labels: Seq[Label], direction: String, ids: Seq[JsValue],
- ts: Long, vertices: Seq[Vertex]) = {
+ ts: Long, vertices: Seq[S2Vertex]) = {
val future = s2.deleteAllAdjacentEdges(vertices.toList, labels, GraphUtil.directions(direction), ts)
if (withWait) {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2rest_play/app/org/apache/s2graph/rest/play/controllers/VertexController.scala
----------------------------------------------------------------------
diff --git a/s2rest_play/app/org/apache/s2graph/rest/play/controllers/VertexController.scala b/s2rest_play/app/org/apache/s2graph/rest/play/controllers/VertexController.scala
index 72e6e82..43f0b15 100644
--- a/s2rest_play/app/org/apache/s2graph/rest/play/controllers/VertexController.scala
+++ b/s2rest_play/app/org/apache/s2graph/rest/play/controllers/VertexController.scala
@@ -21,7 +21,7 @@ package org.apache.s2graph.rest.play.controllers
import org.apache.s2graph.core.rest.RequestParser
import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{ExceptionHandler, Graph, GraphExceptions}
+import org.apache.s2graph.core.{ExceptionHandler, S2Graph, GraphExceptions}
import org.apache.s2graph.rest.play.actors.QueueActor
import org.apache.s2graph.rest.play.config.Config
import play.api.libs.json.{JsValue, Json}
@@ -30,7 +30,7 @@ import play.api.mvc.{Controller, Result}
import scala.concurrent.Future
object VertexController extends Controller {
- private val s2: Graph = org.apache.s2graph.rest.play.Global.s2graph
+ private val s2: S2Graph = org.apache.s2graph.rest.play.Global.s2graph
private val requestParser: RequestParser = org.apache.s2graph.rest.play.Global.s2parser
private val walLogHandler: ExceptionHandler = org.apache.s2graph.rest.play.Global.wallLogHandler
[03/10] incubator-s2graph git commit: [S2GRAPH-131]: Add actual
implementation on interfaces from TinkerPop3 structure package. - Change
core.Edge/Vertex/Graph to core.S2Edge/S2Vertex/S2Graph. - Implement base
interfaces for tinkerpop3 structure packag
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
new file mode 100644
index 0000000..cd1e7f0
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
@@ -0,0 +1,969 @@
+/*
+ * 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.s2graph.core
+
+import java.util
+import java.util.function.{Consumer, BiConsumer}
+
+import org.apache.s2graph.core.S2Edge.{Props, State}
+import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
+import org.apache.s2graph.core.types._
+import org.apache.s2graph.core.utils.logger
+import org.apache.tinkerpop.gremlin.structure
+import org.apache.tinkerpop.gremlin.structure.{Edge, Graph, Vertex, Direction, Property}
+import play.api.libs.json.{JsNumber, JsObject, Json}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{Map => MutableMap}
+import scala.util.hashing.MurmurHash3
+
+case class SnapshotEdge(graph: S2Graph,
+ srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ label: Label,
+ dir: Int,
+ op: Byte,
+ version: Long,
+ private val propsWithTs: Props,
+ pendingEdgeOpt: Option[S2Edge],
+ statusCode: Byte = 0,
+ lockTs: Option[Long],
+ tsInnerValOpt: Option[InnerValLike] = None) {
+ lazy val direction = GraphUtil.fromDirection(dir)
+ lazy val operation = GraphUtil.fromOp(op)
+ lazy val edge = toEdge
+ lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
+// if (!propsWithTs.contains(LabelMeta.timestamp.name)) throw new Exception("Timestamp is required.")
+
+// val label = Label.findById(labelWithDir.labelId)
+ lazy val schemaVer = label.schemaVersion
+ lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.toString().toLong
+
+ def propsToKeyValuesWithTs = HBaseSerializable.propsToKeyValuesWithTs(propsWithTs.asScala.map(kv => kv._2.labelMeta.seq -> kv._2.innerValWithTs).toSeq)
+
+ def allPropsDeleted = S2Edge.allPropsDeleted(propsWithTs)
+
+ def toEdge: S2Edge = {
+ S2Edge(graph, srcVertex, tgtVertex, label, dir, op,
+ version, propsWithTs, pendingEdgeOpt = pendingEdgeOpt,
+ statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ def propsWithName = (for {
+ (_, v) <- propsWithTs.asScala
+ meta = v.labelMeta
+ jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
+ } yield meta.name -> jsValue) ++ Map("version" -> JsNumber(version))
+
+ // only for debug
+ def toLogString() = {
+ List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, propsWithName).mkString("\t")
+ }
+
+ def property[V](key: String, value: V, ts: Long): S2Property[V] = {
+ val labelMeta = label.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on IndexEdge."))
+ val newProps = new S2Property(edge, labelMeta, key, value, ts)
+ propsWithTs.put(key, newProps)
+ newProps
+ }
+ override def hashCode(): Int = {
+ MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId)
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case e: SnapshotEdge =>
+ srcVertex.innerId == e.srcVertex.innerId &&
+ tgtVertex.innerId == e.tgtVertex.innerId &&
+ labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
+ pendingEdgeOpt == e.pendingEdgeOpt && lockTs == lockTs && statusCode == statusCode
+ case _ => false
+ }
+
+ override def toString(): String = {
+ Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> label.label, "direction" -> direction,
+ "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString,
+ "statusCode" -> statusCode, "lockTs" -> lockTs).toString
+ }
+}
+
+case class IndexEdge(graph: S2Graph,
+ srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ label: Label,
+ dir: Int,
+ op: Byte,
+ version: Long,
+ labelIndexSeq: Byte,
+ private val propsWithTs: Props,
+ tsInnerValOpt: Option[InnerValLike] = None) {
+// if (!props.contains(LabelMeta.timeStampSeq)) throw new Exception("Timestamp is required.")
+ // assert(props.contains(LabelMeta.timeStampSeq))
+ lazy val direction = GraphUtil.fromDirection(dir)
+ lazy val operation = GraphUtil.fromOp(op)
+ lazy val edge = toEdge
+ lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
+
+ lazy val isInEdge = labelWithDir.dir == GraphUtil.directions("in")
+ lazy val isOutEdge = !isInEdge
+
+ lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.toString.toLong
+ lazy val degreeEdge = propsWithTs.containsKey(LabelMeta.degree.name)
+
+ lazy val schemaVer = label.schemaVersion
+ lazy val labelIndex = LabelIndex.findByLabelIdAndSeq(labelWithDir.labelId, labelIndexSeq).get
+ lazy val defaultIndexMetas = labelIndex.sortKeyTypes.map { meta =>
+ val innerVal = toInnerVal(meta.defaultValue, meta.dataType, schemaVer)
+ meta.seq -> innerVal
+ }.toMap
+
+ lazy val labelIndexMetaSeqs = labelIndex.sortKeyTypes
+
+ /** TODO: make sure call of this class fill props as this assumes */
+ lazy val orders = for (meta <- labelIndexMetaSeqs) yield {
+ propsWithTs.get(meta.name) match {
+ case null =>
+
+ /**
+ * TODO: agly hack
+ * now we double store target vertex.innerId/srcVertex.innerId for easy development. later fix this to only store id once
+ */
+ val v = meta match {
+ case LabelMeta.timestamp=> InnerVal.withLong(version, schemaVer)
+ case LabelMeta.to => toEdge.tgtVertex.innerId
+ case LabelMeta.from => toEdge.srcVertex.innerId
+ // for now, it does not make sense to build index on srcVertex.innerId since all edges have same data.
+ // throw new RuntimeException("_from on indexProps is not supported")
+ case _ => toInnerVal(meta.defaultValue, meta.dataType, schemaVer)
+ }
+
+ meta -> v
+ case v => meta -> v.innerVal
+ }
+ }
+
+ lazy val ordersKeyMap = orders.map { case (meta, _) => meta.name }.toSet
+ lazy val metas = for ((meta, v) <- propsWithTs.asScala if !ordersKeyMap.contains(meta)) yield v.labelMeta -> v.innerVal
+
+// lazy val propsWithTs = props.map { case (k, v) => k -> InnerValLikeWithTs(v, version) }
+
+ //TODO:
+ // lazy val kvs = Graph.client.indexedEdgeSerializer(this).toKeyValues.toList
+
+ lazy val hasAllPropsForIndex = orders.length == labelIndexMetaSeqs.length
+
+ def propsWithName = for {
+ (_, v) <- propsWithTs.asScala
+ meta = v.labelMeta
+ jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
+ } yield meta.name -> jsValue
+
+
+ def toEdge: S2Edge = S2Edge(graph, srcVertex, tgtVertex, label, dir, op, version, propsWithTs, tsInnerValOpt = tsInnerValOpt)
+
+ // only for debug
+ def toLogString() = {
+ List(version, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, Json.toJson(propsWithName)).mkString("\t")
+ }
+
+ def property(key: String): Option[InnerValLikeWithTs] = {
+ label.metaPropsInvMap.get(key).map(labelMeta => property(labelMeta))
+ }
+
+ def property(labelMeta: LabelMeta): InnerValLikeWithTs = {
+// propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse(label.metaPropsDefaultMapInner(labelMeta))
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ propsWithTs.get(labelMeta.name).innerValWithTs
+ } else {
+ label.metaPropsDefaultMapInner(labelMeta)
+ }
+ }
+
+ def updatePropsWithTs(others: Props = S2Edge.EmptyProps): Props = {
+ if (others.isEmpty) propsWithTs
+ else {
+ val iter = others.entrySet().iterator()
+ while (iter.hasNext) {
+ val e = iter.next()
+ propsWithTs.put(e.getKey, e.getValue)
+ }
+ propsWithTs
+ }
+ }
+
+ def property[V](key: String, value: V, ts: Long): S2Property[V] = {
+ val labelMeta = label.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on IndexEdge."))
+ val newProps = new S2Property(edge, labelMeta, key, value, ts)
+ propsWithTs.put(key, newProps)
+ newProps
+ }
+ override def hashCode(): Int = {
+ MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId + "," + labelIndexSeq)
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case e: IndexEdge =>
+ srcVertex.innerId == e.srcVertex.innerId &&
+ tgtVertex.innerId == e.tgtVertex.innerId &&
+ labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
+ labelIndexSeq == e.labelIndexSeq
+ case _ => false
+ }
+
+ override def toString(): String = {
+ Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> label.label, "direction" -> dir,
+ "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString
+ ).toString
+ }
+}
+
+case class S2Edge(innerGraph: S2Graph,
+ srcVertex: S2Vertex,
+ var tgtVertex: S2Vertex,
+ innerLabel: Label,
+ dir: Int,
+ var op: Byte = GraphUtil.defaultOpByte,
+ var version: Long = System.currentTimeMillis(),
+ propsWithTs: Props = S2Edge.EmptyProps,
+ parentEdges: Seq[EdgeWithScore] = Nil,
+ originalEdgeOpt: Option[S2Edge] = None,
+ pendingEdgeOpt: Option[S2Edge] = None,
+ statusCode: Byte = 0,
+ lockTs: Option[Long] = None,
+ var tsInnerValOpt: Option[InnerValLike] = None) extends GraphElement with Edge {
+
+ lazy val labelWithDir = LabelWithDirection(innerLabel.id.get, dir)
+ lazy val schemaVer = innerLabel.schemaVersion
+ lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.value match {
+ case b: BigDecimal => b.longValue()
+ case l: Long => l
+ case i: Int => i.toLong
+ case _ => throw new RuntimeException("ts should be in [BigDecimal/Long/Int].")
+ }
+
+ lazy val operation = GraphUtil.fromOp(op)
+ lazy val tsInnerVal = tsInnerValOpt.get.value
+ lazy val srcId = srcVertex.innerIdVal
+ lazy val tgtId = tgtVertex.innerIdVal
+ lazy val labelName = innerLabel.label
+ lazy val direction = GraphUtil.fromDirection(dir)
+
+ def toIndexEdge(labelIndexSeq: Byte): IndexEdge = IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelIndexSeq, propsWithTs)
+
+ def serializePropsWithTs(): Array[Byte] = HBaseSerializable.propsToKeyValuesWithTs(propsWithTs.asScala.map(kv => kv._2.labelMeta.seq -> kv._2.innerValWithTs).toSeq)
+
+ def updatePropsWithTs(others: Props = S2Edge.EmptyProps): Props = {
+ val emptyProp = S2Edge.EmptyProps
+
+ propsWithTs.forEach(new BiConsumer[String, S2Property[_]] {
+ override def accept(key: String, value: S2Property[_]): Unit = emptyProp.put(key, value)
+ })
+
+ others.forEach(new BiConsumer[String, S2Property[_]] {
+ override def accept(key: String, value: S2Property[_]): Unit = emptyProp.put(key, value)
+ })
+
+ emptyProp
+ }
+
+ def propertyValue(key: String): Option[InnerValLikeWithTs] = {
+ key match {
+ case "from" | "_from" => Option(InnerValLikeWithTs(srcVertex.innerId, ts))
+ case "to" | "_to" => Option(InnerValLikeWithTs(tgtVertex.innerId, ts))
+ case "label" => Option(InnerValLikeWithTs(InnerVal.withStr(innerLabel.label, schemaVer), ts))
+ case "direction" => Option(InnerValLikeWithTs(InnerVal.withStr(direction, schemaVer), ts))
+ case _ =>
+ innerLabel.metaPropsInvMap.get(key).map(labelMeta => propertyValueInner(labelMeta))
+ }
+ }
+
+ def propertyValueInner(labelMeta: LabelMeta): InnerValLikeWithTs= {
+ // propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse()
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ propsWithTs.get(labelMeta.name).innerValWithTs
+ } else {
+ innerLabel.metaPropsDefaultMapInner(labelMeta)
+ }
+ }
+
+ def propertyValues(keys: Seq[String] = Nil): Map[LabelMeta, InnerValLikeWithTs] = {
+ val labelMetas = for {
+ key <- keys
+ labelMeta <- innerLabel.metaPropsInvMap.get(key)
+ } yield labelMeta
+
+ propertyValuesInner(labelMetas)
+ }
+
+ def propertyValuesInner(labelMetas: Seq[LabelMeta] = Nil): Map[LabelMeta, InnerValLikeWithTs] = {
+ if (labelMetas.isEmpty) {
+ innerLabel.metaPropsDefaultMapInner.map { case (labelMeta, defaultVal) =>
+ labelMeta -> propertyValueInner(labelMeta)
+ }
+ } else {
+ // This is important since timestamp is required for all edges.
+ (LabelMeta.timestamp +: labelMetas).map { labelMeta =>
+ labelMeta -> propertyValueInner(labelMeta)
+ }.toMap
+ }
+ }
+
+// if (!props.contains(LabelMeta.timestamp)) throw new Exception("Timestamp is required.")
+ // assert(propsWithTs.contains(LabelMeta.timeStampSeq))
+
+ lazy val properties = toProps()
+
+ def props = propsWithTs.asScala.mapValues(_.innerVal)
+
+
+ private def toProps(): Map[String, Any] = {
+ for {
+ (labelMeta, defaultVal) <- innerLabel.metaPropsDefaultMapInner
+ } yield {
+ // labelMeta.name -> propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse(defaultVal).innerVal.value
+ val value =
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ propsWithTs.get(labelMeta.name).value
+ } else {
+ defaultVal.innerVal.value
+ }
+ labelMeta.name -> value
+ }
+ }
+
+ def relatedEdges = {
+ if (labelWithDir.isDirected) {
+ val skipReverse = innerLabel.extraOptions.get("skipReverse").map(_.as[Boolean]).getOrElse(false)
+ if (skipReverse) List(this) else List(this, duplicateEdge)
+ } else {
+// val outDir = labelWithDir.copy(dir = GraphUtil.directions("out"))
+// val base = copy(labelWithDir = outDir)
+ val base = copy(dir = GraphUtil.directions("out"))
+ List(base, base.reverseSrcTgtEdge)
+ }
+ }
+
+ // def relatedEdges = List(this)
+
+ def srcForVertex = {
+ val belongLabelIds = Seq(labelWithDir.labelId)
+ if (labelWithDir.dir == GraphUtil.directions("in")) {
+ innerGraph.newVertex(VertexId(innerLabel.tgtColumn, tgtVertex.innerId), tgtVertex.ts, tgtVertex.props, belongLabelIds = belongLabelIds)
+ } else {
+ innerGraph.newVertex(VertexId(innerLabel.srcColumn, srcVertex.innerId), srcVertex.ts, srcVertex.props, belongLabelIds = belongLabelIds)
+ }
+ }
+
+ def tgtForVertex = {
+ val belongLabelIds = Seq(labelWithDir.labelId)
+ if (labelWithDir.dir == GraphUtil.directions("in")) {
+ innerGraph.newVertex(VertexId(innerLabel.srcColumn, srcVertex.innerId), srcVertex.ts, srcVertex.props, belongLabelIds = belongLabelIds)
+ } else {
+ innerGraph.newVertex(VertexId(innerLabel.tgtColumn, tgtVertex.innerId), tgtVertex.ts, tgtVertex.props, belongLabelIds = belongLabelIds)
+ }
+ }
+
+ def duplicateEdge = reverseSrcTgtEdge.reverseDirEdge
+
+// def reverseDirEdge = copy(labelWithDir = labelWithDir.dirToggled)
+ def reverseDirEdge = copy(dir = GraphUtil.toggleDir(dir))
+
+ def reverseSrcTgtEdge = copy(srcVertex = tgtVertex, tgtVertex = srcVertex)
+
+ def labelOrders = LabelIndex.findByLabelIdAll(labelWithDir.labelId)
+
+ override def serviceName = innerLabel.serviceName
+
+ override def queueKey = Seq(ts.toString, tgtVertex.serviceName).mkString("|")
+
+ override def queuePartitionKey = Seq(srcVertex.innerId, tgtVertex.innerId).mkString("|")
+
+ override def isAsync = innerLabel.isAsync
+
+ def isDegree = propsWithTs.containsKey(LabelMeta.degree.name)
+
+// def propsPlusTs = propsWithTs.get(LabelMeta.timeStampSeq) match {
+// case Some(_) => props
+// case None => props ++ Map(LabelMeta.timeStampSeq -> InnerVal.withLong(ts, schemaVer))
+// }
+
+ def propsPlusTsValid = propsWithTs.asScala.filter(kv => LabelMeta.isValidSeq(kv._2.labelMeta.seq)).asJava
+
+ def edgesWithIndex = for (labelOrder <- labelOrders) yield {
+ IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelOrder.seq, propsWithTs, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ def edgesWithIndexValid = for (labelOrder <- labelOrders) yield {
+ IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelOrder.seq, propsPlusTsValid, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ /** force direction as out on invertedEdge */
+ def toSnapshotEdge: SnapshotEdge = {
+ val (smaller, larger) = (srcForVertex, tgtForVertex)
+
+// val newLabelWithDir = LabelWithDirection(labelWithDir.labelId, GraphUtil.directions("out"))
+
+ property(LabelMeta.timestamp.name, ts, ts)
+ val ret = SnapshotEdge(innerGraph, smaller, larger, innerLabel,
+ GraphUtil.directions("out"), op, version, propsWithTs,
+ pendingEdgeOpt = pendingEdgeOpt, statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
+ ret
+ }
+
+ def defaultPropsWithName = Json.obj("from" -> srcVertex.innerId.toString(), "to" -> tgtVertex.innerId.toString(),
+ "label" -> innerLabel.label, "service" -> innerLabel.serviceName)
+
+ def propsWithName =
+ for {
+ (_, v) <- propsWithTs.asScala
+ meta = v.labelMeta
+ jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
+ } yield meta.name -> jsValue
+
+
+ def updateTgtVertex(id: InnerValLike) = {
+ val newId = TargetVertexId(tgtVertex.id.column, id)
+ val newTgtVertex = innerGraph.newVertex(newId, tgtVertex.ts, tgtVertex.props)
+ S2Edge(innerGraph, srcVertex, newTgtVertex, innerLabel, dir, op, version, propsWithTs, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ def rank(r: RankParam): Double =
+ if (r.keySeqAndWeights.size <= 0) 1.0f
+ else {
+ var sum: Double = 0
+
+ for ((labelMeta, w) <- r.keySeqAndWeights) {
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ val innerValWithTs = propsWithTs.get(labelMeta.name)
+ val cost = try innerValWithTs.innerVal.toString.toDouble catch {
+ case e: Exception =>
+ logger.error("toInnerval failed in rank", e)
+ 1.0
+ }
+ sum += w * cost
+ }
+ }
+ sum
+ }
+
+ def toLogString: String = {
+ val allPropsWithName = defaultPropsWithName ++ Json.toJson(propsWithName).asOpt[JsObject].getOrElse(Json.obj())
+ List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, innerLabel.label, allPropsWithName).mkString("\t")
+ }
+
+ override def hashCode(): Int = {
+ MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId)
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case e: S2Edge =>
+ srcVertex.innerId == e.srcVertex.innerId &&
+ tgtVertex.innerId == e.tgtVertex.innerId &&
+ labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
+ pendingEdgeOpt == e.pendingEdgeOpt && lockTs == lockTs && statusCode == statusCode &&
+ parentEdges == e.parentEdges && originalEdgeOpt == originalEdgeOpt
+ case _ => false
+ }
+
+ override def toString(): String = {
+ Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> labelName, "direction" -> direction,
+ "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString,
+ "parentEdges" -> parentEdges, "originalEdge" -> originalEdgeOpt, "statusCode" -> statusCode, "lockTs" -> lockTs
+ ).toString
+ }
+
+ def checkProperty(key: String): Boolean = propsWithTs.containsKey(key)
+
+ def copyEdge(srcVertex: S2Vertex = srcVertex,
+ tgtVertex: S2Vertex = tgtVertex,
+ innerLabel: Label = innerLabel,
+ dir: Int = dir,
+ op: Byte = op,
+ version: Long = version,
+ propsWithTs: State = S2Edge.propsToState(this.propsWithTs),
+ parentEdges: Seq[EdgeWithScore] = parentEdges,
+ originalEdgeOpt: Option[S2Edge] = originalEdgeOpt,
+ pendingEdgeOpt: Option[S2Edge] = pendingEdgeOpt,
+ statusCode: Byte = statusCode,
+ lockTs: Option[Long] = lockTs,
+ tsInnerValOpt: Option[InnerValLike] = tsInnerValOpt,
+ ts: Long = ts): S2Edge = {
+ val edge = new S2Edge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, S2Edge.EmptyProps,
+ parentEdges, originalEdgeOpt, pendingEdgeOpt, statusCode, lockTs, tsInnerValOpt)
+ S2Edge.fillPropsWithTs(edge, propsWithTs)
+ edge.property(LabelMeta.timestamp.name, ts, ts)
+ edge
+ }
+
+ def copyEdgeWithState(state: State, ts: Long): S2Edge = {
+ val newEdge = copy(propsWithTs = S2Edge.EmptyProps)
+ S2Edge.fillPropsWithTs(newEdge, state)
+ newEdge.property(LabelMeta.timestamp.name, ts, ts)
+ newEdge
+ }
+
+ def copyEdgeWithState(state: State): S2Edge = {
+ val newEdge = copy(propsWithTs = S2Edge.EmptyProps)
+ S2Edge.fillPropsWithTs(newEdge, state)
+ newEdge
+ }
+
+ override def vertices(direction: Direction): util.Iterator[structure.Vertex] = {
+ val arr = new util.ArrayList[Vertex]()
+ direction match {
+ case Direction.OUT => arr.add(srcVertex)
+ case Direction.IN => arr.add(tgtVertex)
+ case _ =>
+ arr.add(srcVertex)
+ arr.add(tgtVertex)
+ }
+ arr.iterator()
+ }
+
+ override def properties[V](keys: String*): util.Iterator[Property[V]] = {
+ val ls = new util.ArrayList[Property[V]]()
+ keys.foreach { key => ls.add(property(key)) }
+ ls.iterator()
+ }
+
+ override def property[V](key: String): Property[V] = {
+ val labelMeta = innerLabel.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on Edge."))
+ if (propsWithTs.containsKey(key)) propsWithTs.get(key).asInstanceOf[Property[V]]
+ else {
+ val default = innerLabel.metaPropsDefaultMapInner(labelMeta)
+ property(key, default.innerVal.value, default.ts).asInstanceOf[Property[V]]
+ }
+ }
+
+ override def property[V](key: String, value: V): Property[V] = {
+ property(key, value, System.currentTimeMillis())
+ }
+
+ def property[V](key: String, value: V, ts: Long): Property[V] = {
+ val labelMeta = innerLabel.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on Edge."))
+ val newProp = new S2Property[V](this, labelMeta, key, value, ts)
+ propsWithTs.put(key, newProp)
+ newProp
+ }
+
+ override def remove(): Unit = {}
+
+ override def graph(): Graph = innerGraph
+
+ override def id(): AnyRef = (srcVertex.innerId, labelWithDir, tgtVertex.innerId)
+
+ override def label(): String = innerLabel.label
+}
+
+
+case class EdgeMutate(edgesToDelete: List[IndexEdge] = List.empty[IndexEdge],
+ edgesToInsert: List[IndexEdge] = List.empty[IndexEdge],
+ newSnapshotEdge: Option[SnapshotEdge] = None) {
+
+ def toLogString: String = {
+ val l = (0 until 50).map(_ => "-").mkString("")
+ val deletes = s"deletes: ${edgesToDelete.map(e => e.toLogString).mkString("\n")}"
+ val inserts = s"inserts: ${edgesToInsert.map(e => e.toLogString).mkString("\n")}"
+ val updates = s"snapshot: ${newSnapshotEdge.map(e => e.toLogString).mkString("\n")}"
+
+ List("\n", l, deletes, inserts, updates, l, "\n").mkString("\n")
+ }
+}
+
+object S2Edge {
+ val incrementVersion = 1L
+ val minTsVal = 0L
+
+ /** now version information is required also **/
+ type Props = java.util.Map[String, S2Property[_]]
+ type State = Map[LabelMeta, InnerValLikeWithTs]
+ type PropsPairWithTs = (State, State, Long, String)
+ type MergeState = PropsPairWithTs => (State, Boolean)
+ type UpdateFunc = (Option[S2Edge], S2Edge, MergeState)
+
+ def EmptyProps = new java.util.HashMap[String, S2Property[_]]
+ def EmptyState = Map.empty[LabelMeta, InnerValLikeWithTs]
+ def sameProps(base: Props, other: Props): Boolean = {
+ if (base.size != other.size) false
+ else {
+ var ret = true
+ val iter = base.entrySet().iterator()
+ while (iter.hasNext) {
+ val e = iter.next()
+ if (!other.containsKey(e.getKey)) ret = false
+ else if (e.getValue != other.get(e.getKey)) ret = false
+ else {
+
+ }
+ }
+ val otherIter = other.entrySet().iterator()
+ while (otherIter.hasNext) {
+ val e = otherIter.next()
+ if (!base.containsKey(e.getKey)) ret = false
+ else if (e.getValue != base.get(e.getKey)) ret = false
+ else {
+
+ }
+ }
+ ret
+ }
+// base.sameElements(other)
+ }
+ def fillPropsWithTs(snapshotEdge: SnapshotEdge, state: State): Unit = {
+ state.foreach { case (k, v) => snapshotEdge.property(k.name, v.innerVal.value, v.ts) }
+ }
+ def fillPropsWithTs(indexEdge: IndexEdge, state: State): Unit = {
+ state.foreach { case (k, v) => indexEdge.property(k.name, v.innerVal.value, v.ts) }
+ }
+ def fillPropsWithTs(edge: S2Edge, state: State): Unit = {
+ state.foreach { case (k, v) => edge.property(k.name, v.innerVal.value, v.ts) }
+ }
+
+ def propsToState(props: Props): State = {
+ props.asScala.map { case (k, v) =>
+ v.labelMeta -> v.innerValWithTs
+ }.toMap
+ }
+
+ def stateToProps(edge: S2Edge, state: State): Props = {
+ state.foreach { case (k, v) =>
+ edge.property(k.name, v.innerVal.value, v.ts)
+ }
+ edge.propsWithTs
+ }
+
+ def allPropsDeleted(props: Map[LabelMeta, InnerValLikeWithTs]): Boolean =
+ if (!props.contains(LabelMeta.lastDeletedAt)) false
+ else {
+ val lastDeletedAt = props.get(LabelMeta.lastDeletedAt).get.ts
+ val propsWithoutLastDeletedAt = props - LabelMeta.lastDeletedAt
+
+ propsWithoutLastDeletedAt.forall { case (_, v) => v.ts <= lastDeletedAt }
+ }
+
+ def allPropsDeleted(props: Props): Boolean =
+ if (!props.containsKey(LabelMeta.lastDeletedAt.name)) false
+ else {
+ val lastDeletedAt = props.get(LabelMeta.lastDeletedAt.name).ts
+ props.remove(LabelMeta.lastDeletedAt.name)
+// val propsWithoutLastDeletedAt = props
+//
+// propsWithoutLastDeletedAt.forall { case (_, v) => v.ts <= lastDeletedAt }
+ var ret = true
+ val iter = props.entrySet().iterator()
+ while (iter.hasNext) {
+ val e = iter.next()
+ if (e.getValue.ts > lastDeletedAt) ret = false
+ }
+ ret
+ }
+
+ def buildDeleteBulk(invertedEdge: Option[S2Edge], requestEdge: S2Edge): (S2Edge, EdgeMutate) = {
+ // assert(invertedEdge.isEmpty)
+ // assert(requestEdge.op == GraphUtil.operations("delete"))
+
+ val edgesToDelete = requestEdge.relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
+ val edgeInverted = Option(requestEdge.toSnapshotEdge)
+
+ (requestEdge, EdgeMutate(edgesToDelete, edgesToInsert = Nil, newSnapshotEdge = edgeInverted))
+ }
+
+ def buildOperation(invertedEdge: Option[S2Edge], requestEdges: Seq[S2Edge]): (S2Edge, EdgeMutate) = {
+ // logger.debug(s"oldEdge: ${invertedEdge.map(_.toStringRaw)}")
+ // logger.debug(s"requestEdge: ${requestEdge.toStringRaw}")
+ val oldPropsWithTs =
+ if (invertedEdge.isEmpty) Map.empty[LabelMeta, InnerValLikeWithTs]
+ else propsToState(invertedEdge.get.propsWithTs)
+
+ val funcs = requestEdges.map { edge =>
+ if (edge.op == GraphUtil.operations("insert")) {
+ edge.innerLabel.consistencyLevel match {
+ case "strong" => S2Edge.mergeUpsert _
+ case _ => S2Edge.mergeInsertBulk _
+ }
+ } else if (edge.op == GraphUtil.operations("insertBulk")) {
+ S2Edge.mergeInsertBulk _
+ } else if (edge.op == GraphUtil.operations("delete")) {
+ edge.innerLabel.consistencyLevel match {
+ case "strong" => S2Edge.mergeDelete _
+ case _ => throw new RuntimeException("not supported")
+ }
+ }
+ else if (edge.op == GraphUtil.operations("update")) S2Edge.mergeUpdate _
+ else if (edge.op == GraphUtil.operations("increment")) S2Edge.mergeIncrement _
+ else throw new RuntimeException(s"not supported operation on edge: $edge")
+ }
+
+ val oldTs = invertedEdge.map(_.ts).getOrElse(minTsVal)
+ val requestWithFuncs = requestEdges.zip(funcs).filter(oldTs != _._1.ts).sortBy(_._1.ts)
+
+ if (requestWithFuncs.isEmpty) {
+ (requestEdges.head, EdgeMutate())
+ } else {
+ val requestEdge = requestWithFuncs.last._1
+ var prevPropsWithTs = oldPropsWithTs
+
+ for {
+ (requestEdge, func) <- requestWithFuncs
+ } {
+ val (_newPropsWithTs, _) = func(prevPropsWithTs, propsToState(requestEdge.propsWithTs), requestEdge.ts, requestEdge.schemaVer)
+ prevPropsWithTs = _newPropsWithTs
+ // logger.debug(s"${requestEdge.toLogString}\n$oldPropsWithTs\n$prevPropsWithTs\n")
+ }
+ val requestTs = requestEdge.ts
+ /** version should be monotoniously increasing so our RPC mutation should be applied safely */
+ val newVersion = invertedEdge.map(e => e.version + incrementVersion).getOrElse(requestTs)
+ val maxTs = prevPropsWithTs.map(_._2.ts).max
+ val newTs = if (maxTs > requestTs) maxTs else requestTs
+ val propsWithTs = prevPropsWithTs ++
+ Map(LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(newTs, requestEdge.innerLabel.schemaVersion), newTs))
+
+ val edgeMutate = buildMutation(invertedEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
+
+ // logger.debug(s"${edgeMutate.toLogString}\n${propsWithTs}")
+ // logger.error(s"$propsWithTs")
+ val newEdge = requestEdge.copy(propsWithTs = EmptyProps)
+ fillPropsWithTs(newEdge, propsWithTs)
+ (newEdge, edgeMutate)
+ }
+ }
+
+ def filterOutWithLabelOption(ls: Seq[IndexEdge]): Seq[IndexEdge] = ls.filter { ie =>
+ ie.labelIndex.dir match {
+ case None =>
+ // both direction use same indices that is defined when label creation.
+ true
+ case Some(dir) =>
+ if (dir != ie.dir) {
+ // current labelIndex's direction is different with indexEdge's direction so don't touch
+ false
+ } else {
+ ie.labelIndex.writeOption.map { option =>
+ val hashValueOpt = ie.orders.find { case (k, v) => k == LabelMeta.fromHash }.map{ case (k, v) => v.value.toString.toLong }
+ option.sample(ie, hashValueOpt)
+ }.getOrElse(true)
+ }
+ }
+ }
+
+ def buildMutation(snapshotEdgeOpt: Option[S2Edge],
+ requestEdge: S2Edge,
+ newVersion: Long,
+ oldPropsWithTs: Map[LabelMeta, InnerValLikeWithTs],
+ newPropsWithTs: Map[LabelMeta, InnerValLikeWithTs]): EdgeMutate = {
+
+ if (oldPropsWithTs == newPropsWithTs) {
+ // all requests should be dropped. so empty mutation.
+ EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = None)
+ } else {
+ val withOutDeletedAt = newPropsWithTs.filter(kv => kv._1 != LabelMeta.lastDeletedAtSeq)
+ val newOp = snapshotEdgeOpt match {
+ case None => requestEdge.op
+ case Some(old) =>
+ val oldMaxTs = old.propsWithTs.asScala.map(_._2.ts).max
+ if (oldMaxTs > requestEdge.ts) old.op
+ else requestEdge.op
+ }
+
+ val newSnapshotEdge = requestEdge.copy(op = newOp, version = newVersion).copyEdgeWithState(newPropsWithTs)
+
+ val newSnapshotEdgeOpt = Option(newSnapshotEdge.toSnapshotEdge)
+ // delete request must always update snapshot.
+ if (withOutDeletedAt == oldPropsWithTs && newPropsWithTs.contains(LabelMeta.lastDeletedAt)) {
+ // no mutation on indexEdges. only snapshotEdge should be updated to record lastDeletedAt.
+ EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = newSnapshotEdgeOpt)
+ } else {
+ val edgesToDelete = snapshotEdgeOpt match {
+ case Some(snapshotEdge) if snapshotEdge.op != GraphUtil.operations("delete") =>
+ snapshotEdge.copy(op = GraphUtil.defaultOpByte)
+ .relatedEdges.flatMap { relEdge => filterOutWithLabelOption(relEdge.edgesWithIndexValid) }
+ case _ => Nil
+ }
+
+ val edgesToInsert =
+ if (newPropsWithTs.isEmpty || allPropsDeleted(newPropsWithTs)) Nil
+ else {
+ val newEdge = requestEdge.copy(
+ version = newVersion,
+ propsWithTs = S2Edge.EmptyProps,
+ op = GraphUtil.defaultOpByte
+ )
+ newPropsWithTs.foreach { case (k, v) => newEdge.property(k.name, v.innerVal.value, v.ts) }
+
+ newEdge.relatedEdges.flatMap { relEdge => filterOutWithLabelOption(relEdge.edgesWithIndexValid) }
+ }
+
+
+ EdgeMutate(edgesToDelete = edgesToDelete,
+ edgesToInsert = edgesToInsert,
+ newSnapshotEdge = newSnapshotEdgeOpt)
+ }
+ }
+ }
+
+ def mergeUpsert(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ propsWithTs.get(k) match {
+ case Some(newValWithTs) =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
+ else {
+ shouldReplace = true
+ newValWithTs
+ }
+ Some(k -> v)
+
+ case None =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ if (oldValWithTs.ts >= requestTs || k.seq < 0) Some(k -> oldValWithTs)
+ else {
+ shouldReplace = true
+ None
+ }
+ }
+ }
+ val existInNew =
+ for {
+ (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
+ } yield {
+ shouldReplace = true
+ Some(k -> newValWithTs)
+ }
+
+ ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
+ }
+
+ def mergeUpdate(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ propsWithTs.get(k) match {
+ case Some(newValWithTs) =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
+ else {
+ shouldReplace = true
+ newValWithTs
+ }
+ Some(k -> v)
+ case None =>
+ // important: update need to merge previous valid values.
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ Some(k -> oldValWithTs)
+ }
+ }
+ val existInNew = for {
+ (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
+ } yield {
+ shouldReplace = true
+ Some(k -> newValWithTs)
+ }
+
+ ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
+ }
+
+ def mergeIncrement(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ propsWithTs.get(k) match {
+ case Some(newValWithTs) =>
+ if (k == LabelMeta.timestamp) {
+ val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
+ else {
+ shouldReplace = true
+ newValWithTs
+ }
+ Some(k -> v)
+ } else {
+ if (oldValWithTs.ts >= newValWithTs.ts) {
+ Some(k -> oldValWithTs)
+ } else {
+ assert(oldValWithTs.ts < newValWithTs.ts && oldValWithTs.ts >= lastDeletedAt)
+ shouldReplace = true
+ // incr(t0), incr(t2), d(t1) => deleted
+ Some(k -> InnerValLikeWithTs(oldValWithTs.innerVal + newValWithTs.innerVal, oldValWithTs.ts))
+ }
+ }
+
+ case None =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ Some(k -> oldValWithTs)
+ // if (oldValWithTs.ts >= lastDeletedAt) Some(k -> oldValWithTs) else None
+ }
+ }
+ val existInNew = for {
+ (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
+ } yield {
+ shouldReplace = true
+ Some(k -> newValWithTs)
+ }
+
+ ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
+ }
+
+ def mergeDelete(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt) match {
+ case Some(prevDeletedAt) =>
+ if (prevDeletedAt.ts >= requestTs) prevDeletedAt.ts
+ else {
+ shouldReplace = true
+ requestTs
+ }
+ case None => {
+ shouldReplace = true
+ requestTs
+ }
+ }
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ if (k == LabelMeta.timestamp) {
+ if (oldValWithTs.ts >= requestTs) Some(k -> oldValWithTs)
+ else {
+ shouldReplace = true
+ Some(k -> InnerValLikeWithTs.withLong(requestTs, requestTs, version))
+ }
+ } else {
+ if (oldValWithTs.ts >= lastDeletedAt) Some(k -> oldValWithTs)
+ else {
+ shouldReplace = true
+ None
+ }
+ }
+ }
+ val mustExistInNew = Map(LabelMeta.lastDeletedAt -> InnerValLikeWithTs.withLong(lastDeletedAt, lastDeletedAt, version))
+ ((existInOld.flatten ++ mustExistInNew).toMap, shouldReplace)
+ }
+
+ def mergeInsertBulk(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ val (_, propsWithTs, _, _) = propsPairWithTs
+ (propsWithTs, true)
+ }
+
+// def fromString(s: String): Option[Edge] = Graph.toEdge(s)
+
+
+}
[09/10] incubator-s2graph git commit: - fix bug on
wide.IndexEdgeDeserializable. - bug fix on IndexEdgeTest.
Posted by st...@apache.org.
- fix bug on wide.IndexEdgeDeserializable.
- bug fix on IndexEdgeTest.
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/0f21ad68
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/0f21ad68
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/0f21ad68
Branch: refs/heads/master
Commit: 0f21ad686afcc4559c26929272e1bf4b0402385c
Parents: a08df11
Author: DO YUNG YOON <st...@apache.org>
Authored: Wed Nov 30 22:26:34 2016 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Wed Nov 30 22:26:34 2016 +0900
----------------------------------------------------------------------
.../scala/org/apache/s2graph/core/S2Edge.scala | 4 +-
.../wide/IndexEdgeDeserializable.scala | 1 +
.../core/storage/hbase/IndexEdgeTest.scala | 205 +++++++++----------
3 files changed, 105 insertions(+), 105 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/0f21ad68/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
index 5a8408d..2960265 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
@@ -476,7 +476,8 @@ case class S2Edge(innerGraph: S2Graph,
case e: S2Edge =>
srcVertex.innerId == e.srcVertex.innerId &&
tgtVertex.innerId == e.tgtVertex.innerId &&
- labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
+ labelWithDir == e.labelWithDir && S2Edge.sameProps(propsWithTs, e.propsWithTs) &&
+ op == e.op && version == e.version &&
pendingEdgeOpt == e.pendingEdgeOpt && lockTs == lockTs && statusCode == statusCode &&
parentEdges == e.parentEdges && originalEdgeOpt == originalEdgeOpt
case _ => false
@@ -624,7 +625,6 @@ object S2Edge {
}
ret
}
-// base.sameElements(other)
}
def fillPropsWithTs(snapshotEdge: SnapshotEdge, state: State): Unit = {
state.foreach { case (k, v) => snapshotEdge.property(k.name, v.innerVal.value, v.ts) }
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/0f21ad68/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
index 706d8cb..31a1a89 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeDeserializable.scala
@@ -128,6 +128,7 @@ class IndexEdgeDeserializable(graph: S2Graph,
TargetVertexId(ServiceColumn.Default, vId.innerVal)
} else tgtVertexIdRaw
+ edge.property(LabelMeta.timestamp.name, tsVal, version)
edge.tgtVertex = graph.newVertex(tgtVertexId, version)
edge.op = op
edge.tsInnerValOpt = Option(InnerVal.withLong(tsVal, schemaVer))
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/0f21ad68/s2core/src/test/scala/org/apache/s2graph/core/storage/hbase/IndexEdgeTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/storage/hbase/IndexEdgeTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/storage/hbase/IndexEdgeTest.scala
index acbc689..5409d61 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/storage/hbase/IndexEdgeTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/storage/hbase/IndexEdgeTest.scala
@@ -1,103 +1,102 @@
-///*
-// * 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.s2graph.core.storage.hbase
-//
-//import org.apache.s2graph.core.mysqls.{Model, Label, LabelIndex, LabelMeta}
-//import org.apache.s2graph.core.types._
-//import org.apache.s2graph.core.{QueryParam, IndexEdge, TestCommonWithModels, Vertex}
-//import org.scalatest.{FunSuite, Matchers}
-//
-//
-//class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
-// initTests()
-//
-// val testLabelMeta = LabelMeta(Option(-1), labelV2.id.get, "test", 1.toByte, "0.0", "double")
-// /**
-// * check if storage serializer/deserializer can translate from/to bytes array.
-// * @param l: label for edge.
-// * @param ts: timestamp for edge.
-// * @param to: to VertexId for edge.
-// * @param props: expected props of edge.
-// */
-// def check(l: Label, ts: Long, to: InnerValLike, props: Map[LabelMeta, InnerValLikeWithTs]): Unit = {
-// val from = InnerVal.withLong(1, l.schemaVersion)
-// val vertexId = SourceVertexId(HBaseType.DEFAULT_COL_ID, from)
-// val tgtVertexId = TargetVertexId(HBaseType.DEFAULT_COL_ID, to)
-// val vertex = Vertex(vertexId, ts)
-// val tgtVertex = Vertex(tgtVertexId, ts)
-// val labelWithDir = LabelWithDirection(l.id.get, 0)
-// val labelOpt = Option(l)
-// val edge = graph.newEdge(vertex, tgtVertex, l, labelWithDir.dir, 0, ts, props, tsInnerValOpt = Option(InnerVal.withLong(ts, l.schemaVersion)))
-// val indexEdge = edge.edgesWithIndex.find(_.labelIndexSeq == LabelIndex.DefaultSeq).head
-// val _indexEdgeOpt = graph.getStorage(l).indexEdgeDeserializer(l.schemaVersion).fromKeyValues(labelOpt,
-// graph.getStorage(l).indexEdgeSerializer(indexEdge).toKeyValues, l.schemaVersion, None)
-//
-//
-// _indexEdgeOpt should not be empty
-// indexEdge should be(_indexEdgeOpt.get)
-// }
-//
-//
-// /** note that props have to be properly set up for equals */
-// test("test serializer/deserializer for index edge.") {
-// val ts = System.currentTimeMillis()
-// for {
-// l <- Seq(label, labelV2, labelV3, labelV4)
-// } {
-// val to = InnerVal.withLong(101, l.schemaVersion)
-// val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
-// val props = Map(LabelMeta.timestamp -> tsInnerValWithTs,
-// testLabelMeta -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion))
-//
-// check(l, ts, to, props)
-// }
-// }
-//
-// test("test serializer/deserializer for degree edge.") {
-// val ts = System.currentTimeMillis()
-// for {
-// l <- Seq(label, labelV2, labelV3, labelV4)
-// } {
-// val to = InnerVal.withStr("0", l.schemaVersion)
-// val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
-// val props = Map(
-// LabelMeta.degree -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion),
-// LabelMeta.timestamp -> tsInnerValWithTs)
-//
-// check(l, ts, to, props)
-// }
-// }
-//
-// test("test serializer/deserializer for incrementCount index edge.") {
-// val ts = System.currentTimeMillis()
-// for {
-// l <- Seq(label, labelV2, labelV3, labelV4)
-// } {
-// val to = InnerVal.withLong(101, l.schemaVersion)
-// val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
-// val props = Map(LabelMeta.timestamp -> tsInnerValWithTs,
-// testLabelMeta -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion),
-// LabelMeta.count -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion))
-//
-//
-// check(l, ts, to, props)
-// }
-// }
-//}
+/*
+ * 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.s2graph.core.storage.hbase
+
+import org.apache.s2graph.core.mysqls._
+import org.apache.s2graph.core.types._
+import org.apache.s2graph.core.TestCommonWithModels
+import org.scalatest.{FunSuite, Matchers}
+
+
+class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
+ initTests()
+
+ val testLabelMeta = LabelMeta(Option(-1), labelV2.id.get, "affinity_score", 1.toByte, "0.0", "double")
+ /**
+ * check if storage serializer/deserializer can translate from/to bytes array.
+ * @param l: label for edge.
+ * @param ts: timestamp for edge.
+ * @param to: to VertexId for edge.
+ * @param props: expected props of edge.
+ */
+ def check(l: Label, ts: Long, to: InnerValLike, props: Map[LabelMeta, InnerValLikeWithTs]): Unit = {
+ val from = InnerVal.withLong(1, l.schemaVersion)
+ val vertexId = SourceVertexId(ServiceColumn.Default, from)
+ val tgtVertexId = TargetVertexId(ServiceColumn.Default, to)
+ val vertex = graph.newVertex(vertexId, ts)
+ val tgtVertex = graph.newVertex(tgtVertexId, ts)
+ val labelWithDir = LabelWithDirection(l.id.get, 0)
+ val labelOpt = Option(l)
+ val edge = graph.newEdge(vertex, tgtVertex, l, labelWithDir.dir, 0, ts, props, tsInnerValOpt = Option(InnerVal.withLong(ts, l.schemaVersion)))
+ val indexEdge = edge.edgesWithIndex.find(_.labelIndexSeq == LabelIndex.DefaultSeq).head
+ val _indexEdgeOpt = graph.getStorage(l).indexEdgeDeserializer(l.schemaVersion).fromKeyValues(labelOpt,
+ graph.getStorage(l).indexEdgeSerializer(indexEdge).toKeyValues, l.schemaVersion, None)
+
+ _indexEdgeOpt should not be empty
+ edge == _indexEdgeOpt.get should be(true)
+ }
+
+
+ /** note that props have to be properly set up for equals */
+ test("test serializer/deserializer for index edge.") {
+ val ts = System.currentTimeMillis()
+ for {
+ l <- Seq(label, labelV2, labelV3, labelV4)
+ } {
+ val to = InnerVal.withLong(101, l.schemaVersion)
+ val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
+ val props = Map(LabelMeta.timestamp -> tsInnerValWithTs,
+ testLabelMeta -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion))
+
+ check(l, ts, to, props)
+ }
+ }
+
+ test("test serializer/deserializer for degree edge.") {
+ val ts = System.currentTimeMillis()
+ for {
+ l <- Seq(label, labelV2, labelV3, labelV4)
+ } {
+ val to = InnerVal.withStr("0", l.schemaVersion)
+ val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
+ val props = Map(
+ LabelMeta.degree -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion),
+ LabelMeta.timestamp -> tsInnerValWithTs)
+
+ check(l, ts, to, props)
+ }
+ }
+
+ test("test serializer/deserializer for incrementCount index edge.") {
+ val ts = System.currentTimeMillis()
+ for {
+ l <- Seq(label, labelV2, labelV3, labelV4)
+ } {
+ val to = InnerVal.withLong(101, l.schemaVersion)
+ val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
+ val props = Map(LabelMeta.timestamp -> tsInnerValWithTs,
+ testLabelMeta -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion),
+ LabelMeta.count -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion))
+
+
+ check(l, ts, to, props)
+ }
+ }
+}
[02/10] incubator-s2graph git commit: [S2GRAPH-131]: Add actual
implementation on interfaces from TinkerPop3 structure package. - Change
core.Edge/Vertex/Graph to core.S2Edge/S2Vertex/S2Graph. - Implement base
interfaces for tinkerpop3 structure packag
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
new file mode 100644
index 0000000..51a80f9
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
@@ -0,0 +1,1397 @@
+/*
+ * 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.s2graph.core
+
+import java.util
+import java.util.concurrent.{Executors, TimeUnit}
+
+import com.typesafe.config.{Config, ConfigFactory}
+import org.apache.commons.configuration.Configuration
+import org.apache.s2graph.core.GraphExceptions.{FetchTimeoutException, LabelNotExistException}
+import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core.mysqls._
+import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage
+import org.apache.s2graph.core.storage.{SKeyValue, Storage}
+import org.apache.s2graph.core.types._
+import org.apache.s2graph.core.utils.{DeferCache, Extensions, logger}
+import org.apache.tinkerpop.gremlin.process.computer.GraphComputer
+import org.apache.tinkerpop.gremlin.structure
+import org.apache.tinkerpop.gremlin.structure.Graph.Variables
+import org.apache.tinkerpop.gremlin.structure.util.ElementHelper
+import org.apache.tinkerpop.gremlin.structure.{Edge, Graph, T, Transaction}
+import play.api.libs.json.{JsObject, Json}
+
+import scala.annotation.tailrec
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.concurrent._
+import scala.concurrent.duration.Duration
+import scala.util.{Random, Try}
+
+
+object S2Graph {
+
+ type HashKey = (Int, Int, Int, Int, Boolean)
+ type FilterHashKey = (Int, Int)
+
+
+ val DefaultScore = 1.0
+
+ private val DefaultConfigs: Map[String, AnyRef] = Map(
+ "hbase.zookeeper.quorum" -> "localhost",
+ "hbase.table.name" -> "s2graph",
+ "hbase.table.compression.algorithm" -> "gz",
+ "phase" -> "dev",
+ "db.default.driver" -> "org.h2.Driver",
+ "db.default.url" -> "jdbc:h2:file:./var/metastore;MODE=MYSQL",
+ "db.default.password" -> "graph",
+ "db.default.user" -> "graph",
+ "cache.max.size" -> java.lang.Integer.valueOf(10000),
+ "cache.ttl.seconds" -> java.lang.Integer.valueOf(60),
+ "hbase.client.retries.number" -> java.lang.Integer.valueOf(20),
+ "hbase.rpcs.buffered_flush_interval" -> java.lang.Short.valueOf(100.toShort),
+ "hbase.rpc.timeout" -> java.lang.Integer.valueOf(1000),
+ "max.retry.number" -> java.lang.Integer.valueOf(100),
+ "lock.expire.time" -> java.lang.Integer.valueOf(1000 * 60 * 10),
+ "max.back.off" -> java.lang.Integer.valueOf(100),
+ "back.off.timeout" -> java.lang.Integer.valueOf(1000),
+ "hbase.fail.prob" -> java.lang.Double.valueOf(-0.1),
+ "delete.all.fetch.size" -> java.lang.Integer.valueOf(1000),
+ "delete.all.fetch.count" -> java.lang.Integer.valueOf(200),
+ "future.cache.max.size" -> java.lang.Integer.valueOf(100000),
+ "future.cache.expire.after.write" -> java.lang.Integer.valueOf(10000),
+ "future.cache.expire.after.access" -> java.lang.Integer.valueOf(5000),
+ "future.cache.metric.interval" -> java.lang.Integer.valueOf(60000),
+ "query.future.cache.max.size" -> java.lang.Integer.valueOf(1000),
+ "query.future.cache.expire.after.write" -> java.lang.Integer.valueOf(10000),
+ "query.future.cache.expire.after.access" -> java.lang.Integer.valueOf(5000),
+ "query.future.cache.metric.interval" -> java.lang.Integer.valueOf(60000),
+ "s2graph.storage.backend" -> "hbase",
+ "query.hardlimit" -> java.lang.Integer.valueOf(100000),
+ "hbase.zookeeper.znode.parent" -> "/hbase",
+ "query.log.sample.rate" -> Double.box(0.05)
+ )
+
+ var DefaultConfig: Config = ConfigFactory.parseMap(DefaultConfigs)
+
+
+
+ def initStorage(graph: S2Graph, config: Config)(ec: ExecutionContext): Storage[_, _] = {
+ val storageBackend = config.getString("s2graph.storage.backend")
+ logger.info(s"[InitStorage]: $storageBackend")
+
+ storageBackend match {
+ case "hbase" => new AsynchbaseStorage(graph, config)(ec)
+ case _ => throw new RuntimeException("not supported storage.")
+ }
+ }
+
+ def parseCacheConfig(config: Config, prefix: String): Config = {
+ import scala.collection.JavaConversions._
+
+ val kvs = new java.util.HashMap[String, AnyRef]()
+ for {
+ entry <- config.entrySet()
+ (k, v) = (entry.getKey, entry.getValue) if k.startsWith(prefix)
+ } yield {
+ val newKey = k.replace(prefix, "")
+ kvs.put(newKey, v.unwrapped())
+ }
+ ConfigFactory.parseMap(kvs)
+ }
+
+ /** Global helper functions */
+ @tailrec
+ final def randomInt(sampleNumber: Int, range: Int, set: Set[Int] = Set.empty[Int]): Set[Int] = {
+ if (range < sampleNumber || set.size == sampleNumber) set
+ else randomInt(sampleNumber, range, set + Random.nextInt(range))
+ }
+
+ def sample(queryRequest: QueryRequest, edges: Seq[EdgeWithScore], n: Int): Seq[EdgeWithScore] = {
+ if (edges.size <= n) {
+ edges
+ } else {
+ val plainEdges = if (queryRequest.queryParam.offset == 0) {
+ edges.tail
+ } else edges
+
+ val randoms = randomInt(n, plainEdges.size)
+ var samples = List.empty[EdgeWithScore]
+ var idx = 0
+ plainEdges.foreach { e =>
+ if (randoms.contains(idx)) samples = e :: samples
+ idx += 1
+ }
+ samples
+ }
+ }
+
+ def normalize(edgeWithScores: Seq[EdgeWithScore]): Seq[EdgeWithScore] = {
+ val sum = edgeWithScores.foldLeft(0.0) { case (acc, cur) => acc + cur.score }
+ edgeWithScores.map { edgeWithScore =>
+ edgeWithScore.copy(score = edgeWithScore.score / sum)
+ }
+ }
+
+ def alreadyVisitedVertices(edgeWithScoreLs: Seq[EdgeWithScore]): Map[(LabelWithDirection, S2Vertex), Boolean] = {
+ val vertices = for {
+ edgeWithScore <- edgeWithScoreLs
+ edge = edgeWithScore.edge
+ vertex = if (edge.labelWithDir.dir == GraphUtil.directions("out")) edge.tgtVertex else edge.srcVertex
+ } yield (edge.labelWithDir, vertex) -> true
+
+ vertices.toMap
+ }
+
+ /** common methods for filter out, transform, aggregate queryResult */
+ def convertEdges(queryParam: QueryParam, edge: S2Edge, nextStepOpt: Option[Step]): Seq[S2Edge] = {
+ for {
+ convertedEdge <- queryParam.edgeTransformer.transform(queryParam, edge, nextStepOpt) if !edge.isDegree
+ } yield convertedEdge
+ }
+
+ def processTimeDecay(queryParam: QueryParam, edge: S2Edge) = {
+ /* process time decay */
+ val tsVal = queryParam.timeDecay match {
+ case None => 1.0
+ case Some(timeDecay) =>
+ val tsVal = try {
+ val innerValWithTsOpt = edge.propertyValue(timeDecay.labelMeta.name)
+ innerValWithTsOpt.map { innerValWithTs =>
+ val innerVal = innerValWithTs.innerVal
+ timeDecay.labelMeta.dataType match {
+ case InnerVal.LONG => innerVal.value match {
+ case n: BigDecimal => n.bigDecimal.longValue()
+ case _ => innerVal.toString().toLong
+ }
+ case _ => innerVal.toString().toLong
+ }
+ } getOrElse (edge.ts)
+ } catch {
+ case e: Exception =>
+ logger.error(s"processTimeDecay error. ${edge.toLogString}", e)
+ edge.ts
+ }
+ val timeDiff = queryParam.timestamp - tsVal
+ timeDecay.decay(timeDiff)
+ }
+
+ tsVal
+ }
+
+ def processDuplicates[T](queryParam: QueryParam,
+ duplicates: Seq[(FilterHashKey, T)])(implicit ev: WithScore[T]): Seq[(FilterHashKey, T)] = {
+
+ if (queryParam.label.consistencyLevel != "strong") {
+ //TODO:
+ queryParam.duplicatePolicy match {
+ case DuplicatePolicy.First => Seq(duplicates.head)
+ case DuplicatePolicy.Raw => duplicates
+ case DuplicatePolicy.CountSum =>
+ val countSum = duplicates.size
+ val (headFilterHashKey, headEdgeWithScore) = duplicates.head
+ Seq(headFilterHashKey -> ev.withNewScore(headEdgeWithScore, countSum))
+ case _ =>
+ val scoreSum = duplicates.foldLeft(0.0) { case (prev, current) => prev + ev.score(current._2) }
+ val (headFilterHashKey, headEdgeWithScore) = duplicates.head
+ Seq(headFilterHashKey -> ev.withNewScore(headEdgeWithScore, scoreSum))
+ }
+ } else {
+ duplicates
+ }
+ }
+
+ def toHashKey(queryParam: QueryParam, edge: S2Edge, isDegree: Boolean): (HashKey, FilterHashKey) = {
+ val src = edge.srcVertex.innerId.hashCode()
+ val tgt = edge.tgtVertex.innerId.hashCode()
+ val hashKey = (src, edge.labelWithDir.labelId, edge.labelWithDir.dir, tgt, isDegree)
+ val filterHashKey = (src, tgt)
+
+ (hashKey, filterHashKey)
+ }
+
+ def filterEdges(q: Query,
+ stepIdx: Int,
+ queryRequests: Seq[QueryRequest],
+ queryResultLsFuture: Future[Seq[StepResult]],
+ queryParams: Seq[QueryParam],
+ alreadyVisited: Map[(LabelWithDirection, S2Vertex), Boolean] = Map.empty,
+ buildLastStepInnerResult: Boolean = true,
+ parentEdges: Map[VertexId, Seq[EdgeWithScore]])
+ (implicit ec: scala.concurrent.ExecutionContext): Future[StepResult] = {
+
+ queryResultLsFuture.map { queryRequestWithResultLs =>
+ val (cursors, failCount) = {
+ val _cursors = ArrayBuffer.empty[Array[Byte]]
+ var _failCount = 0
+
+ queryRequestWithResultLs.foreach { stepResult =>
+ _cursors.append(stepResult.cursors: _*)
+ _failCount += stepResult.failCount
+ }
+
+ _cursors -> _failCount
+ }
+
+
+ if (queryRequestWithResultLs.isEmpty) StepResult.Empty.copy(failCount = failCount)
+ else {
+ val isLastStep = stepIdx == q.steps.size - 1
+ val queryOption = q.queryOption
+ val step = q.steps(stepIdx)
+
+ val currentStepResults = queryRequests.view.zip(queryRequestWithResultLs)
+ val shouldBuildInnerResults = !isLastStep || buildLastStepInnerResult
+ val degrees = queryRequestWithResultLs.flatMap(_.degreeEdges)
+
+ if (shouldBuildInnerResults) {
+ val _results = buildResult(q, stepIdx, currentStepResults, parentEdges) { (edgeWithScore, propsSelectColumns) =>
+ edgeWithScore
+ }
+
+ /** process step group by */
+ val results = StepResult.filterOutStepGroupBy(_results, step.groupBy)
+ StepResult(edgeWithScores = results, grouped = Nil, degreeEdges = degrees, cursors = cursors, failCount = failCount)
+
+ } else {
+ val _results = buildResult(q, stepIdx, currentStepResults, parentEdges) { (edgeWithScore, propsSelectColumns) =>
+ val edge = edgeWithScore.edge
+ val score = edgeWithScore.score
+ val label = edgeWithScore.label
+
+ /** Select */
+ val mergedPropsWithTs = edge.propertyValuesInner(propsSelectColumns)
+
+// val newEdge = edge.copy(propsWithTs = mergedPropsWithTs)
+ val newEdge = edge.copyEdgeWithState(mergedPropsWithTs)
+
+ val newEdgeWithScore = edgeWithScore.copy(edge = newEdge)
+ /** OrderBy */
+ val orderByValues =
+ if (queryOption.orderByKeys.isEmpty) (score, edge.tsInnerVal, None, None)
+ else StepResult.toTuple4(newEdgeWithScore.toValues(queryOption.orderByKeys))
+
+ /** StepGroupBy */
+ val stepGroupByValues = newEdgeWithScore.toValues(step.groupBy.keys)
+
+ /** GroupBy */
+ val groupByValues = newEdgeWithScore.toValues(queryOption.groupBy.keys)
+
+ /** FilterOut */
+ val filterOutValues = newEdgeWithScore.toValues(queryOption.filterOutFields)
+
+ newEdgeWithScore.copy(orderByValues = orderByValues,
+ stepGroupByValues = stepGroupByValues,
+ groupByValues = groupByValues,
+ filterOutValues = filterOutValues)
+ }
+
+ /** process step group by */
+ val results = StepResult.filterOutStepGroupBy(_results, step.groupBy)
+
+ /** process ordered list */
+ val ordered = if (queryOption.groupBy.keys.isEmpty) StepResult.orderBy(queryOption, results) else Nil
+
+ /** process grouped list */
+ val grouped =
+ if (queryOption.groupBy.keys.isEmpty) Nil
+ else {
+ val agg = new scala.collection.mutable.HashMap[StepResult.GroupByKey, (Double, StepResult.Values)]()
+ results.groupBy { edgeWithScore =>
+ // edgeWithScore.groupByValues.map(_.map(_.toString))
+ edgeWithScore.groupByValues
+ }.foreach { case (k, ls) =>
+ val (scoreSum, merged) = StepResult.mergeOrdered(ls, Nil, queryOption)
+
+ val newScoreSum = scoreSum
+
+ /**
+ * watch out here. by calling toString on Any, we lose type information which will be used
+ * later for toJson.
+ */
+ if (merged.nonEmpty) {
+ val newKey = merged.head.groupByValues
+ agg += (newKey -> (newScoreSum, merged))
+ }
+ }
+ agg.toSeq.sortBy(_._2._1 * -1)
+ }
+
+ StepResult(edgeWithScores = ordered, grouped = grouped, degreeEdges = degrees, cursors = cursors, failCount = failCount)
+ }
+ }
+ }
+ }
+
+ private def toEdgeWithScores(queryRequest: QueryRequest,
+ stepResult: StepResult,
+ parentEdges: Map[VertexId, Seq[EdgeWithScore]]): Seq[EdgeWithScore] = {
+ val queryOption = queryRequest.query.queryOption
+ val queryParam = queryRequest.queryParam
+ val prevScore = queryRequest.prevStepScore
+ val labelWeight = queryRequest.labelWeight
+ val edgeWithScores = stepResult.edgeWithScores
+
+ val shouldBuildParents = queryOption.returnTree || queryParam.whereHasParent
+ val parents = if (shouldBuildParents) {
+ parentEdges.getOrElse(queryRequest.vertex.id, Nil).map { edgeWithScore =>
+ val edge = edgeWithScore.edge
+ val score = edgeWithScore.score
+ val label = edgeWithScore.label
+
+ /** Select */
+ val mergedPropsWithTs =
+ if (queryOption.selectColumns.isEmpty) {
+ edge.propertyValuesInner()
+ } else {
+ val initial = Map(LabelMeta.timestamp -> edge.propertyValueInner(LabelMeta.timestamp))
+ edge.propertyValues(queryOption.selectColumns) ++ initial
+ }
+
+ val newEdge = edge.copyEdgeWithState(mergedPropsWithTs)
+ edgeWithScore.copy(edge = newEdge)
+ }
+ } else Nil
+
+ // skip
+ if (queryOption.ignorePrevStepCache) stepResult.edgeWithScores
+ else {
+ val degreeScore = 0.0
+
+ val sampled =
+ if (queryRequest.queryParam.sample >= 0) sample(queryRequest, edgeWithScores, queryRequest.queryParam.sample)
+ else edgeWithScores
+
+ val withScores = for {
+ edgeWithScore <- sampled
+ } yield {
+ val edge = edgeWithScore.edge
+ val edgeScore = edgeWithScore.score
+ val score = queryParam.scorePropagateOp match {
+ case "plus" => edgeScore + prevScore
+ case "divide" =>
+ if ((prevScore + queryParam.scorePropagateShrinkage) == 0) 0
+ else edgeScore / (prevScore + queryParam.scorePropagateShrinkage)
+ case _ => edgeScore * prevScore
+ }
+
+ val tsVal = processTimeDecay(queryParam, edge)
+ val newScore = degreeScore + score
+ // val newEdge = if (queryOption.returnTree) edge.copy(parentEdges = parents) else edge
+ val newEdge = edge.copy(parentEdges = parents)
+ edgeWithScore.copy(edge = newEdge, score = newScore * labelWeight * tsVal)
+ }
+
+ val normalized =
+ if (queryParam.shouldNormalize) normalize(withScores)
+ else withScores
+
+ normalized
+ }
+ }
+
+ private def buildResult[T](query: Query,
+ stepIdx: Int,
+ stepResultLs: Seq[(QueryRequest, StepResult)],
+ parentEdges: Map[VertexId, Seq[EdgeWithScore]])
+ (createFunc: (EdgeWithScore, Seq[LabelMeta]) => T)
+ (implicit ev: WithScore[T]): ListBuffer[T] = {
+ import scala.collection._
+
+ val results = ListBuffer.empty[T]
+ val sequentialLs: ListBuffer[(HashKey, FilterHashKey, T, QueryParam)] = ListBuffer.empty
+ val duplicates: mutable.HashMap[HashKey, ListBuffer[(FilterHashKey, T)]] = mutable.HashMap.empty
+ val edgesToExclude: mutable.HashSet[FilterHashKey] = mutable.HashSet.empty
+ val edgesToInclude: mutable.HashSet[FilterHashKey] = mutable.HashSet.empty
+
+ var numOfDuplicates = 0
+ val queryOption = query.queryOption
+ val step = query.steps(stepIdx)
+ val excludeLabelWithDirSet = step.queryParams.filter(_.exclude).map(l => l.labelWithDir).toSet
+ val includeLabelWithDirSet = step.queryParams.filter(_.include).map(l => l.labelWithDir).toSet
+
+ stepResultLs.foreach { case (queryRequest, stepInnerResult) =>
+ val queryParam = queryRequest.queryParam
+ val label = queryParam.label
+ val shouldBeExcluded = excludeLabelWithDirSet.contains(queryParam.labelWithDir)
+ val shouldBeIncluded = includeLabelWithDirSet.contains(queryParam.labelWithDir)
+
+ val propsSelectColumns = (for {
+ column <- queryOption.propsSelectColumns
+ labelMeta <- label.metaPropsInvMap.get(column)
+ } yield labelMeta)
+
+ for {
+ edgeWithScore <- toEdgeWithScores(queryRequest, stepInnerResult, parentEdges)
+ } {
+ val edge = edgeWithScore.edge
+ val (hashKey, filterHashKey) = toHashKey(queryParam, edge, isDegree = false)
+ // params += (hashKey -> queryParam) //
+
+ /** check if this edge should be exlcuded. */
+ if (shouldBeExcluded) {
+ edgesToExclude.add(filterHashKey)
+ } else {
+ if (shouldBeIncluded) {
+ edgesToInclude.add(filterHashKey)
+ }
+ val newEdgeWithScore = createFunc(edgeWithScore, propsSelectColumns)
+
+ sequentialLs += ((hashKey, filterHashKey, newEdgeWithScore, queryParam))
+ duplicates.get(hashKey) match {
+ case None =>
+ val newLs = ListBuffer.empty[(FilterHashKey, T)]
+ newLs += (filterHashKey -> newEdgeWithScore)
+ duplicates += (hashKey -> newLs) //
+ case Some(old) =>
+ numOfDuplicates += 1
+ old += (filterHashKey -> newEdgeWithScore) //
+ }
+ }
+ }
+ }
+
+
+ if (numOfDuplicates == 0) {
+ // no duplicates at all.
+ for {
+ (hashKey, filterHashKey, edgeWithScore, _) <- sequentialLs
+ if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
+ } {
+ results += edgeWithScore
+ }
+ } else {
+ // need to resolve duplicates.
+ val seen = new mutable.HashSet[HashKey]()
+ for {
+ (hashKey, filterHashKey, edgeWithScore, queryParam) <- sequentialLs
+ if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
+ if !seen.contains(hashKey)
+ } {
+ // val queryParam = params(hashKey)
+ processDuplicates(queryParam, duplicates(hashKey)).foreach { case (_, duplicate) =>
+ if (ev.score(duplicate) >= queryParam.threshold) {
+ seen += hashKey
+ results += duplicate
+ }
+ }
+ }
+ }
+ results
+ }
+
+}
+
+class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph {
+
+ import S2Graph._
+
+ val config = _config.withFallback(S2Graph.DefaultConfig)
+
+ Model.apply(config)
+ Model.loadCache()
+
+ val MaxRetryNum = config.getInt("max.retry.number")
+ val MaxBackOff = config.getInt("max.back.off")
+ val BackoffTimeout = config.getInt("back.off.timeout")
+ val DeleteAllFetchCount = config.getInt("delete.all.fetch.count")
+ val DeleteAllFetchSize = config.getInt("delete.all.fetch.size")
+ val FailProb = config.getDouble("hbase.fail.prob")
+ val LockExpireDuration = config.getInt("lock.expire.time")
+ val MaxSize = config.getInt("future.cache.max.size")
+ val ExpireAfterWrite = config.getInt("future.cache.expire.after.write")
+ val ExpireAfterAccess = config.getInt("future.cache.expire.after.access")
+ val WaitTimeout = Duration(60, TimeUnit.SECONDS)
+ val scheduledEx = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor())
+
+ private def confWithFallback(conf: Config): Config = {
+ conf.withFallback(config)
+ }
+
+ /**
+ * TODO: we need to some way to handle malformed configuration for storage.
+ */
+ val storagePool: scala.collection.mutable.Map[String, Storage[_, _]] = {
+ val labels = Label.findAll()
+ val services = Service.findAll()
+
+ val labelConfigs = labels.flatMap(_.toStorageConfig)
+ val serviceConfigs = services.flatMap(_.toStorageConfig)
+
+ val configs = (labelConfigs ++ serviceConfigs).map { conf =>
+ confWithFallback(conf)
+ }.toSet
+
+ val pools = new java.util.HashMap[Config, Storage[_, _]]()
+ configs.foreach { config =>
+ pools.put(config, S2Graph.initStorage(this, config)(ec))
+ }
+
+ val m = new java.util.concurrent.ConcurrentHashMap[String, Storage[_, _]]()
+
+ labels.foreach { label =>
+ if (label.storageConfigOpt.isDefined) {
+ m += (s"label:${label.label}" -> pools(label.storageConfigOpt.get))
+ }
+ }
+
+ services.foreach { service =>
+ if (service.storageConfigOpt.isDefined) {
+ m += (s"service:${service.serviceName}" -> pools(service.storageConfigOpt.get))
+ }
+ }
+
+ m
+ }
+
+ val defaultStorage: Storage[_, _] = S2Graph.initStorage(this, config)(ec)
+
+ /** QueryLevel FutureCache */
+ val queryFutureCache = new DeferCache[StepResult, Promise, Future](parseCacheConfig(config, "query."), empty = StepResult.Empty)
+
+ for {
+ entry <- config.entrySet() if S2Graph.DefaultConfigs.contains(entry.getKey)
+ (k, v) = (entry.getKey, entry.getValue)
+ } logger.info(s"[Initialized]: $k, ${this.config.getAnyRef(k)}")
+
+ def getStorage(service: Service): Storage[_, _] = {
+ storagePool.getOrElse(s"service:${service.serviceName}", defaultStorage)
+ }
+
+ def getStorage(label: Label): Storage[_, _] = {
+ storagePool.getOrElse(s"label:${label.label}", defaultStorage)
+ }
+
+ def flushStorage(): Unit = {
+ storagePool.foreach { case (_, storage) =>
+
+ /** flush is blocking */
+ storage.flush()
+ }
+ }
+
+ def fallback = Future.successful(StepResult.Empty)
+
+ def checkEdges(edges: Seq[S2Edge]): Future[StepResult] = {
+ val futures = for {
+ edge <- edges
+ } yield {
+ fetchSnapshotEdge(edge).map { case (queryParam, edgeOpt, kvOpt) =>
+ edgeOpt.toSeq.map(e => EdgeWithScore(e, 1.0, queryParam.label))
+ }
+ }
+
+ Future.sequence(futures).map { edgeWithScoreLs =>
+ val edgeWithScores = edgeWithScoreLs.flatten
+ StepResult(edgeWithScores = edgeWithScores, grouped = Nil, degreeEdges = Nil)
+ }
+ }
+
+ // def checkEdges(edges: Seq[Edge]): Future[StepResult] = storage.checkEdges(edges)
+
+ def getEdges(q: Query): Future[StepResult] = {
+ Try {
+ if (q.steps.isEmpty) {
+ // TODO: this should be get vertex query.
+ fallback
+ } else {
+ val filterOutFuture = q.queryOption.filterOutQuery match {
+ case None => fallback
+ case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
+ }
+ for {
+ stepResult <- getEdgesStepInner(q)
+ filterOutInnerResult <- filterOutFuture
+ } yield {
+ if (filterOutInnerResult.isEmpty) stepResult
+ else StepResult.filterOut(this, q.queryOption, stepResult, filterOutInnerResult)
+ }
+ }
+ } recover {
+ case e: Exception =>
+ logger.error(s"getEdgesAsync: $e", e)
+ fallback
+ } get
+ }
+
+ def getEdgesStepInner(q: Query, buildLastStepInnerResult: Boolean = false): Future[StepResult] = {
+ Try {
+ if (q.steps.isEmpty) fallback
+ else {
+
+ val queryOption = q.queryOption
+ def fetch: Future[StepResult] = {
+ val startStepInnerResult = QueryResult.fromVertices(this, q)
+ q.steps.zipWithIndex.foldLeft(Future.successful(startStepInnerResult)) { case (prevStepInnerResultFuture, (step, stepIdx)) =>
+ for {
+ prevStepInnerResult <- prevStepInnerResultFuture
+ currentStepInnerResult <- fetchStep(q, stepIdx, prevStepInnerResult, buildLastStepInnerResult)
+ } yield {
+ currentStepInnerResult.copy(
+ accumulatedCursors = prevStepInnerResult.accumulatedCursors :+ currentStepInnerResult.cursors,
+ failCount = currentStepInnerResult.failCount + prevStepInnerResult.failCount
+ )
+ }
+ }
+ }
+
+ fetch
+ }
+ } recover {
+ case e: Exception =>
+ logger.error(s"getEdgesAsync: $e", e)
+ fallback
+ } get
+ }
+
+ def fetchStep(orgQuery: Query,
+ stepIdx: Int,
+ stepInnerResult: StepResult,
+ buildLastStepInnerResult: Boolean = false): Future[StepResult] = {
+ if (stepInnerResult.isEmpty) Future.successful(StepResult.Empty)
+ else {
+ val edgeWithScoreLs = stepInnerResult.edgeWithScores
+
+ val q = orgQuery
+ val queryOption = orgQuery.queryOption
+ val prevStepOpt = if (stepIdx > 0) Option(q.steps(stepIdx - 1)) else None
+ val prevStepThreshold = prevStepOpt.map(_.nextStepScoreThreshold).getOrElse(QueryParam.DefaultThreshold)
+ val prevStepLimit = prevStepOpt.map(_.nextStepLimit).getOrElse(-1)
+ val step = q.steps(stepIdx)
+
+ val alreadyVisited =
+ if (stepIdx == 0) Map.empty[(LabelWithDirection, S2Vertex), Boolean]
+ else alreadyVisitedVertices(stepInnerResult.edgeWithScores)
+
+ val initial = (Map.empty[S2Vertex, Double], Map.empty[S2Vertex, ArrayBuffer[EdgeWithScore]])
+ val (sums, grouped) = edgeWithScoreLs.foldLeft(initial) { case ((sum, group), edgeWithScore) =>
+ val key = edgeWithScore.edge.tgtVertex
+ val newScore = sum.getOrElse(key, 0.0) + edgeWithScore.score
+ val buffer = group.getOrElse(key, ArrayBuffer.empty[EdgeWithScore])
+ buffer += edgeWithScore
+ (sum + (key -> newScore), group + (key -> buffer))
+ }
+ val groupedByFiltered = sums.filter(t => t._2 >= prevStepThreshold)
+ val prevStepTgtVertexIdEdges = grouped.map(t => t._1.id -> t._2)
+
+ val nextStepSrcVertices = if (prevStepLimit >= 0) {
+ groupedByFiltered.toSeq.sortBy(-1 * _._2).take(prevStepLimit)
+ } else {
+ groupedByFiltered.toSeq
+ }
+
+ val queryRequests = for {
+ (vertex, prevStepScore) <- nextStepSrcVertices
+ queryParam <- step.queryParams
+ } yield {
+ val labelWeight = step.labelWeights.getOrElse(queryParam.labelWithDir.labelId, 1.0)
+ val newPrevStepScore = if (queryOption.shouldPropagateScore) prevStepScore else 1.0
+ QueryRequest(q, stepIdx, vertex, queryParam, newPrevStepScore, labelWeight)
+ }
+
+ val fetchedLs = fetches(queryRequests, prevStepTgtVertexIdEdges)
+
+ filterEdges(orgQuery, stepIdx, queryRequests,
+ fetchedLs, orgQuery.steps(stepIdx).queryParams, alreadyVisited, buildLastStepInnerResult, prevStepTgtVertexIdEdges)(ec)
+ }
+ }
+
+
+ /**
+ * responsible to fire parallel fetch call into storage and create future that will return merged result.
+ *
+ * @param queryRequests
+ * @param prevStepEdges
+ * @return
+ */
+ def fetches(queryRequests: Seq[QueryRequest],
+ prevStepEdges: Map[VertexId, Seq[EdgeWithScore]]): Future[Seq[StepResult]] = {
+
+ val reqWithIdxs = queryRequests.zipWithIndex
+ val requestsPerLabel = reqWithIdxs.groupBy(t => t._1.queryParam.label)
+ val aggFuture = requestsPerLabel.foldLeft(Future.successful(Map.empty[Int, StepResult])) { case (prevFuture, (label, reqWithIdxs)) =>
+ for {
+ prev <- prevFuture
+ cur <- getStorage(label).fetches(reqWithIdxs.map(_._1), prevStepEdges)
+ } yield {
+ prev ++ reqWithIdxs.map(_._2).zip(cur).toMap
+ }
+ }
+ aggFuture.map { agg => agg.toSeq.sortBy(_._1).map(_._2) }
+ }
+
+
+ def getEdgesMultiQuery(mq: MultiQuery): Future[StepResult] = {
+ Try {
+ if (mq.queries.isEmpty) fallback
+ else {
+ val filterOutFuture = mq.queryOption.filterOutQuery match {
+ case None => fallback
+ case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
+ }
+
+ val multiQueryFutures = Future.sequence(mq.queries.map { query => getEdges(query) })
+ for {
+ multiQueryResults <- multiQueryFutures
+ filterOutInnerResult <- filterOutFuture
+ } yield {
+ StepResult.merges(mq.queryOption, multiQueryResults, mq.weights, filterOutInnerResult)
+ }
+ }
+ } recover {
+ case e: Exception =>
+ logger.error(s"getEdgesAsync: $e", e)
+ fallback
+ } get
+ }
+
+
+ def fetchSnapshotEdge(edge: S2Edge): Future[(QueryParam, Option[S2Edge], Option[SKeyValue])] = {
+ /** TODO: Fix this. currently fetchSnapshotEdge should not use future cache
+ * so use empty cacheKey.
+ * */
+ val queryParam = QueryParam(labelName = edge.innerLabel.label,
+ direction = GraphUtil.fromDirection(edge.labelWithDir.dir),
+ tgtVertexIdOpt = Option(edge.tgtVertex.innerIdVal),
+ cacheTTLInMillis = -1)
+ val q = Query.toQuery(Seq(edge.srcVertex), Seq(queryParam))
+ val queryRequest = QueryRequest(q, 0, edge.srcVertex, queryParam)
+
+ val storage = getStorage(edge.innerLabel)
+ storage.fetchSnapshotEdgeKeyValues(queryRequest).map { kvs =>
+ val (edgeOpt, kvOpt) =
+ if (kvs.isEmpty) (None, None)
+ else {
+ val snapshotEdgeOpt = storage.toSnapshotEdge(kvs.head, queryRequest, isInnerCall = true, parentEdges = Nil)
+ val _kvOpt = kvs.headOption
+ (snapshotEdgeOpt, _kvOpt)
+ }
+ (queryParam, edgeOpt, kvOpt)
+ } recoverWith { case ex: Throwable =>
+ logger.error(s"fetchQueryParam failed. fallback return.", ex)
+ throw FetchTimeoutException(s"${edge.toLogString}")
+ }
+ }
+
+ def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
+ val verticesWithIdx = vertices.zipWithIndex
+ val futures = verticesWithIdx.groupBy { case (v, idx) => v.service }.map { case (service, vertexGroup) =>
+ getStorage(service).getVertices(vertexGroup.map(_._1)).map(_.zip(vertexGroup.map(_._2)))
+ }
+
+ Future.sequence(futures).map { ls =>
+ ls.flatten.toSeq.sortBy(_._2).map(_._1)
+ }
+ }
+
+ /** mutate */
+ def deleteAllAdjacentEdges(srcVertices: Seq[S2Vertex],
+ labels: Seq[Label],
+ dir: Int,
+ ts: Long): Future[Boolean] = {
+
+ val requestTs = ts
+ val vertices = srcVertices
+ /** create query per label */
+ val queries = for {
+ label <- labels
+ } yield {
+ val queryParam = QueryParam(labelName = label.label, direction = GraphUtil.fromDirection(dir),
+ offset = 0, limit = DeleteAllFetchSize, duplicatePolicy = DuplicatePolicy.Raw)
+ val step = Step(List(queryParam))
+ Query(vertices, Vector(step))
+ }
+
+ // Extensions.retryOnSuccessWithBackoff(MaxRetryNum, Random.nextInt(MaxBackOff) + 1) {
+ val retryFuture = Extensions.retryOnSuccess(DeleteAllFetchCount) {
+ fetchAndDeleteAll(queries, requestTs)
+ } { case (allDeleted, deleteSuccess) =>
+ allDeleted && deleteSuccess
+ }.map { case (allDeleted, deleteSuccess) => allDeleted && deleteSuccess }
+
+ retryFuture onFailure {
+ case ex =>
+ logger.error(s"[Error]: deleteAllAdjacentEdges failed.")
+ }
+
+ retryFuture
+ }
+
+ def fetchAndDeleteAll(queries: Seq[Query], requestTs: Long): Future[(Boolean, Boolean)] = {
+ val future = for {
+ stepInnerResultLs <- Future.sequence(queries.map(getEdgesStepInner(_, true)))
+ (allDeleted, ret) <- deleteAllFetchedEdgesLs(stepInnerResultLs, requestTs)
+ } yield {
+ // logger.debug(s"fetchAndDeleteAll: ${allDeleted}, ${ret}")
+ (allDeleted, ret)
+ }
+
+ Extensions.retryOnFailure(MaxRetryNum) {
+ future
+ } {
+ logger.error(s"fetch and deleteAll failed.")
+ (true, false)
+ }
+
+ }
+
+ def deleteAllFetchedEdgesLs(stepInnerResultLs: Seq[StepResult],
+ requestTs: Long): Future[(Boolean, Boolean)] = {
+ stepInnerResultLs.foreach { stepInnerResult =>
+ if (stepInnerResult.isFailure) throw new RuntimeException("fetched result is fallback.")
+ }
+ val futures = for {
+ stepInnerResult <- stepInnerResultLs
+ deleteStepInnerResult = buildEdgesToDelete(stepInnerResult, requestTs)
+ if deleteStepInnerResult.edgeWithScores.nonEmpty
+ } yield {
+ val head = deleteStepInnerResult.edgeWithScores.head
+ val label = head.edge.innerLabel
+ val ret = label.schemaVersion match {
+ case HBaseType.VERSION3 | HBaseType.VERSION4 =>
+ if (label.consistencyLevel == "strong") {
+ /**
+ * read: snapshotEdge on queryResult = O(N)
+ * write: N x (relatedEdges x indices(indexedEdge) + 1(snapshotEdge))
+ */
+ mutateEdges(deleteStepInnerResult.edgeWithScores.map(_.edge), withWait = true).map(_.forall(identity))
+ } else {
+ getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
+ }
+ case _ =>
+
+ /**
+ * read: x
+ * write: N x ((1(snapshotEdge) + 2(1 for incr, 1 for delete) x indices)
+ */
+ getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
+ }
+ ret
+ }
+
+ if (futures.isEmpty) {
+ // all deleted.
+ Future.successful(true -> true)
+ } else {
+ Future.sequence(futures).map { rets => false -> rets.forall(identity) }
+ }
+ }
+
+ def buildEdgesToDelete(stepInnerResult: StepResult, requestTs: Long): StepResult = {
+ val filtered = stepInnerResult.edgeWithScores.filter { edgeWithScore =>
+ (edgeWithScore.edge.ts < requestTs) && !edgeWithScore.edge.isDegree
+ }
+ if (filtered.isEmpty) StepResult.Empty
+ else {
+ val head = filtered.head
+ val label = head.edge.innerLabel
+ val edgeWithScoreLs = filtered.map { edgeWithScore =>
+ val edge = edgeWithScore.edge
+ val copiedEdge = label.consistencyLevel match {
+ case "strong" =>
+ edge.copyEdge(op = GraphUtil.operations("delete"),
+ version = requestTs, propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()), ts = requestTs)
+ case _ =>
+ edge.copyEdge(propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()), ts = requestTs)
+ }
+// val (newOp, newVersion, newPropsWithTs) = label.consistencyLevel match {
+// case "strong" =>
+// val edge = edgeWithScore.edge
+// edge.property(LabelMeta.timestamp.name, requestTs)
+// val _newPropsWithTs = edge.updatePropsWithTs()
+//
+// (GraphUtil.operations("delete"), requestTs, _newPropsWithTs)
+// case _ =>
+// val oldEdge = edgeWithScore.edge
+// (oldEdge.op, oldEdge.version, oldEdge.updatePropsWithTs())
+// }
+//
+// val copiedEdge =
+// edgeWithScore.edge.copy(op = newOp, version = newVersion, propsWithTs = newPropsWithTs)
+
+ val edgeToDelete = edgeWithScore.copy(edge = copiedEdge)
+ // logger.debug(s"delete edge from deleteAll: ${edgeToDelete.edge.toLogString}")
+ edgeToDelete
+ }
+ //Degree edge?
+ StepResult(edgeWithScores = edgeWithScoreLs, grouped = Nil, degreeEdges = Nil, false)
+ }
+ }
+
+ // def deleteAllAdjacentEdges(srcVertices: List[Vertex], labels: Seq[Label], dir: Int, ts: Long): Future[Boolean] =
+ // storage.deleteAllAdjacentEdges(srcVertices, labels, dir, ts)
+
+ def mutateElements(elements: Seq[GraphElement],
+ withWait: Boolean = false): Future[Seq[Boolean]] = {
+
+ val edgeBuffer = ArrayBuffer[(S2Edge, Int)]()
+ val vertexBuffer = ArrayBuffer[(S2Vertex, Int)]()
+
+ elements.zipWithIndex.foreach {
+ case (e: S2Edge, idx: Int) => edgeBuffer.append((e, idx))
+ case (v: S2Vertex, idx: Int) => vertexBuffer.append((v, idx))
+ case any@_ => logger.error(s"Unknown type: ${any}")
+ }
+
+ val edgeFutureWithIdx = mutateEdges(edgeBuffer.map(_._1), withWait).map { result =>
+ edgeBuffer.map(_._2).zip(result)
+ }
+
+ val vertexFutureWithIdx = mutateVertices(vertexBuffer.map(_._1), withWait).map { result =>
+ vertexBuffer.map(_._2).zip(result)
+ }
+
+ val graphFuture = for {
+ edgesMutated <- edgeFutureWithIdx
+ verticesMutated <- vertexFutureWithIdx
+ } yield (edgesMutated ++ verticesMutated).sortBy(_._1).map(_._2)
+
+ graphFuture
+
+ }
+
+ // def mutateEdges(edges: Seq[Edge], withWait: Boolean = false): Future[Seq[Boolean]] = storage.mutateEdges(edges, withWait)
+
+ def mutateEdges(edges: Seq[S2Edge], withWait: Boolean = false): Future[Seq[Boolean]] = {
+ val edgeWithIdxs = edges.zipWithIndex
+
+ val (strongEdges, weakEdges) =
+ edgeWithIdxs.partition { case (edge, idx) =>
+ val e = edge
+ e.innerLabel.consistencyLevel == "strong" && e.op != GraphUtil.operations("insertBulk")
+ }
+
+ val weakEdgesFutures = weakEdges.groupBy { case (edge, idx) => edge.innerLabel.hbaseZkAddr }.map { case (zkQuorum, edgeWithIdxs) =>
+ val futures = edgeWithIdxs.groupBy(_._1.innerLabel).map { case (label, edgeGroup) =>
+ val storage = getStorage(label)
+ val edges = edgeGroup.map(_._1)
+ val idxs = edgeGroup.map(_._2)
+
+ /** multiple edges with weak consistency level will be processed as batch */
+ val mutations = edges.flatMap { edge =>
+ val (_, edgeUpdate) =
+ if (edge.op == GraphUtil.operations("delete")) S2Edge.buildDeleteBulk(None, edge)
+ else S2Edge.buildOperation(None, Seq(edge))
+
+ storage.buildVertexPutsAsync(edge) ++ storage.indexedEdgeMutations(edgeUpdate) ++ storage.snapshotEdgeMutations(edgeUpdate) ++ storage.increments(edgeUpdate)
+ }
+
+ storage.writeToStorage(zkQuorum, mutations, withWait).map { ret =>
+ idxs.map(idx => idx -> ret)
+ }
+ }
+ Future.sequence(futures)
+ }
+ val (strongDeleteAll, strongEdgesAll) = strongEdges.partition { case (edge, idx) => edge.op == GraphUtil.operations("deleteAll") }
+
+ val deleteAllFutures = strongDeleteAll.map { case (edge, idx) =>
+ deleteAllAdjacentEdges(Seq(edge.srcVertex), Seq(edge.innerLabel), edge.labelWithDir.dir, edge.ts).map(idx -> _)
+ }
+
+ val strongEdgesFutures = strongEdgesAll.groupBy { case (edge, idx) => edge.innerLabel }.map { case (label, edgeGroup) =>
+ val edges = edgeGroup.map(_._1)
+ val idxs = edgeGroup.map(_._2)
+ val storage = getStorage(label)
+ storage.mutateStrongEdges(edges, withWait = true).map { rets =>
+ idxs.zip(rets)
+ }
+ }
+
+ for {
+ weak <- Future.sequence(weakEdgesFutures)
+ deleteAll <- Future.sequence(deleteAllFutures)
+ strong <- Future.sequence(strongEdgesFutures)
+ } yield {
+ (deleteAll ++ weak.flatten.flatten ++ strong.flatten).sortBy(_._1).map(_._2)
+ }
+ }
+
+ def mutateVertices(vertices: Seq[S2Vertex], withWait: Boolean = false): Future[Seq[Boolean]] = {
+ val verticesWithIdx = vertices.zipWithIndex
+ val futures = verticesWithIdx.groupBy { case (v, idx) => v.service }.map { case (service, vertexGroup) =>
+ getStorage(service).mutateVertices(vertexGroup.map(_._1), withWait).map(_.zip(vertexGroup.map(_._2)))
+ }
+ Future.sequence(futures).map { ls => ls.flatten.toSeq.sortBy(_._2).map(_._1) }
+ }
+
+ def incrementCounts(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]] = {
+ val edgesWithIdx = edges.zipWithIndex
+ val futures = edgesWithIdx.groupBy { case (e, idx) => e.innerLabel }.map { case (label, edgeGroup) =>
+ getStorage(label).incrementCounts(edgeGroup.map(_._1), withWait).map(_.zip(edgeGroup.map(_._2)))
+ }
+ Future.sequence(futures).map { ls => ls.flatten.toSeq.sortBy(_._2).map(_._1) }
+ }
+
+ def updateDegree(edge: S2Edge, degreeVal: Long = 0): Future[Boolean] = {
+ val label = edge.innerLabel
+
+ val storage = getStorage(label)
+ val kvs = storage.buildDegreePuts(edge, degreeVal)
+
+ storage.writeToStorage(edge.innerLabel.service.cluster, kvs, withWait = true)
+ }
+
+ def shutdown(): Unit = {
+ flushStorage()
+ Model.shutdown()
+ }
+
+ def toGraphElement(s: String, labelMapping: Map[String, String] = Map.empty): Option[GraphElement] = Try {
+ val parts = GraphUtil.split(s)
+ val logType = parts(2)
+ val element = if (logType == "edge" | logType == "e") {
+ /** current only edge is considered to be bulk loaded */
+ labelMapping.get(parts(5)) match {
+ case None =>
+ case Some(toReplace) =>
+ parts(5) = toReplace
+ }
+ toEdge(parts)
+ } else if (logType == "vertex" | logType == "v") {
+ toVertex(parts)
+ } else {
+ throw new GraphExceptions.JsonParseException("log type is not exist in log.")
+ }
+
+ element
+ } recover {
+ case e: Exception =>
+ logger.error(s"[toElement]: $e", e)
+ None
+ } get
+
+
+ def toVertex(s: String): Option[S2Vertex] = {
+ toVertex(GraphUtil.split(s))
+ }
+
+ def toEdge(s: String): Option[S2Edge] = {
+ toEdge(GraphUtil.split(s))
+ }
+
+ def toEdge(parts: Array[String]): Option[S2Edge] = Try {
+ val (ts, operation, logType, srcId, tgtId, label) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
+ val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
+ val tempDirection = if (parts.length >= 8) parts(7) else "out"
+ val direction = if (tempDirection != "out" && tempDirection != "in") "out" else tempDirection
+ val edge = toEdge(srcId, tgtId, label, direction, props, ts.toLong, operation)
+ Option(edge)
+ } recover {
+ case e: Exception =>
+ logger.error(s"[toEdge]: $e", e)
+ throw e
+ } get
+
+ def toVertex(parts: Array[String]): Option[S2Vertex] = Try {
+ val (ts, operation, logType, srcId, serviceName, colName) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
+ val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
+ val vertex = toVertex(serviceName, colName, srcId, props, ts.toLong, operation)
+ Option(vertex)
+ } recover {
+ case e: Throwable =>
+ logger.error(s"[toVertex]: $e", e)
+ throw e
+ } get
+
+ def toEdge(srcId: Any,
+ tgtId: Any,
+ labelName: String,
+ direction: String,
+ props: Map[String, Any] = Map.empty,
+ ts: Long = System.currentTimeMillis(),
+ operation: String = "insert"): S2Edge = {
+ val label = Label.findByName(labelName).getOrElse(throw new LabelNotExistException(labelName))
+
+ val srcVertexIdInnerVal = toInnerVal(srcId.toString, label.srcColumn.columnType, label.schemaVersion)
+ val tgtVertexIdInnerVal = toInnerVal(tgtId.toString, label.tgtColumn.columnType, label.schemaVersion)
+
+ val srcVertex = newVertex(SourceVertexId(label.srcColumn, srcVertexIdInnerVal), System.currentTimeMillis())
+ val tgtVertex = newVertex(TargetVertexId(label.tgtColumn, tgtVertexIdInnerVal), System.currentTimeMillis())
+ val dir = GraphUtil.toDir(direction).getOrElse(throw new RuntimeException(s"$direction is not supported."))
+
+ val propsPlusTs = props ++ Map(LabelMeta.timestamp.name -> ts)
+ val propsWithTs = label.propsToInnerValsWithTs(propsPlusTs, ts)
+ val op = GraphUtil.toOp(operation).getOrElse(throw new RuntimeException(s"$operation is not supported."))
+
+ new S2Edge(this, srcVertex, tgtVertex, label, dir, op = op, version = ts).copyEdgeWithState(propsWithTs)
+ }
+
+ def toVertex(serviceName: String,
+ columnName: String,
+ id: Any,
+ props: Map[String, Any] = Map.empty,
+ ts: Long = System.currentTimeMillis(),
+ operation: String = "insert"): S2Vertex = {
+
+ val service = Service.findByName(serviceName).getOrElse(throw new RuntimeException(s"$serviceName is not found."))
+ val column = ServiceColumn.find(service.id.get, columnName).getOrElse(throw new RuntimeException(s"$columnName is not found."))
+ val op = GraphUtil.toOp(operation).getOrElse(throw new RuntimeException(s"$operation is not supported."))
+
+ val srcVertexId = VertexId(column, toInnerVal(id.toString, column.columnType, column.schemaVersion))
+ val propsInner = column.propsToInnerVals(props) ++
+ Map(ColumnMeta.timestamp -> InnerVal.withLong(ts, column.schemaVersion))
+
+ val vertex = new S2Vertex(this, srcVertexId, ts, S2Vertex.EmptyProps, op)
+ S2Vertex.fillPropsWithTs(vertex, propsInner)
+ vertex
+ }
+
+ /**
+ * helper to create new Edge instance from given parameters on memory(not actually stored in storage).
+ *
+ * Since we are using mutable map for property value(propsWithTs),
+ * we should make sure that reference for mutable map never be shared between multiple Edge instances.
+ * To guarantee this, we never create Edge directly, but rather use this helper which is available on S2Graph.
+ *
+ * Note that we are using following convention
+ * 1. `add*` for method that actually store instance into storage,
+ * 2. `new*` for method that only create instance on memory, but not store it into storage.
+ *
+ * @param srcVertex
+ * @param tgtVertex
+ * @param innerLabel
+ * @param dir
+ * @param op
+ * @param version
+ * @param propsWithTs
+ * @param parentEdges
+ * @param originalEdgeOpt
+ * @param pendingEdgeOpt
+ * @param statusCode
+ * @param lockTs
+ * @param tsInnerValOpt
+ * @return
+ */
+ def newEdge(srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ innerLabel: Label,
+ dir: Int,
+ op: Byte = GraphUtil.defaultOpByte,
+ version: Long = System.currentTimeMillis(),
+ propsWithTs: S2Edge.State,
+ parentEdges: Seq[EdgeWithScore] = Nil,
+ originalEdgeOpt: Option[S2Edge] = None,
+ pendingEdgeOpt: Option[S2Edge] = None,
+ statusCode: Byte = 0,
+ lockTs: Option[Long] = None,
+ tsInnerValOpt: Option[InnerValLike] = None): S2Edge = {
+ val edge = new S2Edge(this, srcVertex, tgtVertex, innerLabel, dir, op, version, S2Edge.EmptyProps,
+ parentEdges, originalEdgeOpt, pendingEdgeOpt, statusCode, lockTs, tsInnerValOpt)
+ S2Edge.fillPropsWithTs(edge, propsWithTs)
+ edge
+ }
+
+ /**
+ * helper to create new SnapshotEdge instance from given parameters on memory(not actually stored in storage).
+ *
+ * Note that this is only available to S2Graph, not structure.Graph so only internal code should use this method.
+ * @param srcVertex
+ * @param tgtVertex
+ * @param label
+ * @param dir
+ * @param op
+ * @param version
+ * @param propsWithTs
+ * @param pendingEdgeOpt
+ * @param statusCode
+ * @param lockTs
+ * @param tsInnerValOpt
+ * @return
+ */
+ private[core] def newSnapshotEdge(srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ label: Label,
+ dir: Int,
+ op: Byte,
+ version: Long,
+ propsWithTs: S2Edge.State,
+ pendingEdgeOpt: Option[S2Edge],
+ statusCode: Byte = 0,
+ lockTs: Option[Long],
+ tsInnerValOpt: Option[InnerValLike] = None): SnapshotEdge = {
+ val snapshotEdge = new SnapshotEdge(this, srcVertex, tgtVertex, label, dir, op, version, S2Edge.EmptyProps,
+ pendingEdgeOpt, statusCode, lockTs, tsInnerValOpt)
+ S2Edge.fillPropsWithTs(snapshotEdge, propsWithTs)
+ snapshotEdge
+ }
+
+ /**
+ * internal helper to actually store a single edge based on given peramters.
+ *
+ * Note that this is used from S2Vertex to implement blocking interface from Tp3.
+ * Once tp3 provide AsyncStep, then this can be changed to return Java's CompletableFuture.
+ *
+ * @param srcVertex
+ * @param tgtVertex
+ * @param labelName
+ * @param direction
+ * @param props
+ * @param ts
+ * @param operation
+ * @return
+ */
+ private[core] def addEdgeInner(srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ labelName: String,
+ direction: String = "out",
+ props: Map[String, AnyRef] = Map.empty,
+ ts: Long = System.currentTimeMillis(),
+ operation: String = "insert"): S2Edge = {
+ Await.result(addEdgeInnerAsync(srcVertex, tgtVertex, labelName, direction, props, ts, operation), WaitTimeout)
+ }
+
+ private[core] def addEdgeInnerAsync(srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ labelName: String,
+ direction: String = "out",
+ props: Map[String, AnyRef] = Map.empty,
+ ts: Long = System.currentTimeMillis(),
+ operation: String = "insert"): Future[S2Edge] = {
+ // Validations on input parameter
+ val label = Label.findByName(labelName).getOrElse(throw new LabelNotExistException(labelName))
+ val dir = GraphUtil.toDir(direction).getOrElse(throw new RuntimeException(s"$direction is not supported."))
+// if (srcVertex.id.column != label.srcColumnWithDir(dir)) throw new RuntimeException(s"srcVertex's column[${srcVertex.id.column}] is not matched to label's srcColumn[${label.srcColumnWithDir(dir)}")
+// if (tgtVertex.id.column != label.tgtColumnWithDir(dir)) throw new RuntimeException(s"tgtVertex's column[${tgtVertex.id.column}] is not matched to label's tgtColumn[${label.tgtColumnWithDir(dir)}")
+
+ // Convert given Map[String, AnyRef] property into internal class.
+ val propsPlusTs = props ++ Map(LabelMeta.timestamp.name -> ts)
+ val propsWithTs = label.propsToInnerValsWithTs(propsPlusTs, ts)
+ val op = GraphUtil.toOp(operation).getOrElse(throw new RuntimeException(s"$operation is not supported."))
+
+ val edge = newEdge(srcVertex, tgtVertex, label, dir, op = op, version = ts, propsWithTs = propsWithTs)
+ // store edge into storage withWait option.
+ mutateEdges(Seq(edge), withWait = true).map { rets =>
+ if (!rets.headOption.getOrElse(false)) throw new RuntimeException("add edge failed.")
+ else edge
+ }
+ }
+
+
+ def newVertexId(serviceName: String)(columnName: String)(id: Any): VertexId = {
+ val service = Service.findByName(serviceName).getOrElse(throw new RuntimeException(s"$serviceName is not found."))
+ val column = ServiceColumn.find(service.id.get, columnName).getOrElse(throw new RuntimeException(s"$columnName is not found."))
+ newVertexId(service, column, id)
+ }
+
+ /**
+ * helper to create S2Graph's internal VertexId instance with given parameters.
+ * @param service
+ * @param column
+ * @param id
+ * @return
+ */
+ def newVertexId(service: Service,
+ column: ServiceColumn,
+ id: Any): VertexId = {
+ val innerVal = CanInnerValLike.anyToInnerValLike.toInnerVal(id)(column.schemaVersion)
+ new VertexId(column, innerVal)
+ }
+
+ def newVertex(id: VertexId,
+ ts: Long = System.currentTimeMillis(),
+ props: S2Vertex.Props = S2Vertex.EmptyProps,
+ op: Byte = 0,
+ belongLabelIds: Seq[Int] = Seq.empty): S2Vertex = {
+ val vertex = new S2Vertex(this, id, ts, S2Vertex.EmptyProps, op, belongLabelIds)
+ S2Vertex.fillPropsWithTs(vertex, props)
+ vertex
+ }
+
+ def getVertex(vertexId: VertexId): Option[S2Vertex] = {
+ val v = newVertex(vertexId)
+ Await.result(getVertices(Seq(v)).map { vertices => vertices.headOption }, WaitTimeout)
+ }
+
+ def fetchEdges(vertex: S2Vertex, labelNames: Seq[String], direction: String = "out"): util.Iterator[Edge] = {
+ Await.result(fetchEdgesAsync(vertex, labelNames, direction), WaitTimeout)
+ }
+
+ def fetchEdgesAsync(vertex: S2Vertex, labelNames: Seq[String], direction: String = "out"): Future[util.Iterator[Edge]] = {
+ val queryParams = labelNames.map { l =>
+ QueryParam(labelName = l, direction = direction)
+ }
+ val query = Query.toQuery(Seq(vertex), queryParams)
+ getEdges(query).map { stepResult =>
+ val ls = new util.ArrayList[Edge]()
+ stepResult.edgeWithScores.foreach(es => ls.add(es.edge))
+ ls.iterator()
+ }
+ }
+
+ override def vertices(vertexIds: AnyRef*): util.Iterator[structure.Vertex] = {
+ val vertices = for {
+ vertexId <- vertexIds if vertexId.isInstanceOf[VertexId]
+ } yield newVertex(vertexId.asInstanceOf[VertexId])
+
+ val future = getVertices(vertices).map { vs =>
+ val ls = new util.ArrayList[structure.Vertex]()
+ ls.addAll(vs)
+ ls.iterator()
+ }
+ Await.result(future, WaitTimeout)
+ }
+
+ override def tx(): Transaction = ???
+
+ override def edges(objects: AnyRef*): util.Iterator[structure.Edge] = ???
+
+ override def variables(): Variables = ???
+
+ override def configuration(): Configuration = ???
+
+ override def addVertex(kvs: AnyRef*): structure.Vertex = {
+ val kvsMap = ElementHelper.asMap(kvs: _*).asScala.toMap
+ val id = kvsMap.getOrElse(T.id.toString, throw new RuntimeException("T.id is required."))
+ val serviceColumnNames = kvsMap.getOrElse(T.label.toString, throw new RuntimeException("ServiceName::ColumnName is required.")).toString
+ val names = serviceColumnNames.split(S2Vertex.VertexLabelDelimiter)
+ if (names.length != 2) throw new RuntimeException("malformed data on vertex label.")
+ val serviceName = names(0)
+ val columnName = names(1)
+
+ val vertex = toVertex(serviceName, columnName, id, kvsMap)
+ val future = mutateVertices(Seq(vertex), withWait = true).map { vs =>
+ if (vs.forall(identity)) vertex
+ else throw new RuntimeException("addVertex failed.")
+ }
+ Await.result(future, WaitTimeout)
+ }
+
+ def addVertex(id: VertexId,
+ ts: Long = System.currentTimeMillis(),
+ props: S2Vertex.Props = S2Vertex.EmptyProps,
+ op: Byte = 0,
+ belongLabelIds: Seq[Int] = Seq.empty): S2Vertex = {
+ val vertex = newVertex(id, ts, props, op, belongLabelIds)
+ val future = mutateVertices(Seq(vertex), withWait = true).map { rets =>
+ if (rets.forall(identity)) vertex
+ else throw new RuntimeException("addVertex failed.")
+ }
+ Await.result(future, WaitTimeout)
+ }
+
+ override def close(): Unit = {
+ shutdown()
+ }
+
+ override def compute[C <: GraphComputer](aClass: Class[C]): C = ???
+
+ override def compute(): GraphComputer = ???
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/S2Property.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Property.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Property.scala
index 46f5ecf..6a47e46 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Property.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Property.scala
@@ -27,16 +27,18 @@ import org.apache.tinkerpop.gremlin.structure.{Property}
import scala.util.hashing.MurmurHash3
-case class S2Property[V](element: Edge,
+case class S2Property[V](element: S2Edge,
labelMeta: LabelMeta,
key: String,
- value: V,
+ v: V,
ts: Long) extends Property[V] {
import CanInnerValLike._
lazy val innerVal = anyToInnerValLike.toInnerVal(value)(element.innerLabel.schemaVersion)
lazy val innerValWithTs = InnerValLikeWithTs(innerVal, ts)
+ val value = castValue(v, labelMeta.dataType).asInstanceOf[V]
+
def bytes: Array[Byte] = {
innerVal.bytes
}
@@ -64,4 +66,5 @@ case class S2Property[V](element: Edge,
override def toString(): String = {
Map("labelMeta" -> labelMeta.toString, "key" -> key, "value" -> value, "ts" -> ts).toString
}
+
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/S2Vertex.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Vertex.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Vertex.scala
new file mode 100644
index 0000000..7fd2ac4
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Vertex.scala
@@ -0,0 +1,221 @@
+/*
+ * 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.s2graph.core
+
+import java.util
+import java.util.function.{Consumer, BiConsumer}
+
+import org.apache.s2graph.core.S2Vertex.Props
+import org.apache.s2graph.core.mysqls.{ColumnMeta, LabelMeta, Service, ServiceColumn}
+import org.apache.s2graph.core.types._
+import org.apache.tinkerpop.gremlin.structure.VertexProperty.Cardinality
+import org.apache.tinkerpop.gremlin.structure.util.ElementHelper
+import org.apache.tinkerpop.gremlin.structure.{Direction, Vertex, Edge, VertexProperty}
+import play.api.libs.json.Json
+import scala.collection.JavaConverters._
+
+case class S2Vertex(graph: S2Graph,
+ id: VertexId,
+ ts: Long = System.currentTimeMillis(),
+ props: Props = S2Vertex.EmptyProps,
+ op: Byte = 0,
+ belongLabelIds: Seq[Int] = Seq.empty) extends GraphElement with Vertex {
+
+ val innerId = id.innerId
+
+ val innerIdVal = innerId.value
+
+ lazy val properties = for {
+ (k, v) <- props.asScala
+ } yield v.columnMeta.name -> v.value
+
+ def schemaVer = serviceColumn.schemaVersion
+
+ def serviceColumn = ServiceColumn.findById(id.colId)
+
+ def columnName = serviceColumn.columnName
+
+ lazy val service = Service.findById(serviceColumn.serviceId)
+
+ lazy val (hbaseZkAddr, hbaseTableName) = (service.cluster, service.hTableName)
+
+ def defaultProps = {
+ val default = S2Vertex.EmptyProps
+ val newProps = new S2VertexProperty(this, ColumnMeta.lastModifiedAtColumn, ColumnMeta.lastModifiedAtColumn.name, ts)
+ default.put(ColumnMeta.lastModifiedAtColumn.name, newProps)
+ default
+ }
+
+ // lazy val kvs = Graph.client.vertexSerializer(this).toKeyValues
+
+ /** TODO: make this as configurable */
+ override def serviceName = service.serviceName
+
+ override def isAsync = false
+
+ override def queueKey = Seq(ts.toString, serviceName).mkString("|")
+
+ override def queuePartitionKey = id.innerId.toString
+
+ def propsWithName = for {
+ (k, v) <- props.asScala
+ } yield (v.columnMeta.name -> v.value.toString)
+
+ override def hashCode() = {
+ val hash = id.hashCode()
+ // logger.debug(s"Vertex.hashCode: $this -> $hash")
+ hash
+ }
+
+ override def equals(obj: Any) = {
+ obj match {
+ case otherVertex: S2Vertex =>
+ val ret = id == otherVertex.id
+ // logger.debug(s"Vertex.equals: $this, $obj => $ret")
+ ret
+ case _ => false
+ }
+ }
+
+ override def toString(): String = {
+ Map("id" -> id.toString(), "ts" -> ts, "props" -> "", "op" -> op, "belongLabelIds" -> belongLabelIds).toString()
+ }
+
+ def toLogString(): String = {
+ val (serviceName, columnName) =
+ if (!id.storeColId) ("", "")
+ else (serviceColumn.service.serviceName, serviceColumn.columnName)
+
+ if (propsWithName.nonEmpty)
+ Seq(ts, GraphUtil.fromOp(op), "v", id.innerId, serviceName, columnName, Json.toJson(propsWithName)).mkString("\t")
+ else
+ Seq(ts, GraphUtil.fromOp(op), "v", id.innerId, serviceName, columnName).mkString("\t")
+ }
+
+ def copyVertexWithState(props: Props): S2Vertex = {
+ val newVertex = copy(props = S2Vertex.EmptyProps)
+ S2Vertex.fillPropsWithTs(newVertex, props)
+ newVertex
+ }
+
+ override def vertices(direction: Direction, edgeLabels: String*): util.Iterator[Vertex] = {
+ val arr = new util.ArrayList[Vertex]()
+ edges(direction, edgeLabels: _*).forEachRemaining(new Consumer[Edge] {
+ override def accept(edge: Edge): Unit = {
+ direction match {
+ case Direction.OUT => arr.add(edge.inVertex())
+ case Direction.IN => arr.add(edge.outVertex())
+ case _ =>
+ arr.add(edge.inVertex())
+ arr.add(edge.outVertex())
+ }
+ }
+ })
+ arr.iterator()
+ }
+
+ override def edges(direction: Direction, labelNames: String*): util.Iterator[Edge] = {
+ graph.fetchEdges(this, labelNames, direction.name())
+ }
+
+ override def property[V](cardinality: Cardinality, key: String, value: V, objects: AnyRef*): VertexProperty[V] = {
+ cardinality match {
+ case Cardinality.single =>
+ val columnMeta = serviceColumn.metasInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on Vertex."))
+ val newProps = new S2VertexProperty[V](this, columnMeta, key, value)
+ props.put(key, newProps)
+ newProps
+ case _ => throw new RuntimeException("only single cardinality is supported.")
+ }
+ }
+
+ override def addEdge(label: String, vertex: Vertex, kvs: AnyRef*): S2Edge = {
+ vertex match {
+ case otherV: S2Vertex =>
+ val props = ElementHelper.asMap(kvs: _*).asScala.toMap
+ //TODO: direction, operation, _timestamp need to be reserved property key.
+ val direction = props.get("direction").getOrElse("out").toString
+ val ts = props.get(LabelMeta.timestamp.name).map(_.toString.toLong).getOrElse(System.currentTimeMillis())
+ val operation = props.get("operation").map(_.toString).getOrElse("insert")
+
+ graph.addEdgeInner(this, otherV, label, direction, props, ts, operation)
+ case _ => throw new RuntimeException("only S2Graph vertex can be used.")
+ }
+ }
+
+ override def property[V](key: String): VertexProperty[V] = {
+ props.get(key).asInstanceOf[S2VertexProperty[V]]
+ }
+
+ override def properties[V](keys: String*): util.Iterator[VertexProperty[V]] = {
+ val ls = for {
+ key <- keys
+ } yield {
+ property[V](key)
+ }
+ ls.iterator.asJava
+ }
+
+ override def remove(): Unit = ???
+
+ override def label(): String = service.serviceName + S2Vertex.VertexLabelDelimiter + serviceColumn.columnName
+}
+
+object S2Vertex {
+
+ val VertexLabelDelimiter = "::"
+
+ type Props = java.util.Map[String, S2VertexProperty[_]]
+ type State = Map[ColumnMeta, InnerValLike]
+ def EmptyProps = new java.util.HashMap[String, S2VertexProperty[_]]()
+ def EmptyState = Map.empty[ColumnMeta, InnerValLike]
+
+ def toPropKey(labelId: Int): Int = Byte.MaxValue + labelId
+
+ def toLabelId(propKey: Int): Int = propKey - Byte.MaxValue
+
+ def isLabelId(propKey: Int): Boolean = propKey > Byte.MaxValue
+
+ def fillPropsWithTs(vertex: S2Vertex, props: Props): Unit = {
+ props.forEach(new BiConsumer[String, S2VertexProperty[_]] {
+ override def accept(key: String, p: S2VertexProperty[_]): Unit = {
+ vertex.property(Cardinality.single, key, p.value)
+ }
+ })
+ }
+
+ def fillPropsWithTs(vertex: S2Vertex, state: State): Unit = {
+ state.foreach { case (k, v) => vertex.property(Cardinality.single, k.name, v.value) }
+ }
+
+ def propsToState(props: Props): State = {
+ props.asScala.map { case (k, v) =>
+ v.columnMeta -> v.innerVal
+ }.toMap
+ }
+
+ def stateToProps(vertex: S2Vertex, state: State): Props = {
+ state.foreach { case (k, v) =>
+ vertex.property(k.name, v.value)
+ }
+ vertex.props
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/S2VertexProperty.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2VertexProperty.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2VertexProperty.scala
index 0f9f87b..9f8c682 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2VertexProperty.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2VertexProperty.scala
@@ -25,23 +25,44 @@ import org.apache.s2graph.core.mysqls.ColumnMeta
import org.apache.s2graph.core.types.CanInnerValLike
import org.apache.tinkerpop.gremlin.structure.{Property, VertexProperty, Vertex => TpVertex}
-case class S2VertexProperty[V](element: TpVertex,
+import scala.util.hashing.MurmurHash3
+
+case class S2VertexProperty[V](element: S2Vertex,
columnMeta: ColumnMeta,
key: String,
- value: V) extends VertexProperty[V] {
- implicit val encodingVer = columnMeta.serviceColumn.schemaVersion
- val innerVal = CanInnerValLike.anyToInnerValLike.toInnerVal(value)
+ v: V) extends VertexProperty[V] {
+ import CanInnerValLike._
+ implicit lazy val encodingVer = element.serviceColumn.schemaVersion
+ lazy val innerVal = CanInnerValLike.anyToInnerValLike.toInnerVal(value)
def toBytes: Array[Byte] = {
innerVal.bytes
}
+ val value = castValue(v, columnMeta.dataType).asInstanceOf[V]
+
override def properties[U](strings: String*): util.Iterator[Property[U]] = ???
- override def property[V](s: String, v: V): Property[V] = ???
+ override def property[V](key: String, value: V): Property[V] = ???
override def remove(): Unit = ???
override def id(): AnyRef = ???
override def isPresent: Boolean = ???
+
+ override def hashCode(): Int = {
+ MurmurHash3.stringHash(columnMeta.columnId + "," + columnMeta.id.get + "," + key + "," + value)
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case p: S2VertexProperty[_] =>
+ columnMeta.columnId == p.columnMeta.columnId &&
+ columnMeta.seq == p.columnMeta.seq &&
+ key == p.key && value == p.value
+ case _ => false
+ }
+
+ override def toString(): String = {
+ Map("columnMeta" -> columnMeta.toString, "key" -> key, "value" -> value).toString
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala b/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala
deleted file mode 100644
index 57c9824..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.s2graph.core
-
-import java.util
-
-import org.apache.s2graph.core.JSONParser._
-import org.apache.s2graph.core.mysqls.{ColumnMeta, Service, ServiceColumn}
-import org.apache.s2graph.core.types.{InnerVal, InnerValLike, SourceVertexId, VertexId}
-import org.apache.tinkerpop.gremlin.structure
-import org.apache.tinkerpop.gremlin.structure.VertexProperty.Cardinality
-import org.apache.tinkerpop.gremlin.structure.{Vertex => TpVertex, Direction, Edge, VertexProperty, Graph}
-import play.api.libs.json.Json
-
-case class Vertex(graph: Graph,
- id: VertexId,
- ts: Long = System.currentTimeMillis(),
- props: Map[Int, InnerValLike] = Map.empty[Int, InnerValLike],
- op: Byte = 0,
- belongLabelIds: Seq[Int] = Seq.empty) extends GraphElement with TpVertex {
-
- val innerId = id.innerId
-
- val innerIdVal = innerId.value
-
- lazy val properties = for {
- (k, v) <- props
- meta <- serviceColumn.metasMap.get(k)
- } yield meta.name -> v.value
-
- def schemaVer = serviceColumn.schemaVersion
-
- def serviceColumn = ServiceColumn.findById(id.colId)
-
- def columnName = serviceColumn.columnName
-
- def service = Service.findById(serviceColumn.serviceId)
-
- lazy val (hbaseZkAddr, hbaseTableName) = (service.cluster, service.hTableName)
-
- def defaultProps = Map(ColumnMeta.lastModifiedAtColumnSeq.toInt -> InnerVal.withLong(ts, schemaVer))
-
- // lazy val kvs = Graph.client.vertexSerializer(this).toKeyValues
-
- /** TODO: make this as configurable */
- override def serviceName = service.serviceName
-
- override def isAsync = false
-
- override def queueKey = Seq(ts.toString, serviceName).mkString("|")
-
- override def queuePartitionKey = id.innerId.toString
-
- def propsWithName = for {
- (seq, v) <- props
- meta <- ColumnMeta.findByIdAndSeq(id.colId, seq.toByte)
- } yield (meta.name -> v.toString)
-
- def toEdgeVertex() = graph.newVertex(SourceVertexId(id.column, innerId), ts, props, op)
-
-
- override def hashCode() = {
- val hash = id.hashCode()
- // logger.debug(s"Vertex.hashCode: $this -> $hash")
- hash
- }
-
- override def equals(obj: Any) = {
- obj match {
- case otherVertex: Vertex =>
- val ret = id == otherVertex.id
- // logger.debug(s"Vertex.equals: $this, $obj => $ret")
- ret
- case _ => false
- }
- }
-
- def withProps(newProps: Map[Int, InnerValLike]) = graph.newVertex(id, ts, newProps, op)
-
- def toLogString(): String = {
- val (serviceName, columnName) =
- if (!id.storeColId) ("", "")
- else (serviceColumn.service.serviceName, serviceColumn.columnName)
-
- if (propsWithName.nonEmpty)
- Seq(ts, GraphUtil.fromOp(op), "v", id.innerId, serviceName, columnName, Json.toJson(propsWithName)).mkString("\t")
- else
- Seq(ts, GraphUtil.fromOp(op), "v", id.innerId, serviceName, columnName).mkString("\t")
- }
-
- override def vertices(direction: Direction, strings: String*): util.Iterator[TpVertex] = ???
-
- override def edges(direction: Direction, strings: String*): util.Iterator[structure.Edge] = ???
-
- override def property[V](cardinality: Cardinality, s: String, v: V, objects: AnyRef*): VertexProperty[V] = ???
-
- override def addEdge(s: String, vertex: TpVertex, objects: AnyRef*): Edge = ???
-
- override def properties[V](strings: String*): util.Iterator[VertexProperty[V]] = ???
-
- override def remove(): Unit = ???
-
- override def label(): String = ???
-}
-
-object Vertex {
-
- def toPropKey(labelId: Int): Int = Byte.MaxValue + labelId
-
- def toLabelId(propKey: Int): Int = propKey - Byte.MaxValue
-
- def isLabelId(propKey: Int): Boolean = propKey > Byte.MaxValue
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ColumnMeta.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ColumnMeta.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ColumnMeta.scala
index f6c174d..09d02d1 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ColumnMeta.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ColumnMeta.scala
@@ -30,6 +30,8 @@ object ColumnMeta extends Model[ColumnMeta] {
val lastModifiedAtColumn = ColumnMeta(Some(0), 0, "lastModifiedAt", lastModifiedAtColumnSeq, "long")
val maxValue = Byte.MaxValue
+ val timestamp = ColumnMeta(None, -1, "_timestamp", timeStampSeq, "long")
+
def apply(rs: WrappedResultSet): ColumnMeta = {
ColumnMeta(Some(rs.int("id")), rs.int("column_id"), rs.string("name"), rs.byte("seq"), rs.string("data_type").toLowerCase())
}
@@ -125,6 +127,14 @@ object ColumnMeta extends Model[ColumnMeta] {
}
case class ColumnMeta(id: Option[Int], columnId: Int, name: String, seq: Byte, dataType: String) {
- lazy val serviceColumn = ServiceColumn.findById(columnId)
lazy val toJson = Json.obj("name" -> name, "dataType" -> dataType)
+ override def equals(other: Any): Boolean = {
+ if (!other.isInstanceOf[ColumnMeta]) false
+ else {
+ val o = other.asInstanceOf[ColumnMeta]
+ // labelId == o.labelId &&
+ seq == o.seq
+ }
+ }
+ override def hashCode(): Int = seq.toInt
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
index 6636649..4a7e931 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
@@ -191,7 +191,6 @@ case class LabelMeta(id: Option[Int],
seq: Byte,
defaultValue: String,
dataType: String) {
- lazy val label = Label.findById(labelId)
lazy val toJson = Json.obj("name" -> name, "defaultValue" -> defaultValue, "dataType" -> dataType)
override def equals(other: Any): Boolean = {
if (!other.isInstanceOf[LabelMeta]) false
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ServiceColumn.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ServiceColumn.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ServiceColumn.scala
index 85b6929..ebbbf88 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ServiceColumn.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/ServiceColumn.scala
@@ -29,12 +29,19 @@ import org.apache.s2graph.core.types.{HBaseType, InnerValLikeWithTs, InnerValLik
import play.api.libs.json.Json
import scalikejdbc._
object ServiceColumn extends Model[ServiceColumn] {
- val Default = ServiceColumn(Option(HBaseType.DEFAULT_COL_ID), 0, "default", "string", "v4")
+ val Default = ServiceColumn(Option(0), -1, "default", "string", "v4")
def apply(rs: WrappedResultSet): ServiceColumn = {
ServiceColumn(rs.intOpt("id"), rs.int("service_id"), rs.string("column_name"), rs.string("column_type").toLowerCase(), rs.string("schema_version"))
}
+// def findByServiceAndColumn(serviceName: String,
+// columnName: String,
+// useCache: Boolean = true)(implicit session: DBSession): Option[ServiceColumn] = {
+// val service = Service.findByName(serviceName).getOrElse(throw new RuntimeException(s"$serviceName is not found."))
+// find(service.id.get, columnName, useCache)
+// }
+
def findById(id: Int)(implicit session: DBSession = AutoSession): ServiceColumn = {
// val cacheKey = s"id=$id"
val cacheKey = "id=" + id
@@ -95,18 +102,18 @@ object ServiceColumn extends Model[ServiceColumn] {
case class ServiceColumn(id: Option[Int], serviceId: Int, columnName: String, columnType: String, schemaVersion: String) {
lazy val service = Service.findById(serviceId)
- lazy val metas = ColumnMeta.findAllByColumn(id.get)
+ lazy val metas = ColumnMeta.timestamp +: ColumnMeta.findAllByColumn(id.get) :+ ColumnMeta.lastModifiedAtColumn
lazy val metasMap = metas.map { meta => meta.seq.toInt -> meta } toMap
lazy val metasInvMap = metas.map { meta => meta.name -> meta} toMap
lazy val metaNamesMap = (ColumnMeta.lastModifiedAtColumn :: metas).map(x => (x.seq.toInt, x.name)) toMap
lazy val toJson = Json.obj("serviceName" -> service.serviceName, "columnName" -> columnName, "columnType" -> columnType)
- def propsToInnerVals(props: Map[String, Any]): Map[Int, InnerValLike] = {
+ def propsToInnerVals(props: Map[String, Any]): Map[ColumnMeta, InnerValLike] = {
for {
(k, v) <- props
labelMeta <- metasInvMap.get(k)
innerVal = toInnerVal(v.toString, labelMeta.dataType, schemaVersion)
- } yield labelMeta.seq.toInt -> innerVal
+ } yield labelMeta -> innerVal
}
def innerValsToProps(props: Map[Int, InnerValLike]): Map[String, Any] = {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala b/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
index aa018a9..d754bb7 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
@@ -22,7 +22,7 @@ package org.apache.s2graph.core.parsers
import org.apache.s2graph.core.GraphExceptions.{LabelNotExistException, WhereParserException}
import org.apache.s2graph.core.mysqls.{Label, LabelMeta}
import org.apache.s2graph.core.types.InnerValLike
-import org.apache.s2graph.core.{Edge, GraphUtil}
+import org.apache.s2graph.core.{S2Edge, GraphUtil}
import org.apache.s2graph.core.JSONParser._
import org.apache.s2graph.core.utils.logger
@@ -33,7 +33,7 @@ import scala.util.parsing.combinator.JavaTokenParsers
trait ExtractValue {
val parent = "_parent."
- def propToInnerVal(edge: Edge, key: String) = {
+ def propToInnerVal(edge: S2Edge, key: String) = {
val (propKey, parentEdge) = findParentEdge(edge, key)
val label = parentEdge.innerLabel
@@ -47,7 +47,7 @@ trait ExtractValue {
}
}
- def valueToCompare(edge: Edge, key: String, value: String) = {
+ def valueToCompare(edge: S2Edge, key: String, value: String) = {
val label = edge.innerLabel
if (value.startsWith(parent) || label.metaPropsInvMap.contains(value)) propToInnerVal(edge, value)
else {
@@ -65,11 +65,11 @@ trait ExtractValue {
}
@tailrec
- private def findParent(edge: Edge, depth: Int): Edge =
+ private def findParent(edge: S2Edge, depth: Int): S2Edge =
if (depth > 0) findParent(edge.parentEdges.head.edge, depth - 1)
else edge
- private def findParentEdge(edge: Edge, key: String): (String, Edge) = {
+ private def findParentEdge(edge: S2Edge, key: String): (String, S2Edge) = {
if (!key.startsWith(parent)) (key, edge)
else {
val split = key.split(parent)
@@ -88,9 +88,9 @@ trait Clause extends ExtractValue {
def or(otherField: Clause): Clause = Or(this, otherField)
- def filter(edge: Edge): Boolean
+ def filter(edge: S2Edge): Boolean
- def binaryOp(binOp: (InnerValLike, InnerValLike) => Boolean)(propKey: String, value: String)(edge: Edge): Boolean = {
+ def binaryOp(binOp: (InnerValLike, InnerValLike) => Boolean)(propKey: String, value: String)(edge: S2Edge): Boolean = {
val propValue = propToInnerVal(edge, propKey)
val compValue = valueToCompare(edge, propKey, value)
@@ -105,20 +105,20 @@ object Where {
}
}
case class Where(clauses: Seq[Clause] = Seq.empty[Clause]) {
- def filter(edge: Edge) =
+ def filter(edge: S2Edge) =
if (clauses.isEmpty) true else clauses.map(_.filter(edge)).forall(identity)
}
case class Gt(propKey: String, value: String) extends Clause {
- override def filter(edge: Edge): Boolean = binaryOp(_ > _)(propKey, value)(edge)
+ override def filter(edge: S2Edge): Boolean = binaryOp(_ > _)(propKey, value)(edge)
}
case class Lt(propKey: String, value: String) extends Clause {
- override def filter(edge: Edge): Boolean = binaryOp(_ < _)(propKey, value)(edge)
+ override def filter(edge: S2Edge): Boolean = binaryOp(_ < _)(propKey, value)(edge)
}
case class Eq(propKey: String, value: String) extends Clause {
- override def filter(edge: Edge): Boolean = binaryOp(_ == _)(propKey, value)(edge)
+ override def filter(edge: S2Edge): Boolean = binaryOp(_ == _)(propKey, value)(edge)
}
case class InWithoutParent(label: Label, propKey: String, values: Set[String]) extends Clause {
@@ -144,7 +144,7 @@ case class InWithoutParent(label: Label, propKey: String, values: Set[String]) e
toInnerVal(value, dataType, label.schemaVersion)
}
- override def filter(edge: Edge): Boolean = {
+ override def filter(edge: S2Edge): Boolean = {
if (edge.dir == GraphUtil.directions("in")) {
val propVal = propToInnerVal(edge, propKey)
innerValLikeLsIn.contains(propVal)
@@ -156,7 +156,7 @@ case class InWithoutParent(label: Label, propKey: String, values: Set[String]) e
}
case class IN(propKey: String, values: Set[String]) extends Clause {
- override def filter(edge: Edge): Boolean = {
+ override def filter(edge: S2Edge): Boolean = {
val propVal = propToInnerVal(edge, propKey)
values.exists { value =>
valueToCompare(edge, propKey, value) == propVal
@@ -165,7 +165,7 @@ case class IN(propKey: String, values: Set[String]) extends Clause {
}
case class Between(propKey: String, minValue: String, maxValue: String) extends Clause {
- override def filter(edge: Edge): Boolean = {
+ override def filter(edge: S2Edge): Boolean = {
val propVal = propToInnerVal(edge, propKey)
val minVal = valueToCompare(edge, propKey, minValue)
val maxVal = valueToCompare(edge, propKey, maxValue)
@@ -175,15 +175,15 @@ case class Between(propKey: String, minValue: String, maxValue: String) extends
}
case class Not(self: Clause) extends Clause {
- override def filter(edge: Edge) = !self.filter(edge)
+ override def filter(edge: S2Edge) = !self.filter(edge)
}
case class And(left: Clause, right: Clause) extends Clause {
- override def filter(edge: Edge) = left.filter(edge) && right.filter(edge)
+ override def filter(edge: S2Edge) = left.filter(edge) && right.filter(edge)
}
case class Or(left: Clause, right: Clause) extends Clause {
- override def filter(edge: Edge) = left.filter(edge) || right.filter(edge)
+ override def filter(edge: S2Edge) = left.filter(edge) || right.filter(edge)
}
object WhereParser {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala b/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
index 13e02a0..62d1e40 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
@@ -118,7 +118,7 @@ object RequestParser {
}
-class RequestParser(graph: Graph) {
+class RequestParser(graph: S2Graph) {
import Management.JsonModel._
import RequestParser._
@@ -261,7 +261,7 @@ class RequestParser(graph: Graph) {
GroupBy(keys)
}.getOrElse(GroupBy.Empty)
- def toVertices(labelName: String, direction: String, ids: Seq[JsValue]): Seq[Vertex] = {
+ def toVertices(labelName: String, direction: String, ids: Seq[JsValue]): Seq[S2Vertex] = {
val vertices = for {
label <- Label.findByName(labelName).toSeq
serviceColumn = if (direction == "out") label.srcColumn else label.tgtColumn
@@ -547,12 +547,12 @@ class RequestParser(graph: Graph) {
elementsWithTsv
}
- def parseJsonFormat(jsValue: JsValue, operation: String): Seq[(Edge, String)] = {
+ def parseJsonFormat(jsValue: JsValue, operation: String): Seq[(S2Edge, String)] = {
val jsValues = toJsValues(jsValue)
jsValues.flatMap(toEdgeWithTsv(_, operation))
}
- private def toEdgeWithTsv(jsValue: JsValue, operation: String): Seq[(Edge, String)] = {
+ private def toEdgeWithTsv(jsValue: JsValue, operation: String): Seq[(S2Edge, String)] = {
val srcIds = (jsValue \ "from").asOpt[JsValue].map(Seq(_)).getOrElse(Nil) ++ (jsValue \ "froms").asOpt[Seq[JsValue]].getOrElse(Nil)
val tgtIds = (jsValue \ "to").asOpt[JsValue].map(Seq(_)).getOrElse(Nil) ++ (jsValue \ "tos").asOpt[Seq[JsValue]].getOrElse(Nil)
@@ -580,7 +580,7 @@ class RequestParser(graph: Graph) {
toJsValues(jsValue).map(toVertex(_, operation, serviceName, columnName))
}
- def toVertex(jsValue: JsValue, operation: String, serviceName: Option[String] = None, columnName: Option[String] = None): Vertex = {
+ def toVertex(jsValue: JsValue, operation: String, serviceName: Option[String] = None, columnName: Option[String] = None): S2Vertex = {
val id = parse[JsValue](jsValue, "id")
val ts = parseOption[Long](jsValue, "timestamp").getOrElse(System.currentTimeMillis())
val sName = if (serviceName.isEmpty) parse[String](jsValue, "serviceName") else serviceName.get
[05/10] incubator-s2graph git commit: [S2GRAPH-131]: Add actual
implementation on interfaces from TinkerPop3 structure package. - Change
core.Edge/Vertex/Graph to core.S2Edge/S2Vertex/S2Graph. - Implement base
interfaces for tinkerpop3 structure packag
Posted by st...@apache.org.
[S2GRAPH-131]: Add actual implementation on interfaces from TinkerPop3 structure package.
- Change core.Edge/Vertex/Graph to core.S2Edge/S2Vertex/S2Graph.
- Implement base interfaces for tinkerpop3 structure package.
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/e8c0bf20
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/e8c0bf20
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/e8c0bf20
Branch: refs/heads/master
Commit: e8c0bf20b517a2e9a752df63c156fc265b7365b6
Parents: 189bc41
Author: DO YUNG YOON <st...@apache.org>
Authored: Mon Nov 28 12:09:52 2016 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Mon Nov 28 12:09:52 2016 +0900
----------------------------------------------------------------------
.../loader/subscriber/GraphSubscriber.scala | 14 +-
.../loader/subscriber/TransferToHFile.scala | 6 +-
.../s2graph/loader/subscriber/WalLogStat.scala | 2 +-
.../loader/subscriber/WalLogToHDFS.scala | 2 +-
.../scala/org/apache/s2graph/core/Edge.scala | 956 ------------
.../scala/org/apache/s2graph/core/Graph.scala | 1238 ----------------
.../org/apache/s2graph/core/Management.scala | 2 +-
.../org/apache/s2graph/core/PostProcess.scala | 8 +-
.../org/apache/s2graph/core/QueryParam.scala | 23 +-
.../org/apache/s2graph/core/QueryResult.scala | 10 +-
.../scala/org/apache/s2graph/core/S2Edge.scala | 969 ++++++++++++
.../scala/org/apache/s2graph/core/S2Graph.scala | 1397 ++++++++++++++++++
.../org/apache/s2graph/core/S2Property.scala | 7 +-
.../org/apache/s2graph/core/S2Vertex.scala | 221 +++
.../apache/s2graph/core/S2VertexProperty.scala | 31 +-
.../scala/org/apache/s2graph/core/Vertex.scala | 132 --
.../apache/s2graph/core/mysqls/ColumnMeta.scala | 12 +-
.../apache/s2graph/core/mysqls/LabelMeta.scala | 1 -
.../s2graph/core/mysqls/ServiceColumn.scala | 15 +-
.../s2graph/core/parsers/WhereParser.scala | 34 +-
.../s2graph/core/rest/RequestParser.scala | 10 +-
.../apache/s2graph/core/rest/RestHandler.scala | 4 +-
.../apache/s2graph/core/storage/Storage.scala | 96 +-
.../core/storage/hbase/AsynchbaseStorage.scala | 10 +-
.../tall/IndexEdgeDeserializable.scala | 10 +-
.../wide/IndexEdgeDeserializable.scala | 8 +-
.../tall/SnapshotEdgeDeserializable.scala | 4 +-
.../wide/SnapshotEdgeDeserializable.scala | 4 +-
.../serde/vertex/VertexDeserializable.scala | 23 +-
.../serde/vertex/VertexSerializable.scala | 13 +-
.../s2graph/core/types/InnerValLike.scala | 17 +
.../apache/s2graph/core/types/VertexId.scala | 2 +-
.../org/apache/s2graph/core/EdgeTest.scala | 210 ---
.../core/Integrate/IntegrateCommon.scala | 6 +-
.../Integrate/tinkerpop/S2S2GraphTest.scala | 130 ++
.../org/apache/s2graph/core/S2EdgeTest.scala | 210 +++
.../s2graph/core/TestCommonWithModels.scala | 4 +-
.../core/benchmark/BenchmarkCommon.scala | 2 +-
.../s2graph/core/parsers/WhereParserTest.scala | 2 +-
.../s2graph/counter/helper/CounterAdmin.scala | 4 +-
.../counter/core/RankingCounterSpec.scala | 4 +-
.../loader/core/CounterEtlFunctions.scala | 14 +-
.../loader/core/CounterEtlFunctionsSpec.scala | 4 +-
.../org/apache/s2graph/rest/netty/Server.scala | 4 +-
.../apache/s2graph/rest/play/Bootstrap.scala | 6 +-
.../s2graph/rest/play/actors/QueueActor.scala | 8 +-
.../rest/play/controllers/EdgeController.scala | 12 +-
.../play/controllers/VertexController.scala | 4 +-
48 files changed, 3179 insertions(+), 2726 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
----------------------------------------------------------------------
diff --git a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
index b25bc84..2352cdf 100644
--- a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
+++ b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
@@ -48,7 +48,7 @@ object GraphConfig {
if (kafkaBrokerList.isEmpty) Map("hbase.zookeeper.quorum" -> zkQuorum, "db.default.url" -> database, "cache.ttl.seconds" -> cacheTTL)
else Map("hbase.zookeeper.quorum" -> zkQuorum, "db.default.url" -> database, "kafka.metadata.broker.list" -> kafkaBrokers, "cache.ttl.seconds" -> cacheTTL)
- ConfigFactory.parseMap(newConf).withFallback(Graph.DefaultConfig)
+ ConfigFactory.parseMap(newConf).withFallback(S2Graph.DefaultConfig)
}
}
@@ -64,7 +64,7 @@ object GraphSubscriberHelper extends WithKafka {
private val sleepPeriod = 10000
private val maxTryNum = 10
- var g: Graph = null
+ var g: S2Graph = null
var management: Management = null
val conns = new scala.collection.mutable.HashMap[String, Connection]()
@@ -80,7 +80,7 @@ object GraphSubscriberHelper extends WithKafka {
if (g == null) {
val ec = ExecutionContext.Implicits.global
- g = new Graph(config)(ec)
+ g = new S2Graph(config)(ec)
management = new Management(g)
}
}
@@ -107,12 +107,12 @@ object GraphSubscriberHelper extends WithKafka {
(for (msg <- msgs) yield {
statFunc("total", 1)
g.toGraphElement(msg, labelMapping) match {
- case Some(e) if e.isInstanceOf[Edge] =>
+ case Some(e) if e.isInstanceOf[S2Edge] =>
statFunc("EdgeParseOk", 1)
- e.asInstanceOf[Edge]
- case Some(v) if v.isInstanceOf[Vertex] =>
+ e.asInstanceOf[S2Edge]
+ case Some(v) if v.isInstanceOf[S2Vertex] =>
statFunc("VertexParseOk", 1)
- v.asInstanceOf[Vertex]
+ v.asInstanceOf[S2Vertex]
case Some(x) =>
throw new RuntimeException(s">>>>> GraphSubscriber.toGraphElements: parsing failed. ${x.serviceName}")
case None =>
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
----------------------------------------------------------------------
diff --git a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
index d1da319..dce085e 100644
--- a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
+++ b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
@@ -57,7 +57,7 @@ object TransferToHFile extends SparkApp {
/** build key values */
case class DegreeKey(vertexIdStr: String, labelName: String, direction: String)
- private def insertBulkForLoaderAsync(edge: Edge, createRelEdges: Boolean = true): List[PutRequest] = {
+ private def insertBulkForLoaderAsync(edge: S2Edge, createRelEdges: Boolean = true): List[PutRequest] = {
val relEdges = if (createRelEdges) edge.relatedEdges else List(edge)
buildPutRequests(edge.toSnapshotEdge) ++ relEdges.toList.flatMap { e =>
e.edgesWithIndex.flatMap { indexEdge => buildPutRequests(indexEdge) }
@@ -125,8 +125,8 @@ object TransferToHFile extends SparkApp {
def toKeyValues(strs: Seq[String], labelMapping: Map[String, String], autoEdgeCreate: Boolean): Iterator[KeyValue] = {
val kvs = for {
s <- strs
- element <- GraphSubscriberHelper.g.toGraphElement(s, labelMapping).toSeq if element.isInstanceOf[Edge]
- edge = element.asInstanceOf[Edge]
+ element <- GraphSubscriberHelper.g.toGraphElement(s, labelMapping).toSeq if element.isInstanceOf[S2Edge]
+ edge = element.asInstanceOf[S2Edge]
putRequest <- insertBulkForLoaderAsync(edge, autoEdgeCreate)
} yield {
val p = putRequest
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogStat.scala
----------------------------------------------------------------------
diff --git a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogStat.scala b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogStat.scala
index 5b68754..40f936d 100644
--- a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogStat.scala
+++ b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogStat.scala
@@ -21,7 +21,7 @@ package org.apache.s2graph.loader.subscriber
import kafka.producer.KeyedMessage
import kafka.serializer.StringDecoder
-import org.apache.s2graph.core.Graph
+import org.apache.s2graph.core.S2Graph$
import org.apache.s2graph.spark.spark.{WithKafka, SparkApp}
import org.apache.spark.streaming.Durations._
import org.apache.spark.streaming.kafka.HasOffsetRanges
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogToHDFS.scala
----------------------------------------------------------------------
diff --git a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogToHDFS.scala b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogToHDFS.scala
index 0f69dc7..23c3cda 100644
--- a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogToHDFS.scala
+++ b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/WalLogToHDFS.scala
@@ -22,7 +22,7 @@ package org.apache.s2graph.loader.subscriber
import java.text.SimpleDateFormat
import java.util.Date
import kafka.serializer.StringDecoder
-import org.apache.s2graph.core.Graph
+import org.apache.s2graph.core.S2Graph$
import org.apache.s2graph.spark.spark.{WithKafka, SparkApp, HashMapParam}
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.streaming.Durations._
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala b/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala
deleted file mode 100644
index f10b4db..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala
+++ /dev/null
@@ -1,956 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.s2graph.core
-
-import java.util
-import java.util.function.BiConsumer
-
-import org.apache.s2graph.core.Edge.{Props, State}
-import org.apache.s2graph.core.GraphExceptions.LabelNotExistException
-import org.apache.s2graph.core.JSONParser._
-import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
-import org.apache.s2graph.core.types._
-import org.apache.s2graph.core.utils.logger
-import org.apache.tinkerpop.gremlin.structure
-import org.apache.tinkerpop.gremlin.structure.{Direction, Edge => TpEdge, Graph => TpGraph, Property}
-import play.api.libs.json.{JsNumber, JsObject, Json}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{Map => MutableMap}
-import scala.util.hashing.MurmurHash3
-
-case class SnapshotEdge(graph: Graph,
- srcVertex: Vertex,
- tgtVertex: Vertex,
- label: Label,
- dir: Int,
- op: Byte,
- version: Long,
- private val propsWithTs: Props,
- pendingEdgeOpt: Option[Edge],
- statusCode: Byte = 0,
- lockTs: Option[Long],
- tsInnerValOpt: Option[InnerValLike] = None) {
- lazy val direction = GraphUtil.fromDirection(dir)
- lazy val operation = GraphUtil.fromOp(op)
- lazy val edge = toEdge
- lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
-// if (!propsWithTs.contains(LabelMeta.timestamp.name)) throw new Exception("Timestamp is required.")
-
-// val label = Label.findById(labelWithDir.labelId)
- lazy val schemaVer = label.schemaVersion
- lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.toString().toLong
-
- def propsToKeyValuesWithTs = HBaseSerializable.propsToKeyValuesWithTs(propsWithTs.asScala.map(kv => kv._2.labelMeta.seq -> kv._2.innerValWithTs).toSeq)
-
- def allPropsDeleted = Edge.allPropsDeleted(propsWithTs)
-
- def toEdge: Edge = {
- Edge(graph, srcVertex, tgtVertex, label, dir, op,
- version, propsWithTs, pendingEdgeOpt = pendingEdgeOpt,
- statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
- }
-
- def propsWithName = (for {
- (_, v) <- propsWithTs.asScala
- meta = v.labelMeta
- jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
- } yield meta.name -> jsValue) ++ Map("version" -> JsNumber(version))
-
- // only for debug
- def toLogString() = {
- List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, propsWithName).mkString("\t")
- }
-
- def property[V](key: String, value: V, ts: Long): S2Property[V] = {
- val labelMeta = label.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on IndexEdge."))
- val newProps = new S2Property(edge, labelMeta, key, value, ts)
- propsWithTs.put(key, newProps)
- newProps
- }
- override def hashCode(): Int = {
- MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId)
- }
-
- override def equals(other: Any): Boolean = other match {
- case e: SnapshotEdge =>
- srcVertex.innerId == e.srcVertex.innerId &&
- tgtVertex.innerId == e.tgtVertex.innerId &&
- labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
- pendingEdgeOpt == e.pendingEdgeOpt && lockTs == lockTs && statusCode == statusCode
- case _ => false
- }
-
- override def toString(): String = {
- Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> label.label, "direction" -> direction,
- "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString,
- "statusCode" -> statusCode, "lockTs" -> lockTs).toString
- }
-}
-
-case class IndexEdge(graph: Graph,
- srcVertex: Vertex,
- tgtVertex: Vertex,
- label: Label,
- dir: Int,
- op: Byte,
- version: Long,
- labelIndexSeq: Byte,
- private val propsWithTs: Props,
- tsInnerValOpt: Option[InnerValLike] = None) {
-// if (!props.contains(LabelMeta.timeStampSeq)) throw new Exception("Timestamp is required.")
- // assert(props.contains(LabelMeta.timeStampSeq))
- lazy val direction = GraphUtil.fromDirection(dir)
- lazy val operation = GraphUtil.fromOp(op)
- lazy val edge = toEdge
- lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
-
- lazy val isInEdge = labelWithDir.dir == GraphUtil.directions("in")
- lazy val isOutEdge = !isInEdge
-
- lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.toString.toLong
- lazy val degreeEdge = propsWithTs.containsKey(LabelMeta.degree.name)
-
- lazy val schemaVer = label.schemaVersion
- lazy val labelIndex = LabelIndex.findByLabelIdAndSeq(labelWithDir.labelId, labelIndexSeq).get
- lazy val defaultIndexMetas = labelIndex.sortKeyTypes.map { meta =>
- val innerVal = toInnerVal(meta.defaultValue, meta.dataType, schemaVer)
- meta.seq -> innerVal
- }.toMap
-
- lazy val labelIndexMetaSeqs = labelIndex.sortKeyTypes
-
- /** TODO: make sure call of this class fill props as this assumes */
- lazy val orders = for (meta <- labelIndexMetaSeqs) yield {
- propsWithTs.get(meta.name) match {
- case null =>
-
- /**
- * TODO: agly hack
- * now we double store target vertex.innerId/srcVertex.innerId for easy development. later fix this to only store id once
- */
- val v = meta match {
- case LabelMeta.timestamp=> InnerVal.withLong(version, schemaVer)
- case LabelMeta.to => toEdge.tgtVertex.innerId
- case LabelMeta.from => toEdge.srcVertex.innerId
- // for now, it does not make sense to build index on srcVertex.innerId since all edges have same data.
- // throw new RuntimeException("_from on indexProps is not supported")
- case _ => toInnerVal(meta.defaultValue, meta.dataType, schemaVer)
- }
-
- meta -> v
- case v => meta -> v.innerVal
- }
- }
-
- lazy val ordersKeyMap = orders.map { case (meta, _) => meta.name }.toSet
- lazy val metas = for ((meta, v) <- propsWithTs.asScala if !ordersKeyMap.contains(meta)) yield v.labelMeta -> v.innerVal
-
-// lazy val propsWithTs = props.map { case (k, v) => k -> InnerValLikeWithTs(v, version) }
-
- //TODO:
- // lazy val kvs = Graph.client.indexedEdgeSerializer(this).toKeyValues.toList
-
- lazy val hasAllPropsForIndex = orders.length == labelIndexMetaSeqs.length
-
- def propsWithName = for {
- (_, v) <- propsWithTs.asScala
- meta = v.labelMeta
- jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
- } yield meta.name -> jsValue
-
-
- def toEdge: Edge = Edge(graph, srcVertex, tgtVertex, label, dir, op, version, propsWithTs, tsInnerValOpt = tsInnerValOpt)
-
- // only for debug
- def toLogString() = {
- List(version, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, Json.toJson(propsWithName)).mkString("\t")
- }
-
- def property(key: String): Option[InnerValLikeWithTs] = {
- label.metaPropsInvMap.get(key).map(labelMeta => property(labelMeta))
- }
-
- def property(labelMeta: LabelMeta): InnerValLikeWithTs = {
-// propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse(label.metaPropsDefaultMapInner(labelMeta))
- if (propsWithTs.containsKey(labelMeta.name)) {
- propsWithTs.get(labelMeta.name).innerValWithTs
- } else {
- label.metaPropsDefaultMapInner(labelMeta)
- }
- }
-
- def updatePropsWithTs(others: Props = Edge.EmptyProps): Props = {
- if (others.isEmpty) propsWithTs
- else {
- val iter = others.entrySet().iterator()
- while (iter.hasNext) {
- val e = iter.next()
- propsWithTs.put(e.getKey, e.getValue)
- }
- propsWithTs
- }
- }
-
- def property[V](key: String, value: V, ts: Long): S2Property[V] = {
- val labelMeta = label.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on IndexEdge."))
- val newProps = new S2Property(edge, labelMeta, key, value, ts)
- propsWithTs.put(key, newProps)
- newProps
- }
- override def hashCode(): Int = {
- MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId + "," + labelIndexSeq)
- }
-
- override def equals(other: Any): Boolean = other match {
- case e: IndexEdge =>
- srcVertex.innerId == e.srcVertex.innerId &&
- tgtVertex.innerId == e.tgtVertex.innerId &&
- labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
- labelIndexSeq == e.labelIndexSeq
- case _ => false
- }
-
- override def toString(): String = {
- Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> label.label, "direction" -> dir,
- "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString
- ).toString
- }
-}
-
-case class Edge(innerGraph: Graph,
- srcVertex: Vertex,
- var tgtVertex: Vertex,
- innerLabel: Label,
- dir: Int,
- var op: Byte = GraphUtil.defaultOpByte,
- var version: Long = System.currentTimeMillis(),
- propsWithTs: Props = Edge.EmptyProps,
- parentEdges: Seq[EdgeWithScore] = Nil,
- originalEdgeOpt: Option[Edge] = None,
- pendingEdgeOpt: Option[Edge] = None,
- statusCode: Byte = 0,
- lockTs: Option[Long] = None,
- var tsInnerValOpt: Option[InnerValLike] = None) extends GraphElement with TpEdge {
-
- lazy val labelWithDir = LabelWithDirection(innerLabel.id.get, dir)
- lazy val schemaVer = innerLabel.schemaVersion
- lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.value match {
- case b: BigDecimal => b.longValue()
- case l: Long => l
- case i: Int => i.toLong
- case _ => throw new RuntimeException("ts should be in [BigDecimal/Long/Int].")
- }
-
- lazy val operation = GraphUtil.fromOp(op)
- lazy val tsInnerVal = tsInnerValOpt.get.value
- lazy val srcId = srcVertex.innerIdVal
- lazy val tgtId = tgtVertex.innerIdVal
- lazy val labelName = innerLabel.label
- lazy val direction = GraphUtil.fromDirection(dir)
-
- def toIndexEdge(labelIndexSeq: Byte): IndexEdge = IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelIndexSeq, propsWithTs)
-
- def serializePropsWithTs(): Array[Byte] = HBaseSerializable.propsToKeyValuesWithTs(propsWithTs.asScala.map(kv => kv._2.labelMeta.seq -> kv._2.innerValWithTs).toSeq)
-
- def updatePropsWithTs(others: Props = Edge.EmptyProps): Props = {
- val emptyProp = Edge.EmptyProps
-
- propsWithTs.forEach(new BiConsumer[String, S2Property[_]] {
- override def accept(key: String, value: S2Property[_]): Unit = emptyProp.put(key, value)
- })
-
- others.forEach(new BiConsumer[String, S2Property[_]] {
- override def accept(key: String, value: S2Property[_]): Unit = emptyProp.put(key, value)
- })
-
- emptyProp
- }
-
- def propertyValue(key: String): Option[InnerValLikeWithTs] = {
- key match {
- case "from" | "_from" => Option(InnerValLikeWithTs(srcVertex.innerId, ts))
- case "to" | "_to" => Option(InnerValLikeWithTs(tgtVertex.innerId, ts))
- case "label" => Option(InnerValLikeWithTs(InnerVal.withStr(innerLabel.label, schemaVer), ts))
- case "direction" => Option(InnerValLikeWithTs(InnerVal.withStr(direction, schemaVer), ts))
- case _ =>
- innerLabel.metaPropsInvMap.get(key).map(labelMeta => propertyValueInner(labelMeta))
- }
- }
-
- def propertyValueInner(labelMeta: LabelMeta): InnerValLikeWithTs= {
- // propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse()
- if (propsWithTs.containsKey(labelMeta.name)) {
- propsWithTs.get(labelMeta.name).innerValWithTs
- } else {
- innerLabel.metaPropsDefaultMapInner(labelMeta)
- }
- }
-
- def propertyValues(keys: Seq[String] = Nil): Map[LabelMeta, InnerValLikeWithTs] = {
- val labelMetas = for {
- key <- keys
- labelMeta <- innerLabel.metaPropsInvMap.get(key)
- } yield labelMeta
-
- propertyValuesInner(labelMetas)
- }
-
- def propertyValuesInner(labelMetas: Seq[LabelMeta] = Nil): Map[LabelMeta, InnerValLikeWithTs] = {
- if (labelMetas.isEmpty) {
- innerLabel.metaPropsDefaultMapInner.map { case (labelMeta, defaultVal) =>
- labelMeta -> propertyValueInner(labelMeta)
- }
- } else {
- // This is important since timestamp is required for all edges.
- (LabelMeta.timestamp +: labelMetas).map { labelMeta =>
- labelMeta -> propertyValueInner(labelMeta)
- }.toMap
- }
- }
-
-// if (!props.contains(LabelMeta.timestamp)) throw new Exception("Timestamp is required.")
- // assert(propsWithTs.contains(LabelMeta.timeStampSeq))
-
- lazy val properties = toProps()
-
- def props = propsWithTs.asScala.mapValues(_.innerVal)
-
-
- private def toProps(): Map[String, Any] = {
- for {
- (labelMeta, defaultVal) <- innerLabel.metaPropsDefaultMapInner
- } yield {
- // labelMeta.name -> propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse(defaultVal).innerVal.value
- val value =
- if (propsWithTs.containsKey(labelMeta.name)) {
- propsWithTs.get(labelMeta.name).value
- } else {
- defaultVal.innerVal.value
- }
- labelMeta.name -> value
- }
- }
-
- def relatedEdges = {
- if (labelWithDir.isDirected) {
- val skipReverse = innerLabel.extraOptions.get("skipReverse").map(_.as[Boolean]).getOrElse(false)
- if (skipReverse) List(this) else List(this, duplicateEdge)
- } else {
-// val outDir = labelWithDir.copy(dir = GraphUtil.directions("out"))
-// val base = copy(labelWithDir = outDir)
- val base = copy(dir = GraphUtil.directions("out"))
- List(base, base.reverseSrcTgtEdge)
- }
- }
-
- // def relatedEdges = List(this)
-
- def srcForVertex = {
- val belongLabelIds = Seq(labelWithDir.labelId)
- if (labelWithDir.dir == GraphUtil.directions("in")) {
- innerGraph.newVertex(VertexId(innerLabel.tgtColumn, tgtVertex.innerId), tgtVertex.ts, tgtVertex.props, belongLabelIds = belongLabelIds)
- } else {
- innerGraph.newVertex(VertexId(innerLabel.srcColumn, srcVertex.innerId), srcVertex.ts, srcVertex.props, belongLabelIds = belongLabelIds)
- }
- }
-
- def tgtForVertex = {
- val belongLabelIds = Seq(labelWithDir.labelId)
- if (labelWithDir.dir == GraphUtil.directions("in")) {
- innerGraph.newVertex(VertexId(innerLabel.srcColumn, srcVertex.innerId), srcVertex.ts, srcVertex.props, belongLabelIds = belongLabelIds)
- } else {
- innerGraph.newVertex(VertexId(innerLabel.tgtColumn, tgtVertex.innerId), tgtVertex.ts, tgtVertex.props, belongLabelIds = belongLabelIds)
- }
- }
-
- def duplicateEdge = reverseSrcTgtEdge.reverseDirEdge
-
-// def reverseDirEdge = copy(labelWithDir = labelWithDir.dirToggled)
- def reverseDirEdge = copy(dir = GraphUtil.toggleDir(dir))
-
- def reverseSrcTgtEdge = copy(srcVertex = tgtVertex, tgtVertex = srcVertex)
-
- def labelOrders = LabelIndex.findByLabelIdAll(labelWithDir.labelId)
-
- override def serviceName = innerLabel.serviceName
-
- override def queueKey = Seq(ts.toString, tgtVertex.serviceName).mkString("|")
-
- override def queuePartitionKey = Seq(srcVertex.innerId, tgtVertex.innerId).mkString("|")
-
- override def isAsync = innerLabel.isAsync
-
- def isDegree = propsWithTs.containsKey(LabelMeta.degree.name)
-
-// def propsPlusTs = propsWithTs.get(LabelMeta.timeStampSeq) match {
-// case Some(_) => props
-// case None => props ++ Map(LabelMeta.timeStampSeq -> InnerVal.withLong(ts, schemaVer))
-// }
-
- def propsPlusTsValid = propsWithTs.asScala.filter(kv => LabelMeta.isValidSeq(kv._2.labelMeta.seq)).asJava
-
- def edgesWithIndex = for (labelOrder <- labelOrders) yield {
- IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelOrder.seq, propsWithTs, tsInnerValOpt = tsInnerValOpt)
- }
-
- def edgesWithIndexValid = for (labelOrder <- labelOrders) yield {
- IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelOrder.seq, propsPlusTsValid, tsInnerValOpt = tsInnerValOpt)
- }
-
- /** force direction as out on invertedEdge */
- def toSnapshotEdge: SnapshotEdge = {
- val (smaller, larger) = (srcForVertex, tgtForVertex)
-
-// val newLabelWithDir = LabelWithDirection(labelWithDir.labelId, GraphUtil.directions("out"))
-
- property(LabelMeta.timestamp.name, ts, ts)
- val ret = SnapshotEdge(innerGraph, smaller, larger, innerLabel,
- GraphUtil.directions("out"), op, version, propsWithTs,
- pendingEdgeOpt = pendingEdgeOpt, statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
- ret
- }
-
- def defaultPropsWithName = Json.obj("from" -> srcVertex.innerId.toString(), "to" -> tgtVertex.innerId.toString(),
- "label" -> innerLabel.label, "service" -> innerLabel.serviceName)
-
- def propsWithName =
- for {
- (_, v) <- propsWithTs.asScala
- meta = v.labelMeta
- jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
- } yield meta.name -> jsValue
-
-
- def updateTgtVertex(id: InnerValLike) = {
- val newId = TargetVertexId(tgtVertex.id.column, id)
- val newTgtVertex = innerGraph.newVertex(newId, tgtVertex.ts, tgtVertex.props)
- Edge(innerGraph, srcVertex, newTgtVertex, innerLabel, dir, op, version, propsWithTs, tsInnerValOpt = tsInnerValOpt)
- }
-
- def rank(r: RankParam): Double =
- if (r.keySeqAndWeights.size <= 0) 1.0f
- else {
- var sum: Double = 0
-
- for ((labelMeta, w) <- r.keySeqAndWeights) {
- if (propsWithTs.containsKey(labelMeta.name)) {
- val innerValWithTs = propsWithTs.get(labelMeta.name)
- val cost = try innerValWithTs.innerVal.toString.toDouble catch {
- case e: Exception =>
- logger.error("toInnerval failed in rank", e)
- 1.0
- }
- sum += w * cost
- }
- }
- sum
- }
-
- def toLogString: String = {
- val allPropsWithName = defaultPropsWithName ++ Json.toJson(propsWithName).asOpt[JsObject].getOrElse(Json.obj())
- List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, innerLabel.label, allPropsWithName).mkString("\t")
- }
-
- override def hashCode(): Int = {
- MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId)
- }
-
- override def equals(other: Any): Boolean = other match {
- case e: Edge =>
- srcVertex.innerId == e.srcVertex.innerId &&
- tgtVertex.innerId == e.tgtVertex.innerId &&
- labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
- pendingEdgeOpt == e.pendingEdgeOpt && lockTs == lockTs && statusCode == statusCode &&
- parentEdges == e.parentEdges && originalEdgeOpt == originalEdgeOpt
- case _ => false
- }
-
- override def toString(): String = {
- Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> labelName, "direction" -> direction,
- "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString,
- "parentEdges" -> parentEdges, "originalEdge" -> originalEdgeOpt, "statusCode" -> statusCode, "lockTs" -> lockTs
- ).toString
- }
-
- def checkProperty(key: String): Boolean = propsWithTs.containsKey(key)
-
- def copyEdge(srcVertex: Vertex = srcVertex,
- tgtVertex: Vertex = tgtVertex,
- innerLabel: Label = innerLabel,
- dir: Int = dir,
- op: Byte = op,
- version: Long = version,
- propsWithTs: State = Edge.propsToState(this.propsWithTs),
- parentEdges: Seq[EdgeWithScore] = parentEdges,
- originalEdgeOpt: Option[Edge] = originalEdgeOpt,
- pendingEdgeOpt: Option[Edge] = pendingEdgeOpt,
- statusCode: Byte = statusCode,
- lockTs: Option[Long] = lockTs,
- tsInnerValOpt: Option[InnerValLike] = tsInnerValOpt,
- ts: Long = ts): Edge = {
- val edge = new Edge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, Edge.EmptyProps,
- parentEdges, originalEdgeOpt, pendingEdgeOpt, statusCode, lockTs, tsInnerValOpt)
- Edge.fillPropsWithTs(edge, propsWithTs)
- edge.property(LabelMeta.timestamp.name, ts, ts)
- edge
- }
-
- def copyEdgeWithState(state: State, ts: Long): Edge = {
- val newEdge = copy(propsWithTs = Edge.EmptyProps)
- Edge.fillPropsWithTs(newEdge, state)
- newEdge.property(LabelMeta.timestamp.name, ts, ts)
- newEdge
- }
-
- def copyEdgeWithState(state: State): Edge = {
- val newEdge = copy(propsWithTs = Edge.EmptyProps)
- Edge.fillPropsWithTs(newEdge, state)
- newEdge
- }
-
- override def vertices(direction: Direction): util.Iterator[structure.Vertex] = ???
-
- override def properties[V](strings: String*): util.Iterator[Property[V]] = ???
-
- override def property[V](key: String): Property[V] = {
- val labelMeta = innerLabel.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on Edge."))
- if (propsWithTs.containsKey(key)) propsWithTs.get(key).asInstanceOf[Property[V]]
- else {
- val default = innerLabel.metaPropsDefaultMapInner(labelMeta)
- property(key, default.innerVal.value, default.ts).asInstanceOf[Property[V]]
- }
- }
-
- override def property[V](key: String, value: V): Property[V] = {
- property(key, value, System.currentTimeMillis())
- }
-
- def property[V](key: String, value: V, ts: Long): Property[V] = {
- val labelMeta = innerLabel.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on Edge."))
- val newProp = new S2Property[V](this, labelMeta, key, value, ts)
- propsWithTs.put(key, newProp)
- newProp
- }
-
- override def remove(): Unit = {}
-
- override def graph(): TpGraph = innerGraph
-
- override def id(): AnyRef = (srcVertex.innerId, labelWithDir, tgtVertex.innerId)
-
- override def label(): String = innerLabel.label
-}
-
-
-case class EdgeMutate(edgesToDelete: List[IndexEdge] = List.empty[IndexEdge],
- edgesToInsert: List[IndexEdge] = List.empty[IndexEdge],
- newSnapshotEdge: Option[SnapshotEdge] = None) {
-
- def toLogString: String = {
- val l = (0 until 50).map(_ => "-").mkString("")
- val deletes = s"deletes: ${edgesToDelete.map(e => e.toLogString).mkString("\n")}"
- val inserts = s"inserts: ${edgesToInsert.map(e => e.toLogString).mkString("\n")}"
- val updates = s"snapshot: ${newSnapshotEdge.map(e => e.toLogString).mkString("\n")}"
-
- List("\n", l, deletes, inserts, updates, l, "\n").mkString("\n")
- }
-}
-
-object Edge {
- val incrementVersion = 1L
- val minTsVal = 0L
-
- /** now version information is required also **/
- type Props = java.util.Map[String, S2Property[_]]
- type State = Map[LabelMeta, InnerValLikeWithTs]
- type PropsPairWithTs = (State, State, Long, String)
- type MergeState = PropsPairWithTs => (State, Boolean)
- type UpdateFunc = (Option[Edge], Edge, MergeState)
-
- def EmptyProps = new java.util.HashMap[String, S2Property[_]]
- def EmptyState = Map.empty[LabelMeta, InnerValLikeWithTs]
- def sameProps(base: Props, other: Props): Boolean = {
- if (base.size != other.size) false
- else {
- var ret = true
- val iter = base.entrySet().iterator()
- while (iter.hasNext) {
- val e = iter.next()
- if (!other.containsKey(e.getKey)) ret = false
- else if (e.getValue != other.get(e.getKey)) ret = false
- else {
-
- }
- }
- val otherIter = other.entrySet().iterator()
- while (otherIter.hasNext) {
- val e = otherIter.next()
- if (!base.containsKey(e.getKey)) ret = false
- else if (e.getValue != base.get(e.getKey)) ret = false
- else {
-
- }
- }
- ret
- }
-// base.sameElements(other)
- }
- def fillPropsWithTs(snapshotEdge: SnapshotEdge, state: State): Unit = {
- state.foreach { case (k, v) => snapshotEdge.property(k.name, v.innerVal.value, v.ts) }
- }
- def fillPropsWithTs(indexEdge: IndexEdge, state: State): Unit = {
- state.foreach { case (k, v) => indexEdge.property(k.name, v.innerVal.value, v.ts) }
- }
- def fillPropsWithTs(edge: Edge, state: State): Unit = {
- state.foreach { case (k, v) => edge.property(k.name, v.innerVal.value, v.ts) }
- }
-
- def propsToState(props: Props): State = {
- props.asScala.map { case (k, v) =>
- v.labelMeta -> v.innerValWithTs
- }.toMap
- }
-
- def stateToProps(edge: Edge, state: State): Props = {
- state.foreach { case (k, v) =>
- edge.property(k.name, v.innerVal.value, v.ts)
- }
- edge.propsWithTs
- }
-
- def allPropsDeleted(props: Map[LabelMeta, InnerValLikeWithTs]): Boolean =
- if (!props.contains(LabelMeta.lastDeletedAt)) false
- else {
- val lastDeletedAt = props.get(LabelMeta.lastDeletedAt).get.ts
- val propsWithoutLastDeletedAt = props - LabelMeta.lastDeletedAt
-
- propsWithoutLastDeletedAt.forall { case (_, v) => v.ts <= lastDeletedAt }
- }
-
- def allPropsDeleted(props: Props): Boolean =
- if (!props.containsKey(LabelMeta.lastDeletedAt.name)) false
- else {
- val lastDeletedAt = props.get(LabelMeta.lastDeletedAt.name).ts
- props.remove(LabelMeta.lastDeletedAt.name)
-// val propsWithoutLastDeletedAt = props
-//
-// propsWithoutLastDeletedAt.forall { case (_, v) => v.ts <= lastDeletedAt }
- var ret = true
- val iter = props.entrySet().iterator()
- while (iter.hasNext) {
- val e = iter.next()
- if (e.getValue.ts > lastDeletedAt) ret = false
- }
- ret
- }
-
- def buildDeleteBulk(invertedEdge: Option[Edge], requestEdge: Edge): (Edge, EdgeMutate) = {
- // assert(invertedEdge.isEmpty)
- // assert(requestEdge.op == GraphUtil.operations("delete"))
-
- val edgesToDelete = requestEdge.relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
- val edgeInverted = Option(requestEdge.toSnapshotEdge)
-
- (requestEdge, EdgeMutate(edgesToDelete, edgesToInsert = Nil, newSnapshotEdge = edgeInverted))
- }
-
- def buildOperation(invertedEdge: Option[Edge], requestEdges: Seq[Edge]): (Edge, EdgeMutate) = {
- // logger.debug(s"oldEdge: ${invertedEdge.map(_.toStringRaw)}")
- // logger.debug(s"requestEdge: ${requestEdge.toStringRaw}")
- val oldPropsWithTs =
- if (invertedEdge.isEmpty) Map.empty[LabelMeta, InnerValLikeWithTs]
- else propsToState(invertedEdge.get.propsWithTs)
-
- val funcs = requestEdges.map { edge =>
- if (edge.op == GraphUtil.operations("insert")) {
- edge.innerLabel.consistencyLevel match {
- case "strong" => Edge.mergeUpsert _
- case _ => Edge.mergeInsertBulk _
- }
- } else if (edge.op == GraphUtil.operations("insertBulk")) {
- Edge.mergeInsertBulk _
- } else if (edge.op == GraphUtil.operations("delete")) {
- edge.innerLabel.consistencyLevel match {
- case "strong" => Edge.mergeDelete _
- case _ => throw new RuntimeException("not supported")
- }
- }
- else if (edge.op == GraphUtil.operations("update")) Edge.mergeUpdate _
- else if (edge.op == GraphUtil.operations("increment")) Edge.mergeIncrement _
- else throw new RuntimeException(s"not supported operation on edge: $edge")
- }
-
- val oldTs = invertedEdge.map(_.ts).getOrElse(minTsVal)
- val requestWithFuncs = requestEdges.zip(funcs).filter(oldTs != _._1.ts).sortBy(_._1.ts)
-
- if (requestWithFuncs.isEmpty) {
- (requestEdges.head, EdgeMutate())
- } else {
- val requestEdge = requestWithFuncs.last._1
- var prevPropsWithTs = oldPropsWithTs
-
- for {
- (requestEdge, func) <- requestWithFuncs
- } {
- val (_newPropsWithTs, _) = func(prevPropsWithTs, propsToState(requestEdge.propsWithTs), requestEdge.ts, requestEdge.schemaVer)
- prevPropsWithTs = _newPropsWithTs
- // logger.debug(s"${requestEdge.toLogString}\n$oldPropsWithTs\n$prevPropsWithTs\n")
- }
- val requestTs = requestEdge.ts
- /** version should be monotoniously increasing so our RPC mutation should be applied safely */
- val newVersion = invertedEdge.map(e => e.version + incrementVersion).getOrElse(requestTs)
- val maxTs = prevPropsWithTs.map(_._2.ts).max
- val newTs = if (maxTs > requestTs) maxTs else requestTs
- val propsWithTs = prevPropsWithTs ++
- Map(LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(newTs, requestEdge.innerLabel.schemaVersion), newTs))
-
- val edgeMutate = buildMutation(invertedEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
-
- // logger.debug(s"${edgeMutate.toLogString}\n${propsWithTs}")
- // logger.error(s"$propsWithTs")
- val newEdge = requestEdge.copy(propsWithTs = EmptyProps)
- fillPropsWithTs(newEdge, propsWithTs)
- (newEdge, edgeMutate)
- }
- }
-
- def filterOutWithLabelOption(ls: Seq[IndexEdge]): Seq[IndexEdge] = ls.filter { ie =>
- ie.labelIndex.dir match {
- case None =>
- // both direction use same indices that is defined when label creation.
- true
- case Some(dir) =>
- if (dir != ie.dir) {
- // current labelIndex's direction is different with indexEdge's direction so don't touch
- false
- } else {
- ie.labelIndex.writeOption.map { option =>
- val hashValueOpt = ie.orders.find { case (k, v) => k == LabelMeta.fromHash }.map{ case (k, v) => v.value.toString.toLong }
- option.sample(ie, hashValueOpt)
- }.getOrElse(true)
- }
- }
- }
-
- def buildMutation(snapshotEdgeOpt: Option[Edge],
- requestEdge: Edge,
- newVersion: Long,
- oldPropsWithTs: Map[LabelMeta, InnerValLikeWithTs],
- newPropsWithTs: Map[LabelMeta, InnerValLikeWithTs]): EdgeMutate = {
-
- if (oldPropsWithTs == newPropsWithTs) {
- // all requests should be dropped. so empty mutation.
- EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = None)
- } else {
- val withOutDeletedAt = newPropsWithTs.filter(kv => kv._1 != LabelMeta.lastDeletedAtSeq)
- val newOp = snapshotEdgeOpt match {
- case None => requestEdge.op
- case Some(old) =>
- val oldMaxTs = old.propsWithTs.asScala.map(_._2.ts).max
- if (oldMaxTs > requestEdge.ts) old.op
- else requestEdge.op
- }
-
- val newSnapshotEdge = requestEdge.copy(op = newOp, version = newVersion).copyEdgeWithState(newPropsWithTs)
-
- val newSnapshotEdgeOpt = Option(newSnapshotEdge.toSnapshotEdge)
- // delete request must always update snapshot.
- if (withOutDeletedAt == oldPropsWithTs && newPropsWithTs.contains(LabelMeta.lastDeletedAt)) {
- // no mutation on indexEdges. only snapshotEdge should be updated to record lastDeletedAt.
- EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = newSnapshotEdgeOpt)
- } else {
- val edgesToDelete = snapshotEdgeOpt match {
- case Some(snapshotEdge) if snapshotEdge.op != GraphUtil.operations("delete") =>
- snapshotEdge.copy(op = GraphUtil.defaultOpByte)
- .relatedEdges.flatMap { relEdge => filterOutWithLabelOption(relEdge.edgesWithIndexValid) }
- case _ => Nil
- }
-
- val edgesToInsert =
- if (newPropsWithTs.isEmpty || allPropsDeleted(newPropsWithTs)) Nil
- else {
- val newEdge = requestEdge.copy(
- version = newVersion,
- propsWithTs = Edge.EmptyProps,
- op = GraphUtil.defaultOpByte
- )
- newPropsWithTs.foreach { case (k, v) => newEdge.property(k.name, v.innerVal.value, v.ts) }
-
- newEdge.relatedEdges.flatMap { relEdge => filterOutWithLabelOption(relEdge.edgesWithIndexValid) }
- }
-
-
- EdgeMutate(edgesToDelete = edgesToDelete,
- edgesToInsert = edgesToInsert,
- newSnapshotEdge = newSnapshotEdgeOpt)
- }
- }
- }
-
- def mergeUpsert(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
- var shouldReplace = false
- val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
- val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
- val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
- propsWithTs.get(k) match {
- case Some(newValWithTs) =>
- assert(oldValWithTs.ts >= lastDeletedAt)
- val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
- else {
- shouldReplace = true
- newValWithTs
- }
- Some(k -> v)
-
- case None =>
- assert(oldValWithTs.ts >= lastDeletedAt)
- if (oldValWithTs.ts >= requestTs || k.seq < 0) Some(k -> oldValWithTs)
- else {
- shouldReplace = true
- None
- }
- }
- }
- val existInNew =
- for {
- (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
- } yield {
- shouldReplace = true
- Some(k -> newValWithTs)
- }
-
- ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
- }
-
- def mergeUpdate(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
- var shouldReplace = false
- val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
- val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
- val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
- propsWithTs.get(k) match {
- case Some(newValWithTs) =>
- assert(oldValWithTs.ts >= lastDeletedAt)
- val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
- else {
- shouldReplace = true
- newValWithTs
- }
- Some(k -> v)
- case None =>
- // important: update need to merge previous valid values.
- assert(oldValWithTs.ts >= lastDeletedAt)
- Some(k -> oldValWithTs)
- }
- }
- val existInNew = for {
- (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
- } yield {
- shouldReplace = true
- Some(k -> newValWithTs)
- }
-
- ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
- }
-
- def mergeIncrement(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
- var shouldReplace = false
- val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
- val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
- val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
- propsWithTs.get(k) match {
- case Some(newValWithTs) =>
- if (k == LabelMeta.timestamp) {
- val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
- else {
- shouldReplace = true
- newValWithTs
- }
- Some(k -> v)
- } else {
- if (oldValWithTs.ts >= newValWithTs.ts) {
- Some(k -> oldValWithTs)
- } else {
- assert(oldValWithTs.ts < newValWithTs.ts && oldValWithTs.ts >= lastDeletedAt)
- shouldReplace = true
- // incr(t0), incr(t2), d(t1) => deleted
- Some(k -> InnerValLikeWithTs(oldValWithTs.innerVal + newValWithTs.innerVal, oldValWithTs.ts))
- }
- }
-
- case None =>
- assert(oldValWithTs.ts >= lastDeletedAt)
- Some(k -> oldValWithTs)
- // if (oldValWithTs.ts >= lastDeletedAt) Some(k -> oldValWithTs) else None
- }
- }
- val existInNew = for {
- (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
- } yield {
- shouldReplace = true
- Some(k -> newValWithTs)
- }
-
- ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
- }
-
- def mergeDelete(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
- var shouldReplace = false
- val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
- val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt) match {
- case Some(prevDeletedAt) =>
- if (prevDeletedAt.ts >= requestTs) prevDeletedAt.ts
- else {
- shouldReplace = true
- requestTs
- }
- case None => {
- shouldReplace = true
- requestTs
- }
- }
- val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
- if (k == LabelMeta.timestamp) {
- if (oldValWithTs.ts >= requestTs) Some(k -> oldValWithTs)
- else {
- shouldReplace = true
- Some(k -> InnerValLikeWithTs.withLong(requestTs, requestTs, version))
- }
- } else {
- if (oldValWithTs.ts >= lastDeletedAt) Some(k -> oldValWithTs)
- else {
- shouldReplace = true
- None
- }
- }
- }
- val mustExistInNew = Map(LabelMeta.lastDeletedAt -> InnerValLikeWithTs.withLong(lastDeletedAt, lastDeletedAt, version))
- ((existInOld.flatten ++ mustExistInNew).toMap, shouldReplace)
- }
-
- def mergeInsertBulk(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
- val (_, propsWithTs, _, _) = propsPairWithTs
- (propsWithTs, true)
- }
-
-// def fromString(s: String): Option[Edge] = Graph.toEdge(s)
-
-
-}
[07/10] incubator-s2graph git commit: add optimization parameter for
S2Graph#vertices.
Posted by st...@apache.org.
add optimization parameter for S2Graph#vertices.
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/42e4fa13
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/42e4fa13
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/42e4fa13
Branch: refs/heads/master
Commit: 42e4fa1311a9bf883919a0f96d97e81bccb5cd3b
Parents: 7310631
Author: DO YUNG YOON <st...@apache.org>
Authored: Mon Nov 28 23:52:58 2016 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Mon Nov 28 23:52:58 2016 +0900
----------------------------------------------------------------------
.../scala/org/apache/s2graph/core/S2Graph.scala | 26 ++++++++++++++++----
1 file changed, 21 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/42e4fa13/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
index 51a80f9..b3f3ac8 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
@@ -1336,17 +1336,33 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
}
+ /**
+ * used by graph.traversal().V()
+ * @param vertexIds: array of VertexId values. note that last parameter can be used to control if actually fetch vertices from storage or not.
+ * since S2Graph use user-provided id as part of edge, it is possible to
+ * fetch edge without fetch start vertex. default is false which means we are not fetching vertices from storage.
+ * @return
+ */
override def vertices(vertexIds: AnyRef*): util.Iterator[structure.Vertex] = {
+ val fetchVertices = vertexIds.lastOption.map { lastParam =>
+ if (lastParam.isInstanceOf[Boolean]) lastParam.asInstanceOf[Boolean]
+ else false
+ }.getOrElse(false)
+
val vertices = for {
vertexId <- vertexIds if vertexId.isInstanceOf[VertexId]
} yield newVertex(vertexId.asInstanceOf[VertexId])
- val future = getVertices(vertices).map { vs =>
- val ls = new util.ArrayList[structure.Vertex]()
- ls.addAll(vs)
- ls.iterator()
+ if (fetchVertices) {
+ val future = getVertices(vertices).map { vs =>
+ val ls = new util.ArrayList[structure.Vertex]()
+ ls.addAll(vs)
+ ls.iterator()
+ }
+ Await.result(future, WaitTimeout)
+ } else {
+ vertices.iterator
}
- Await.result(future, WaitTimeout)
}
override def tx(): Transaction = ???
[08/10] incubator-s2graph git commit: Merge branch 'master' into
S2GRAPH-131
Posted by st...@apache.org.
Merge branch 'master' into S2GRAPH-131
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/a08df114
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/a08df114
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/a08df114
Branch: refs/heads/master
Commit: a08df114e81fcf262b34a3c2125481e6d8587493
Parents: 42e4fa1 f74c224
Author: DO YUNG YOON <st...@apache.org>
Authored: Wed Nov 30 21:10:48 2016 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Wed Nov 30 21:10:48 2016 +0900
----------------------------------------------------------------------
CHANGES | 4 ++++
s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala | 2 +-
2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/a08df114/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
----------------------------------------------------------------------
diff --cc s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
index cd1e7f0,0000000..5a8408d
mode 100644,000000..100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
@@@ -1,969 -1,0 +1,969 @@@
+/*
+ * 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.s2graph.core
+
+import java.util
+import java.util.function.{Consumer, BiConsumer}
+
+import org.apache.s2graph.core.S2Edge.{Props, State}
+import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
+import org.apache.s2graph.core.types._
+import org.apache.s2graph.core.utils.logger
+import org.apache.tinkerpop.gremlin.structure
+import org.apache.tinkerpop.gremlin.structure.{Edge, Graph, Vertex, Direction, Property}
+import play.api.libs.json.{JsNumber, JsObject, Json}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{Map => MutableMap}
+import scala.util.hashing.MurmurHash3
+
+case class SnapshotEdge(graph: S2Graph,
+ srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ label: Label,
+ dir: Int,
+ op: Byte,
+ version: Long,
+ private val propsWithTs: Props,
+ pendingEdgeOpt: Option[S2Edge],
+ statusCode: Byte = 0,
+ lockTs: Option[Long],
+ tsInnerValOpt: Option[InnerValLike] = None) {
+ lazy val direction = GraphUtil.fromDirection(dir)
+ lazy val operation = GraphUtil.fromOp(op)
+ lazy val edge = toEdge
+ lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
+// if (!propsWithTs.contains(LabelMeta.timestamp.name)) throw new Exception("Timestamp is required.")
+
+// val label = Label.findById(labelWithDir.labelId)
+ lazy val schemaVer = label.schemaVersion
+ lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.toString().toLong
+
+ def propsToKeyValuesWithTs = HBaseSerializable.propsToKeyValuesWithTs(propsWithTs.asScala.map(kv => kv._2.labelMeta.seq -> kv._2.innerValWithTs).toSeq)
+
+ def allPropsDeleted = S2Edge.allPropsDeleted(propsWithTs)
+
+ def toEdge: S2Edge = {
+ S2Edge(graph, srcVertex, tgtVertex, label, dir, op,
+ version, propsWithTs, pendingEdgeOpt = pendingEdgeOpt,
+ statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ def propsWithName = (for {
+ (_, v) <- propsWithTs.asScala
+ meta = v.labelMeta
+ jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
+ } yield meta.name -> jsValue) ++ Map("version" -> JsNumber(version))
+
+ // only for debug
+ def toLogString() = {
+ List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, propsWithName).mkString("\t")
+ }
+
+ def property[V](key: String, value: V, ts: Long): S2Property[V] = {
+ val labelMeta = label.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on IndexEdge."))
+ val newProps = new S2Property(edge, labelMeta, key, value, ts)
+ propsWithTs.put(key, newProps)
+ newProps
+ }
+ override def hashCode(): Int = {
+ MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId)
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case e: SnapshotEdge =>
+ srcVertex.innerId == e.srcVertex.innerId &&
+ tgtVertex.innerId == e.tgtVertex.innerId &&
+ labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
+ pendingEdgeOpt == e.pendingEdgeOpt && lockTs == lockTs && statusCode == statusCode
+ case _ => false
+ }
+
+ override def toString(): String = {
+ Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> label.label, "direction" -> direction,
+ "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString,
+ "statusCode" -> statusCode, "lockTs" -> lockTs).toString
+ }
+}
+
+case class IndexEdge(graph: S2Graph,
+ srcVertex: S2Vertex,
+ tgtVertex: S2Vertex,
+ label: Label,
+ dir: Int,
+ op: Byte,
+ version: Long,
+ labelIndexSeq: Byte,
+ private val propsWithTs: Props,
+ tsInnerValOpt: Option[InnerValLike] = None) {
+// if (!props.contains(LabelMeta.timeStampSeq)) throw new Exception("Timestamp is required.")
+ // assert(props.contains(LabelMeta.timeStampSeq))
+ lazy val direction = GraphUtil.fromDirection(dir)
+ lazy val operation = GraphUtil.fromOp(op)
+ lazy val edge = toEdge
+ lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
+
+ lazy val isInEdge = labelWithDir.dir == GraphUtil.directions("in")
+ lazy val isOutEdge = !isInEdge
+
+ lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.toString.toLong
+ lazy val degreeEdge = propsWithTs.containsKey(LabelMeta.degree.name)
+
+ lazy val schemaVer = label.schemaVersion
+ lazy val labelIndex = LabelIndex.findByLabelIdAndSeq(labelWithDir.labelId, labelIndexSeq).get
+ lazy val defaultIndexMetas = labelIndex.sortKeyTypes.map { meta =>
+ val innerVal = toInnerVal(meta.defaultValue, meta.dataType, schemaVer)
+ meta.seq -> innerVal
+ }.toMap
+
+ lazy val labelIndexMetaSeqs = labelIndex.sortKeyTypes
+
+ /** TODO: make sure call of this class fill props as this assumes */
+ lazy val orders = for (meta <- labelIndexMetaSeqs) yield {
+ propsWithTs.get(meta.name) match {
+ case null =>
+
+ /**
+ * TODO: agly hack
+ * now we double store target vertex.innerId/srcVertex.innerId for easy development. later fix this to only store id once
+ */
+ val v = meta match {
+ case LabelMeta.timestamp=> InnerVal.withLong(version, schemaVer)
+ case LabelMeta.to => toEdge.tgtVertex.innerId
+ case LabelMeta.from => toEdge.srcVertex.innerId
+ // for now, it does not make sense to build index on srcVertex.innerId since all edges have same data.
+ // throw new RuntimeException("_from on indexProps is not supported")
+ case _ => toInnerVal(meta.defaultValue, meta.dataType, schemaVer)
+ }
+
+ meta -> v
+ case v => meta -> v.innerVal
+ }
+ }
+
+ lazy val ordersKeyMap = orders.map { case (meta, _) => meta.name }.toSet
+ lazy val metas = for ((meta, v) <- propsWithTs.asScala if !ordersKeyMap.contains(meta)) yield v.labelMeta -> v.innerVal
+
+// lazy val propsWithTs = props.map { case (k, v) => k -> InnerValLikeWithTs(v, version) }
+
+ //TODO:
+ // lazy val kvs = Graph.client.indexedEdgeSerializer(this).toKeyValues.toList
+
+ lazy val hasAllPropsForIndex = orders.length == labelIndexMetaSeqs.length
+
+ def propsWithName = for {
+ (_, v) <- propsWithTs.asScala
+ meta = v.labelMeta
+ jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
+ } yield meta.name -> jsValue
+
+
+ def toEdge: S2Edge = S2Edge(graph, srcVertex, tgtVertex, label, dir, op, version, propsWithTs, tsInnerValOpt = tsInnerValOpt)
+
+ // only for debug
+ def toLogString() = {
+ List(version, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, Json.toJson(propsWithName)).mkString("\t")
+ }
+
+ def property(key: String): Option[InnerValLikeWithTs] = {
+ label.metaPropsInvMap.get(key).map(labelMeta => property(labelMeta))
+ }
+
+ def property(labelMeta: LabelMeta): InnerValLikeWithTs = {
+// propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse(label.metaPropsDefaultMapInner(labelMeta))
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ propsWithTs.get(labelMeta.name).innerValWithTs
+ } else {
+ label.metaPropsDefaultMapInner(labelMeta)
+ }
+ }
+
+ def updatePropsWithTs(others: Props = S2Edge.EmptyProps): Props = {
+ if (others.isEmpty) propsWithTs
+ else {
+ val iter = others.entrySet().iterator()
+ while (iter.hasNext) {
+ val e = iter.next()
+ propsWithTs.put(e.getKey, e.getValue)
+ }
+ propsWithTs
+ }
+ }
+
+ def property[V](key: String, value: V, ts: Long): S2Property[V] = {
+ val labelMeta = label.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on IndexEdge."))
+ val newProps = new S2Property(edge, labelMeta, key, value, ts)
+ propsWithTs.put(key, newProps)
+ newProps
+ }
+ override def hashCode(): Int = {
+ MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId + "," + labelIndexSeq)
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case e: IndexEdge =>
+ srcVertex.innerId == e.srcVertex.innerId &&
+ tgtVertex.innerId == e.tgtVertex.innerId &&
+ labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
+ labelIndexSeq == e.labelIndexSeq
+ case _ => false
+ }
+
+ override def toString(): String = {
+ Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> label.label, "direction" -> dir,
+ "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString
+ ).toString
+ }
+}
+
+case class S2Edge(innerGraph: S2Graph,
+ srcVertex: S2Vertex,
+ var tgtVertex: S2Vertex,
+ innerLabel: Label,
+ dir: Int,
+ var op: Byte = GraphUtil.defaultOpByte,
+ var version: Long = System.currentTimeMillis(),
+ propsWithTs: Props = S2Edge.EmptyProps,
+ parentEdges: Seq[EdgeWithScore] = Nil,
+ originalEdgeOpt: Option[S2Edge] = None,
+ pendingEdgeOpt: Option[S2Edge] = None,
+ statusCode: Byte = 0,
+ lockTs: Option[Long] = None,
+ var tsInnerValOpt: Option[InnerValLike] = None) extends GraphElement with Edge {
+
+ lazy val labelWithDir = LabelWithDirection(innerLabel.id.get, dir)
+ lazy val schemaVer = innerLabel.schemaVersion
+ lazy val ts = propsWithTs.get(LabelMeta.timestamp.name).innerVal.value match {
+ case b: BigDecimal => b.longValue()
+ case l: Long => l
+ case i: Int => i.toLong
+ case _ => throw new RuntimeException("ts should be in [BigDecimal/Long/Int].")
+ }
+
+ lazy val operation = GraphUtil.fromOp(op)
+ lazy val tsInnerVal = tsInnerValOpt.get.value
+ lazy val srcId = srcVertex.innerIdVal
+ lazy val tgtId = tgtVertex.innerIdVal
+ lazy val labelName = innerLabel.label
+ lazy val direction = GraphUtil.fromDirection(dir)
+
+ def toIndexEdge(labelIndexSeq: Byte): IndexEdge = IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelIndexSeq, propsWithTs)
+
+ def serializePropsWithTs(): Array[Byte] = HBaseSerializable.propsToKeyValuesWithTs(propsWithTs.asScala.map(kv => kv._2.labelMeta.seq -> kv._2.innerValWithTs).toSeq)
+
+ def updatePropsWithTs(others: Props = S2Edge.EmptyProps): Props = {
+ val emptyProp = S2Edge.EmptyProps
+
+ propsWithTs.forEach(new BiConsumer[String, S2Property[_]] {
+ override def accept(key: String, value: S2Property[_]): Unit = emptyProp.put(key, value)
+ })
+
+ others.forEach(new BiConsumer[String, S2Property[_]] {
+ override def accept(key: String, value: S2Property[_]): Unit = emptyProp.put(key, value)
+ })
+
+ emptyProp
+ }
+
+ def propertyValue(key: String): Option[InnerValLikeWithTs] = {
+ key match {
+ case "from" | "_from" => Option(InnerValLikeWithTs(srcVertex.innerId, ts))
+ case "to" | "_to" => Option(InnerValLikeWithTs(tgtVertex.innerId, ts))
+ case "label" => Option(InnerValLikeWithTs(InnerVal.withStr(innerLabel.label, schemaVer), ts))
+ case "direction" => Option(InnerValLikeWithTs(InnerVal.withStr(direction, schemaVer), ts))
+ case _ =>
+ innerLabel.metaPropsInvMap.get(key).map(labelMeta => propertyValueInner(labelMeta))
+ }
+ }
+
+ def propertyValueInner(labelMeta: LabelMeta): InnerValLikeWithTs= {
+ // propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse()
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ propsWithTs.get(labelMeta.name).innerValWithTs
+ } else {
+ innerLabel.metaPropsDefaultMapInner(labelMeta)
+ }
+ }
+
+ def propertyValues(keys: Seq[String] = Nil): Map[LabelMeta, InnerValLikeWithTs] = {
+ val labelMetas = for {
+ key <- keys
+ labelMeta <- innerLabel.metaPropsInvMap.get(key)
+ } yield labelMeta
+
+ propertyValuesInner(labelMetas)
+ }
+
+ def propertyValuesInner(labelMetas: Seq[LabelMeta] = Nil): Map[LabelMeta, InnerValLikeWithTs] = {
+ if (labelMetas.isEmpty) {
+ innerLabel.metaPropsDefaultMapInner.map { case (labelMeta, defaultVal) =>
+ labelMeta -> propertyValueInner(labelMeta)
+ }
+ } else {
+ // This is important since timestamp is required for all edges.
+ (LabelMeta.timestamp +: labelMetas).map { labelMeta =>
+ labelMeta -> propertyValueInner(labelMeta)
+ }.toMap
+ }
+ }
+
+// if (!props.contains(LabelMeta.timestamp)) throw new Exception("Timestamp is required.")
+ // assert(propsWithTs.contains(LabelMeta.timeStampSeq))
+
+ lazy val properties = toProps()
+
+ def props = propsWithTs.asScala.mapValues(_.innerVal)
+
+
+ private def toProps(): Map[String, Any] = {
+ for {
+ (labelMeta, defaultVal) <- innerLabel.metaPropsDefaultMapInner
+ } yield {
+ // labelMeta.name -> propsWithTs.get(labelMeta.name).map(_.innerValWithTs).getOrElse(defaultVal).innerVal.value
+ val value =
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ propsWithTs.get(labelMeta.name).value
+ } else {
+ defaultVal.innerVal.value
+ }
+ labelMeta.name -> value
+ }
+ }
+
+ def relatedEdges = {
+ if (labelWithDir.isDirected) {
+ val skipReverse = innerLabel.extraOptions.get("skipReverse").map(_.as[Boolean]).getOrElse(false)
+ if (skipReverse) List(this) else List(this, duplicateEdge)
+ } else {
+// val outDir = labelWithDir.copy(dir = GraphUtil.directions("out"))
+// val base = copy(labelWithDir = outDir)
+ val base = copy(dir = GraphUtil.directions("out"))
+ List(base, base.reverseSrcTgtEdge)
+ }
+ }
+
+ // def relatedEdges = List(this)
+
+ def srcForVertex = {
+ val belongLabelIds = Seq(labelWithDir.labelId)
+ if (labelWithDir.dir == GraphUtil.directions("in")) {
+ innerGraph.newVertex(VertexId(innerLabel.tgtColumn, tgtVertex.innerId), tgtVertex.ts, tgtVertex.props, belongLabelIds = belongLabelIds)
+ } else {
+ innerGraph.newVertex(VertexId(innerLabel.srcColumn, srcVertex.innerId), srcVertex.ts, srcVertex.props, belongLabelIds = belongLabelIds)
+ }
+ }
+
+ def tgtForVertex = {
+ val belongLabelIds = Seq(labelWithDir.labelId)
+ if (labelWithDir.dir == GraphUtil.directions("in")) {
+ innerGraph.newVertex(VertexId(innerLabel.srcColumn, srcVertex.innerId), srcVertex.ts, srcVertex.props, belongLabelIds = belongLabelIds)
+ } else {
+ innerGraph.newVertex(VertexId(innerLabel.tgtColumn, tgtVertex.innerId), tgtVertex.ts, tgtVertex.props, belongLabelIds = belongLabelIds)
+ }
+ }
+
+ def duplicateEdge = reverseSrcTgtEdge.reverseDirEdge
+
+// def reverseDirEdge = copy(labelWithDir = labelWithDir.dirToggled)
+ def reverseDirEdge = copy(dir = GraphUtil.toggleDir(dir))
+
+ def reverseSrcTgtEdge = copy(srcVertex = tgtVertex, tgtVertex = srcVertex)
+
+ def labelOrders = LabelIndex.findByLabelIdAll(labelWithDir.labelId)
+
+ override def serviceName = innerLabel.serviceName
+
+ override def queueKey = Seq(ts.toString, tgtVertex.serviceName).mkString("|")
+
+ override def queuePartitionKey = Seq(srcVertex.innerId, tgtVertex.innerId).mkString("|")
+
+ override def isAsync = innerLabel.isAsync
+
+ def isDegree = propsWithTs.containsKey(LabelMeta.degree.name)
+
+// def propsPlusTs = propsWithTs.get(LabelMeta.timeStampSeq) match {
+// case Some(_) => props
+// case None => props ++ Map(LabelMeta.timeStampSeq -> InnerVal.withLong(ts, schemaVer))
+// }
+
+ def propsPlusTsValid = propsWithTs.asScala.filter(kv => LabelMeta.isValidSeq(kv._2.labelMeta.seq)).asJava
+
+ def edgesWithIndex = for (labelOrder <- labelOrders) yield {
+ IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelOrder.seq, propsWithTs, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ def edgesWithIndexValid = for (labelOrder <- labelOrders) yield {
+ IndexEdge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, labelOrder.seq, propsPlusTsValid, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ /** force direction as out on invertedEdge */
+ def toSnapshotEdge: SnapshotEdge = {
+ val (smaller, larger) = (srcForVertex, tgtForVertex)
+
+// val newLabelWithDir = LabelWithDirection(labelWithDir.labelId, GraphUtil.directions("out"))
+
+ property(LabelMeta.timestamp.name, ts, ts)
+ val ret = SnapshotEdge(innerGraph, smaller, larger, innerLabel,
+ GraphUtil.directions("out"), op, version, propsWithTs,
+ pendingEdgeOpt = pendingEdgeOpt, statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
+ ret
+ }
+
+ def defaultPropsWithName = Json.obj("from" -> srcVertex.innerId.toString(), "to" -> tgtVertex.innerId.toString(),
+ "label" -> innerLabel.label, "service" -> innerLabel.serviceName)
+
+ def propsWithName =
+ for {
+ (_, v) <- propsWithTs.asScala
+ meta = v.labelMeta
+ jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
+ } yield meta.name -> jsValue
+
+
+ def updateTgtVertex(id: InnerValLike) = {
+ val newId = TargetVertexId(tgtVertex.id.column, id)
+ val newTgtVertex = innerGraph.newVertex(newId, tgtVertex.ts, tgtVertex.props)
+ S2Edge(innerGraph, srcVertex, newTgtVertex, innerLabel, dir, op, version, propsWithTs, tsInnerValOpt = tsInnerValOpt)
+ }
+
+ def rank(r: RankParam): Double =
+ if (r.keySeqAndWeights.size <= 0) 1.0f
+ else {
+ var sum: Double = 0
+
+ for ((labelMeta, w) <- r.keySeqAndWeights) {
+ if (propsWithTs.containsKey(labelMeta.name)) {
+ val innerValWithTs = propsWithTs.get(labelMeta.name)
+ val cost = try innerValWithTs.innerVal.toString.toDouble catch {
+ case e: Exception =>
+ logger.error("toInnerval failed in rank", e)
+ 1.0
+ }
+ sum += w * cost
+ }
+ }
+ sum
+ }
+
+ def toLogString: String = {
+ val allPropsWithName = defaultPropsWithName ++ Json.toJson(propsWithName).asOpt[JsObject].getOrElse(Json.obj())
+ List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, innerLabel.label, allPropsWithName).mkString("\t")
+ }
+
+ override def hashCode(): Int = {
+ MurmurHash3.stringHash(srcVertex.innerId + "," + labelWithDir + "," + tgtVertex.innerId)
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case e: S2Edge =>
+ srcVertex.innerId == e.srcVertex.innerId &&
+ tgtVertex.innerId == e.tgtVertex.innerId &&
+ labelWithDir == e.labelWithDir && op == e.op && version == e.version &&
+ pendingEdgeOpt == e.pendingEdgeOpt && lockTs == lockTs && statusCode == statusCode &&
+ parentEdges == e.parentEdges && originalEdgeOpt == originalEdgeOpt
+ case _ => false
+ }
+
+ override def toString(): String = {
+ Map("srcVertex" -> srcVertex.toString, "tgtVertex" -> tgtVertex.toString, "label" -> labelName, "direction" -> direction,
+ "operation" -> operation, "version" -> version, "props" -> propsWithTs.asScala.map(kv => kv._1 -> kv._2.value).toString,
+ "parentEdges" -> parentEdges, "originalEdge" -> originalEdgeOpt, "statusCode" -> statusCode, "lockTs" -> lockTs
+ ).toString
+ }
+
+ def checkProperty(key: String): Boolean = propsWithTs.containsKey(key)
+
+ def copyEdge(srcVertex: S2Vertex = srcVertex,
+ tgtVertex: S2Vertex = tgtVertex,
+ innerLabel: Label = innerLabel,
+ dir: Int = dir,
+ op: Byte = op,
+ version: Long = version,
+ propsWithTs: State = S2Edge.propsToState(this.propsWithTs),
+ parentEdges: Seq[EdgeWithScore] = parentEdges,
+ originalEdgeOpt: Option[S2Edge] = originalEdgeOpt,
+ pendingEdgeOpt: Option[S2Edge] = pendingEdgeOpt,
+ statusCode: Byte = statusCode,
+ lockTs: Option[Long] = lockTs,
+ tsInnerValOpt: Option[InnerValLike] = tsInnerValOpt,
+ ts: Long = ts): S2Edge = {
+ val edge = new S2Edge(innerGraph, srcVertex, tgtVertex, innerLabel, dir, op, version, S2Edge.EmptyProps,
+ parentEdges, originalEdgeOpt, pendingEdgeOpt, statusCode, lockTs, tsInnerValOpt)
+ S2Edge.fillPropsWithTs(edge, propsWithTs)
+ edge.property(LabelMeta.timestamp.name, ts, ts)
+ edge
+ }
+
+ def copyEdgeWithState(state: State, ts: Long): S2Edge = {
+ val newEdge = copy(propsWithTs = S2Edge.EmptyProps)
+ S2Edge.fillPropsWithTs(newEdge, state)
+ newEdge.property(LabelMeta.timestamp.name, ts, ts)
+ newEdge
+ }
+
+ def copyEdgeWithState(state: State): S2Edge = {
+ val newEdge = copy(propsWithTs = S2Edge.EmptyProps)
+ S2Edge.fillPropsWithTs(newEdge, state)
+ newEdge
+ }
+
+ override def vertices(direction: Direction): util.Iterator[structure.Vertex] = {
+ val arr = new util.ArrayList[Vertex]()
+ direction match {
+ case Direction.OUT => arr.add(srcVertex)
+ case Direction.IN => arr.add(tgtVertex)
+ case _ =>
+ arr.add(srcVertex)
+ arr.add(tgtVertex)
+ }
+ arr.iterator()
+ }
+
+ override def properties[V](keys: String*): util.Iterator[Property[V]] = {
+ val ls = new util.ArrayList[Property[V]]()
+ keys.foreach { key => ls.add(property(key)) }
+ ls.iterator()
+ }
+
+ override def property[V](key: String): Property[V] = {
+ val labelMeta = innerLabel.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on Edge."))
+ if (propsWithTs.containsKey(key)) propsWithTs.get(key).asInstanceOf[Property[V]]
+ else {
+ val default = innerLabel.metaPropsDefaultMapInner(labelMeta)
+ property(key, default.innerVal.value, default.ts).asInstanceOf[Property[V]]
+ }
+ }
+
+ override def property[V](key: String, value: V): Property[V] = {
+ property(key, value, System.currentTimeMillis())
+ }
+
+ def property[V](key: String, value: V, ts: Long): Property[V] = {
+ val labelMeta = innerLabel.metaPropsInvMap.getOrElse(key, throw new RuntimeException(s"$key is not configured on Edge."))
+ val newProp = new S2Property[V](this, labelMeta, key, value, ts)
+ propsWithTs.put(key, newProp)
+ newProp
+ }
+
+ override def remove(): Unit = {}
+
+ override def graph(): Graph = innerGraph
+
+ override def id(): AnyRef = (srcVertex.innerId, labelWithDir, tgtVertex.innerId)
+
+ override def label(): String = innerLabel.label
+}
+
+
+case class EdgeMutate(edgesToDelete: List[IndexEdge] = List.empty[IndexEdge],
+ edgesToInsert: List[IndexEdge] = List.empty[IndexEdge],
+ newSnapshotEdge: Option[SnapshotEdge] = None) {
+
+ def toLogString: String = {
+ val l = (0 until 50).map(_ => "-").mkString("")
+ val deletes = s"deletes: ${edgesToDelete.map(e => e.toLogString).mkString("\n")}"
+ val inserts = s"inserts: ${edgesToInsert.map(e => e.toLogString).mkString("\n")}"
+ val updates = s"snapshot: ${newSnapshotEdge.map(e => e.toLogString).mkString("\n")}"
+
+ List("\n", l, deletes, inserts, updates, l, "\n").mkString("\n")
+ }
+}
+
+object S2Edge {
+ val incrementVersion = 1L
+ val minTsVal = 0L
+
+ /** now version information is required also **/
+ type Props = java.util.Map[String, S2Property[_]]
+ type State = Map[LabelMeta, InnerValLikeWithTs]
+ type PropsPairWithTs = (State, State, Long, String)
+ type MergeState = PropsPairWithTs => (State, Boolean)
+ type UpdateFunc = (Option[S2Edge], S2Edge, MergeState)
+
+ def EmptyProps = new java.util.HashMap[String, S2Property[_]]
+ def EmptyState = Map.empty[LabelMeta, InnerValLikeWithTs]
+ def sameProps(base: Props, other: Props): Boolean = {
+ if (base.size != other.size) false
+ else {
+ var ret = true
+ val iter = base.entrySet().iterator()
+ while (iter.hasNext) {
+ val e = iter.next()
+ if (!other.containsKey(e.getKey)) ret = false
+ else if (e.getValue != other.get(e.getKey)) ret = false
+ else {
+
+ }
+ }
+ val otherIter = other.entrySet().iterator()
+ while (otherIter.hasNext) {
+ val e = otherIter.next()
+ if (!base.containsKey(e.getKey)) ret = false
+ else if (e.getValue != base.get(e.getKey)) ret = false
+ else {
+
+ }
+ }
+ ret
+ }
+// base.sameElements(other)
+ }
+ def fillPropsWithTs(snapshotEdge: SnapshotEdge, state: State): Unit = {
+ state.foreach { case (k, v) => snapshotEdge.property(k.name, v.innerVal.value, v.ts) }
+ }
+ def fillPropsWithTs(indexEdge: IndexEdge, state: State): Unit = {
+ state.foreach { case (k, v) => indexEdge.property(k.name, v.innerVal.value, v.ts) }
+ }
+ def fillPropsWithTs(edge: S2Edge, state: State): Unit = {
+ state.foreach { case (k, v) => edge.property(k.name, v.innerVal.value, v.ts) }
+ }
+
+ def propsToState(props: Props): State = {
+ props.asScala.map { case (k, v) =>
+ v.labelMeta -> v.innerValWithTs
+ }.toMap
+ }
+
+ def stateToProps(edge: S2Edge, state: State): Props = {
+ state.foreach { case (k, v) =>
+ edge.property(k.name, v.innerVal.value, v.ts)
+ }
+ edge.propsWithTs
+ }
+
+ def allPropsDeleted(props: Map[LabelMeta, InnerValLikeWithTs]): Boolean =
+ if (!props.contains(LabelMeta.lastDeletedAt)) false
+ else {
+ val lastDeletedAt = props.get(LabelMeta.lastDeletedAt).get.ts
+ val propsWithoutLastDeletedAt = props - LabelMeta.lastDeletedAt
+
+ propsWithoutLastDeletedAt.forall { case (_, v) => v.ts <= lastDeletedAt }
+ }
+
+ def allPropsDeleted(props: Props): Boolean =
+ if (!props.containsKey(LabelMeta.lastDeletedAt.name)) false
+ else {
+ val lastDeletedAt = props.get(LabelMeta.lastDeletedAt.name).ts
+ props.remove(LabelMeta.lastDeletedAt.name)
+// val propsWithoutLastDeletedAt = props
+//
+// propsWithoutLastDeletedAt.forall { case (_, v) => v.ts <= lastDeletedAt }
+ var ret = true
+ val iter = props.entrySet().iterator()
- while (iter.hasNext) {
++ while (iter.hasNext && ret) {
+ val e = iter.next()
+ if (e.getValue.ts > lastDeletedAt) ret = false
+ }
+ ret
+ }
+
+ def buildDeleteBulk(invertedEdge: Option[S2Edge], requestEdge: S2Edge): (S2Edge, EdgeMutate) = {
+ // assert(invertedEdge.isEmpty)
+ // assert(requestEdge.op == GraphUtil.operations("delete"))
+
+ val edgesToDelete = requestEdge.relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
+ val edgeInverted = Option(requestEdge.toSnapshotEdge)
+
+ (requestEdge, EdgeMutate(edgesToDelete, edgesToInsert = Nil, newSnapshotEdge = edgeInverted))
+ }
+
+ def buildOperation(invertedEdge: Option[S2Edge], requestEdges: Seq[S2Edge]): (S2Edge, EdgeMutate) = {
+ // logger.debug(s"oldEdge: ${invertedEdge.map(_.toStringRaw)}")
+ // logger.debug(s"requestEdge: ${requestEdge.toStringRaw}")
+ val oldPropsWithTs =
+ if (invertedEdge.isEmpty) Map.empty[LabelMeta, InnerValLikeWithTs]
+ else propsToState(invertedEdge.get.propsWithTs)
+
+ val funcs = requestEdges.map { edge =>
+ if (edge.op == GraphUtil.operations("insert")) {
+ edge.innerLabel.consistencyLevel match {
+ case "strong" => S2Edge.mergeUpsert _
+ case _ => S2Edge.mergeInsertBulk _
+ }
+ } else if (edge.op == GraphUtil.operations("insertBulk")) {
+ S2Edge.mergeInsertBulk _
+ } else if (edge.op == GraphUtil.operations("delete")) {
+ edge.innerLabel.consistencyLevel match {
+ case "strong" => S2Edge.mergeDelete _
+ case _ => throw new RuntimeException("not supported")
+ }
+ }
+ else if (edge.op == GraphUtil.operations("update")) S2Edge.mergeUpdate _
+ else if (edge.op == GraphUtil.operations("increment")) S2Edge.mergeIncrement _
+ else throw new RuntimeException(s"not supported operation on edge: $edge")
+ }
+
+ val oldTs = invertedEdge.map(_.ts).getOrElse(minTsVal)
+ val requestWithFuncs = requestEdges.zip(funcs).filter(oldTs != _._1.ts).sortBy(_._1.ts)
+
+ if (requestWithFuncs.isEmpty) {
+ (requestEdges.head, EdgeMutate())
+ } else {
+ val requestEdge = requestWithFuncs.last._1
+ var prevPropsWithTs = oldPropsWithTs
+
+ for {
+ (requestEdge, func) <- requestWithFuncs
+ } {
+ val (_newPropsWithTs, _) = func(prevPropsWithTs, propsToState(requestEdge.propsWithTs), requestEdge.ts, requestEdge.schemaVer)
+ prevPropsWithTs = _newPropsWithTs
+ // logger.debug(s"${requestEdge.toLogString}\n$oldPropsWithTs\n$prevPropsWithTs\n")
+ }
+ val requestTs = requestEdge.ts
+ /** version should be monotoniously increasing so our RPC mutation should be applied safely */
+ val newVersion = invertedEdge.map(e => e.version + incrementVersion).getOrElse(requestTs)
+ val maxTs = prevPropsWithTs.map(_._2.ts).max
+ val newTs = if (maxTs > requestTs) maxTs else requestTs
+ val propsWithTs = prevPropsWithTs ++
+ Map(LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(newTs, requestEdge.innerLabel.schemaVersion), newTs))
+
+ val edgeMutate = buildMutation(invertedEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
+
+ // logger.debug(s"${edgeMutate.toLogString}\n${propsWithTs}")
+ // logger.error(s"$propsWithTs")
+ val newEdge = requestEdge.copy(propsWithTs = EmptyProps)
+ fillPropsWithTs(newEdge, propsWithTs)
+ (newEdge, edgeMutate)
+ }
+ }
+
+ def filterOutWithLabelOption(ls: Seq[IndexEdge]): Seq[IndexEdge] = ls.filter { ie =>
+ ie.labelIndex.dir match {
+ case None =>
+ // both direction use same indices that is defined when label creation.
+ true
+ case Some(dir) =>
+ if (dir != ie.dir) {
+ // current labelIndex's direction is different with indexEdge's direction so don't touch
+ false
+ } else {
+ ie.labelIndex.writeOption.map { option =>
+ val hashValueOpt = ie.orders.find { case (k, v) => k == LabelMeta.fromHash }.map{ case (k, v) => v.value.toString.toLong }
+ option.sample(ie, hashValueOpt)
+ }.getOrElse(true)
+ }
+ }
+ }
+
+ def buildMutation(snapshotEdgeOpt: Option[S2Edge],
+ requestEdge: S2Edge,
+ newVersion: Long,
+ oldPropsWithTs: Map[LabelMeta, InnerValLikeWithTs],
+ newPropsWithTs: Map[LabelMeta, InnerValLikeWithTs]): EdgeMutate = {
+
+ if (oldPropsWithTs == newPropsWithTs) {
+ // all requests should be dropped. so empty mutation.
+ EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = None)
+ } else {
+ val withOutDeletedAt = newPropsWithTs.filter(kv => kv._1 != LabelMeta.lastDeletedAtSeq)
+ val newOp = snapshotEdgeOpt match {
+ case None => requestEdge.op
+ case Some(old) =>
+ val oldMaxTs = old.propsWithTs.asScala.map(_._2.ts).max
+ if (oldMaxTs > requestEdge.ts) old.op
+ else requestEdge.op
+ }
+
+ val newSnapshotEdge = requestEdge.copy(op = newOp, version = newVersion).copyEdgeWithState(newPropsWithTs)
+
+ val newSnapshotEdgeOpt = Option(newSnapshotEdge.toSnapshotEdge)
+ // delete request must always update snapshot.
+ if (withOutDeletedAt == oldPropsWithTs && newPropsWithTs.contains(LabelMeta.lastDeletedAt)) {
+ // no mutation on indexEdges. only snapshotEdge should be updated to record lastDeletedAt.
+ EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = newSnapshotEdgeOpt)
+ } else {
+ val edgesToDelete = snapshotEdgeOpt match {
+ case Some(snapshotEdge) if snapshotEdge.op != GraphUtil.operations("delete") =>
+ snapshotEdge.copy(op = GraphUtil.defaultOpByte)
+ .relatedEdges.flatMap { relEdge => filterOutWithLabelOption(relEdge.edgesWithIndexValid) }
+ case _ => Nil
+ }
+
+ val edgesToInsert =
+ if (newPropsWithTs.isEmpty || allPropsDeleted(newPropsWithTs)) Nil
+ else {
+ val newEdge = requestEdge.copy(
+ version = newVersion,
+ propsWithTs = S2Edge.EmptyProps,
+ op = GraphUtil.defaultOpByte
+ )
+ newPropsWithTs.foreach { case (k, v) => newEdge.property(k.name, v.innerVal.value, v.ts) }
+
+ newEdge.relatedEdges.flatMap { relEdge => filterOutWithLabelOption(relEdge.edgesWithIndexValid) }
+ }
+
+
+ EdgeMutate(edgesToDelete = edgesToDelete,
+ edgesToInsert = edgesToInsert,
+ newSnapshotEdge = newSnapshotEdgeOpt)
+ }
+ }
+ }
+
+ def mergeUpsert(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ propsWithTs.get(k) match {
+ case Some(newValWithTs) =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
+ else {
+ shouldReplace = true
+ newValWithTs
+ }
+ Some(k -> v)
+
+ case None =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ if (oldValWithTs.ts >= requestTs || k.seq < 0) Some(k -> oldValWithTs)
+ else {
+ shouldReplace = true
+ None
+ }
+ }
+ }
+ val existInNew =
+ for {
+ (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
+ } yield {
+ shouldReplace = true
+ Some(k -> newValWithTs)
+ }
+
+ ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
+ }
+
+ def mergeUpdate(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ propsWithTs.get(k) match {
+ case Some(newValWithTs) =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
+ else {
+ shouldReplace = true
+ newValWithTs
+ }
+ Some(k -> v)
+ case None =>
+ // important: update need to merge previous valid values.
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ Some(k -> oldValWithTs)
+ }
+ }
+ val existInNew = for {
+ (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
+ } yield {
+ shouldReplace = true
+ Some(k -> newValWithTs)
+ }
+
+ ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
+ }
+
+ def mergeIncrement(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt).map(v => v.ts).getOrElse(minTsVal)
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ propsWithTs.get(k) match {
+ case Some(newValWithTs) =>
+ if (k == LabelMeta.timestamp) {
+ val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
+ else {
+ shouldReplace = true
+ newValWithTs
+ }
+ Some(k -> v)
+ } else {
+ if (oldValWithTs.ts >= newValWithTs.ts) {
+ Some(k -> oldValWithTs)
+ } else {
+ assert(oldValWithTs.ts < newValWithTs.ts && oldValWithTs.ts >= lastDeletedAt)
+ shouldReplace = true
+ // incr(t0), incr(t2), d(t1) => deleted
+ Some(k -> InnerValLikeWithTs(oldValWithTs.innerVal + newValWithTs.innerVal, oldValWithTs.ts))
+ }
+ }
+
+ case None =>
+ assert(oldValWithTs.ts >= lastDeletedAt)
+ Some(k -> oldValWithTs)
+ // if (oldValWithTs.ts >= lastDeletedAt) Some(k -> oldValWithTs) else None
+ }
+ }
+ val existInNew = for {
+ (k, newValWithTs) <- propsWithTs if !oldPropsWithTs.contains(k) && newValWithTs.ts > lastDeletedAt
+ } yield {
+ shouldReplace = true
+ Some(k -> newValWithTs)
+ }
+
+ ((existInOld.flatten ++ existInNew.flatten).toMap, shouldReplace)
+ }
+
+ def mergeDelete(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ var shouldReplace = false
+ val (oldPropsWithTs, propsWithTs, requestTs, version) = propsPairWithTs
+ val lastDeletedAt = oldPropsWithTs.get(LabelMeta.lastDeletedAt) match {
+ case Some(prevDeletedAt) =>
+ if (prevDeletedAt.ts >= requestTs) prevDeletedAt.ts
+ else {
+ shouldReplace = true
+ requestTs
+ }
+ case None => {
+ shouldReplace = true
+ requestTs
+ }
+ }
+ val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
+ if (k == LabelMeta.timestamp) {
+ if (oldValWithTs.ts >= requestTs) Some(k -> oldValWithTs)
+ else {
+ shouldReplace = true
+ Some(k -> InnerValLikeWithTs.withLong(requestTs, requestTs, version))
+ }
+ } else {
+ if (oldValWithTs.ts >= lastDeletedAt) Some(k -> oldValWithTs)
+ else {
+ shouldReplace = true
+ None
+ }
+ }
+ }
+ val mustExistInNew = Map(LabelMeta.lastDeletedAt -> InnerValLikeWithTs.withLong(lastDeletedAt, lastDeletedAt, version))
+ ((existInOld.flatten ++ mustExistInNew).toMap, shouldReplace)
+ }
+
+ def mergeInsertBulk(propsPairWithTs: PropsPairWithTs): (State, Boolean) = {
+ val (_, propsWithTs, _, _) = propsPairWithTs
+ (propsWithTs, true)
+ }
+
+// def fromString(s: String): Option[Edge] = Graph.toEdge(s)
+
+
+}
[10/10] incubator-s2graph git commit: [S2GRAPH-131]: Add actual
implementation on interfaces from TinkerPop3 structure package.
Posted by st...@apache.org.
[S2GRAPH-131]: Add actual implementation on interfaces from TinkerPop3 structure package.
JIRA:
[S2GRAPH-131] https://issues.apache.org/jira/browse/S2GRAPH-131
Pull Request:
Closes #101
Authors
DO YUNG YOON: steamshon@apache.org
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/247b2cb9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/247b2cb9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/247b2cb9
Branch: refs/heads/master
Commit: 247b2cb9d67f41e56cee1facba5430221db939c9
Parents: 0f21ad6
Author: DO YUNG YOON <st...@apache.org>
Authored: Thu Dec 1 17:30:46 2016 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Thu Dec 1 17:30:46 2016 +0900
----------------------------------------------------------------------
CHANGES | 2 ++
1 file changed, 2 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/247b2cb9/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 484cf88..860d2f5 100644
--- a/CHANGES
+++ b/CHANGES
@@ -231,6 +231,8 @@ Release 0.1.0 - unreleased
S2GRAPH-129: Restrict direct access on Edge's properties from other classes. (Committed by DOYUNG YOON).
S2GRAPH-130: Edge.propsWithTs data type should be changed into mutable to support setter interface exist in tp3 (Committed by DOYUNG YOON).
+
+ S2GRAPH-131: Add actual implementation on interfaces from TinkerPop3 structure package. (Committed by DOYUNG YOON).
TEST
[04/10] incubator-s2graph git commit: [S2GRAPH-131]: Add actual
implementation on interfaces from TinkerPop3 structure package. - Change
core.Edge/Vertex/Graph to core.S2Edge/S2Vertex/S2Graph. - Implement base
interfaces for tinkerpop3 structure packag
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala b/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala
deleted file mode 100644
index 38477b4..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala
+++ /dev/null
@@ -1,1238 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.s2graph.core
-
-import java.util
-import java.util.concurrent.Executors
-
-import com.typesafe.config.{Config, ConfigFactory}
-import org.apache.commons.configuration.Configuration
-import org.apache.s2graph.core.GraphExceptions.{FetchAllStepFailException, FetchTimeoutException, LabelNotExistException}
-import org.apache.s2graph.core.JSONParser._
-import org.apache.s2graph.core.mysqls._
-import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage
-import org.apache.s2graph.core.storage.{SKeyValue, Storage}
-import org.apache.s2graph.core.types._
-import org.apache.s2graph.core.utils.{DeferCache, Extensions, SafeUpdateCache, logger}
-import org.apache.tinkerpop.gremlin.process.computer.GraphComputer
-import org.apache.tinkerpop.gremlin.structure
-import org.apache.tinkerpop.gremlin.structure.Graph.Variables
-import org.apache.tinkerpop.gremlin.structure.{Graph => TpGraph, Transaction}
-import play.api.libs.json.{JsObject, Json}
-import scala.annotation.tailrec
-import scala.collection.JavaConversions._
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-import scala.concurrent._
-import scala.util.{Random, Try}
-
-object Graph {
-
- type HashKey = (Int, Int, Int, Int, Boolean)
- type FilterHashKey = (Int, Int)
-
-
- val DefaultScore = 1.0
-
- private val DefaultConfigs: Map[String, AnyRef] = Map(
- "hbase.zookeeper.quorum" -> "localhost",
- "hbase.table.name" -> "s2graph",
- "hbase.table.compression.algorithm" -> "gz",
- "phase" -> "dev",
- "db.default.driver" -> "org.h2.Driver",
- "db.default.url" -> "jdbc:h2:file:./var/metastore;MODE=MYSQL",
- "db.default.password" -> "graph",
- "db.default.user" -> "graph",
- "cache.max.size" -> java.lang.Integer.valueOf(10000),
- "cache.ttl.seconds" -> java.lang.Integer.valueOf(60),
- "hbase.client.retries.number" -> java.lang.Integer.valueOf(20),
- "hbase.rpcs.buffered_flush_interval" -> java.lang.Short.valueOf(100.toShort),
- "hbase.rpc.timeout" -> java.lang.Integer.valueOf(1000),
- "max.retry.number" -> java.lang.Integer.valueOf(100),
- "lock.expire.time" -> java.lang.Integer.valueOf(1000 * 60 * 10),
- "max.back.off" -> java.lang.Integer.valueOf(100),
- "back.off.timeout" -> java.lang.Integer.valueOf(1000),
- "hbase.fail.prob" -> java.lang.Double.valueOf(-0.1),
- "delete.all.fetch.size" -> java.lang.Integer.valueOf(1000),
- "delete.all.fetch.count" -> java.lang.Integer.valueOf(200),
- "future.cache.max.size" -> java.lang.Integer.valueOf(100000),
- "future.cache.expire.after.write" -> java.lang.Integer.valueOf(10000),
- "future.cache.expire.after.access" -> java.lang.Integer.valueOf(5000),
- "future.cache.metric.interval" -> java.lang.Integer.valueOf(60000),
- "query.future.cache.max.size" -> java.lang.Integer.valueOf(1000),
- "query.future.cache.expire.after.write" -> java.lang.Integer.valueOf(10000),
- "query.future.cache.expire.after.access" -> java.lang.Integer.valueOf(5000),
- "query.future.cache.metric.interval" -> java.lang.Integer.valueOf(60000),
- "s2graph.storage.backend" -> "hbase",
- "query.hardlimit" -> java.lang.Integer.valueOf(100000),
- "hbase.zookeeper.znode.parent" -> "/hbase",
- "query.log.sample.rate" -> Double.box(0.05)
- )
-
- var DefaultConfig: Config = ConfigFactory.parseMap(DefaultConfigs)
-
-
-
- def initStorage(graph: Graph, config: Config)(ec: ExecutionContext): Storage[_, _] = {
- val storageBackend = config.getString("s2graph.storage.backend")
- logger.info(s"[InitStorage]: $storageBackend")
-
- storageBackend match {
- case "hbase" => new AsynchbaseStorage(graph, config)(ec)
- case _ => throw new RuntimeException("not supported storage.")
- }
- }
-
- def parseCacheConfig(config: Config, prefix: String): Config = {
- import scala.collection.JavaConversions._
-
- val kvs = new java.util.HashMap[String, AnyRef]()
- for {
- entry <- config.entrySet()
- (k, v) = (entry.getKey, entry.getValue) if k.startsWith(prefix)
- } yield {
- val newKey = k.replace(prefix, "")
- kvs.put(newKey, v.unwrapped())
- }
- ConfigFactory.parseMap(kvs)
- }
-
- /** Global helper functions */
- @tailrec
- final def randomInt(sampleNumber: Int, range: Int, set: Set[Int] = Set.empty[Int]): Set[Int] = {
- if (range < sampleNumber || set.size == sampleNumber) set
- else randomInt(sampleNumber, range, set + Random.nextInt(range))
- }
-
- def sample(queryRequest: QueryRequest, edges: Seq[EdgeWithScore], n: Int): Seq[EdgeWithScore] = {
- if (edges.size <= n) {
- edges
- } else {
- val plainEdges = if (queryRequest.queryParam.offset == 0) {
- edges.tail
- } else edges
-
- val randoms = randomInt(n, plainEdges.size)
- var samples = List.empty[EdgeWithScore]
- var idx = 0
- plainEdges.foreach { e =>
- if (randoms.contains(idx)) samples = e :: samples
- idx += 1
- }
- samples
- }
- }
-
- def normalize(edgeWithScores: Seq[EdgeWithScore]): Seq[EdgeWithScore] = {
- val sum = edgeWithScores.foldLeft(0.0) { case (acc, cur) => acc + cur.score }
- edgeWithScores.map { edgeWithScore =>
- edgeWithScore.copy(score = edgeWithScore.score / sum)
- }
- }
-
- def alreadyVisitedVertices(edgeWithScoreLs: Seq[EdgeWithScore]): Map[(LabelWithDirection, Vertex), Boolean] = {
- val vertices = for {
- edgeWithScore <- edgeWithScoreLs
- edge = edgeWithScore.edge
- vertex = if (edge.labelWithDir.dir == GraphUtil.directions("out")) edge.tgtVertex else edge.srcVertex
- } yield (edge.labelWithDir, vertex) -> true
-
- vertices.toMap
- }
-
- /** common methods for filter out, transform, aggregate queryResult */
- def convertEdges(queryParam: QueryParam, edge: Edge, nextStepOpt: Option[Step]): Seq[Edge] = {
- for {
- convertedEdge <- queryParam.edgeTransformer.transform(queryParam, edge, nextStepOpt) if !edge.isDegree
- } yield convertedEdge
- }
-
- def processTimeDecay(queryParam: QueryParam, edge: Edge) = {
- /* process time decay */
- val tsVal = queryParam.timeDecay match {
- case None => 1.0
- case Some(timeDecay) =>
- val tsVal = try {
- val innerValWithTsOpt = edge.propertyValue(timeDecay.labelMeta.name)
- innerValWithTsOpt.map { innerValWithTs =>
- val innerVal = innerValWithTs.innerVal
- timeDecay.labelMeta.dataType match {
- case InnerVal.LONG => innerVal.value match {
- case n: BigDecimal => n.bigDecimal.longValue()
- case _ => innerVal.toString().toLong
- }
- case _ => innerVal.toString().toLong
- }
- } getOrElse (edge.ts)
- } catch {
- case e: Exception =>
- logger.error(s"processTimeDecay error. ${edge.toLogString}", e)
- edge.ts
- }
- val timeDiff = queryParam.timestamp - tsVal
- timeDecay.decay(timeDiff)
- }
-
- tsVal
- }
-
- def processDuplicates[T](queryParam: QueryParam,
- duplicates: Seq[(FilterHashKey, T)])(implicit ev: WithScore[T]): Seq[(FilterHashKey, T)] = {
-
- if (queryParam.label.consistencyLevel != "strong") {
- //TODO:
- queryParam.duplicatePolicy match {
- case DuplicatePolicy.First => Seq(duplicates.head)
- case DuplicatePolicy.Raw => duplicates
- case DuplicatePolicy.CountSum =>
- val countSum = duplicates.size
- val (headFilterHashKey, headEdgeWithScore) = duplicates.head
- Seq(headFilterHashKey -> ev.withNewScore(headEdgeWithScore, countSum))
- case _ =>
- val scoreSum = duplicates.foldLeft(0.0) { case (prev, current) => prev + ev.score(current._2) }
- val (headFilterHashKey, headEdgeWithScore) = duplicates.head
- Seq(headFilterHashKey -> ev.withNewScore(headEdgeWithScore, scoreSum))
- }
- } else {
- duplicates
- }
- }
-
- def toHashKey(queryParam: QueryParam, edge: Edge, isDegree: Boolean): (HashKey, FilterHashKey) = {
- val src = edge.srcVertex.innerId.hashCode()
- val tgt = edge.tgtVertex.innerId.hashCode()
- val hashKey = (src, edge.labelWithDir.labelId, edge.labelWithDir.dir, tgt, isDegree)
- val filterHashKey = (src, tgt)
-
- (hashKey, filterHashKey)
- }
-
- def filterEdges(q: Query,
- stepIdx: Int,
- queryRequests: Seq[QueryRequest],
- queryResultLsFuture: Future[Seq[StepResult]],
- queryParams: Seq[QueryParam],
- alreadyVisited: Map[(LabelWithDirection, Vertex), Boolean] = Map.empty,
- buildLastStepInnerResult: Boolean = true,
- parentEdges: Map[VertexId, Seq[EdgeWithScore]])
- (implicit ec: scala.concurrent.ExecutionContext): Future[StepResult] = {
-
- queryResultLsFuture.map { queryRequestWithResultLs =>
- val (cursors, failCount) = {
- val _cursors = ArrayBuffer.empty[Array[Byte]]
- var _failCount = 0
-
- queryRequestWithResultLs.foreach { stepResult =>
- _cursors.append(stepResult.cursors: _*)
- _failCount += stepResult.failCount
- }
-
- _cursors -> _failCount
- }
-
-
- if (queryRequestWithResultLs.isEmpty) StepResult.Empty.copy(failCount = failCount)
- else {
- val isLastStep = stepIdx == q.steps.size - 1
- val queryOption = q.queryOption
- val step = q.steps(stepIdx)
-
- val currentStepResults = queryRequests.view.zip(queryRequestWithResultLs)
- val shouldBuildInnerResults = !isLastStep || buildLastStepInnerResult
- val degrees = queryRequestWithResultLs.flatMap(_.degreeEdges)
-
- if (shouldBuildInnerResults) {
- val _results = buildResult(q, stepIdx, currentStepResults, parentEdges) { (edgeWithScore, propsSelectColumns) =>
- edgeWithScore
- }
-
- /** process step group by */
- val results = StepResult.filterOutStepGroupBy(_results, step.groupBy)
- StepResult(edgeWithScores = results, grouped = Nil, degreeEdges = degrees, cursors = cursors, failCount = failCount)
-
- } else {
- val _results = buildResult(q, stepIdx, currentStepResults, parentEdges) { (edgeWithScore, propsSelectColumns) =>
- val edge = edgeWithScore.edge
- val score = edgeWithScore.score
- val label = edgeWithScore.label
-
- /** Select */
- val mergedPropsWithTs = edge.propertyValuesInner(propsSelectColumns)
-
-// val newEdge = edge.copy(propsWithTs = mergedPropsWithTs)
- val newEdge = edge.copyEdgeWithState(mergedPropsWithTs)
-
- val newEdgeWithScore = edgeWithScore.copy(edge = newEdge)
- /** OrderBy */
- val orderByValues =
- if (queryOption.orderByKeys.isEmpty) (score, edge.tsInnerVal, None, None)
- else StepResult.toTuple4(newEdgeWithScore.toValues(queryOption.orderByKeys))
-
- /** StepGroupBy */
- val stepGroupByValues = newEdgeWithScore.toValues(step.groupBy.keys)
-
- /** GroupBy */
- val groupByValues = newEdgeWithScore.toValues(queryOption.groupBy.keys)
-
- /** FilterOut */
- val filterOutValues = newEdgeWithScore.toValues(queryOption.filterOutFields)
-
- newEdgeWithScore.copy(orderByValues = orderByValues,
- stepGroupByValues = stepGroupByValues,
- groupByValues = groupByValues,
- filterOutValues = filterOutValues)
- }
-
- /** process step group by */
- val results = StepResult.filterOutStepGroupBy(_results, step.groupBy)
-
- /** process ordered list */
- val ordered = if (queryOption.groupBy.keys.isEmpty) StepResult.orderBy(queryOption, results) else Nil
-
- /** process grouped list */
- val grouped =
- if (queryOption.groupBy.keys.isEmpty) Nil
- else {
- val agg = new scala.collection.mutable.HashMap[StepResult.GroupByKey, (Double, StepResult.Values)]()
- results.groupBy { edgeWithScore =>
- // edgeWithScore.groupByValues.map(_.map(_.toString))
- edgeWithScore.groupByValues
- }.foreach { case (k, ls) =>
- val (scoreSum, merged) = StepResult.mergeOrdered(ls, Nil, queryOption)
-
- val newScoreSum = scoreSum
-
- /**
- * watch out here. by calling toString on Any, we lose type information which will be used
- * later for toJson.
- */
- if (merged.nonEmpty) {
- val newKey = merged.head.groupByValues
- agg += (newKey -> (newScoreSum, merged))
- }
- }
- agg.toSeq.sortBy(_._2._1 * -1)
- }
-
- StepResult(edgeWithScores = ordered, grouped = grouped, degreeEdges = degrees, cursors = cursors, failCount = failCount)
- }
- }
- }
- }
-
- private def toEdgeWithScores(queryRequest: QueryRequest,
- stepResult: StepResult,
- parentEdges: Map[VertexId, Seq[EdgeWithScore]]): Seq[EdgeWithScore] = {
- val queryOption = queryRequest.query.queryOption
- val queryParam = queryRequest.queryParam
- val prevScore = queryRequest.prevStepScore
- val labelWeight = queryRequest.labelWeight
- val edgeWithScores = stepResult.edgeWithScores
-
- val shouldBuildParents = queryOption.returnTree || queryParam.whereHasParent
- val parents = if (shouldBuildParents) {
- parentEdges.getOrElse(queryRequest.vertex.id, Nil).map { edgeWithScore =>
- val edge = edgeWithScore.edge
- val score = edgeWithScore.score
- val label = edgeWithScore.label
-
- /** Select */
- val mergedPropsWithTs =
- if (queryOption.selectColumns.isEmpty) {
- edge.propertyValuesInner()
- } else {
- val initial = Map(LabelMeta.timestamp -> edge.propertyValueInner(LabelMeta.timestamp))
- edge.propertyValues(queryOption.selectColumns) ++ initial
- }
-
- val newEdge = edge.copyEdgeWithState(mergedPropsWithTs)
- edgeWithScore.copy(edge = newEdge)
- }
- } else Nil
-
- // skip
- if (queryOption.ignorePrevStepCache) stepResult.edgeWithScores
- else {
- val degreeScore = 0.0
-
- val sampled =
- if (queryRequest.queryParam.sample >= 0) sample(queryRequest, edgeWithScores, queryRequest.queryParam.sample)
- else edgeWithScores
-
- val withScores = for {
- edgeWithScore <- sampled
- } yield {
- val edge = edgeWithScore.edge
- val edgeScore = edgeWithScore.score
- val score = queryParam.scorePropagateOp match {
- case "plus" => edgeScore + prevScore
- case "divide" =>
- if ((prevScore + queryParam.scorePropagateShrinkage) == 0) 0
- else edgeScore / (prevScore + queryParam.scorePropagateShrinkage)
- case _ => edgeScore * prevScore
- }
-
- val tsVal = processTimeDecay(queryParam, edge)
- val newScore = degreeScore + score
- // val newEdge = if (queryOption.returnTree) edge.copy(parentEdges = parents) else edge
- val newEdge = edge.copy(parentEdges = parents)
- edgeWithScore.copy(edge = newEdge, score = newScore * labelWeight * tsVal)
- }
-
- val normalized =
- if (queryParam.shouldNormalize) normalize(withScores)
- else withScores
-
- normalized
- }
- }
-
- private def buildResult[T](query: Query,
- stepIdx: Int,
- stepResultLs: Seq[(QueryRequest, StepResult)],
- parentEdges: Map[VertexId, Seq[EdgeWithScore]])
- (createFunc: (EdgeWithScore, Seq[LabelMeta]) => T)
- (implicit ev: WithScore[T]): ListBuffer[T] = {
- import scala.collection._
-
- val results = ListBuffer.empty[T]
- val sequentialLs: ListBuffer[(HashKey, FilterHashKey, T, QueryParam)] = ListBuffer.empty
- val duplicates: mutable.HashMap[HashKey, ListBuffer[(FilterHashKey, T)]] = mutable.HashMap.empty
- val edgesToExclude: mutable.HashSet[FilterHashKey] = mutable.HashSet.empty
- val edgesToInclude: mutable.HashSet[FilterHashKey] = mutable.HashSet.empty
-
- var numOfDuplicates = 0
- val queryOption = query.queryOption
- val step = query.steps(stepIdx)
- val excludeLabelWithDirSet = step.queryParams.filter(_.exclude).map(l => l.labelWithDir).toSet
- val includeLabelWithDirSet = step.queryParams.filter(_.include).map(l => l.labelWithDir).toSet
-
- stepResultLs.foreach { case (queryRequest, stepInnerResult) =>
- val queryParam = queryRequest.queryParam
- val label = queryParam.label
- val shouldBeExcluded = excludeLabelWithDirSet.contains(queryParam.labelWithDir)
- val shouldBeIncluded = includeLabelWithDirSet.contains(queryParam.labelWithDir)
-
- val propsSelectColumns = (for {
- column <- queryOption.propsSelectColumns
- labelMeta <- label.metaPropsInvMap.get(column)
- } yield labelMeta)
-
- for {
- edgeWithScore <- toEdgeWithScores(queryRequest, stepInnerResult, parentEdges)
- } {
- val edge = edgeWithScore.edge
- val (hashKey, filterHashKey) = toHashKey(queryParam, edge, isDegree = false)
- // params += (hashKey -> queryParam) //
-
- /** check if this edge should be exlcuded. */
- if (shouldBeExcluded) {
- edgesToExclude.add(filterHashKey)
- } else {
- if (shouldBeIncluded) {
- edgesToInclude.add(filterHashKey)
- }
- val newEdgeWithScore = createFunc(edgeWithScore, propsSelectColumns)
-
- sequentialLs += ((hashKey, filterHashKey, newEdgeWithScore, queryParam))
- duplicates.get(hashKey) match {
- case None =>
- val newLs = ListBuffer.empty[(FilterHashKey, T)]
- newLs += (filterHashKey -> newEdgeWithScore)
- duplicates += (hashKey -> newLs) //
- case Some(old) =>
- numOfDuplicates += 1
- old += (filterHashKey -> newEdgeWithScore) //
- }
- }
- }
- }
-
-
- if (numOfDuplicates == 0) {
- // no duplicates at all.
- for {
- (hashKey, filterHashKey, edgeWithScore, _) <- sequentialLs
- if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
- } {
- results += edgeWithScore
- }
- } else {
- // need to resolve duplicates.
- val seen = new mutable.HashSet[HashKey]()
- for {
- (hashKey, filterHashKey, edgeWithScore, queryParam) <- sequentialLs
- if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
- if !seen.contains(hashKey)
- } {
- // val queryParam = params(hashKey)
- processDuplicates(queryParam, duplicates(hashKey)).foreach { case (_, duplicate) =>
- if (ev.score(duplicate) >= queryParam.threshold) {
- seen += hashKey
- results += duplicate
- }
- }
- }
- }
- results
- }
-
-}
-
-class Graph(_config: Config)(implicit val ec: ExecutionContext) extends TpGraph {
-
- import Graph._
-
- val config = _config.withFallback(Graph.DefaultConfig)
-
- Model.apply(config)
- Model.loadCache()
-
- val MaxRetryNum = config.getInt("max.retry.number")
- val MaxBackOff = config.getInt("max.back.off")
- val BackoffTimeout = config.getInt("back.off.timeout")
- val DeleteAllFetchCount = config.getInt("delete.all.fetch.count")
- val DeleteAllFetchSize = config.getInt("delete.all.fetch.size")
- val FailProb = config.getDouble("hbase.fail.prob")
- val LockExpireDuration = config.getInt("lock.expire.time")
- val MaxSize = config.getInt("future.cache.max.size")
- val ExpireAfterWrite = config.getInt("future.cache.expire.after.write")
- val ExpireAfterAccess = config.getInt("future.cache.expire.after.access")
-
- val scheduledEx = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor())
-
- private def confWithFallback(conf: Config): Config = {
- conf.withFallback(config)
- }
-
- /**
- * TODO: we need to some way to handle malformed configuration for storage.
- */
- val storagePool: scala.collection.mutable.Map[String, Storage[_, _]] = {
- val labels = Label.findAll()
- val services = Service.findAll()
-
- val labelConfigs = labels.flatMap(_.toStorageConfig)
- val serviceConfigs = services.flatMap(_.toStorageConfig)
-
- val configs = (labelConfigs ++ serviceConfigs).map { conf =>
- confWithFallback(conf)
- }.toSet
-
- val pools = new java.util.HashMap[Config, Storage[_, _]]()
- configs.foreach { config =>
- pools.put(config, Graph.initStorage(this, config)(ec))
- }
-
- val m = new java.util.concurrent.ConcurrentHashMap[String, Storage[_, _]]()
-
- labels.foreach { label =>
- if (label.storageConfigOpt.isDefined) {
- m += (s"label:${label.label}" -> pools(label.storageConfigOpt.get))
- }
- }
-
- services.foreach { service =>
- if (service.storageConfigOpt.isDefined) {
- m += (s"service:${service.serviceName}" -> pools(service.storageConfigOpt.get))
- }
- }
-
- m
- }
-
- val defaultStorage: Storage[_, _] = Graph.initStorage(this, config)(ec)
-
- /** QueryLevel FutureCache */
- val queryFutureCache = new DeferCache[StepResult, Promise, Future](parseCacheConfig(config, "query."), empty = StepResult.Empty)
-
- for {
- entry <- config.entrySet() if Graph.DefaultConfigs.contains(entry.getKey)
- (k, v) = (entry.getKey, entry.getValue)
- } logger.info(s"[Initialized]: $k, ${this.config.getAnyRef(k)}")
-
- def getStorage(service: Service): Storage[_, _] = {
- storagePool.getOrElse(s"service:${service.serviceName}", defaultStorage)
- }
-
- def getStorage(label: Label): Storage[_, _] = {
- storagePool.getOrElse(s"label:${label.label}", defaultStorage)
- }
-
- def flushStorage(): Unit = {
- storagePool.foreach { case (_, storage) =>
-
- /** flush is blocking */
- storage.flush()
- }
- }
-
- def fallback = Future.successful(StepResult.Empty)
-
- def checkEdges(edges: Seq[Edge]): Future[StepResult] = {
- val futures = for {
- edge <- edges
- } yield {
- fetchSnapshotEdge(edge).map { case (queryParam, edgeOpt, kvOpt) =>
- edgeOpt.toSeq.map(e => EdgeWithScore(e, 1.0, queryParam.label))
- }
- }
-
- Future.sequence(futures).map { edgeWithScoreLs =>
- val edgeWithScores = edgeWithScoreLs.flatten
- StepResult(edgeWithScores = edgeWithScores, grouped = Nil, degreeEdges = Nil)
- }
- }
-
- // def checkEdges(edges: Seq[Edge]): Future[StepResult] = storage.checkEdges(edges)
-
- def getEdges(q: Query): Future[StepResult] = {
- Try {
- if (q.steps.isEmpty) {
- // TODO: this should be get vertex query.
- fallback
- } else {
- val filterOutFuture = q.queryOption.filterOutQuery match {
- case None => fallback
- case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
- }
- for {
- stepResult <- getEdgesStepInner(q)
- filterOutInnerResult <- filterOutFuture
- } yield {
- if (filterOutInnerResult.isEmpty) stepResult
- else StepResult.filterOut(this, q.queryOption, stepResult, filterOutInnerResult)
- }
- }
- } recover {
- case e: Exception =>
- logger.error(s"getEdgesAsync: $e", e)
- fallback
- } get
- }
-
- def getEdgesStepInner(q: Query, buildLastStepInnerResult: Boolean = false): Future[StepResult] = {
- Try {
- if (q.steps.isEmpty) fallback
- else {
-
- val queryOption = q.queryOption
- def fetch: Future[StepResult] = {
- val startStepInnerResult = QueryResult.fromVertices(this, q)
- q.steps.zipWithIndex.foldLeft(Future.successful(startStepInnerResult)) { case (prevStepInnerResultFuture, (step, stepIdx)) =>
- for {
- prevStepInnerResult <- prevStepInnerResultFuture
- currentStepInnerResult <- fetchStep(q, stepIdx, prevStepInnerResult, buildLastStepInnerResult)
- } yield {
- currentStepInnerResult.copy(
- accumulatedCursors = prevStepInnerResult.accumulatedCursors :+ currentStepInnerResult.cursors,
- failCount = currentStepInnerResult.failCount + prevStepInnerResult.failCount
- )
- }
- }
- }
-
- fetch
- }
- } recover {
- case e: Exception =>
- logger.error(s"getEdgesAsync: $e", e)
- fallback
- } get
- }
-
- def fetchStep(orgQuery: Query,
- stepIdx: Int,
- stepInnerResult: StepResult,
- buildLastStepInnerResult: Boolean = false): Future[StepResult] = {
- if (stepInnerResult.isEmpty) Future.successful(StepResult.Empty)
- else {
- val edgeWithScoreLs = stepInnerResult.edgeWithScores
-
- val q = orgQuery
- val queryOption = orgQuery.queryOption
- val prevStepOpt = if (stepIdx > 0) Option(q.steps(stepIdx - 1)) else None
- val prevStepThreshold = prevStepOpt.map(_.nextStepScoreThreshold).getOrElse(QueryParam.DefaultThreshold)
- val prevStepLimit = prevStepOpt.map(_.nextStepLimit).getOrElse(-1)
- val step = q.steps(stepIdx)
-
- val alreadyVisited =
- if (stepIdx == 0) Map.empty[(LabelWithDirection, Vertex), Boolean]
- else alreadyVisitedVertices(stepInnerResult.edgeWithScores)
-
- val initial = (Map.empty[Vertex, Double], Map.empty[Vertex, ArrayBuffer[EdgeWithScore]])
- val (sums, grouped) = edgeWithScoreLs.foldLeft(initial) { case ((sum, group), edgeWithScore) =>
- val key = edgeWithScore.edge.tgtVertex
- val newScore = sum.getOrElse(key, 0.0) + edgeWithScore.score
- val buffer = group.getOrElse(key, ArrayBuffer.empty[EdgeWithScore])
- buffer += edgeWithScore
- (sum + (key -> newScore), group + (key -> buffer))
- }
- val groupedByFiltered = sums.filter(t => t._2 >= prevStepThreshold)
- val prevStepTgtVertexIdEdges = grouped.map(t => t._1.id -> t._2)
-
- val nextStepSrcVertices = if (prevStepLimit >= 0) {
- groupedByFiltered.toSeq.sortBy(-1 * _._2).take(prevStepLimit)
- } else {
- groupedByFiltered.toSeq
- }
-
- val queryRequests = for {
- (vertex, prevStepScore) <- nextStepSrcVertices
- queryParam <- step.queryParams
- } yield {
- val labelWeight = step.labelWeights.getOrElse(queryParam.labelWithDir.labelId, 1.0)
- val newPrevStepScore = if (queryOption.shouldPropagateScore) prevStepScore else 1.0
- QueryRequest(q, stepIdx, vertex, queryParam, newPrevStepScore, labelWeight)
- }
-
- val fetchedLs = fetches(queryRequests, prevStepTgtVertexIdEdges)
-
- filterEdges(orgQuery, stepIdx, queryRequests,
- fetchedLs, orgQuery.steps(stepIdx).queryParams, alreadyVisited, buildLastStepInnerResult, prevStepTgtVertexIdEdges)(ec)
- }
- }
-
-
- /**
- * responsible to fire parallel fetch call into storage and create future that will return merged result.
- *
- * @param queryRequests
- * @param prevStepEdges
- * @return
- */
- def fetches(queryRequests: Seq[QueryRequest],
- prevStepEdges: Map[VertexId, Seq[EdgeWithScore]]): Future[Seq[StepResult]] = {
-
- val reqWithIdxs = queryRequests.zipWithIndex
- val requestsPerLabel = reqWithIdxs.groupBy(t => t._1.queryParam.label)
- val aggFuture = requestsPerLabel.foldLeft(Future.successful(Map.empty[Int, StepResult])) { case (prevFuture, (label, reqWithIdxs)) =>
- for {
- prev <- prevFuture
- cur <- getStorage(label).fetches(reqWithIdxs.map(_._1), prevStepEdges)
- } yield {
- prev ++ reqWithIdxs.map(_._2).zip(cur).toMap
- }
- }
- aggFuture.map { agg => agg.toSeq.sortBy(_._1).map(_._2) }
- }
-
-
- def getEdgesMultiQuery(mq: MultiQuery): Future[StepResult] = {
- Try {
- if (mq.queries.isEmpty) fallback
- else {
- val filterOutFuture = mq.queryOption.filterOutQuery match {
- case None => fallback
- case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
- }
-
- val multiQueryFutures = Future.sequence(mq.queries.map { query => getEdges(query) })
- for {
- multiQueryResults <- multiQueryFutures
- filterOutInnerResult <- filterOutFuture
- } yield {
- StepResult.merges(mq.queryOption, multiQueryResults, mq.weights, filterOutInnerResult)
- }
- }
- } recover {
- case e: Exception =>
- logger.error(s"getEdgesAsync: $e", e)
- fallback
- } get
- }
-
-
- def fetchSnapshotEdge(edge: Edge): Future[(QueryParam, Option[Edge], Option[SKeyValue])] = {
- /** TODO: Fix this. currently fetchSnapshotEdge should not use future cache
- * so use empty cacheKey.
- * */
- val queryParam = QueryParam(labelName = edge.innerLabel.label,
- direction = GraphUtil.fromDirection(edge.labelWithDir.dir),
- tgtVertexIdOpt = Option(edge.tgtVertex.innerIdVal),
- cacheTTLInMillis = -1)
- val q = Query.toQuery(Seq(edge.srcVertex), queryParam)
- val queryRequest = QueryRequest(q, 0, edge.srcVertex, queryParam)
-
- val storage = getStorage(edge.innerLabel)
- storage.fetchSnapshotEdgeKeyValues(queryRequest).map { kvs =>
- val (edgeOpt, kvOpt) =
- if (kvs.isEmpty) (None, None)
- else {
- val snapshotEdgeOpt = storage.toSnapshotEdge(kvs.head, queryRequest, isInnerCall = true, parentEdges = Nil)
- val _kvOpt = kvs.headOption
- (snapshotEdgeOpt, _kvOpt)
- }
- (queryParam, edgeOpt, kvOpt)
- } recoverWith { case ex: Throwable =>
- logger.error(s"fetchQueryParam failed. fallback return.", ex)
- throw FetchTimeoutException(s"${edge.toLogString}")
- }
- }
-
- def getVertices(vertices: Seq[Vertex]): Future[Seq[Vertex]] = {
- val verticesWithIdx = vertices.zipWithIndex
- val futures = verticesWithIdx.groupBy { case (v, idx) => v.service }.map { case (service, vertexGroup) =>
- getStorage(service).getVertices(vertexGroup.map(_._1)).map(_.zip(vertexGroup.map(_._2)))
- }
-
- Future.sequence(futures).map { ls =>
- ls.flatten.toSeq.sortBy(_._2).map(_._1)
- }
- }
-
- /** mutate */
- def deleteAllAdjacentEdges(srcVertices: Seq[Vertex],
- labels: Seq[Label],
- dir: Int,
- ts: Long): Future[Boolean] = {
-
- val requestTs = ts
- val vertices = srcVertices
- /** create query per label */
- val queries = for {
- label <- labels
- } yield {
- val queryParam = QueryParam(labelName = label.label, direction = GraphUtil.fromDirection(dir),
- offset = 0, limit = DeleteAllFetchSize, duplicatePolicy = DuplicatePolicy.Raw)
- val step = Step(List(queryParam))
- Query(vertices, Vector(step))
- }
-
- // Extensions.retryOnSuccessWithBackoff(MaxRetryNum, Random.nextInt(MaxBackOff) + 1) {
- val retryFuture = Extensions.retryOnSuccess(DeleteAllFetchCount) {
- fetchAndDeleteAll(queries, requestTs)
- } { case (allDeleted, deleteSuccess) =>
- allDeleted && deleteSuccess
- }.map { case (allDeleted, deleteSuccess) => allDeleted && deleteSuccess }
-
- retryFuture onFailure {
- case ex =>
- logger.error(s"[Error]: deleteAllAdjacentEdges failed.")
- }
-
- retryFuture
- }
-
- def fetchAndDeleteAll(queries: Seq[Query], requestTs: Long): Future[(Boolean, Boolean)] = {
- val future = for {
- stepInnerResultLs <- Future.sequence(queries.map(getEdgesStepInner(_, true)))
- (allDeleted, ret) <- deleteAllFetchedEdgesLs(stepInnerResultLs, requestTs)
- } yield {
- // logger.debug(s"fetchAndDeleteAll: ${allDeleted}, ${ret}")
- (allDeleted, ret)
- }
-
- Extensions.retryOnFailure(MaxRetryNum) {
- future
- } {
- logger.error(s"fetch and deleteAll failed.")
- (true, false)
- }
-
- }
-
- def deleteAllFetchedEdgesLs(stepInnerResultLs: Seq[StepResult],
- requestTs: Long): Future[(Boolean, Boolean)] = {
- stepInnerResultLs.foreach { stepInnerResult =>
- if (stepInnerResult.isFailure) throw new RuntimeException("fetched result is fallback.")
- }
- val futures = for {
- stepInnerResult <- stepInnerResultLs
- deleteStepInnerResult = buildEdgesToDelete(stepInnerResult, requestTs)
- if deleteStepInnerResult.edgeWithScores.nonEmpty
- } yield {
- val head = deleteStepInnerResult.edgeWithScores.head
- val label = head.edge.innerLabel
- val ret = label.schemaVersion match {
- case HBaseType.VERSION3 | HBaseType.VERSION4 =>
- if (label.consistencyLevel == "strong") {
- /**
- * read: snapshotEdge on queryResult = O(N)
- * write: N x (relatedEdges x indices(indexedEdge) + 1(snapshotEdge))
- */
- mutateEdges(deleteStepInnerResult.edgeWithScores.map(_.edge), withWait = true).map(_.forall(identity))
- } else {
- getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
- }
- case _ =>
-
- /**
- * read: x
- * write: N x ((1(snapshotEdge) + 2(1 for incr, 1 for delete) x indices)
- */
- getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
- }
- ret
- }
-
- if (futures.isEmpty) {
- // all deleted.
- Future.successful(true -> true)
- } else {
- Future.sequence(futures).map { rets => false -> rets.forall(identity) }
- }
- }
-
- def buildEdgesToDelete(stepInnerResult: StepResult, requestTs: Long): StepResult = {
- val filtered = stepInnerResult.edgeWithScores.filter { edgeWithScore =>
- (edgeWithScore.edge.ts < requestTs) && !edgeWithScore.edge.isDegree
- }
- if (filtered.isEmpty) StepResult.Empty
- else {
- val head = filtered.head
- val label = head.edge.innerLabel
- val edgeWithScoreLs = filtered.map { edgeWithScore =>
- val edge = edgeWithScore.edge
- val copiedEdge = label.consistencyLevel match {
- case "strong" =>
- edge.copyEdge(op = GraphUtil.operations("delete"),
- version = requestTs, propsWithTs = Edge.propsToState(edge.updatePropsWithTs()), ts = requestTs)
- case _ =>
- edge.copyEdge(propsWithTs = Edge.propsToState(edge.updatePropsWithTs()), ts = requestTs)
- }
-// val (newOp, newVersion, newPropsWithTs) = label.consistencyLevel match {
-// case "strong" =>
-// val edge = edgeWithScore.edge
-// edge.property(LabelMeta.timestamp.name, requestTs)
-// val _newPropsWithTs = edge.updatePropsWithTs()
-//
-// (GraphUtil.operations("delete"), requestTs, _newPropsWithTs)
-// case _ =>
-// val oldEdge = edgeWithScore.edge
-// (oldEdge.op, oldEdge.version, oldEdge.updatePropsWithTs())
-// }
-//
-// val copiedEdge =
-// edgeWithScore.edge.copy(op = newOp, version = newVersion, propsWithTs = newPropsWithTs)
-
- val edgeToDelete = edgeWithScore.copy(edge = copiedEdge)
- // logger.debug(s"delete edge from deleteAll: ${edgeToDelete.edge.toLogString}")
- edgeToDelete
- }
- //Degree edge?
- StepResult(edgeWithScores = edgeWithScoreLs, grouped = Nil, degreeEdges = Nil, false)
- }
- }
-
- // def deleteAllAdjacentEdges(srcVertices: List[Vertex], labels: Seq[Label], dir: Int, ts: Long): Future[Boolean] =
- // storage.deleteAllAdjacentEdges(srcVertices, labels, dir, ts)
-
- def mutateElements(elements: Seq[GraphElement],
- withWait: Boolean = false): Future[Seq[Boolean]] = {
-
- val edgeBuffer = ArrayBuffer[(Edge, Int)]()
- val vertexBuffer = ArrayBuffer[(Vertex, Int)]()
-
- elements.zipWithIndex.foreach {
- case (e: Edge, idx: Int) => edgeBuffer.append((e, idx))
- case (v: Vertex, idx: Int) => vertexBuffer.append((v, idx))
- case any@_ => logger.error(s"Unknown type: ${any}")
- }
-
- val edgeFutureWithIdx = mutateEdges(edgeBuffer.map(_._1), withWait).map { result =>
- edgeBuffer.map(_._2).zip(result)
- }
-
- val vertexFutureWithIdx = mutateVertices(vertexBuffer.map(_._1), withWait).map { result =>
- vertexBuffer.map(_._2).zip(result)
- }
-
- val graphFuture = for {
- edgesMutated <- edgeFutureWithIdx
- verticesMutated <- vertexFutureWithIdx
- } yield (edgesMutated ++ verticesMutated).sortBy(_._1).map(_._2)
-
- graphFuture
-
- }
-
- // def mutateEdges(edges: Seq[Edge], withWait: Boolean = false): Future[Seq[Boolean]] = storage.mutateEdges(edges, withWait)
-
- def mutateEdges(edges: Seq[Edge], withWait: Boolean = false): Future[Seq[Boolean]] = {
- val edgeWithIdxs = edges.zipWithIndex
-
- val (strongEdges, weakEdges) =
- edgeWithIdxs.partition { case (edge, idx) =>
- val e = edge
- e.innerLabel.consistencyLevel == "strong" && e.op != GraphUtil.operations("insertBulk")
- }
-
- val weakEdgesFutures = weakEdges.groupBy { case (edge, idx) => edge.innerLabel.hbaseZkAddr }.map { case (zkQuorum, edgeWithIdxs) =>
- val futures = edgeWithIdxs.groupBy(_._1.innerLabel).map { case (label, edgeGroup) =>
- val storage = getStorage(label)
- val edges = edgeGroup.map(_._1)
- val idxs = edgeGroup.map(_._2)
-
- /** multiple edges with weak consistency level will be processed as batch */
- val mutations = edges.flatMap { edge =>
- val (_, edgeUpdate) =
- if (edge.op == GraphUtil.operations("delete")) Edge.buildDeleteBulk(None, edge)
- else Edge.buildOperation(None, Seq(edge))
-
- storage.buildVertexPutsAsync(edge) ++ storage.indexedEdgeMutations(edgeUpdate) ++ storage.snapshotEdgeMutations(edgeUpdate) ++ storage.increments(edgeUpdate)
- }
-
- storage.writeToStorage(zkQuorum, mutations, withWait).map { ret =>
- idxs.map(idx => idx -> ret)
- }
- }
- Future.sequence(futures)
- }
- val (strongDeleteAll, strongEdgesAll) = strongEdges.partition { case (edge, idx) => edge.op == GraphUtil.operations("deleteAll") }
-
- val deleteAllFutures = strongDeleteAll.map { case (edge, idx) =>
- deleteAllAdjacentEdges(Seq(edge.srcVertex), Seq(edge.innerLabel), edge.labelWithDir.dir, edge.ts).map(idx -> _)
- }
-
- val strongEdgesFutures = strongEdgesAll.groupBy { case (edge, idx) => edge.innerLabel }.map { case (label, edgeGroup) =>
- val edges = edgeGroup.map(_._1)
- val idxs = edgeGroup.map(_._2)
- val storage = getStorage(label)
- storage.mutateStrongEdges(edges, withWait = true).map { rets =>
- idxs.zip(rets)
- }
- }
-
- for {
- weak <- Future.sequence(weakEdgesFutures)
- deleteAll <- Future.sequence(deleteAllFutures)
- strong <- Future.sequence(strongEdgesFutures)
- } yield {
- (deleteAll ++ weak.flatten.flatten ++ strong.flatten).sortBy(_._1).map(_._2)
- }
- }
-
- def mutateVertices(vertices: Seq[Vertex], withWait: Boolean = false): Future[Seq[Boolean]] = {
- val verticesWithIdx = vertices.zipWithIndex
- val futures = verticesWithIdx.groupBy { case (v, idx) => v.service }.map { case (service, vertexGroup) =>
- getStorage(service).mutateVertices(vertexGroup.map(_._1), withWait).map(_.zip(vertexGroup.map(_._2)))
- }
- Future.sequence(futures).map { ls => ls.flatten.toSeq.sortBy(_._2).map(_._1) }
- }
-
- def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]] = {
- val edgesWithIdx = edges.zipWithIndex
- val futures = edgesWithIdx.groupBy { case (e, idx) => e.innerLabel }.map { case (label, edgeGroup) =>
- getStorage(label).incrementCounts(edgeGroup.map(_._1), withWait).map(_.zip(edgeGroup.map(_._2)))
- }
- Future.sequence(futures).map { ls => ls.flatten.toSeq.sortBy(_._2).map(_._1) }
- }
-
- def updateDegree(edge: Edge, degreeVal: Long = 0): Future[Boolean] = {
- val label = edge.innerLabel
-
- val storage = getStorage(label)
- val kvs = storage.buildDegreePuts(edge, degreeVal)
-
- storage.writeToStorage(edge.innerLabel.service.cluster, kvs, withWait = true)
- }
-
- def shutdown(): Unit = {
- flushStorage()
- Model.shutdown()
- }
-
- def addEdge(srcId: Any,
- tgtId: Any,
- labelName: String,
- direction: String = "out",
- props: Map[String, Any] = Map.empty,
- ts: Long = System.currentTimeMillis(),
- operation: String = "insert",
- withWait: Boolean = true): Future[Boolean] = {
-
- val innerEdges = Seq(toEdge(srcId, tgtId, labelName, direction, props.toMap, ts, operation))
- mutateEdges(innerEdges, withWait).map(_.headOption.getOrElse(false))
- }
-
- def addVertex(serviceName: String,
- columnName: String,
- id: Any,
- props: Map[String, Any] = Map.empty,
- ts: Long = System.currentTimeMillis(),
- operation: String = "insert",
- withWait: Boolean = true): Future[Boolean] = {
- val innerVertices = Seq(toVertex(serviceName, columnName, id, props.toMap, ts, operation))
- mutateVertices(innerVertices, withWait).map(_.headOption.getOrElse(false))
- }
-
- def toGraphElement(s: String, labelMapping: Map[String, String] = Map.empty): Option[GraphElement] = Try {
- val parts = GraphUtil.split(s)
- val logType = parts(2)
- val element = if (logType == "edge" | logType == "e") {
- /** current only edge is considered to be bulk loaded */
- labelMapping.get(parts(5)) match {
- case None =>
- case Some(toReplace) =>
- parts(5) = toReplace
- }
- toEdge(parts)
- } else if (logType == "vertex" | logType == "v") {
- toVertex(parts)
- } else {
- throw new GraphExceptions.JsonParseException("log type is not exist in log.")
- }
-
- element
- } recover {
- case e: Exception =>
- logger.error(s"[toElement]: $e", e)
- None
- } get
-
-
- def toVertex(s: String): Option[Vertex] = {
- toVertex(GraphUtil.split(s))
- }
-
- def toEdge(s: String): Option[Edge] = {
- toEdge(GraphUtil.split(s))
- }
-
- def toEdge(parts: Array[String]): Option[Edge] = Try {
- val (ts, operation, logType, srcId, tgtId, label) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
- val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
- val tempDirection = if (parts.length >= 8) parts(7) else "out"
- val direction = if (tempDirection != "out" && tempDirection != "in") "out" else tempDirection
- val edge = toEdge(srcId, tgtId, label, direction, props, ts.toLong, operation)
- Option(edge)
- } recover {
- case e: Exception =>
- logger.error(s"[toEdge]: $e", e)
- throw e
- } get
-
- def toVertex(parts: Array[String]): Option[Vertex] = Try {
- val (ts, operation, logType, srcId, serviceName, colName) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
- val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
- val vertex = toVertex(serviceName, colName, srcId, props, ts.toLong, operation)
- Option(vertex)
- } recover {
- case e: Throwable =>
- logger.error(s"[toVertex]: $e", e)
- throw e
- } get
-
-
- def newSnapshotEdge(srcVertex: Vertex,
- tgtVertex: Vertex,
- label: Label,
- dir: Int,
- op: Byte,
- version: Long,
- propsWithTs: Edge.State,
- pendingEdgeOpt: Option[Edge],
- statusCode: Byte = 0,
- lockTs: Option[Long],
- tsInnerValOpt: Option[InnerValLike] = None): SnapshotEdge = {
- val snapshotEdge = new SnapshotEdge(this, srcVertex, tgtVertex, label, dir, op, version, Edge.EmptyProps,
- pendingEdgeOpt, statusCode, lockTs, tsInnerValOpt)
- Edge.fillPropsWithTs(snapshotEdge, propsWithTs)
- snapshotEdge
- }
-
- def newEdge(srcVertex: Vertex,
- tgtVertex: Vertex,
- innerLabel: Label,
- dir: Int,
- op: Byte = GraphUtil.defaultOpByte,
- version: Long = System.currentTimeMillis(),
- propsWithTs: Edge.State,
- parentEdges: Seq[EdgeWithScore] = Nil,
- originalEdgeOpt: Option[Edge] = None,
- pendingEdgeOpt: Option[Edge] = None,
- statusCode: Byte = 0,
- lockTs: Option[Long] = None,
- tsInnerValOpt: Option[InnerValLike] = None): Edge = {
- val edge = new Edge(this, srcVertex, tgtVertex, innerLabel, dir, op, version, Edge.EmptyProps,
- parentEdges, originalEdgeOpt, pendingEdgeOpt, statusCode, lockTs, tsInnerValOpt)
- Edge.fillPropsWithTs(edge, propsWithTs)
- edge
- }
- def toEdge(srcId: Any,
- tgtId: Any,
- labelName: String,
- direction: String,
- props: Map[String, Any] = Map.empty,
- ts: Long = System.currentTimeMillis(),
- operation: String = "insert"): Edge = {
- val label = Label.findByName(labelName).getOrElse(throw new LabelNotExistException(labelName))
-
- val srcVertexId = toInnerVal(srcId.toString, label.srcColumn.columnType, label.schemaVersion)
- val tgtVertexId = toInnerVal(tgtId.toString, label.tgtColumn.columnType, label.schemaVersion)
-
- val srcColId = label.srcColumn.id.get
- val tgtColId = label.tgtColumn.id.get
-
- val srcVertex = newVertex(SourceVertexId(label.srcColumn, srcVertexId), System.currentTimeMillis())
- val tgtVertex = newVertex(TargetVertexId(label.tgtColumn, tgtVertexId), System.currentTimeMillis())
- val dir = GraphUtil.toDir(direction).getOrElse(throw new RuntimeException(s"$direction is not supported."))
-
- val labelWithDir = LabelWithDirection(label.id.get, dir)
- val propsPlusTs = props ++ Map(LabelMeta.timestamp.name -> ts)
- val propsWithTs = label.propsToInnerValsWithTs(propsPlusTs, ts)
- val op = GraphUtil.toOp(operation).getOrElse(throw new RuntimeException(s"$operation is not supported."))
-
- new Edge(this, srcVertex, tgtVertex, label, dir, op = op, version = ts).copyEdgeWithState(propsWithTs)
- }
-
- def newVertex(id: VertexId,
- ts: Long = System.currentTimeMillis(),
- props: Map[Int, InnerValLike] = Map.empty[Int, InnerValLike],
- op: Byte = 0,
- belongLabelIds: Seq[Int] = Seq.empty): Vertex = {
- new Vertex(this, id, ts, props, op, belongLabelIds)
- }
- def toVertex(serviceName: String,
- columnName: String,
- id: Any,
- props: Map[String, Any] = Map.empty,
- ts: Long = System.currentTimeMillis(),
- operation: String = "insert"): Vertex = {
-
- val service = Service.findByName(serviceName).getOrElse(throw new RuntimeException(s"$serviceName is not found."))
- val column = ServiceColumn.find(service.id.get, columnName).getOrElse(throw new RuntimeException(s"$columnName is not found."))
- val op = GraphUtil.toOp(operation).getOrElse(throw new RuntimeException(s"$operation is not supported."))
-
- val srcVertexId = VertexId(column, toInnerVal(id.toString, column.columnType, column.schemaVersion))
- val propsInner = column.propsToInnerVals(props) ++
- Map(ColumnMeta.timeStampSeq.toInt -> InnerVal.withLong(ts, column.schemaVersion))
-
- new Vertex(this, srcVertexId, ts, propsInner, op)
- }
-
- override def vertices(objects: AnyRef*): util.Iterator[structure.Vertex] = ???
-
- override def tx(): Transaction = ???
-
- override def edges(objects: AnyRef*): util.Iterator[structure.Edge] = ???
-
- override def variables(): Variables = ???
-
- override def configuration(): Configuration = ???
-
- override def addVertex(objects: AnyRef*): structure.Vertex = ???
-
- override def close(): Unit = ???
-
- override def compute[C <: GraphComputer](aClass: Class[C]): C = ???
-
- override def compute(): GraphComputer = ???
-}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/Management.scala b/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
index 60900be..064a3d1 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
@@ -256,7 +256,7 @@ object Management {
}
}
-class Management(graph: Graph) {
+class Management(graph: S2Graph) {
import Management._
def createStorageTable(zkAddr: String,
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala b/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
index 083159f..b22eb65 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
@@ -53,7 +53,7 @@ object PostProcess {
case _ => Json.obj("message" -> ex.getMessage)
}
- def s2EdgeParent(graph: Graph,
+ def s2EdgeParent(graph: S2Graph,
queryOption: QueryOption,
parentEdges: Seq[EdgeWithScore]): JsValue = {
if (parentEdges.isEmpty) JsNull
@@ -141,7 +141,7 @@ object PostProcess {
}
}
- def s2VertexToJson(s2Vertex: Vertex): Option[JsValue] = {
+ def s2VertexToJson(s2Vertex: S2Vertex): Option[JsValue] = {
val props = for {
(k, v) <- s2Vertex.properties
jsVal <- anyValToJsValue(v)
@@ -160,7 +160,7 @@ object PostProcess {
}
}
- def verticesToJson(s2Vertices: Seq[Vertex]): JsValue =
+ def verticesToJson(s2Vertices: Seq[S2Vertex]): JsValue =
Json.toJson(s2Vertices.flatMap(s2VertexToJson(_)))
def withOptionalFields(queryOption: QueryOption,
@@ -189,7 +189,7 @@ object PostProcess {
case _ => js
}
- def toJson(orgQuery: Option[JsValue])(graph: Graph,
+ def toJson(orgQuery: Option[JsValue])(graph: S2Graph,
queryOption: QueryOption,
stepResult: StepResult): JsValue = {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala b/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
index 170fd0b..eb36258 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
@@ -27,7 +27,7 @@ import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
import org.apache.s2graph.core.parsers.{Where, WhereParser}
import org.apache.s2graph.core.rest.TemplateHelper
import org.apache.s2graph.core.storage.StorageSerializable._
-import org.apache.s2graph.core.types.{InnerVal, InnerValLike, InnerValLikeWithTs, LabelWithDirection}
+import org.apache.s2graph.core.types._
import org.hbase.async.ColumnRangeFilter
import play.api.libs.json.{JsString, JsNull, JsValue, Json}
@@ -39,7 +39,7 @@ object Query {
def apply(query: Query): Query = {
Query(query.vertices, query.steps, query.queryOption, query.jsonQuery)
}
- def toQuery(srcVertices: Seq[Vertex], queryParam: QueryParam) = Query(srcVertices, Vector(Step(List(queryParam))))
+ def toQuery(srcVertices: Seq[S2Vertex], queryParams: Seq[QueryParam]) = Query(srcVertices, Vector(Step(queryParams)))
}
@@ -96,7 +96,7 @@ case class QueryOption(removeCycle: Boolean = false,
}
-case class Query(vertices: Seq[Vertex] = Seq.empty[Vertex],
+case class Query(vertices: Seq[S2Vertex] = Seq.empty[S2Vertex],
steps: IndexedSeq[Step] = Vector.empty[Step],
queryOption: QueryOption = QueryOption(),
jsonQuery: JsValue = JsNull) {
@@ -162,7 +162,7 @@ case class EdgeTransformer(jsValue: JsValue) {
}
}
- def toInnerValOpt(queryParam: QueryParam, edge: Edge, fieldName: String): Option[InnerValLike] = {
+ def toInnerValOpt(queryParam: QueryParam, edge: S2Edge, fieldName: String): Option[InnerValLike] = {
fieldName match {
case LabelMeta.to.name => Option(edge.tgtVertex.innerId)
case LabelMeta.from.name => Option(edge.srcVertex.innerId)
@@ -170,7 +170,7 @@ case class EdgeTransformer(jsValue: JsValue) {
}
}
- def transform(queryParam: QueryParam, edge: Edge, nextStepOpt: Option[Step]): Seq[Edge] = {
+ def transform(queryParam: QueryParam, edge: S2Edge, nextStepOpt: Option[Step]): Seq[S2Edge] = {
if (isDefault) Seq(edge)
else {
val edges = for {
@@ -218,7 +218,7 @@ case class Step(queryParams: Seq[QueryParam],
}
}
-case class VertexParam(vertices: Seq[Vertex]) {
+case class VertexParam(vertices: Seq[S2Vertex]) {
var filters: Option[Map[Byte, InnerValLike]] = None
def has(what: Option[Map[Byte, InnerValLike]]): VertexParam = {
@@ -306,11 +306,10 @@ case class QueryParam(labelName: String,
else label.indexNameMap.getOrElse(indexName, throw new RuntimeException(s"$indexName indexName is not found.")).seq
lazy val tgtVertexInnerIdOpt = tgtVertexIdOpt.map { id =>
- val tmp = label.tgtColumnWithDir(dir)
- toInnerVal(id, tmp.columnType, tmp.schemaVersion)
+ CanInnerValLike.anyToInnerValLike.toInnerVal(id)(label.tgtColumnWithDir(dir).schemaVersion)
}
- def buildInterval(edgeOpt: Option[Edge]) = intervalOpt match {
+ def buildInterval(edgeOpt: Option[S2Edge]) = intervalOpt match {
case None => Array.empty[Byte] -> Array.empty[Byte]
case Some(interval) =>
val (froms, tos) = interval
@@ -358,7 +357,7 @@ case class QueryParam(labelName: String,
Bytes.add(bytes, optionalCacheKey)
}
- private def convertToInner(kvs: Seq[(String, JsValue)], edgeOpt: Option[Edge]): Seq[(LabelMeta, InnerValLike)] = {
+ private def convertToInner(kvs: Seq[(String, JsValue)], edgeOpt: Option[S2Edge]): Seq[(LabelMeta, InnerValLike)] = {
kvs.map { case (propKey, propValJs) =>
propValJs match {
case JsString(in) if edgeOpt.isDefined && in.contains("_parent.") =>
@@ -376,7 +375,7 @@ case class QueryParam(labelName: String,
val propVal =
if (InnerVal.isNumericType(labelMeta.dataType)) {
- InnerVal.withLong(edge.property(labelMeta.name).value.asInstanceOf[BigDecimal].longValue() + padding, label.schemaVersion)
+ InnerVal.withLong(edge.property(labelMeta.name).value.toString.toLong + padding, label.schemaVersion)
} else {
edge.property(labelMeta.name).asInstanceOf[S2Property[_]].innerVal
}
@@ -391,7 +390,7 @@ case class QueryParam(labelName: String,
}
}
- def paddingInterval(len: Byte, froms: Seq[(String, JsValue)], tos: Seq[(String, JsValue)], edgeOpt: Option[Edge] = None) = {
+ def paddingInterval(len: Byte, froms: Seq[(String, JsValue)], tos: Seq[(String, JsValue)], edgeOpt: Option[S2Edge] = None) = {
val fromInnerVal = convertToInner(froms, edgeOpt)
val toInnerVal = convertToInner(tos, edgeOpt)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/e8c0bf20/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala b/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
index 3753d0f..bad8361 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
@@ -27,7 +27,7 @@ import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import scala.collection.{Seq, mutable}
object QueryResult {
- def fromVertices(graph: Graph,
+ def fromVertices(graph: S2Graph,
query: Query): StepResult = {
if (query.steps.isEmpty || query.steps.head.queryParams.isEmpty) {
StepResult.Empty
@@ -41,7 +41,7 @@ object QueryResult {
vertex <- query.vertices
} yield {
val edge = graph.newEdge(vertex, vertex, label, queryParam.labelWithDir.dir, propsWithTs = propsWithTs)
- val edgeWithScore = EdgeWithScore(edge, Graph.DefaultScore, queryParam.label)
+ val edgeWithScore = EdgeWithScore(edge, S2Graph.DefaultScore, queryParam.label)
edgeWithScore
}
StepResult(edgeWithScores = edgeWithScores, grouped = Nil, degreeEdges = Nil, false)
@@ -51,7 +51,7 @@ object QueryResult {
case class QueryRequest(query: Query,
stepIdx: Int,
- vertex: Vertex,
+ vertex: S2Vertex,
queryParam: QueryParam,
prevStepScore: Double = 1.0,
labelWeight: Double = 1.0) {
@@ -73,7 +73,7 @@ object WithScore {
}
}
-case class EdgeWithScore(edge: Edge,
+case class EdgeWithScore(edge: S2Edge,
score: Double,
label: Label,
orderByValues: (Any, Any, Any, Any) = StepResult.EmptyOrderByValues,
@@ -283,7 +283,7 @@ object StepResult {
}
//TODO: Optimize this.
- def filterOut(graph: Graph,
+ def filterOut(graph: S2Graph,
queryOption: QueryOption,
baseStepResult: StepResult,
filterOutStepResult: StepResult): StepResult = {
[06/10] incubator-s2graph git commit: rename S2S2GraphTest into
S2GraphTest.
Posted by st...@apache.org.
rename S2S2GraphTest into S2GraphTest.
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/73106316
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/73106316
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/73106316
Branch: refs/heads/master
Commit: 73106316e50a67f81c054ecaa97a61119b5fff1f
Parents: e8c0bf2
Author: DO YUNG YOON <st...@apache.org>
Authored: Mon Nov 28 16:28:00 2016 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Mon Nov 28 16:28:00 2016 +0900
----------------------------------------------------------------------
.../core/Integrate/tinkerpop/S2GraphTest.scala | 130 +++++++++++++++++++
.../Integrate/tinkerpop/S2S2GraphTest.scala | 130 -------------------
2 files changed, 130 insertions(+), 130 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/73106316/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2GraphTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2GraphTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2GraphTest.scala
new file mode 100644
index 0000000..3eb04ff
--- /dev/null
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2GraphTest.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.s2graph.core.Integrate.tinkerpop
+
+import org.apache.s2graph.core.mysqls.Label
+import org.apache.s2graph.core.utils.logger
+import org.apache.s2graph.core.{S2Graph, TestCommonWithModels, S2Vertex}
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal
+import org.apache.tinkerpop.gremlin.structure.{Edge, Vertex, T}
+import org.scalatest.{FunSuite, Matchers}
+
+
+class S2GraphTest extends FunSuite with Matchers with TestCommonWithModels {
+
+ import scala.collection.JavaConversions._
+ import scala.concurrent.ExecutionContext.Implicits.global
+
+ initTests()
+
+ val g = new S2Graph(config)
+
+ def printEdges(edges: Seq[Edge]): Unit = {
+ edges.foreach { edge =>
+ logger.debug(s"[FetchedEdge]: $edge")
+ }
+ }
+
+ import scala.language.implicitConversions
+
+ def newVertexId(id: Any, label: Label = labelV2) = g.newVertexId(label.srcService, label.srcColumn, id)
+
+ def addVertex(id: AnyRef, label: Label = labelV2) =
+ g.addVertex(T.label, label.srcService.serviceName + S2Vertex.VertexLabelDelimiter + label.srcColumnName,
+ T.id, id).asInstanceOf[S2Vertex]
+
+ val srcId = Long.box(20)
+ val range = (100 until 110)
+ testData(srcId, range)
+
+ // val testProps = Seq(
+ // Prop("affinity_score", "0.0", DOUBLE),
+ // Prop("is_blocked", "false", BOOLEAN),
+ // Prop("time", "0", INT),
+ // Prop("weight", "0", INT),
+ // Prop("is_hidden", "true", BOOLEAN),
+ // Prop("phone_number", "xxx-xxx-xxxx", STRING),
+ // Prop("score", "0.1", FLOAT),
+ // Prop("age", "10", INT)
+ // )
+ def testData(srcId: AnyRef, range: Range, label: Label = labelV2) = {
+ val src = addVertex(srcId)
+
+ for {
+ i <- range
+ } {
+ val tgt = addVertex(Int.box(i))
+
+ src.addEdge(labelV2.label, tgt,
+ "age", Int.box(10),
+ "affinity_score", Double.box(0.1),
+ "is_blocked", Boolean.box(true),
+ "ts", Long.box(i))
+ }
+ }
+
+ test("test traversal.") {
+ val vertices = g.traversal().V(newVertexId(srcId)).out(labelV2.label).toSeq
+
+ vertices.size should be(range.size)
+ range.reverse.zip(vertices).foreach { case (tgtId, vertex) =>
+ val vertexId = g.newVertexId(labelV2.tgtService, labelV2.tgtColumn, tgtId)
+ val expectedId = g.newVertex(vertexId)
+ vertex.asInstanceOf[S2Vertex].innerId should be(expectedId.innerId)
+ }
+ }
+
+ test("test traversal. limit 1") {
+ val vertexIdParams = Seq(newVertexId(srcId))
+ val t: GraphTraversal[Vertex, Double] = g.traversal().V(vertexIdParams: _*).outE(labelV2.label).limit(1).values("affinity_score")
+ for {
+ affinityScore <- t
+ } {
+ logger.debug(s"$affinityScore")
+ affinityScore should be (0.1)
+ }
+ }
+ test("test traversal. 3") {
+
+ val l = label
+
+ val srcA = addVertex(Long.box(1), l)
+ val srcB = addVertex(Long.box(2), l)
+ val srcC = addVertex(Long.box(3), l)
+
+ val tgtA = addVertex(Long.box(101), l)
+ val tgtC = addVertex(Long.box(103), l)
+
+ srcA.addEdge(l.label, tgtA)
+ srcA.addEdge(l.label, tgtC)
+ tgtC.addEdge(l.label, srcB)
+ tgtA.addEdge(l.label, srcC)
+
+ val vertexIdParams = Seq(srcA.id)
+ val vertices = g.traversal().V(vertexIdParams: _*).out(l.label).out(l.label).toSeq
+ vertices.size should be(2)
+ vertices.foreach { v =>
+ val vertex = v.asInstanceOf[S2Vertex]
+ // TODO: we have too many id. this is ugly and confusing so fix me.
+ vertex.id.innerId == srcB.id.innerId || vertex.id.innerId == srcC.id.innerId should be(true)
+ logger.debug(s"[Vertex]: $v")
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/73106316/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala
deleted file mode 100644
index 3eb04ff..0000000
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/tinkerpop/S2S2GraphTest.scala
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.s2graph.core.Integrate.tinkerpop
-
-import org.apache.s2graph.core.mysqls.Label
-import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{S2Graph, TestCommonWithModels, S2Vertex}
-import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal
-import org.apache.tinkerpop.gremlin.structure.{Edge, Vertex, T}
-import org.scalatest.{FunSuite, Matchers}
-
-
-class S2GraphTest extends FunSuite with Matchers with TestCommonWithModels {
-
- import scala.collection.JavaConversions._
- import scala.concurrent.ExecutionContext.Implicits.global
-
- initTests()
-
- val g = new S2Graph(config)
-
- def printEdges(edges: Seq[Edge]): Unit = {
- edges.foreach { edge =>
- logger.debug(s"[FetchedEdge]: $edge")
- }
- }
-
- import scala.language.implicitConversions
-
- def newVertexId(id: Any, label: Label = labelV2) = g.newVertexId(label.srcService, label.srcColumn, id)
-
- def addVertex(id: AnyRef, label: Label = labelV2) =
- g.addVertex(T.label, label.srcService.serviceName + S2Vertex.VertexLabelDelimiter + label.srcColumnName,
- T.id, id).asInstanceOf[S2Vertex]
-
- val srcId = Long.box(20)
- val range = (100 until 110)
- testData(srcId, range)
-
- // val testProps = Seq(
- // Prop("affinity_score", "0.0", DOUBLE),
- // Prop("is_blocked", "false", BOOLEAN),
- // Prop("time", "0", INT),
- // Prop("weight", "0", INT),
- // Prop("is_hidden", "true", BOOLEAN),
- // Prop("phone_number", "xxx-xxx-xxxx", STRING),
- // Prop("score", "0.1", FLOAT),
- // Prop("age", "10", INT)
- // )
- def testData(srcId: AnyRef, range: Range, label: Label = labelV2) = {
- val src = addVertex(srcId)
-
- for {
- i <- range
- } {
- val tgt = addVertex(Int.box(i))
-
- src.addEdge(labelV2.label, tgt,
- "age", Int.box(10),
- "affinity_score", Double.box(0.1),
- "is_blocked", Boolean.box(true),
- "ts", Long.box(i))
- }
- }
-
- test("test traversal.") {
- val vertices = g.traversal().V(newVertexId(srcId)).out(labelV2.label).toSeq
-
- vertices.size should be(range.size)
- range.reverse.zip(vertices).foreach { case (tgtId, vertex) =>
- val vertexId = g.newVertexId(labelV2.tgtService, labelV2.tgtColumn, tgtId)
- val expectedId = g.newVertex(vertexId)
- vertex.asInstanceOf[S2Vertex].innerId should be(expectedId.innerId)
- }
- }
-
- test("test traversal. limit 1") {
- val vertexIdParams = Seq(newVertexId(srcId))
- val t: GraphTraversal[Vertex, Double] = g.traversal().V(vertexIdParams: _*).outE(labelV2.label).limit(1).values("affinity_score")
- for {
- affinityScore <- t
- } {
- logger.debug(s"$affinityScore")
- affinityScore should be (0.1)
- }
- }
- test("test traversal. 3") {
-
- val l = label
-
- val srcA = addVertex(Long.box(1), l)
- val srcB = addVertex(Long.box(2), l)
- val srcC = addVertex(Long.box(3), l)
-
- val tgtA = addVertex(Long.box(101), l)
- val tgtC = addVertex(Long.box(103), l)
-
- srcA.addEdge(l.label, tgtA)
- srcA.addEdge(l.label, tgtC)
- tgtC.addEdge(l.label, srcB)
- tgtA.addEdge(l.label, srcC)
-
- val vertexIdParams = Seq(srcA.id)
- val vertices = g.traversal().V(vertexIdParams: _*).out(l.label).out(l.label).toSeq
- vertices.size should be(2)
- vertices.foreach { v =>
- val vertex = v.asInstanceOf[S2Vertex]
- // TODO: we have too many id. this is ugly and confusing so fix me.
- vertex.id.innerId == srcB.id.innerId || vertex.id.innerId == srcC.id.innerId should be(true)
- logger.debug(s"[Vertex]: $v")
- }
- }
-}
\ No newline at end of file