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 2017/11/19 02:28:59 UTC

[16/23] incubator-s2graph git commit: add S2GraphLike.

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/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 8e3fdf3..6ebc90c 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
@@ -28,9 +28,9 @@ import org.apache.s2graph.core.storage.serde.StorageDeserializable._
 import org.apache.s2graph.core.types._
 
 object IndexEdgeDeserializable{
-  def getNewInstance(graph: S2Graph) = new IndexEdgeDeserializable(graph)
+  def getNewInstance(graph: S2GraphLike) = new IndexEdgeDeserializable(graph)
 }
-class IndexEdgeDeserializable(graph: S2Graph,
+class IndexEdgeDeserializable(graph: S2GraphLike,
                               bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong,
                               tallSchemaVersions: Set[String] = Set(HBaseType.VERSION4)) extends Deserializable[S2EdgeLike] {
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/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 f5d9af9..79e3f2e 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
@@ -27,7 +27,7 @@ import org.apache.s2graph.core.storage._
 import org.apache.s2graph.core.storage.serde.Deserializable
 import org.apache.s2graph.core.types._
 
-class IndexEdgeDeserializable(graph: S2Graph,
+class IndexEdgeDeserializable(graph: S2GraphLike,
                               bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[S2EdgeLike] {
 
    type QualifierRaw = (Array[(LabelMeta, InnerValLike)], VertexId, Byte, Boolean, Int)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/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 408822a..4c97f6e 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
@@ -27,7 +27,7 @@ import org.apache.s2graph.core.types._
 import org.apache.s2graph.core._
 import org.apache.s2graph.core.storage.serde.Deserializable
 
-class SnapshotEdgeDeserializable(graph: S2Graph) extends Deserializable[SnapshotEdge] {
+class SnapshotEdgeDeserializable(graph: S2GraphLike) extends Deserializable[SnapshotEdge] {
 
   def statusCodeWithOp(byte: Byte): (Byte, Byte) = {
     val statusCode = byte >> 4

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/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 ff8eb80..a6a64a2 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
@@ -27,7 +27,7 @@ import org.apache.s2graph.core.types.{HBaseType, LabelWithDirection, SourceVerte
 import org.apache.s2graph.core._
 import org.apache.s2graph.core.storage.serde.Deserializable
 
-class SnapshotEdgeDeserializable(graph: S2Graph) extends Deserializable[SnapshotEdge] {
+class SnapshotEdgeDeserializable(graph: S2GraphLike) extends Deserializable[SnapshotEdge] {
 
   def statusCodeWithOp(byte: Byte): (Byte, Byte) = {
     val statusCode = byte >> 4

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/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
deleted file mode 100644
index 87f0947..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexDeserializable.scala
+++ /dev/null
@@ -1,73 +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.storage.serde.vertex
-//
-//import org.apache.s2graph.core.mysqls.ColumnMeta
-//import org.apache.s2graph.core.storage.serde.StorageDeserializable._
-//import org.apache.s2graph.core.storage.CanSKeyValue
-//import org.apache.s2graph.core.storage.serde.Deserializable
-//import org.apache.s2graph.core.types.{HBaseType, InnerVal, InnerValLike, VertexId}
-//import org.apache.s2graph.core.{S2Graph, S2Vertex}
-//
-//import scala.collection.mutable.ListBuffer
-//
-//class VertexDeserializable(graph: S2Graph,
-//                           bytesToInt: (Array[Byte], Int) => Int = bytesToInt) extends Deserializable[S2Vertex] {
-//  def fromKeyValues[T: CanSKeyValue](_kvs: Seq[T],
-//                                          cacheElementOpt: Option[S2Vertex]): Option[S2Vertex] = {
-//    try {
-//      val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
-//      val kv = kvs.head
-//      val version = HBaseType.DEFAULT_VERSION
-//      val (vertexId, _) = VertexId.fromBytes(kv.row, 0, kv.row.length, version)
-//
-//      var maxTs = Long.MinValue
-//      val propsMap = new collection.mutable.HashMap[ColumnMeta, InnerValLike]
-//      val belongLabelIds = new ListBuffer[Int]
-//
-//      for {
-//        kv <- kvs
-//      } {
-//        val propKey =
-//          if (kv.qualifier.length == 1) kv.qualifier.head.toInt
-//          else bytesToInt(kv.qualifier, 0)
-//
-//        val ts = kv.timestamp
-//        if (ts > maxTs) maxTs = ts
-//
-//        if (S2Vertex.isLabelId(propKey)) {
-//          belongLabelIds += S2Vertex.toLabelId(propKey)
-//        } else {
-//          val v = kv.value
-//          val (value, _) = InnerVal.fromBytes(v, 0, v.length, version)
-//          val columnMeta = vertexId.column.metasMap(propKey)
-//          propsMap += (columnMeta -> value)
-//        }
-//      }
-//      assert(maxTs != Long.MinValue)
-//      val vertex = graph.newVertex(vertexId, maxTs, S2Vertex.EmptyProps, belongLabelIds = belongLabelIds)
-//      S2Vertex.fillPropsWithTs(vertex, propsMap.toMap)
-//
-//      Option(vertex)
-//    } catch {
-//      case e: Exception => None
-//    }
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/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
deleted file mode 100644
index aa85574..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/VertexSerializable.scala
+++ /dev/null
@@ -1,62 +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.storage.serde.vertex
-//
-//import org.apache.s2graph.core.S2Vertex
-//import org.apache.s2graph.core.storage.serde.StorageSerializable._
-//import org.apache.s2graph.core.storage.SKeyValue
-//import org.apache.s2graph.core.storage.serde.Serializable
-//
-//import scala.collection.JavaConverters._
-//
-//case class VertexSerializable(vertex: S2Vertex, intToBytes: Int => Array[Byte] = intToBytes) extends Serializable[S2Vertex] {
-//
-//  override val table = vertex.hbaseTableName.getBytes
-//  override val ts = vertex.ts
-//  override val cf = Serializable.vertexCf
-//
-//  override def toRowKey: Array[Byte] = vertex.id.bytes
-//
-//  override def toQualifier: Array[Byte] = Array.empty[Byte]
-//
-//  override def toValue: Array[Byte] = {
-//    val props = for ((k, v) <- vertex.props.asScala ++ vertex.defaultProps.asScala) yield {
-//      v.columnMeta -> v.innerVal.bytes
-//    }
-//    vertexPropsToBytes(props.toSeq)
-//  }
-//
-//  /** vertex override toKeyValues since vertex expect to produce multiple sKeyValues */
-//  override def toKeyValues: Seq[SKeyValue] = {
-//    val row = toRowKey
-//    // serializer all props into value.
-//    Seq(
-//      SKeyValue(vertex.hbaseTableName.getBytes, row, cf, toQualifier, toValue, vertex.ts)
-//    )
-////    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
-//  }
-//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexDeserializable.scala
index 019394e..70160a8 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexDeserializable.scala
@@ -24,9 +24,9 @@ import org.apache.s2graph.core.storage.CanSKeyValue
 import org.apache.s2graph.core.storage.serde.Deserializable
 import org.apache.s2graph.core.storage.serde.StorageDeserializable._
 import org.apache.s2graph.core.types.{HBaseType, InnerValLike, VertexId}
-import org.apache.s2graph.core.{S2Graph, S2Vertex, S2VertexLike}
+import org.apache.s2graph.core.{S2Graph, S2GraphLike, S2Vertex, S2VertexLike}
 
-class VertexDeserializable(graph: S2Graph,
+class VertexDeserializable(graph: S2GraphLike,
                            bytesToInt: (Array[Byte], Int) => Int = bytesToInt) extends Deserializable[S2VertexLike] {
   def fromKeyValues[T: CanSKeyValue](_kvs: Seq[T],
                                           cacheElementOpt: Option[S2VertexLike]): Option[S2VertexLike] = {

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexDeserializable.scala
index ddb90ac..002f577 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexDeserializable.scala
@@ -24,11 +24,11 @@ import org.apache.s2graph.core.storage.CanSKeyValue
 import org.apache.s2graph.core.storage.serde.Deserializable
 import org.apache.s2graph.core.storage.serde.StorageDeserializable._
 import org.apache.s2graph.core.types.{HBaseType, InnerVal, InnerValLike, VertexId}
-import org.apache.s2graph.core.{S2Graph, S2Vertex, S2VertexLike}
+import org.apache.s2graph.core.{S2Graph, S2GraphLike, S2Vertex, S2VertexLike}
 
 import scala.collection.mutable.ListBuffer
 
-class VertexDeserializable(graph: S2Graph,
+class VertexDeserializable(graph: S2GraphLike,
                            bytesToInt: (Array[Byte], Int) => Int = bytesToInt) extends Deserializable[S2VertexLike] {
   def fromKeyValues[T: CanSKeyValue](_kvs: Seq[T],
                                           cacheElementOpt: Option[S2VertexLike]): Option[S2VertexLike] = {

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6403bc9d/s2core/src/test/scala/org/apache/s2graph/core/tinkerpop/S2GraphProvider.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/tinkerpop/S2GraphProvider.scala b/s2core/src/test/scala/org/apache/s2graph/core/tinkerpop/S2GraphProvider.scala
index c7d474b..3a73d20 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/tinkerpop/S2GraphProvider.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/tinkerpop/S2GraphProvider.scala
@@ -86,7 +86,7 @@ class S2GraphProvider extends AbstractGraphProvider {
 
   override def loadGraphData(graph: Graph, loadGraphWith: LoadGraphWith, testClass: Class[_], testName: String): Unit = {
     val s2Graph = graph.asInstanceOf[S2Graph]
-    val mnt = s2Graph.getManagement()
+    val mnt = s2Graph.management
 
     S2GraphFactory.cleanupDefaultSchema
     initTestSchema(testClass, testName)