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/10/30 10:23:29 UTC
[1/8] incubator-s2graph git commit: Separate interfaces from Storage.
Repository: incubator-s2graph
Updated Branches:
refs/heads/master ed9bedf0d -> 128d67c06
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala
new file mode 100644
index 0000000..1f7d863
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala
@@ -0,0 +1,118 @@
+package org.apache.s2graph.core.storage.hbase
+
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core.storage.{IncrementResponse, MutateResponse, SKeyValue, StorageWritable}
+import org.apache.s2graph.core.utils.{Extensions, logger}
+import org.hbase.async.{AtomicIncrementRequest, DeleteRequest, HBaseClient, PutRequest}
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.{ExecutionContext, Future}
+
+class AsynchbaseStorageWritable(val client: HBaseClient,
+ val clientWithFlush: HBaseClient) extends StorageWritable {
+ import Extensions.DeferOps
+
+ private def client(withWait: Boolean): HBaseClient = if (withWait) clientWithFlush else client
+ /**
+ * decide how to store given key values Seq[SKeyValue] into storage using storage's client.
+ * note that this should be return true on all success.
+ * we assumes that each storage implementation has client as member variable.
+ *
+ * @param cluster : where this key values should be stored.
+ * @param kvs : sequence of SKeyValue that need to be stored in storage.
+ * @param withWait : flag to control wait ack from storage.
+ * note that in AsynchbaseStorage(which support asynchronous operations), even with true,
+ * it never block thread, but rather submit work and notified by event loop when storage send ack back.
+ * @return ack message from storage.
+ */
+ override def writeToStorage(cluster: String, kvs: Seq[SKeyValue], withWait: Boolean)(implicit ec: ExecutionContext) = {
+ if (kvs.isEmpty) Future.successful(MutateResponse.Success)
+ else {
+ val _client = client(withWait)
+ val (increments, putAndDeletes) = kvs.partition(_.operation == SKeyValue.Increment)
+
+ /* Asynchbase IncrementRequest does not implement HasQualifiers */
+ val incrementsFutures = increments.map { kv =>
+ val countVal = Bytes.toLong(kv.value)
+ val request = new AtomicIncrementRequest(kv.table, kv.row, kv.cf, kv.qualifier, countVal)
+ val fallbackFn: (Exception => MutateResponse) = { ex =>
+ logger.error(s"mutation failed. $request", ex)
+ new IncrementResponse(false, -1L, -1L)
+ }
+ val future = _client.bufferAtomicIncrement(request).mapWithFallback(0L)(fallbackFn) { resultCount: java.lang.Long =>
+ new IncrementResponse(true, resultCount.longValue(), countVal)
+ }.toFuture(MutateResponse.IncrementFailure)
+
+ if (withWait) future else Future.successful(MutateResponse.IncrementSuccess)
+ }
+
+ /* PutRequest and DeleteRequest accept byte[][] qualifiers/values. */
+ val othersFutures = putAndDeletes.groupBy { kv =>
+ (kv.table.toSeq, kv.row.toSeq, kv.cf.toSeq, kv.operation, kv.timestamp)
+ }.map { case ((table, row, cf, operation, timestamp), groupedKeyValues) =>
+
+ val durability = groupedKeyValues.head.durability
+ val qualifiers = new ArrayBuffer[Array[Byte]]()
+ val values = new ArrayBuffer[Array[Byte]]()
+
+ groupedKeyValues.foreach { kv =>
+ if (kv.qualifier != null) qualifiers += kv.qualifier
+ if (kv.value != null) values += kv.value
+ }
+ val defer = operation match {
+ case SKeyValue.Put =>
+ val put = new PutRequest(table.toArray, row.toArray, cf.toArray, qualifiers.toArray, values.toArray, timestamp)
+ put.setDurable(durability)
+ _client.put(put)
+ case SKeyValue.Delete =>
+ val delete =
+ if (qualifiers.isEmpty)
+ new DeleteRequest(table.toArray, row.toArray, cf.toArray, timestamp)
+ else
+ new DeleteRequest(table.toArray, row.toArray, cf.toArray, qualifiers.toArray, timestamp)
+ delete.setDurable(durability)
+ _client.delete(delete)
+ }
+ if (withWait) {
+ defer.toFuture(new AnyRef()).map(_ => MutateResponse.Success).recover { case ex: Exception =>
+ groupedKeyValues.foreach { kv => logger.error(s"mutation failed. $kv", ex) }
+ MutateResponse.Failure
+ }
+ } else Future.successful(MutateResponse.Success)
+ }
+ for {
+ incrementRets <- Future.sequence(incrementsFutures)
+ otherRets <- Future.sequence(othersFutures)
+ } yield new MutateResponse(isSuccess = (incrementRets ++ otherRets).forall(_.isSuccess))
+ }
+ }
+
+ /**
+ * write requestKeyValue into storage if the current value in storage that is stored matches.
+ * note that we only use SnapshotEdge as place for lock, so this method only change SnapshotEdge.
+ *
+ * Most important thing is this have to be 'atomic' operation.
+ * When this operation is mutating requestKeyValue's snapshotEdge, then other thread need to be
+ * either blocked or failed on write-write conflict case.
+ *
+ * Also while this method is still running, then fetchSnapshotEdgeKeyValues should be synchronized to
+ * prevent wrong data for read.
+ *
+ * Best is use storage's concurrency control(either pessimistic or optimistic) such as transaction,
+ * compareAndSet to synchronize.
+ *
+ * for example, AsynchbaseStorage use HBase's CheckAndSet atomic operation to guarantee 'atomicity'.
+ * for storage that does not support concurrency control, then storage implementation
+ * itself can maintain manual locks that synchronize read(fetchSnapshotEdgeKeyValues)
+ * and write(writeLock).
+ *
+ * @param rpc
+ * @param expectedOpt
+ * @return
+ */
+ override def writeLock(rpc: SKeyValue, expectedOpt: Option[SKeyValue])(implicit ec: ExecutionContext): Future[MutateResponse] = {
+ val put = new PutRequest(rpc.table, rpc.row, rpc.cf, rpc.qualifier, rpc.value, rpc.timestamp)
+ val expected = expectedOpt.map(_.value).getOrElse(Array.empty)
+ client(withWait = true).compareAndSet(put, expected).map(true.booleanValue())(ret => ret.booleanValue()).toFuture(true)
+ .map(r => new MutateResponse(isSuccess = r))
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Deserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Deserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Deserializable.scala
new file mode 100644
index 0000000..349bff3
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Deserializable.scala
@@ -0,0 +1,41 @@
+/*
+ * 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
+
+import org.apache.s2graph.core.types.{LabelWithDirection, VertexId}
+
+
+trait Deserializable[E] extends StorageDeserializable[E] {
+
+ type RowKeyRaw = (VertexId, LabelWithDirection, Byte, Boolean, Int)
+
+// /** version 1 and version 2 share same code for parsing row key part */
+// def parseRow(kv: SKeyValue, version: String = HBaseType.DEFAULT_VERSION): RowKeyRaw = {
+// var pos = 0
+// val (srcVertexId, srcIdLen) = SourceVertexId.fromBytes(kv.row, pos, kv.row.length, version)
+// pos += srcIdLen
+// val labelWithDir = LabelWithDirection(Bytes.toInt(kv.row, pos, 4))
+// pos += 4
+// val (labelIdxSeq, isInverted) = bytesToLabelIndexSeqWithIsInverted(kv.row, pos)
+//
+// val rowLen = srcIdLen + 4 + 1
+// (srcVertexId, labelWithDir, labelIdxSeq, isInverted, rowLen)
+// }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Serializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Serializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Serializable.scala
new file mode 100644
index 0000000..46b4860
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/Serializable.scala
@@ -0,0 +1,27 @@
+/*
+ * 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
+
+object Serializable {
+ val vertexCf = "v".getBytes("UTF-8")
+ val edgeCf = "e".getBytes("UTF-8")
+}
+
+trait Serializable[E] extends StorageSerializable[E]
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageDeserializable.scala
new file mode 100644
index 0000000..dc7690b
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageDeserializable.scala
@@ -0,0 +1,144 @@
+/*
+ * 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
+
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core.mysqls.{ColumnMeta, Label, LabelMeta, ServiceColumn}
+import org.apache.s2graph.core.storage.CanSKeyValue
+import org.apache.s2graph.core.types.{InnerVal, InnerValLike, InnerValLikeWithTs}
+
+object StorageDeserializable {
+ /** Deserializer */
+ def bytesToLabelIndexSeqWithIsInverted(bytes: Array[Byte], offset: Int): (Byte, Boolean) = {
+ val byte = bytes(offset)
+ val isInverted = if ((byte & 1) != 0) true else false
+ val labelOrderSeq = byte >> 1
+ (labelOrderSeq.toByte, isInverted)
+ }
+
+ def bytesToKeyValues(bytes: Array[Byte],
+ offset: Int,
+ length: Int,
+ schemaVer: String,
+ serviceColumn: ServiceColumn): (Array[(ColumnMeta, InnerValLike)], Int) = {
+ var pos = offset
+ val len = bytes(pos)
+ pos += 1
+ val kvs = new Array[(ColumnMeta, InnerValLike)](len)
+ var i = 0
+ while (i < len) {
+ val kSeq = Bytes.toInt(bytes, pos, 4)
+ val k = serviceColumn.metasMap(kSeq)
+ pos += 4
+
+ val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, schemaVer)
+ pos += numOfBytesUsed
+ kvs(i) = (k -> v)
+ i += 1
+ }
+ val ret = (kvs, pos)
+ // logger.debug(s"bytesToProps: $ret")
+ ret
+ }
+
+ def bytesToKeyValues(bytes: Array[Byte],
+ offset: Int,
+ length: Int,
+ schemaVer: String,
+ label: Label): (Array[(LabelMeta, InnerValLike)], Int) = {
+ var pos = offset
+ val len = bytes(pos)
+ pos += 1
+ val kvs = new Array[(LabelMeta, InnerValLike)](len)
+ var i = 0
+ while (i < len) {
+ val k = label.labelMetaMap(bytes(pos))
+ pos += 1
+ val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, schemaVer)
+ pos += numOfBytesUsed
+ kvs(i) = (k -> v)
+ i += 1
+ }
+ val ret = (kvs, pos)
+ // logger.debug(s"bytesToProps: $ret")
+ ret
+ }
+
+ def bytesToKeyValuesWithTs(bytes: Array[Byte],
+ offset: Int,
+ schemaVer: String,
+ label: Label): (Array[(LabelMeta, InnerValLikeWithTs)], Int) = {
+ var pos = offset
+ val len = bytes(pos)
+ pos += 1
+ val kvs = new Array[(LabelMeta, InnerValLikeWithTs)](len)
+ var i = 0
+ while (i < len) {
+ val k = label.labelMetaMap(bytes(pos))
+ pos += 1
+ val (v, numOfBytesUsed) = InnerValLikeWithTs.fromBytes(bytes, pos, 0, schemaVer)
+ pos += numOfBytesUsed
+ kvs(i) = (k -> v)
+ i += 1
+ }
+ val ret = (kvs, pos)
+ // logger.debug(s"bytesToProps: $ret")
+ ret
+ }
+
+ def bytesToProps(bytes: Array[Byte],
+ offset: Int,
+ schemaVer: String): (Array[(LabelMeta, InnerValLike)], Int) = {
+ var pos = offset
+ val len = bytes(pos)
+ pos += 1
+ val kvs = new Array[(LabelMeta, InnerValLike)](len)
+ var i = 0
+ while (i < len) {
+ val k = LabelMeta.empty
+ val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, schemaVer)
+ pos += numOfBytesUsed
+ kvs(i) = (k -> v)
+ i += 1
+ }
+ // logger.error(s"bytesToProps: $kvs")
+ val ret = (kvs, pos)
+
+ ret
+ }
+
+ def bytesToLong(bytes: Array[Byte], offset: Int): Long = Bytes.toLong(bytes, offset)
+
+ def bytesToInt(bytes: Array[Byte], offset: Int): Int = Bytes.toInt(bytes, offset)
+}
+
+trait StorageDeserializable[E] {
+ def fromKeyValues[T: CanSKeyValue](kvs: Seq[T], cacheElementOpt: Option[E]): Option[E]
+// = {
+// try {
+// Option(fromKeyValuesInner(kvs, cacheElementOpt))
+// } catch {
+// case e: Exception =>
+// logger.error(s"${this.getClass.getName} fromKeyValues failed.", e)
+// None
+// }
+// }
+// def fromKeyValuesInner[T: CanSKeyValue](kvs: Seq[T], cacheElementOpt: Option[E]): E
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageSerializable.scala
new file mode 100644
index 0000000..219d097
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/StorageSerializable.scala
@@ -0,0 +1,90 @@
+/*
+ * 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
+
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core.mysqls.{ColumnMeta, LabelMeta}
+import org.apache.s2graph.core.storage.SKeyValue
+import org.apache.s2graph.core.types.{InnerValLike, InnerValLikeWithTs}
+
+object StorageSerializable {
+ /** serializer */
+ def propsToBytes(props: Seq[(LabelMeta, InnerValLike)]): Array[Byte] = {
+ val len = props.length
+ assert(len < Byte.MaxValue)
+ var bytes = Array.fill(1)(len.toByte)
+ for ((_, v) <- props) bytes = Bytes.add(bytes, v.bytes)
+ bytes
+ }
+
+ def vertexPropsToBytes(props: Seq[(ColumnMeta, Array[Byte])]): Array[Byte] = {
+ val len = props.length
+ assert(len < Byte.MaxValue)
+ var bytes = Array.fill(1)(len.toByte)
+ for ((k, v) <- props) bytes = Bytes.add(bytes, Bytes.toBytes(k.seq.toInt), v)
+ bytes
+ }
+
+ def propsToKeyValues(props: Seq[(LabelMeta, InnerValLike)]): Array[Byte] = {
+ val len = props.length
+ assert(len < Byte.MaxValue)
+ var bytes = Array.fill(1)(len.toByte)
+ for ((k, v) <- props) bytes = Bytes.add(bytes, Array.fill(1)(k.seq), v.bytes)
+ bytes
+ }
+
+ def propsToKeyValuesWithTs(props: Seq[(LabelMeta, InnerValLikeWithTs)]): Array[Byte] = {
+ val len = props.length
+ assert(len < Byte.MaxValue)
+ var bytes = Array.fill(1)(len.toByte)
+ for ((k, v) <- props) bytes = Bytes.add(bytes, Array.fill(1)(k.seq), v.bytes)
+ bytes
+ }
+
+ def labelOrderSeqWithIsInverted(labelOrderSeq: Byte, isInverted: Boolean): Array[Byte] = {
+ assert(labelOrderSeq < (1 << 6))
+ val byte = labelOrderSeq << 1 | (if (isInverted) 1 else 0)
+ Array.fill(1)(byte.toByte)
+ }
+
+ def intToBytes(value: Int): Array[Byte] = Bytes.toBytes(value)
+
+ def longToBytes(value: Long): Array[Byte] = Bytes.toBytes(value)
+}
+
+trait StorageSerializable[E] {
+ val cf = Serializable.edgeCf
+
+ def table: Array[Byte]
+ def ts: Long
+
+ def toRowKey: Array[Byte]
+ def toQualifier: Array[Byte]
+ def toValue: Array[Byte]
+
+ def toKeyValues: Seq[SKeyValue] = {
+ val row = toRowKey
+ val qualifier = toQualifier
+ val value = toValue
+ val kv = SKeyValue(table, row, cf, qualifier, value, ts)
+// logger.debug(s"[SER]: ${kv.toLogString}}")
+ Seq(kv)
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 3da8267..2501ed9 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
@@ -22,16 +22,18 @@ package org.apache.s2graph.core.storage.serde.indexedge.tall
import org.apache.hadoop.hbase.util.Bytes
import org.apache.s2graph.core._
import org.apache.s2graph.core.mysqls.{Label, LabelMeta, ServiceColumn}
-import org.apache.s2graph.core.storage.StorageDeserializable._
-import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable, StorageDeserializable}
+import org.apache.s2graph.core.storage.serde._
+import org.apache.s2graph.core.storage.serde.StorageDeserializable._
+import org.apache.s2graph.core.storage.serde.Deserializable
+import org.apache.s2graph.core.storage.CanSKeyValue
import org.apache.s2graph.core.types._
object IndexEdgeDeserializable{
def getNewInstance(graph: S2Graph) = new IndexEdgeDeserializable(graph)
}
class IndexEdgeDeserializable(graph: S2Graph,
- bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[S2Edge] {
- import StorageDeserializable._
+ bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong,
+ tallSchemaVersions: Set[String] = Set(HBaseType.VERSION4)) extends Deserializable[S2Edge] {
type QualifierRaw = (Array[(LabelMeta, InnerValLike)], VertexId, Byte, Boolean, Int)
type ValueRaw = (Array[(LabelMeta, InnerValLike)], Int)
@@ -65,7 +67,7 @@ class IndexEdgeDeserializable(graph: S2Graph,
val edge = graph.newEdge(srcVertex, null,
label, labelWithDir.dir, GraphUtil.defaultOpByte, version, S2Edge.EmptyState)
var tsVal = version
- val isTallSchema = label.schemaVersion == HBaseType.VERSION4
+ val isTallSchema = tallSchemaVersions(label.schemaVersion)
val isDegree = if (isTallSchema) pos == kv.row.length else kv.qualifier.isEmpty
if (isDegree) {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeSerializable.scala
index 632eefa..28982dc 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeSerializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/tall/IndexEdgeSerializable.scala
@@ -21,13 +21,12 @@ package org.apache.s2graph.core.storage.serde.indexedge.tall
import org.apache.hadoop.hbase.util.Bytes
import org.apache.s2graph.core.mysqls.LabelMeta
-import org.apache.s2graph.core.storage.{Serializable, StorageSerializable}
import org.apache.s2graph.core.types.VertexId
import org.apache.s2graph.core.{GraphUtil, IndexEdge}
-import org.apache.s2graph.core.storage.StorageSerializable._
+import org.apache.s2graph.core.storage.serde.StorageSerializable._
+import org.apache.s2graph.core.storage.serde.Serializable
class IndexEdgeSerializable(indexEdge: IndexEdge, longToBytes: Long => Array[Byte] = longToBytes) extends Serializable[IndexEdge] {
- import StorageSerializable._
override def ts = indexEdge.version
override def table = indexEdge.label.hbaseTableName.getBytes("UTF-8")
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 59db07e..68732ce 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
@@ -22,13 +22,13 @@ package org.apache.s2graph.core.storage.serde.indexedge.wide
import org.apache.hadoop.hbase.util.Bytes
import org.apache.s2graph.core._
import org.apache.s2graph.core.mysqls.{Label, LabelMeta, ServiceColumn}
-import org.apache.s2graph.core.storage.StorageDeserializable._
+import org.apache.s2graph.core.storage.serde.StorageDeserializable._
import org.apache.s2graph.core.storage._
+import org.apache.s2graph.core.storage.serde.Deserializable
import org.apache.s2graph.core.types._
class IndexEdgeDeserializable(graph: S2Graph,
bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[S2Edge] {
- import StorageDeserializable._
type QualifierRaw = (Array[(LabelMeta, InnerValLike)], VertexId, Byte, Boolean, Int)
type ValueRaw = (Array[(LabelMeta, InnerValLike)], Int)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeSerializable.scala
index 434db02..34e9a6e 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeSerializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/indexedge/wide/IndexEdgeSerializable.scala
@@ -21,13 +21,12 @@ package org.apache.s2graph.core.storage.serde.indexedge.wide
import org.apache.hadoop.hbase.util.Bytes
import org.apache.s2graph.core.mysqls.LabelMeta
-import org.apache.s2graph.core.storage.{SKeyValue, Serializable, StorageSerializable}
import org.apache.s2graph.core.types.VertexId
import org.apache.s2graph.core.{GraphUtil, IndexEdge}
-import org.apache.s2graph.core.storage.StorageSerializable._
+import org.apache.s2graph.core.storage.serde.StorageSerializable._
+import org.apache.s2graph.core.storage.serde.Serializable
class IndexEdgeSerializable(indexEdge: IndexEdge, longToBytes: Long => Array[Byte] = longToBytes) extends Serializable[IndexEdge] {
- import StorageSerializable._
override def ts = indexEdge.version
override def table = indexEdge.label.hbaseTableName.getBytes("UTF-8")
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 3b55ed8..b618962 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
@@ -20,11 +20,12 @@
package org.apache.s2graph.core.storage.serde.snapshotedge.tall
import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.mysqls.{ServiceColumn, Label, LabelIndex, LabelMeta}
-import org.apache.s2graph.core.storage.StorageDeserializable._
-import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable, SKeyValue, StorageDeserializable}
+import org.apache.s2graph.core.mysqls.{Label, LabelMeta, ServiceColumn}
+import org.apache.s2graph.core.storage.serde.StorageDeserializable._
+import org.apache.s2graph.core.storage.CanSKeyValue
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] {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeSerializable.scala
index 76fb74d..5f00b48 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeSerializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/tall/SnapshotEdgeSerializable.scala
@@ -22,12 +22,12 @@ package org.apache.s2graph.core.storage.serde.snapshotedge.tall
import org.apache.hadoop.hbase.util.Bytes
import org.apache.s2graph.core.SnapshotEdge
import org.apache.s2graph.core.mysqls.LabelIndex
-import org.apache.s2graph.core.storage.{SKeyValue, Serializable, StorageSerializable}
+import org.apache.s2graph.core.storage.serde._
+import org.apache.s2graph.core.storage.serde.StorageSerializable._
import org.apache.s2graph.core.types.SourceAndTargetVertexIdPair
class SnapshotEdgeSerializable(snapshotEdge: SnapshotEdge) extends Serializable[SnapshotEdge] {
- import StorageSerializable._
override def ts = snapshotEdge.version
override def table = snapshotEdge.label.hbaseTableName.getBytes("UTF-8")
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 78ac2f7..8c961ce 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
@@ -20,11 +20,12 @@
package org.apache.s2graph.core.storage.serde.snapshotedge.wide
import org.apache.hadoop.hbase.util.Bytes
-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.{LabelWithDirection, HBaseType, SourceVertexId, TargetVertexId}
+import org.apache.s2graph.core.mysqls.{Label, LabelMeta}
+import org.apache.s2graph.core.storage.serde.StorageDeserializable._
+import org.apache.s2graph.core.storage.CanSKeyValue
+import org.apache.s2graph.core.types.{HBaseType, LabelWithDirection, SourceVertexId, TargetVertexId}
import org.apache.s2graph.core._
+import org.apache.s2graph.core.storage.serde.Deserializable
class SnapshotEdgeDeserializable(graph: S2Graph) extends Deserializable[SnapshotEdge] {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeSerializable.scala
index d2544e0..df84e86 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeSerializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/snapshotedge/wide/SnapshotEdgeSerializable.scala
@@ -22,7 +22,8 @@ package org.apache.s2graph.core.storage.serde.snapshotedge.wide
import org.apache.hadoop.hbase.util.Bytes
import org.apache.s2graph.core.SnapshotEdge
import org.apache.s2graph.core.mysqls.LabelIndex
-import org.apache.s2graph.core.storage.{SKeyValue, Serializable, StorageSerializable}
+import org.apache.s2graph.core.storage.serde.Serializable
+import org.apache.s2graph.core.storage.serde.StorageSerializable._
import org.apache.s2graph.core.types.VertexId
@@ -32,7 +33,6 @@ import org.apache.s2graph.core.types.VertexId
* @param snapshotEdge
*/
class SnapshotEdgeSerializable(snapshotEdge: SnapshotEdge) extends Serializable[SnapshotEdge] {
- import StorageSerializable._
override def ts = snapshotEdge.version
override def table = snapshotEdge.label.hbaseTableName.getBytes("UTF-8")
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 f8921a8..87f0947 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
@@ -1,73 +1,73 @@
-/*
- * 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, Label}
-import org.apache.s2graph.core.storage.StorageDeserializable._
-import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable}
-import org.apache.s2graph.core.types.{HBaseType, InnerVal, InnerValLike, VertexId}
-import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{QueryParam, 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
- }
- }
-}
+///*
+// * 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/39544dc5/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 ee147f1..aa85574 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
@@ -1,52 +1,62 @@
-/*
- * 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.StorageSerializable._
-import org.apache.s2graph.core.storage.{SKeyValue, Serializable}
-import org.apache.s2graph.core.utils.logger
-
-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] = Array.empty[Byte]
-
- /** 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.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
+///*
+// * 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/39544dc5/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
new file mode 100644
index 0000000..648c9df
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexDeserializable.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.tall
+
+import org.apache.s2graph.core.mysqls.ColumnMeta
+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}
+
+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 {
+ assert(_kvs.size == 1)
+
+ 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)
+ val serviceColumn = vertexId.column
+ val schemaVer = serviceColumn.schemaVersion
+
+ val (props, _) = bytesToKeyValues(kv.value, 0, kv.value.length, schemaVer, serviceColumn)
+
+ val propsMap = new collection.mutable.HashMap[ColumnMeta, InnerValLike]
+ props.foreach { case (columnMeta, innerVal) =>
+ propsMap += (columnMeta -> innerVal)
+ }
+
+ val vertex = graph.newVertex(vertexId, kv.timestamp, S2Vertex.EmptyProps, belongLabelIds = Nil)
+ S2Vertex.fillPropsWithTs(vertex, propsMap.toMap)
+
+ Option(vertex)
+ } catch {
+ case e: Exception => None
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexSerializable.scala
new file mode 100644
index 0000000..87f050d
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/tall/VertexSerializable.scala
@@ -0,0 +1,54 @@
+/*
+ * 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.tall
+
+import org.apache.s2graph.core.S2Vertex
+import org.apache.s2graph.core.storage.SKeyValue
+import org.apache.s2graph.core.storage.serde.Serializable
+import org.apache.s2graph.core.storage.serde.StorageSerializable._
+
+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 = (vertex.props.asScala ++ vertex.defaultProps.asScala).toSeq.map { case (_, v) =>
+ v.columnMeta -> v.innerVal.bytes
+ }
+ vertexPropsToBytes(props)
+ }
+
+ /** vertex override toKeyValues since vertex expect to produce multiple sKeyValues */
+ override def toKeyValues: Seq[SKeyValue] = {
+ val row = toRowKey
+ val qualifier = toQualifier
+ val value = toValue
+ Seq(
+ SKeyValue(vertex.hbaseTableName.getBytes, row, cf, qualifier, value, vertex.ts)
+ )
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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
new file mode 100644
index 0000000..bae7941
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexDeserializable.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.wide
+
+import org.apache.s2graph.core.mysqls.ColumnMeta
+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}
+
+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/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexSerializable.scala
new file mode 100644
index 0000000..59db0ab
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/serde/vertex/wide/VertexSerializable.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.wide
+
+import org.apache.s2graph.core.S2Vertex
+import org.apache.s2graph.core.storage.SKeyValue
+import org.apache.s2graph.core.storage.serde.Serializable
+import org.apache.s2graph.core.storage.serde.StorageSerializable._
+
+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] = Array.empty[Byte]
+
+ /** 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.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/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/utils/DeferCache.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/utils/DeferCache.scala b/s2core/src/main/scala/org/apache/s2graph/core/utils/DeferCache.scala
index 9bb99ed..0a1d5f3 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/utils/DeferCache.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/utils/DeferCache.scala
@@ -90,13 +90,13 @@ object DeferCache {
/**
* @param config
- * @param ec
* @param canDefer: implicit evidence to find out implementation of CanDefer.
* @tparam A: actual element type that will be stored in M[_] and C[_].
* @tparam M[_]: container type that will be stored in local cache. ex) Promise, Defer.
* @tparam C[_]: container type that will be returned to client of this class. Ex) Future, Defer.
*/
-class DeferCache[A, M[_], C[_]](config: Config, empty: => A, name: String = "", useMetric: Boolean = false)(implicit ec: ExecutionContext, canDefer: CanDefer[A, M, C]) {
+class DeferCache[A, M[_], C[_]](config: Config, empty: => A,
+ name: String = "", useMetric: Boolean = false)(implicit canDefer: CanDefer[A, M, C]) {
type Value = (Long, C[A])
private val maxSize = config.getInt("future.cache.max.size")
@@ -131,7 +131,7 @@ class DeferCache[A, M[_], C[_]](config: Config, empty: => A, name: String = "",
private def checkAndExpire(cacheKey: Long,
cachedAt: Long,
cacheTTL: Long,
- oldFuture: C[A])(op: => C[A]): C[A] = {
+ oldFuture: C[A])(op: => C[A])(implicit ec: ExecutionContext): C[A] = {
if (System.currentTimeMillis() >= cachedAt + cacheTTL) {
// future is too old. so need to expire and fetch new data from storage.
futureCache.asMap().remove(cacheKey)
@@ -164,7 +164,7 @@ class DeferCache[A, M[_], C[_]](config: Config, empty: => A, name: String = "",
}
}
- def getOrElseUpdate(cacheKey: Long, cacheTTL: Long)(op: => C[A]): C[A] = {
+ def getOrElseUpdate(cacheKey: Long, cacheTTL: Long)(op: => C[A])(implicit ec: ExecutionContext): C[A] = {
val cacheVal = futureCache.getIfPresent(cacheKey)
cacheVal match {
case null =>
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/test/scala/org/apache/s2graph/core/Integrate/CrudTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/CrudTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/CrudTest.scala
index a41152c..4ed7905 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/CrudTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/CrudTest.scala
@@ -270,12 +270,12 @@ class CrudTest extends IntegrateCommon {
val queryJson = querySnapshotEdgeJson(serviceName, columnName, labelName, id)
- if (!rets.forall(identity)) {
+ if (!rets.forall(_.isSuccess)) {
Thread.sleep(graph.LockExpireDuration + 100)
/** expect current request would be ignored */
val bulkEdges = Seq(TestUtil.toEdge(i-1, "u", "e", 0, 0, testLabelName, Json.obj("time" -> 20).toString()))
val rets = TestUtil.insertEdgesSync(bulkEdges: _*)
- if (rets.forall(identity)) {
+ if (rets.forall(_.isSuccess)) {
// check
val jsResult = TestUtil.getEdgesSync(queryJson)
(jsResult \\ "time").head.as[Int] should be(10)
@@ -295,12 +295,12 @@ class CrudTest extends IntegrateCommon {
val queryJson = querySnapshotEdgeJson(serviceName, columnName, labelName, id)
- if (!rets.forall(identity)) {
+ if (!rets.forall(_.isSuccess)) {
Thread.sleep(graph.LockExpireDuration + 100)
/** expect current request would be applied */
val bulkEdges = Seq(TestUtil.toEdge(i+1, "u", "e", 0, 0, testLabelName, Json.obj("time" -> 20).toString()))
val rets = TestUtil.insertEdgesSync(bulkEdges: _*)
- if (rets.forall(identity)) {
+ if (rets.forall(_.isSuccess)) {
// check
val jsResult = TestUtil.getEdgesSync(queryJson)
(jsResult \\ "time").head.as[Int] should be(20)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/test/scala/org/apache/s2graph/core/Integrate/LabelLabelIndexMutateOptionTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/LabelLabelIndexMutateOptionTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/LabelLabelIndexMutateOptionTest.scala
index 4855cfc..1dfcfe6 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/LabelLabelIndexMutateOptionTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/LabelLabelIndexMutateOptionTest.scala
@@ -20,7 +20,7 @@
package org.apache.s2graph.core.Integrate
import org.apache.s2graph.core._
-import org.scalatest.BeforeAndAfterEach
+import org.scalatest.{BeforeAndAfterEach, Tag}
import play.api.libs.json._
class LabelLabelIndexMutateOptionTest extends IntegrateCommon with BeforeAndAfterEach {
@@ -127,7 +127,7 @@ class LabelLabelIndexMutateOptionTest extends IntegrateCommon with BeforeAndAfte
/**
* { "out": {"method": "drop", "storeDegree": false} }
*/
- test("index for in direction should drop in direction edge and store degree") {
+ ignore("index for in direction should drop in direction edge and store degree") {
val edges = getEdgesSync(getQuery(Seq(1, 2, 3), "in", idxDropInStoreDegree))
(edges \ "results").as[Seq[JsValue]].size should be(0)
(edges \\ "_degree").map(_.as[Long]).sum should be(3)
@@ -136,7 +136,7 @@ class LabelLabelIndexMutateOptionTest extends IntegrateCommon with BeforeAndAfte
/**
* { "in": {"method": "drop", "storeDegree": false }, "out": {"method": "drop"} }
*/
- test("index for out direction should drop out direction edge and store degree") {
+ ignore("index for out direction should drop out direction edge and store degree") {
val edges = getEdgesSync(getQuery(Seq(0), "out", idxDropOutStoreDegree))
(edges \ "results").as[Seq[JsValue]].size should be(0)
(edges \\ "_degree").map(_.as[Long]).sum should be(3)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala
new file mode 100644
index 0000000..fd9d2b3
--- /dev/null
+++ b/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala
@@ -0,0 +1,59 @@
+package org.apache.s2graph.core.storage
+
+import org.apache.s2graph.core.mysqls._
+import org.apache.s2graph.core.storage.hbase.AsynchbaseStorageSerDe
+import org.apache.s2graph.core.storage.rocks.RocksStorageSerDe
+import org.apache.s2graph.core.storage.serde.{StorageDeserializable, StorageSerializable}
+import org.apache.s2graph.core.{S2Vertex, TestCommonWithModels}
+import org.scalatest.{FunSuite, Matchers}
+
+class StorageIOTest extends FunSuite with Matchers with TestCommonWithModels {
+
+ initTests()
+
+ test("AsynchbaseStorageIO: VertexSerializer/Deserializer") {
+ def check(vertex: S2Vertex,
+ op: S2Vertex => StorageSerializable[S2Vertex],
+ deserializer: StorageDeserializable[S2Vertex]): Boolean = {
+ val sKeyValues = op(vertex).toKeyValues
+ val deserialized = deserializer.fromKeyValues(sKeyValues, None)
+ vertex == deserialized
+ }
+
+ val serDe: StorageSerDe = new AsynchbaseStorageSerDe(graph)
+ val service = Service.findByName(serviceName, useCache = false).getOrElse {
+ throw new IllegalStateException("service not found.")
+ }
+ val column = ServiceColumn.find(service.id.get, columnName).getOrElse {
+ throw new IllegalStateException("column not found.")
+ }
+
+ val vertexId = graph.newVertexId(service, column, 1L)
+ val vertex = graph.newVertex(vertexId)
+
+ check(vertex, serDe.vertexSerializer, serDe.vertexDeserializer(vertex.serviceColumn.schemaVersion))
+ }
+
+ test("RocksStorageIO: VertexSerializer/Deserializer") {
+ def check(vertex: S2Vertex,
+ op: S2Vertex => StorageSerializable[S2Vertex],
+ deserializer: StorageDeserializable[S2Vertex]): Boolean = {
+ val sKeyValues = op(vertex).toKeyValues
+ val deserialized = deserializer.fromKeyValues(sKeyValues, None)
+ vertex == deserialized
+ }
+
+ val serDe: StorageSerDe = new RocksStorageSerDe(graph)
+ val service = Service.findByName(serviceName, useCache = false).getOrElse {
+ throw new IllegalStateException("service not found.")
+ }
+ val column = ServiceColumn.find(service.id.get, columnName).getOrElse {
+ throw new IllegalStateException("column not found.")
+ }
+
+ val vertexId = graph.newVertexId(service, column, 1L)
+ val vertex = graph.newVertex(vertexId)
+
+ check(vertex, serDe.vertexSerializer, serDe.vertexDeserializer(vertex.serviceColumn.schemaVersion))
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 a5c974e..0cbaa81 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
@@ -46,7 +46,8 @@ class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
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(graph.getStorage(l).indexEdgeSerializer(indexEdge).toKeyValues, None)
+ val kvs = graph.getStorage(l).indexEdgeSerializer(indexEdge).toKeyValues
+ val _indexEdgeOpt = graph.getStorage(l).indexEdgeDeserializer(l.schemaVersion).fromKeyValues(kvs, None)
_indexEdgeOpt should not be empty
edge == _indexEdgeOpt.get should be(true)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala
new file mode 100644
index 0000000..8a8a532
--- /dev/null
+++ b/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala
@@ -0,0 +1,33 @@
+package org.apache.s2graph.core.storage.rocks
+
+import org.apache.s2graph.core.TestCommonWithModels
+import org.apache.s2graph.core.mysqls.{Service, ServiceColumn}
+import org.apache.tinkerpop.gremlin.structure.T
+import org.scalatest.{FunSuite, Matchers}
+
+import scala.collection.JavaConversions._
+
+class RocksStorageTest extends FunSuite with Matchers with TestCommonWithModels {
+ initTests()
+
+ test("VertexTest: shouldNotGetConcurrentModificationException()") {
+ val service = Service.findByName(serviceName, useCache = false).getOrElse {
+ throw new IllegalStateException("service not found.")
+ }
+ val column = ServiceColumn.find(service.id.get, columnName).getOrElse {
+ throw new IllegalStateException("column not found.")
+ }
+
+ val vertexId = graph.newVertexId(service, column, 1L)
+
+ val vertex = graph.newVertex(vertexId)
+ for (i <- (0 until 10)) {
+ vertex.addEdge(labelName, vertex)
+ }
+
+ println(graph.edges().toSeq)
+ println("*" * 100)
+ vertex.remove()
+ println(graph.vertices().toSeq)
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 28da7fe..9a45bd5 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
@@ -25,6 +25,7 @@ import org.apache.s2graph.core._
import org.apache.s2graph.core.mysqls.Label
import org.apache.s2graph.core.rest.RequestParser
import org.apache.s2graph.core.utils.logger
+import org.apache.s2graph.core.storage.{IncrementResponse, MutateResponse}
import org.apache.s2graph.rest.play.actors.QueueActor
import org.apache.s2graph.rest.play.config.Config
import play.api.libs.json._
@@ -92,7 +93,7 @@ object EdgeController extends Controller {
val result = s2.mutateElements(elements.map(_._1), true)
result onComplete { results =>
results.get.zip(elements).map {
- case (false, (e: S2Edge, tsv: String)) =>
+ case (r: MutateResponse, (e: S2Edge, tsv: String)) if !r.isSuccess =>
val kafkaMessages = if(e.op == GraphUtil.operations("deleteAll")){
toDeleteAllFailMessages(Seq(e.srcVertex), Seq(e.innerLabel), e.labelWithDir.dir, e.ts)
} else{
@@ -119,13 +120,13 @@ object EdgeController extends Controller {
val (elementSync, elementAsync) = elementWithIdxs.partition { case ((element, tsv), idx) =>
!skipElement(element.isAsync)
}
- val retToSkip = elementAsync.map(_._2 -> true)
+ val retToSkip = elementAsync.map(_._2 -> MutateResponse.Success)
val elementsToStore = elementSync.map(_._1)
val elementsIdxToStore = elementSync.map(_._2)
mutateElementsWithFailLog(elementsToStore).map { rets =>
elementsIdxToStore.zip(rets) ++ retToSkip
}.map { rets =>
- Json.toJson(rets.sortBy(_._1).map(_._2))
+ Json.toJson(rets.sortBy(_._1).map(_._2.isSuccess))
}.map(jsonResponse(_))
} else {
val rets = elementWithIdxs.map { case ((element, tsv), idx) =>
@@ -232,8 +233,8 @@ object EdgeController extends Controller {
else {
s2.incrementCounts(edges, withWait = true).map { results =>
- val json = results.map { case (isSuccess, resultCount, count) =>
- Json.obj("success" -> isSuccess, "result" -> resultCount, "_count" -> count)
+ val json = results.map { case IncrementResponse(isSuccess, afterCount, beforeCount) =>
+ Json.obj("success" -> isSuccess, "result" -> afterCount, "_count" -> beforeCount)
}
jsonResponse(Json.toJson(json))
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 43f0b15..a6df439 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
@@ -20,8 +20,9 @@
package org.apache.s2graph.rest.play.controllers
import org.apache.s2graph.core.rest.RequestParser
+import org.apache.s2graph.core.storage.MutateResponse
import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{ExceptionHandler, S2Graph, GraphExceptions}
+import org.apache.s2graph.core.{ExceptionHandler, GraphExceptions, S2Graph}
import org.apache.s2graph.rest.play.actors.QueueActor
import org.apache.s2graph.rest.play.config.Config
import play.api.libs.json.{JsValue, Json}
@@ -54,7 +55,7 @@ object VertexController extends Controller {
if (verticesToStore.isEmpty) Future.successful(jsonResponse(Json.toJson(Seq.empty[Boolean])))
else {
if (withWait) {
- val rets = s2.mutateVertices(verticesToStore, withWait = true)
+ val rets = s2.mutateVertices(verticesToStore, withWait = true).map(_.map(_.isSuccess))
rets.map(Json.toJson(_)).map(jsonResponse(_))
} else {
val rets = verticesToStore.map { vertex => QueueActor.router ! vertex; true }
[5/8] incubator-s2graph git commit: run apache-rat.
Posted by st...@apache.org.
run apache-rat.
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/f416194c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/f416194c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/f416194c
Branch: refs/heads/master
Commit: f416194c11d97ee524a4c95852d42a5f0afb446d
Parents: 39544dc
Author: DO YUNG YOON <st...@apache.org>
Authored: Sat Oct 28 09:32:53 2017 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Mon Oct 30 10:12:22 2017 +0900
----------------------------------------------------------------------
s2core/build.sbt | 3 +-
.../scala/org/apache/s2graph/core/S2Graph.scala | 33 +++++++++++++--
.../s2graph/core/features/S2GraphFeatures.scala | 3 +-
.../s2graph/core/storage/MutateResponse.scala | 19 +++++++++
.../apache/s2graph/core/storage/Storage.scala | 32 ---------------
.../apache/s2graph/core/storage/StorageIO.scala | 19 +++++++++
.../core/storage/StorageManagement.scala | 19 +++++++++
.../s2graph/core/storage/StorageReadable.scala | 19 +++++++++
.../s2graph/core/storage/StorageSerDe.scala | 19 +++++++++
.../s2graph/core/storage/StorageWritable.scala | 19 +++++++++
.../storage/WriteWriteConflictResolver.scala | 19 +++++++++
.../hbase/AsynchbaseStorageManagement.scala | 19 +++++++++
.../hbase/AsynchbaseStorageReadable.scala | 21 +++++++++-
.../storage/hbase/AsynchbaseStorageSerDe.scala | 19 +++++++++
.../hbase/AsynchbaseStorageWritable.scala | 19 +++++++++
.../s2graph/core/storage/StorageIOTest.scala | 42 +++++++++-----------
.../core/storage/rocks/RocksStorageTest.scala | 33 ---------------
17 files changed, 261 insertions(+), 96 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/build.sbt
----------------------------------------------------------------------
diff --git a/s2core/build.sbt b/s2core/build.sbt
index e7e602f..8033581 100644
--- a/s2core/build.sbt
+++ b/s2core/build.sbt
@@ -48,8 +48,7 @@ libraryDependencies ++= Seq(
"org.specs2" %% "specs2-core" % specs2Version % "test",
"org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion ,
"org.apache.lucene" % "lucene-core" % "6.6.0",
- "org.apache.lucene" % "lucene-queryparser" % "6.6.0",
- "org.rocksdb" % "rocksdbjni" % "5.8.0"
+ "org.apache.lucene" % "lucene-queryparser" % "6.6.0"
)
libraryDependencies := {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/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 32724b4..92f68dc 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
@@ -31,7 +31,6 @@ import org.apache.s2graph.core.index.IndexProvider
import org.apache.s2graph.core.io.tinkerpop.optimize.S2GraphStepStrategy
import org.apache.s2graph.core.mysqls._
import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage
-import org.apache.s2graph.core.storage.rocks.RocksStorage
import org.apache.s2graph.core.storage.{ MutateResponse, SKeyValue, Storage}
import org.apache.s2graph.core.types._
import org.apache.s2graph.core.utils.{DeferCache, Extensions, logger}
@@ -140,7 +139,6 @@ object S2Graph {
storageBackend match {
case "hbase" => new AsynchbaseStorage(graph, config)
- case "rocks" => new RocksStorage(graph, config)
case _ => throw new RuntimeException("not supported storage.")
}
}
@@ -694,7 +692,7 @@ object S2Graph {
// new Graph.OptOut(test = "org.apache.tinkerpop.gremlin.process.traversal.step.map.VertexTest$Traversals", method = "*", reason = "no"),
// passed: all
-
+
// new Graph.OptOut(test = "org.apache.tinkerpop.gremlin.process.traversal.step.map.UnfoldTest$Traversals", method = "*", reason = "no"),
// passed: all
@@ -1145,7 +1143,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
fallback
} get
}
-
+
def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
def getVertices[Q](storage: Storage[Q])(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
def fromResult(kvs: Seq[SKeyValue],
@@ -1673,6 +1671,33 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
vertex
}
+ def toRequestEdge(queryRequest: QueryRequest, parentEdges: Seq[EdgeWithScore]): S2Edge = {
+ val srcVertex = queryRequest.vertex
+ val queryParam = queryRequest.queryParam
+ val tgtVertexIdOpt = queryParam.tgtVertexInnerIdOpt
+ val label = queryParam.label
+ val labelWithDir = queryParam.labelWithDir
+ val (srcColumn, tgtColumn) = label.srcTgtColumn(labelWithDir.dir)
+ val propsWithTs = label.EmptyPropsWithTs
+
+ tgtVertexIdOpt match {
+ case Some(tgtVertexId) => // _to is given.
+ /* we use toSnapshotEdge so dont need to swap src, tgt */
+ val src = srcVertex.innerId
+ val tgt = tgtVertexId
+ val (srcVId, tgtVId) = (SourceVertexId(srcColumn, src), TargetVertexId(tgtColumn, tgt))
+ val (srcV, tgtV) = (newVertex(srcVId), newVertex(tgtVId))
+
+ newEdge(srcV, tgtV, label, labelWithDir.dir, propsWithTs = propsWithTs)
+ case None =>
+ val src = srcVertex.innerId
+ val srcVId = SourceVertexId(srcColumn, src)
+ val srcV = newVertex(srcVId)
+
+ newEdge(srcV, srcV, label, labelWithDir.dir, propsWithTs = propsWithTs, parentEdges = parentEdges)
+ }
+ }
+
/**
* helper to create new Edge instance from given parameters on memory(not actually stored in storage).
*
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/features/S2GraphFeatures.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/features/S2GraphFeatures.scala b/s2core/src/main/scala/org/apache/s2graph/core/features/S2GraphFeatures.scala
index e2c4179..e21ffc8 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/features/S2GraphFeatures.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/features/S2GraphFeatures.scala
@@ -22,8 +22,9 @@ package org.apache.s2graph.core.features
import org.apache.tinkerpop.gremlin.structure.Graph.Features
import org.apache.tinkerpop.gremlin.structure.Graph.Features.GraphFeatures
-
class S2GraphFeatures extends GraphFeatures {
+
+
override def supportsComputer(): Boolean = false
override def supportsThreadedTransactions(): Boolean = false
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala
index bed1152..a726d17 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala
@@ -1,3 +1,22 @@
+/*
+ * 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
object MutateResponse {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/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 c9353e1..2fe6e42 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
@@ -25,40 +25,8 @@ import org.apache.s2graph.core._
import org.apache.s2graph.core.storage.serde.Deserializable
import org.apache.s2graph.core.storage.serde.indexedge.tall.IndexEdgeDeserializable
import org.apache.s2graph.core.types._
-
import scala.concurrent.{ExecutionContext, Future}
-object Storage {
- def toRequestEdge(graph: S2Graph)(queryRequest: QueryRequest, parentEdges: Seq[EdgeWithScore]): S2Edge = {
- val srcVertex = queryRequest.vertex
- val queryParam = queryRequest.queryParam
- val tgtVertexIdOpt = queryParam.tgtVertexInnerIdOpt
- val label = queryParam.label
- val labelWithDir = queryParam.labelWithDir
- val (srcColumn, tgtColumn) = label.srcTgtColumn(labelWithDir.dir)
- val propsWithTs = label.EmptyPropsWithTs
-
- tgtVertexIdOpt match {
- case Some(tgtVertexId) => // _to is given.
- /* we use toSnapshotEdge so dont need to swap src, tgt */
- val src = srcVertex.innerId
- val tgt = tgtVertexId
- val (srcVId, tgtVId) = (SourceVertexId(srcColumn, src), TargetVertexId(tgtColumn, tgt))
- val (srcV, tgtV) = (graph.newVertex(srcVId), graph.newVertex(tgtVId))
-
- graph.newEdge(srcV, tgtV, label, labelWithDir.dir, propsWithTs = propsWithTs)
- case None =>
- val src = srcVertex.innerId
- val srcVId = SourceVertexId(srcColumn, src)
- val srcV = graph.newVertex(srcVId)
-
- graph.newEdge(srcV, srcV, label, labelWithDir.dir, propsWithTs = propsWithTs, parentEdges = parentEdges)
- }
- }
-
-
-}
-
abstract class Storage[Q](val graph: S2Graph,
val config: Config) {
/* Storage backend specific resource management */
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala
index 2e11f0b..67033f0 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala
@@ -1,3 +1,22 @@
+/*
+ * 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
import org.apache.hadoop.hbase.util.Bytes
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala
index da94767..72a0b69 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala
@@ -1,3 +1,22 @@
+/*
+ * 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
import com.typesafe.config.Config
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
index 96669ca..7a0d8ef 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
@@ -1,3 +1,22 @@
+/*
+ * 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
import org.apache.s2graph.core.GraphExceptions.FetchTimeoutException
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala
index f973e0f..15b3576 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala
@@ -1,3 +1,22 @@
+/*
+ * 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
import org.apache.s2graph.core.{IndexEdge, S2Graph, S2Vertex, SnapshotEdge}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala
index 216aece..80da3a9 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala
@@ -1,3 +1,22 @@
+/*
+ * 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
import scala.concurrent.{ExecutionContext, Future}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
index eab5cab..79b764d 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
@@ -1,3 +1,22 @@
+/*
+ * 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
import java.util.concurrent.{Executors, TimeUnit}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala
index c55c6c7..0fb3173 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala
@@ -1,3 +1,22 @@
+/*
+ * 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 java.util.concurrent.{Executors, TimeUnit}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
index 4ef95b8..1cb6109 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
@@ -1,3 +1,22 @@
+/*
+ * 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 java.util
@@ -272,7 +291,7 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
val cacheTTL = queryParam.cacheTTLInMillis
/* with version 4, request's type is (Scanner, (Int, Int)). otherwise GetRequest. */
- val edge = Storage.toRequestEdge(graph)(queryRequest, parentEdges)
+ val edge = graph.toRequestEdge(queryRequest, parentEdges)
val request = buildRequest(queryRequest, edge)
val (intervalMaxBytes, intervalMinBytes) = queryParam.buildInterval(Option(edge))
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala
index ab1ff19..1bdd74e 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala
@@ -1,3 +1,22 @@
+/*
+ * 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.storage.serde.Deserializable
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala
index 1f7d863..7ca3782 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageWritable.scala
@@ -1,3 +1,22 @@
+/*
+ * 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.hadoop.hbase.util.Bytes
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala
index fd9d2b3..a05be79 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/storage/StorageIOTest.scala
@@ -1,8 +1,26 @@
+/*
+ * 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
import org.apache.s2graph.core.mysqls._
import org.apache.s2graph.core.storage.hbase.AsynchbaseStorageSerDe
-import org.apache.s2graph.core.storage.rocks.RocksStorageSerDe
import org.apache.s2graph.core.storage.serde.{StorageDeserializable, StorageSerializable}
import org.apache.s2graph.core.{S2Vertex, TestCommonWithModels}
import org.scalatest.{FunSuite, Matchers}
@@ -34,26 +52,4 @@ class StorageIOTest extends FunSuite with Matchers with TestCommonWithModels {
check(vertex, serDe.vertexSerializer, serDe.vertexDeserializer(vertex.serviceColumn.schemaVersion))
}
- test("RocksStorageIO: VertexSerializer/Deserializer") {
- def check(vertex: S2Vertex,
- op: S2Vertex => StorageSerializable[S2Vertex],
- deserializer: StorageDeserializable[S2Vertex]): Boolean = {
- val sKeyValues = op(vertex).toKeyValues
- val deserialized = deserializer.fromKeyValues(sKeyValues, None)
- vertex == deserialized
- }
-
- val serDe: StorageSerDe = new RocksStorageSerDe(graph)
- val service = Service.findByName(serviceName, useCache = false).getOrElse {
- throw new IllegalStateException("service not found.")
- }
- val column = ServiceColumn.find(service.id.get, columnName).getOrElse {
- throw new IllegalStateException("column not found.")
- }
-
- val vertexId = graph.newVertexId(service, column, 1L)
- val vertex = graph.newVertex(vertexId)
-
- check(vertex, serDe.vertexSerializer, serDe.vertexDeserializer(vertex.serviceColumn.schemaVersion))
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f416194c/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala
deleted file mode 100644
index 8a8a532..0000000
--- a/s2core/src/test/scala/org/apache/s2graph/core/storage/rocks/RocksStorageTest.scala
+++ /dev/null
@@ -1,33 +0,0 @@
-package org.apache.s2graph.core.storage.rocks
-
-import org.apache.s2graph.core.TestCommonWithModels
-import org.apache.s2graph.core.mysqls.{Service, ServiceColumn}
-import org.apache.tinkerpop.gremlin.structure.T
-import org.scalatest.{FunSuite, Matchers}
-
-import scala.collection.JavaConversions._
-
-class RocksStorageTest extends FunSuite with Matchers with TestCommonWithModels {
- initTests()
-
- test("VertexTest: shouldNotGetConcurrentModificationException()") {
- val service = Service.findByName(serviceName, useCache = false).getOrElse {
- throw new IllegalStateException("service not found.")
- }
- val column = ServiceColumn.find(service.id.get, columnName).getOrElse {
- throw new IllegalStateException("column not found.")
- }
-
- val vertexId = graph.newVertexId(service, column, 1L)
-
- val vertex = graph.newVertex(vertexId)
- for (i <- (0 until 10)) {
- vertex.addEdge(labelName, vertex)
- }
-
- println(graph.edges().toSeq)
- println("*" * 100)
- vertex.remove()
- println(graph.vertices().toSeq)
- }
-}
[3/8] incubator-s2graph git commit: Separate interfaces from Storage.
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 57d4872..c9353e1 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
@@ -20,948 +20,16 @@
package org.apache.s2graph.core.storage
-import org.apache.s2graph.core.GraphExceptions.{NoStackException, FetchTimeoutException}
+import com.typesafe.config.Config
import org.apache.s2graph.core._
-import org.apache.s2graph.core.mysqls.{Label, LabelMeta}
-import org.apache.s2graph.core.parsers.WhereParser
-import org.apache.s2graph.core.storage.serde.indexedge.wide.{IndexEdgeDeserializable, IndexEdgeSerializable}
-import org.apache.s2graph.core.storage.serde.snapshotedge.wide.SnapshotEdgeDeserializable
-import org.apache.s2graph.core.storage.serde.vertex.{VertexDeserializable, VertexSerializable}
+import org.apache.s2graph.core.storage.serde.Deserializable
+import org.apache.s2graph.core.storage.serde.indexedge.tall.IndexEdgeDeserializable
import org.apache.s2graph.core.types._
-import org.apache.s2graph.core.utils.{DeferCache, Extensions, logger}
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-import scala.concurrent.{ExecutionContext, Future, Promise}
-import scala.util.{Random, Try}
-import java.util.concurrent.{Executors, TimeUnit}
+import scala.concurrent.{ExecutionContext, Future}
-import com.typesafe.config.{Config, ConfigFactory}
-import org.apache.hadoop.hbase.util.Bytes
-
-
-abstract class Storage[Q, R](val graph: S2Graph,
- val config: Config)(implicit ec: ExecutionContext) {
- import HBaseType._
- import S2Graph._
-
- val BackoffTimeout = graph.BackoffTimeout
- val MaxRetryNum = graph.MaxRetryNum
- val MaxBackOff = graph.MaxBackOff
- val FailProb = graph.FailProb
- val LockExpireDuration = graph.LockExpireDuration
- val MaxSize = graph.MaxSize
- val ExpireAfterWrite = graph.ExpireAfterWrite
- val ExpireAfterAccess = graph.ExpireAfterAccess
-
- /** retry scheduler */
- val scheduledThreadPool = Executors.newSingleThreadScheduledExecutor()
-
-
- /**
- * Compatibility table
- * | label schema version | snapshot edge | index edge | vertex | note |
- * | v1 | serde.snapshotedge.wide | serde.indexedge.wide | serde.vertex | do not use this. this exist only for backward compatibility issue |
- * | v2 | serde.snapshotedge.wide | serde.indexedge.wide | serde.vertex | do not use this. this exist only for backward compatibility issue |
- * | v3 | serde.snapshotedge.tall | serde.indexedge.wide | serde.vertex | recommended with HBase. current stable schema |
- * | v4 | serde.snapshotedge.tall | serde.indexedge.tall | serde.vertex | experimental schema. use scanner instead of get |
- *
- */
-
- /**
- * create serializer that knows how to convert given snapshotEdge into kvs: Seq[SKeyValue]
- * so we can store this kvs.
- * @param snapshotEdge: snapshotEdge to serialize
- * @return serializer implementation for StorageSerializable which has toKeyValues return Seq[SKeyValue]
- */
- def snapshotEdgeSerializer(snapshotEdge: SnapshotEdge): Serializable[SnapshotEdge] = {
- snapshotEdge.schemaVer match {
-// case VERSION1 |
- case VERSION2 => new serde.snapshotedge.wide.SnapshotEdgeSerializable(snapshotEdge)
- case VERSION3 | VERSION4 => new serde.snapshotedge.tall.SnapshotEdgeSerializable(snapshotEdge)
- case _ => throw new RuntimeException(s"not supported version: ${snapshotEdge.schemaVer}")
- }
- }
-
- /**
- * create serializer that knows how to convert given indexEdge into kvs: Seq[SKeyValue]
- * @param indexEdge: indexEdge to serialize
- * @return serializer implementation
- */
- def indexEdgeSerializer(indexEdge: IndexEdge): Serializable[IndexEdge] = {
- indexEdge.schemaVer match {
-// case VERSION1
- case VERSION2 | VERSION3 => new IndexEdgeSerializable(indexEdge)
- case VERSION4 => new serde.indexedge.tall.IndexEdgeSerializable(indexEdge)
- case _ => throw new RuntimeException(s"not supported version: ${indexEdge.schemaVer}")
-
- }
- }
-
- /**
- * create serializer that knows how to convert given vertex into kvs: Seq[SKeyValue]
- * @param vertex: vertex to serialize
- * @return serializer implementation
- */
- def vertexSerializer(vertex: S2Vertex): Serializable[S2Vertex] = new VertexSerializable(vertex)
-
- /**
- * create deserializer that can parse stored CanSKeyValue into snapshotEdge.
- * note that each storage implementation should implement implicit type class
- * to convert storage dependent dataType into common SKeyValue type by implementing CanSKeyValue
- *
- * ex) Asynchbase use it's KeyValue class and CanSKeyValue object has implicit type conversion method.
- * if any storaage use different class to represent stored byte array,
- * then that storage implementation is responsible to provide implicit type conversion method on CanSKeyValue.
- * */
-
- val snapshotEdgeDeserializer: Deserializable[SnapshotEdge] = new serde.snapshotedge.tall.SnapshotEdgeDeserializable(graph)
-
- def snapshotEdgeDeserializer(schemaVer: String): Deserializable[SnapshotEdge] = snapshotEdgeDeserializer
-
- /** create deserializer that can parse stored CanSKeyValue into indexEdge. */
- val indexEdgeDeserializer: Deserializable[S2Edge] = new serde.indexedge.tall.IndexEdgeDeserializable(graph)
-
- def indexEdgeDeserializer(schemaVer: String) = new serde.indexedge.tall.IndexEdgeDeserializable(graph)
-
- /** create deserializer that can parser stored CanSKeyValue into vertex. */
- val vertexDeserializer: Deserializable[S2Vertex] = new VertexDeserializable(graph)
-
-
- /**
- * decide how to store given key values Seq[SKeyValue] into storage using storage's client.
- * note that this should be return true on all success.
- * we assumes that each storage implementation has client as member variable.
- *
- *
- * @param cluster: where this key values should be stored.
- * @param kvs: sequence of SKeyValue that need to be stored in storage.
- * @param withWait: flag to control wait ack from storage.
- * note that in AsynchbaseStorage(which support asynchronous operations), even with true,
- * it never block thread, but rather submit work and notified by event loop when storage send ack back.
- * @return ack message from storage.
- */
- def writeToStorage(cluster: String, kvs: Seq[SKeyValue], withWait: Boolean): Future[Boolean]
-
-// def writeToStorage(kv: SKeyValue, withWait: Boolean): Future[Boolean]
-
- /**
- * fetch SnapshotEdge for given request from storage.
- * also storage datatype should be converted into SKeyValue.
- * note that return type is Sequence rather than single SKeyValue for simplicity,
- * even though there is assertions sequence.length == 1.
- * @param request
- * @return
- */
- def fetchSnapshotEdgeKeyValues(request: QueryRequest): Future[Seq[SKeyValue]]
-
- /**
- * write requestKeyValue into storage if the current value in storage that is stored matches.
- * note that we only use SnapshotEdge as place for lock, so this method only change SnapshotEdge.
- *
- * Most important thing is this have to be 'atomic' operation.
- * When this operation is mutating requestKeyValue's snapshotEdge, then other thread need to be
- * either blocked or failed on write-write conflict case.
- *
- * Also while this method is still running, then fetchSnapshotEdgeKeyValues should be synchronized to
- * prevent wrong data for read.
- *
- * Best is use storage's concurrency control(either pessimistic or optimistic) such as transaction,
- * compareAndSet to synchronize.
- *
- * for example, AsynchbaseStorage use HBase's CheckAndSet atomic operation to guarantee 'atomicity'.
- * for storage that does not support concurrency control, then storage implementation
- * itself can maintain manual locks that synchronize read(fetchSnapshotEdgeKeyValues)
- * and write(writeLock).
- * @param requestKeyValue
- * @param expectedOpt
- * @return
- */
- def writeLock(requestKeyValue: SKeyValue, expectedOpt: Option[SKeyValue]): Future[Boolean]
-
- /**
- * build proper request which is specific into storage to call fetchIndexEdgeKeyValues or fetchSnapshotEdgeKeyValues.
- * for example, Asynchbase use GetRequest, Scanner so this method is responsible to build
- * client request(GetRequest, Scanner) based on user provided query.
- *
- * @param queryRequest
- * @return
- */
- protected def buildRequest(queryRequest: QueryRequest, edge: S2Edge): Q
-
- /**
- * fetch IndexEdges for given queryParam in queryRequest.
- * this expect previous step starting score to propagate score into next step.
- * also parentEdges is necessary to return full bfs tree when query require it.
- *
- * note that return type is general type.
- * for example, currently we wanted to use Asynchbase
- * so single I/O return type should be Deferred[T].
- *
- * if we use native hbase client, then this return type can be Future[T] or just T.
- *
- * @param queryRequest
- * @param isInnerCall
- * @param parentEdges
- * @return
- */
- def fetch(queryRequest: QueryRequest,
- isInnerCall: Boolean,
- parentEdges: Seq[EdgeWithScore]): R
-
- /**
- * 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]]
-
- /**
- * fetch Vertex for given request from storage.
- *
- * @param request
- * @return
- */
- def fetchVertexKeyValues(request: QueryRequest): Future[Seq[SKeyValue]]
-
- /**
- * decide how to apply given edges(indexProps values + Map(_count -> countVal)) into storage.
- *
- * @param edges
- * @param withWait
- * @return
- */
- 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
- * such as client into storage.
- */
- def flush(): Unit = {
- }
-
- def fetchEdgesAll(): Future[Seq[S2Edge]]
-
- def fetchVerticesAll(): Future[Seq[S2Vertex]]
-
- /**
- * create table on storage.
- * if storage implementation does not support namespace or table, then there is nothing to be done
- *
- * @param zkAddr
- * @param tableName
- * @param cfs
- * @param regionMultiplier
- * @param ttl
- * @param compressionAlgorithm
- */
- def createTable(zkAddr: String,
- tableName: String,
- cfs: List[String],
- regionMultiplier: Int,
- ttl: Option[Int],
- compressionAlgorithm: String,
- replicationScopeOpt: Option[Int] = None,
- totalRegionCount: Option[Int] = None): Unit
-
- def truncateTable(zkAddr: String, tableNameStr: String): Unit = {}
-
- def deleteTable(zkAddr: String, tableNameStr: String): Unit = {}
-
- def shutdown(): Unit
-
- /** Public Interface */
- def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
- def fromResult(kvs: Seq[SKeyValue],
- version: String): Option[S2Vertex] = {
- if (kvs.isEmpty) None
- else vertexDeserializer.fromKeyValues(kvs, None)
-// .map(S2Vertex(graph, _))
- }
-
- val futures = vertices.map { vertex =>
- val queryParam = QueryParam.Empty
- 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)
- } recoverWith { case ex: Throwable =>
- Future.successful(None)
- }
- }
-
- Future.sequence(futures).map { result => result.toList.flatten }
- }
- def mutateStrongEdges(_edges: Seq[S2Edge], withWait: Boolean): Future[Seq[Boolean]] = {
-
- val edgeWithIdxs = _edges.zipWithIndex
- val grouped = edgeWithIdxs.groupBy { case (edge, idx) =>
- (edge.innerLabel, edge.srcVertex.innerId, edge.tgtVertex.innerId)
- } toSeq
-
- val mutateEdges = grouped.map { case ((_, _, _), edgeGroup) =>
- val edges = edgeGroup.map(_._1)
- val idxs = edgeGroup.map(_._2)
- // After deleteAll, process others
- val mutateEdgeFutures = edges.toList match {
- case head :: tail =>
- val edgeFuture = mutateEdgesInner(edges, checkConsistency = true , withWait)
-
- //TODO: decide what we will do on failure on vertex put
- val puts = buildVertexPutsAsync(head)
- val vertexFuture = writeToStorage(head.innerLabel.hbaseZkAddr, puts, withWait)
- Seq(edgeFuture, vertexFuture)
- case Nil => Nil
- }
-
- val composed = for {
-// deleteRet <- Future.sequence(deleteAllFutures)
- mutateRet <- Future.sequence(mutateEdgeFutures)
- } yield mutateRet
-
- composed.map(_.forall(identity)).map { ret => idxs.map( idx => idx -> ret) }
- }
-
- Future.sequence(mutateEdges).map { squashedRets =>
- squashedRets.flatten.sortBy { case (idx, ret) => idx }.map(_._2)
- }
- }
-
- 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)
- } else if (vertex.op == GraphUtil.operations("deleteAll")) {
- logger.info(s"deleteAll for vertex is truncated. $vertex")
- Future.successful(true) // Ignore withWait parameter, because deleteAll operation may takes long time
- } else {
- writeToStorage(vertex.hbaseZkAddr, buildPutsAll(vertex), withWait)
- }
- }
-
- 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[S2Edge],
- checkConsistency: Boolean,
- withWait: Boolean): Future[Boolean] = {
- assert(edges.nonEmpty)
- // TODO:: remove after code review: unreachable code
- if (!checkConsistency) {
-
- val zkQuorum = edges.head.innerLabel.hbaseZkAddr
- val futures = edges.map { edge =>
- val (_, edgeUpdate) = S2Edge.buildOperation(None, Seq(edge))
-
- val (bufferIncr, nonBufferIncr) = increments(edgeUpdate.deepCopy)
- val mutations =
- indexedEdgeMutations(edgeUpdate.deepCopy) ++ snapshotEdgeMutations(edgeUpdate.deepCopy) ++ nonBufferIncr
-
- if (bufferIncr.nonEmpty) writeToStorage(zkQuorum, bufferIncr, withWait = false)
-
- writeToStorage(zkQuorum, mutations, withWait)
- }
- Future.sequence(futures).map { rets => rets.forall(identity) }
- } else {
- fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
- retry(1)(edges, 0, snapshotEdgeOpt)
- }
- }
- }
-
- def exponentialBackOff(tryNum: Int) = {
- // time slot is divided by 10 ms
- val slot = 10
- Random.nextInt(Math.min(BackoffTimeout, slot * Math.pow(2, tryNum)).toInt)
- }
-
- 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}")
- }
-
- Future.successful(false)
- } else {
- val future = commitUpdate(edges, statusCode, fetchedSnapshotEdgeOpt)
- future.onSuccess {
- case success =>
- logger.debug(s"Finished. [$tryNum]\n${edges.head.toLogString}\n")
- }
- future recoverWith {
- case FetchTimeoutException(retryEdge) =>
- logger.info(s"[Try: $tryNum], Fetch fail.\n${retryEdge}")
- /* fetch failed. re-fetch should be done */
- fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
- retry(tryNum + 1)(edges, statusCode, snapshotEdgeOpt)
- }
-
- case PartialFailureException(retryEdge, failedStatusCode, faileReason) =>
- val status = failedStatusCode match {
- case 0 => "AcquireLock failed."
- case 1 => "Mutation failed."
- case 2 => "Increment failed."
- case 3 => "ReleaseLock failed."
- case 4 => "Unknown"
- }
- logger.info(s"[Try: $tryNum], [Status: $status] partial fail.\n${retryEdge.toLogString}\nFailReason: ${faileReason}")
-
- /* retry logic */
- val promise = Promise[Boolean]
- val backOff = exponentialBackOff(tryNum)
- scheduledThreadPool.schedule(new Runnable {
- override def run(): Unit = {
- val future = if (failedStatusCode == 0) {
- // acquire Lock failed. other is mutating so this thead need to re-fetch snapshotEdge.
- /* fetch failed. re-fetch should be done */
- fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
- retry(tryNum + 1)(edges, statusCode, snapshotEdgeOpt)
- }
- } else {
- // partial failure occur while self locked and mutating.
- // assert(fetchedSnapshotEdgeOpt.nonEmpty)
- retry(tryNum + 1)(edges, failedStatusCode, fetchedSnapshotEdgeOpt)
- }
- promise.completeWith(future)
- }
-
- }, backOff, TimeUnit.MILLISECONDS)
- promise.future
-
- case ex: Exception =>
- logger.error("Unknown exception", ex)
- Future.successful(false)
- }
- }
- }
-
- protected def commitUpdate(edges: Seq[S2Edge],
- statusCode: Byte,
- fetchedSnapshotEdgeOpt: Option[S2Edge]): Future[Boolean] = {
-// Future.failed(new PartialFailureException(edges.head, 0, "ahahah"))
- assert(edges.nonEmpty)
-// assert(statusCode == 0 || fetchedSnapshotEdgeOpt.isDefined)
-
- statusCode match {
- case 0 =>
- fetchedSnapshotEdgeOpt match {
- case None =>
- /*
- * no one has never mutated this SN.
- * (squashedEdge, edgeMutate) = Edge.buildOperation(None, edges)
- * pendingE = squashedEdge.copy(statusCode = 1, lockTs = now, version = squashedEdge.ts + 1)
- * lock = (squashedEdge, pendingE)
- * releaseLock = (edgeMutate.newSnapshotEdge, None)
- */
- val (squashedEdge, edgeMutate) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
-
- assert(edgeMutate.newSnapshotEdge.isDefined)
-
- val lockTs = Option(System.currentTimeMillis())
- val pendingEdge = squashedEdge.copy(statusCode = 1, lockTs = lockTs, version = squashedEdge.ts + 1)
- val lockSnapshotEdge = squashedEdge.toSnapshotEdge.copy(pendingEdgeOpt = Option(pendingEdge))
- val releaseLockSnapshotEdge = edgeMutate.newSnapshotEdge.get.copy(statusCode = 0,
- pendingEdgeOpt = None, version = lockSnapshotEdge.version + 1)
-
- commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
-
- case Some(snapshotEdge) =>
- snapshotEdge.pendingEdgeOpt match {
- case None =>
- /*
- * others finished commit on this SN. but there is no contention.
- * (squashedEdge, edgeMutate) = Edge.buildOperation(snapshotEdgeOpt, edges)
- * pendingE = squashedEdge.copy(statusCode = 1, lockTs = now, version = snapshotEdge.version + 1) ?
- * lock = (snapshotEdge, pendingE)
- * releaseLock = (edgeMutate.newSnapshotEdge, None)
- */
- 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)
- } else {
- val lockTs = Option(System.currentTimeMillis())
- val pendingEdge = squashedEdge.copy(statusCode = 1, lockTs = lockTs, version = snapshotEdge.version + 1)
- val lockSnapshotEdge = snapshotEdge.toSnapshotEdge.copy(pendingEdgeOpt = Option(pendingEdge))
- val releaseLockSnapshotEdge = edgeMutate.newSnapshotEdge.get.copy(statusCode = 0,
- pendingEdgeOpt = None, version = lockSnapshotEdge.version + 1)
- commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
- }
- case Some(pendingEdge) =>
- val isLockExpired = pendingEdge.lockTs.get + LockExpireDuration < System.currentTimeMillis()
- if (isLockExpired) {
- /*
- * if pendingEdge.ts == snapshotEdge.ts =>
- * (squashedEdge, edgeMutate) = Edge.buildOperation(None, Seq(pendingEdge))
- * else =>
- * (squashedEdge, edgeMutate) = Edge.buildOperation(snapshotEdgeOpt, Seq(pendingEdge))
- * pendingE = squashedEdge.copy(statusCode = 1, lockTs = now, version = snapshotEdge.version + 1)
- * lock = (snapshotEdge, pendingE)
- * releaseLock = (edgeMutate.newSnapshotEdge, None)
- */
- logger.debug(s"${pendingEdge.toLogString} has been expired.")
- val (squashedEdge, edgeMutate) =
- 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)
- val lockSnapshotEdge = snapshotEdge.toSnapshotEdge.copy(pendingEdgeOpt = Option(newPendingEdge))
- val releaseLockSnapshotEdge = edgeMutate.newSnapshotEdge.get.copy(statusCode = 0,
- pendingEdgeOpt = None, version = lockSnapshotEdge.version + 1)
-
- commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
- } else {
- /*
- * others finished commit on this SN and there is currently contention.
- * this can't be proceed so retry from re-fetch.
- * throw EX
- */
- val (squashedEdge, _) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
- Future.failed(new PartialFailureException(squashedEdge, 0, s"others[${pendingEdge.ts}] is mutating. me[${squashedEdge.ts}]"))
- }
- }
-
- }
- case _ =>
-
- /*
- * statusCode > 0 which means self locked and there has been partial failure either on mutate, increment, releaseLock
- */
-
- /*
- * this succeed to lock this SN. keep doing on commit process.
- * if SN.isEmpty =>
- * no one never succed to commit on this SN.
- * this is first mutation try on this SN.
- * (squashedEdge, edgeMutate) = Edge.buildOperation(SN, edges)
- * else =>
- * assert(SN.pengingEdgeOpt.isEmpty) no-fetch after acquire lock when self retrying.
- * there has been success commit on this SN.
- * (squashedEdge, edgeMutate) = Edge.buildOperation(SN, edges)
- * releaseLock = (edgeMutate.newSnapshotEdge, None)
- */
- val _edges =
- if (fetchedSnapshotEdgeOpt.isDefined && fetchedSnapshotEdgeOpt.get.pendingEdgeOpt.isDefined) fetchedSnapshotEdgeOpt.get.pendingEdgeOpt.get +: edges
- else 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)
- case Some(newSnapshotEdge) => newSnapshotEdge.copy(statusCode = 0, pendingEdgeOpt = None, version = newVersion)
- }
- // lockSnapshotEdge will be ignored.
- commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, releaseLockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
- }
- }
- /**
- * orchestrate commit process.
- * we separate into 4 step to avoid duplicating each step over and over.
- *
- * @param statusCode: current statusCode of this thread to process edges.
- * @param squashedEdge: squashed(in memory) final edge from input edges on same snapshotEdge.
- * @param fetchedSnapshotEdgeOpt: fetched snapshotEdge from storage before commit process begin.
- * @param lockSnapshotEdge: lockEdge that hold necessary data to lock this snapshotEdge for this thread.
- * @param releaseLockSnapshotEdge: releaseLockEdge that will remove lock by storing new final merged states
- * all from current request edges and fetched snapshotEdge.
- * @param edgeMutate: mutations for indexEdge and snapshotEdge.
- * @return
- */
- protected def commitProcess(statusCode: Byte,
- squashedEdge: S2Edge,
- fetchedSnapshotEdgeOpt:Option[S2Edge],
- lockSnapshotEdge: SnapshotEdge,
- releaseLockSnapshotEdge: SnapshotEdge,
- edgeMutate: EdgeMutate): Future[Boolean] = {
- for {
- locked <- acquireLock(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge)
- mutated <- commitIndexEdgeMutations(locked, statusCode, squashedEdge, edgeMutate)
- incremented <- commitIndexEdgeDegreeMutations(mutated, statusCode, squashedEdge, edgeMutate)
- lockReleased <- releaseLock(incremented, statusCode, squashedEdge, releaseLockSnapshotEdge)
- } yield lockReleased
- }
-
- 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")
- logger.debug(msg)
- }
-
- protected def debug(ret: Boolean, phase: String, snapshotEdge: SnapshotEdge, edgeMutate: EdgeMutate) = {
- val msg = Seq(s"[$ret] [$phase]", s"${snapshotEdge.toLogString()}",
- s"${edgeMutate.toLogString}").mkString("\n")
- logger.debug(msg)
- }
-
- /**
- * try to acquire lock on storage for this given snapshotEdge(lockEdge).
- *
- * @param statusCode: current statusCode of this thread to process edges.
- * @param squashedEdge: squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
- * @param fetchedSnapshotEdgeOpt: fetched snapshot edge from storage.
- * @param lockEdge: lockEdge to build RPC request(compareAndSet) into Storage.
- * @return
- */
- protected def acquireLock(statusCode: Byte,
- squashedEdge: S2Edge,
- fetchedSnapshotEdgeOpt: Option[S2Edge],
- lockEdge: SnapshotEdge): Future[Boolean] = {
- if (statusCode >= 1) {
- logger.debug(s"skip acquireLock: [$statusCode]\n${squashedEdge.toLogString}")
- Future.successful(true)
- } else {
- val p = Random.nextDouble()
- if (p < FailProb) {
- Future.failed(new PartialFailureException(squashedEdge, 0, s"$p"))
- } else {
- val lockEdgePut = snapshotEdgeSerializer(lockEdge).toKeyValues.head
- val oldPut = fetchedSnapshotEdgeOpt.map(e => snapshotEdgeSerializer(e.toSnapshotEdge).toKeyValues.head)
- writeLock(lockEdgePut, oldPut).recoverWith { case ex: Exception =>
- logger.error(s"AcquireLock RPC Failed.")
- throw new PartialFailureException(squashedEdge, 0, "AcquireLock RPC Failed")
- }.map { ret =>
- if (ret) {
- val log = Seq(
- "\n",
- "=" * 50,
- s"[Success]: acquireLock",
- s"[RequestEdge]: ${squashedEdge.toLogString}",
- s"[LockEdge]: ${lockEdge.toLogString()}",
- s"[PendingEdge]: ${lockEdge.pendingEdgeOpt.map(_.toLogString).getOrElse("")}",
- "=" * 50, "\n").mkString("\n")
-
- logger.debug(log)
- // debug(ret, "acquireLock", edge.toSnapshotEdge)
- } else {
- throw new PartialFailureException(squashedEdge, 0, "hbase fail.")
- }
- true
- }
- }
- }
- }
-
-
- /**
- * change this snapshot's state on storage from locked into committed by
- * storing new merged states on storage. merge state come from releaseLockEdge.
- * note that releaseLock return Future.failed on predicate failure.
- *
- * @param predicate: indicate if this releaseLock phase should be proceed or not.
- * @param statusCode: releaseLock do not use statusCode, only for debug.
- * @param squashedEdge: squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
- * @param releaseLockEdge: final merged states if all process goes well.
- * @return
- */
- protected def releaseLock(predicate: Boolean,
- statusCode: Byte,
- squashedEdge: S2Edge,
- releaseLockEdge: SnapshotEdge): Future[Boolean] = {
- if (!predicate) {
- Future.failed(new PartialFailureException(squashedEdge, 3, "predicate failed."))
- } else {
- val p = Random.nextDouble()
- if (p < FailProb) Future.failed(new PartialFailureException(squashedEdge, 3, s"$p"))
- else {
- val releaseLockEdgePuts = snapshotEdgeSerializer(releaseLockEdge).toKeyValues
- writeToStorage(squashedEdge.innerLabel.hbaseZkAddr, releaseLockEdgePuts, withWait = true).recoverWith {
- case ex: Exception =>
- logger.error(s"ReleaseLock RPC Failed.")
- throw new PartialFailureException(squashedEdge, 3, "ReleaseLock RPC Failed")
- }.map { ret =>
- if (ret) {
- debug(ret, "releaseLock", squashedEdge.toSnapshotEdge)
- } else {
- val msg = Seq("\nFATAL ERROR\n",
- "=" * 50,
- squashedEdge.toLogString,
- releaseLockEdgePuts,
- "=" * 50,
- "\n"
- )
- logger.error(msg.mkString("\n"))
- // error(ret, "releaseLock", edge.toSnapshotEdge)
- throw new PartialFailureException(squashedEdge, 3, "hbase fail.")
- }
- true
- }
- }
- }
- }
-
- /**
- *
- * @param predicate: indicate if this commitIndexEdgeMutations phase should be proceed or not.
- * @param statusCode: current statusCode of this thread to process edges.
- * @param squashedEdge: squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
- * @param edgeMutate: actual collection of mutations. note that edgeMutate contains snapshotEdge mutations,
- * but in here, we only use indexEdge's mutations.
- * @return
- */
- protected def commitIndexEdgeMutations(predicate: Boolean,
- statusCode: Byte,
- squashedEdge: S2Edge,
- edgeMutate: EdgeMutate): Future[Boolean] = {
- if (!predicate) Future.failed(new PartialFailureException(squashedEdge, 1, "predicate failed."))
- else {
- if (statusCode >= 2) {
- logger.debug(s"skip mutate: [$statusCode]\n${squashedEdge.toLogString}")
- Future.successful(true)
- } else {
- val p = Random.nextDouble()
- if (p < FailProb) Future.failed(new PartialFailureException(squashedEdge, 1, s"$p"))
- else
- writeToStorage(squashedEdge.innerLabel.hbaseZkAddr, indexedEdgeMutations(edgeMutate), withWait = true).map { ret =>
- if (ret) {
- debug(ret, "mutate", squashedEdge.toSnapshotEdge, edgeMutate)
- } else {
- throw new PartialFailureException(squashedEdge, 1, "hbase fail.")
- }
- true
- }
- }
- }
- }
-
- /**
- *
- * @param predicate: indicate if this commitIndexEdgeMutations phase should be proceed or not.
- * @param statusCode: current statusCode of this thread to process edges.
- * @param squashedEdge: squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
- * @param edgeMutate: actual collection of mutations. note that edgeMutate contains snapshotEdge mutations,
- * but in here, we only use indexEdge's degree mutations.
- * @return
- */
- protected def commitIndexEdgeDegreeMutations(predicate: Boolean,
- statusCode: Byte,
- squashedEdge: S2Edge,
- edgeMutate: EdgeMutate): Future[Boolean] = {
-
- def _write(kvs: Seq[SKeyValue], withWait: Boolean): Future[Boolean] = {
- writeToStorage(squashedEdge.innerLabel.hbaseZkAddr, kvs, withWait = withWait).map { ret =>
- if (ret) {
- debug(ret, "increment", squashedEdge.toSnapshotEdge, edgeMutate)
- } else {
- throw new PartialFailureException(squashedEdge, 2, "hbase fail.")
- }
- true
- }
- }
-
- if (!predicate) Future.failed(new PartialFailureException(squashedEdge, 2, "predicate failed."))
- if (statusCode >= 3) {
- logger.debug(s"skip increment: [$statusCode]\n${squashedEdge.toLogString}")
- Future.successful(true)
- } else {
- val p = Random.nextDouble()
- if (p < FailProb) Future.failed(new PartialFailureException(squashedEdge, 2, s"$p"))
- else {
- val (bufferIncr, nonBufferIncr) = increments(edgeMutate.deepCopy)
-
- if (bufferIncr.nonEmpty) _write(bufferIncr, withWait = false)
- _write(nonBufferIncr, withWait = true)
- }
- }
- }
-
- /** end of methods for consistency */
-
- 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")}",
- s"newEdge: ${newEdge.toLogString}",
- s"mutation: \n${edgeMutate.toLogString}",
- "----------------------------------------------").mkString("\n")
-
-
- /** Delete All */
- def deleteAllFetchedEdgesAsyncOld(stepInnerResult: StepResult,
- requestTs: Long,
- retryNum: Int): Future[Boolean] = {
- if (stepInnerResult.isEmpty) Future.successful(true)
- else {
- val head = stepInnerResult.edgeWithScores.head
- val zkQuorum = head.edge.innerLabel.hbaseZkAddr
- val futures = for {
- edgeWithScore <- stepInnerResult.edgeWithScores
- } yield {
- val edge = edgeWithScore.edge
- val score = edgeWithScore.score
-
- 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 = 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 = S2Edge.propsToState(edge.updatePropsWithTs()))
- val reversedIndexedEdgesMutations = edgeRevert.duplicateEdge.edgesWithIndex.flatMap { indexEdge =>
- indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete)) ++
- buildIncrementsAsync(indexEdge, -1L)
- }
-
- val mutations = reversedIndexedEdgesMutations ++ reversedSnapshotEdgeMutations ++ forwardIndexedEdgeMutations
-
- writeToStorage(zkQuorum, mutations, withWait = true)
- }
-
- Future.sequence(futures).map { rets => rets.forall(identity) }
- }
- }
-
- /** End Of Delete All */
-
-
-
-
- /** Parsing Logic: parse from kv from Storage into Edge */
- def toEdge[K: CanSKeyValue](kv: K,
- queryRequest: QueryRequest,
- cacheElementOpt: Option[S2Edge],
- parentEdges: Seq[EdgeWithScore]): Option[S2Edge] = {
- logger.debug(s"toEdge: $kv")
-
- try {
- val queryOption = queryRequest.query.queryOption
- val queryParam = queryRequest.queryParam
- val schemaVer = queryParam.label.schemaVersion
- val indexEdgeOpt = indexEdgeDeserializer(schemaVer).fromKeyValues(Seq(kv), cacheElementOpt)
- if (!queryOption.returnTree) indexEdgeOpt.map(indexEdge => indexEdge.copy(parentEdges = parentEdges))
- else indexEdgeOpt
- } catch {
- case ex: Exception =>
- logger.error(s"Fail on toEdge: ${kv.toString}, ${queryRequest}", ex)
- None
- }
- }
-
- def toSnapshotEdge[K: CanSKeyValue](kv: K,
- queryRequest: QueryRequest,
- cacheElementOpt: Option[SnapshotEdge] = None,
- isInnerCall: Boolean,
- parentEdges: Seq[EdgeWithScore]): Option[S2Edge] = {
-// logger.debug(s"SnapshottoEdge: $kv")
- val queryParam = queryRequest.queryParam
- val schemaVer = queryParam.label.schemaVersion
- val snapshotEdgeOpt = snapshotEdgeDeserializer(schemaVer).fromKeyValues(Seq(kv), cacheElementOpt)
-
- if (isInnerCall) {
- snapshotEdgeOpt.flatMap { snapshotEdge =>
- val edge = snapshotEdge.toEdge.copy(parentEdges = parentEdges)
- if (queryParam.where.map(_.filter(edge)).getOrElse(true)) Option(edge)
- else None
- }
- } else {
- snapshotEdgeOpt.flatMap { snapshotEdge =>
- if (snapshotEdge.allPropsDeleted) None
- else {
- val edge = snapshotEdge.toEdge.copy(parentEdges = parentEdges)
- if (queryParam.where.map(_.filter(edge)).getOrElse(true)) Option(edge)
- else None
- }
- }
- }
- }
-
- val dummyCursor: Array[Byte] = Array.empty
-
- def toEdges[K: CanSKeyValue](kvs: Seq[K],
- queryRequest: QueryRequest,
- prevScore: Double = 1.0,
- isInnerCall: Boolean,
- parentEdges: Seq[EdgeWithScore],
- startOffset: Int = 0,
- len: Int = Int.MaxValue): StepResult = {
-
- val toSKeyValue = implicitly[CanSKeyValue[K]].toSKeyValue _
-
- if (kvs.isEmpty) StepResult.Empty.copy(cursors = Seq(dummyCursor))
- else {
- val queryOption = queryRequest.query.queryOption
- val queryParam = queryRequest.queryParam
- val labelWeight = queryRequest.labelWeight
- val nextStepOpt = queryRequest.nextStepOpt
- val where = queryParam.where.get
- val label = queryParam.label
- val isDefaultTransformer = queryParam.edgeTransformer.isDefault
- val first = kvs.head
- val kv = first
- val schemaVer = queryParam.label.schemaVersion
- val cacheElementOpt =
- if (queryParam.isSnapshotEdge) None
- else indexEdgeDeserializer(schemaVer).fromKeyValues(Seq(kv), None)
-
- val (degreeEdges, keyValues) = cacheElementOpt match {
- case None => (Nil, kvs)
- case Some(cacheElement) =>
- val head = cacheElement
- if (!head.isDegree) (Nil, kvs)
- else (Seq(EdgeWithScore(head, 1.0, label)), kvs.tail)
- }
-
- val lastCursor: Seq[Array[Byte]] = Seq(if (keyValues.nonEmpty) toSKeyValue(keyValues(keyValues.length - 1)).row else dummyCursor)
-
- if (!queryOption.ignorePrevStepCache) {
- val edgeWithScores = for {
- (kv, idx) <- keyValues.zipWithIndex if idx >= startOffset && idx < startOffset + len
- edge <- (if (queryParam.isSnapshotEdge) toSnapshotEdge(kv, queryRequest, None, isInnerCall, parentEdges) else toEdge(kv, queryRequest, cacheElementOpt, parentEdges)).toSeq
- if where == WhereParser.success || where.filter(edge)
- convertedEdge <- if (isDefaultTransformer) Seq(edge) else convertEdges(queryParam, edge, nextStepOpt)
- } yield {
- val score = edge.rank(queryParam.rank)
- EdgeWithScore(convertedEdge, score, label)
- }
- StepResult(edgeWithScores = edgeWithScores, grouped = Nil, degreeEdges = degreeEdges, cursors = lastCursor)
- } else {
- val degreeScore = 0.0
-
- val edgeWithScores = for {
- (kv, idx) <- keyValues.zipWithIndex if idx >= startOffset && idx < startOffset + len
- edge <- (if (queryParam.isSnapshotEdge) toSnapshotEdge(kv, queryRequest, None, isInnerCall, parentEdges) else toEdge(kv, queryRequest, cacheElementOpt, parentEdges)).toSeq
- if where == WhereParser.success || where.filter(edge)
- convertedEdge <- if (isDefaultTransformer) Seq(edge) else convertEdges(queryParam, edge, nextStepOpt)
- } yield {
- val edgeScore = edge.rank(queryParam.rank)
- 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
- EdgeWithScore(convertedEdge.copy(parentEdges = parentEdges), score = newScore * labelWeight * tsVal, label = label)
- }
-
- val sampled =
- if (queryRequest.queryParam.sample >= 0) sample(queryRequest, edgeWithScores, queryRequest.queryParam.sample)
- else edgeWithScores
-
- val normalized = if (queryParam.shouldNormalize) normalize(sampled) else sampled
-
- StepResult(edgeWithScores = normalized, grouped = Nil, degreeEdges = degreeEdges, cursors = lastCursor)
- }
- }
- }
-
- /** End Of Parse Logic */
-
- protected def toRequestEdge(queryRequest: QueryRequest, parentEdges: Seq[EdgeWithScore]): S2Edge = {
+object Storage {
+ def toRequestEdge(graph: S2Graph)(queryRequest: QueryRequest, parentEdges: Seq[EdgeWithScore]): S2Edge = {
val srcVertex = queryRequest.vertex
val queryParam = queryRequest.queryParam
val tgtVertexIdOpt = queryParam.tgtVertexInnerIdOpt
@@ -988,133 +56,131 @@ abstract class Storage[Q, R](val graph: S2Graph,
}
}
- 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.
- * */
- 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 q = Query.toQuery(Seq(edge.srcVertex), queryParam)
+}
- fetchSnapshotEdgeKeyValues(queryRequest).map { kvs =>
- val (edgeOpt, kvOpt) =
- if (kvs.isEmpty) (None, None)
- else {
- val snapshotEdgeOpt = 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 new FetchTimeoutException(s"${edge.toLogString}")
- }
- }
+abstract class Storage[Q](val graph: S2Graph,
+ val config: Config) {
+ /* Storage backend specific resource management */
+ val management: StorageManagement
+
+ /* Physically store given KeyValue into backend storage. */
+ val mutator: StorageWritable
+
+ /*
+ * Given QueryRequest/Vertex/Edge, fetch KeyValue from storage
+ * then convert them into Edge/Vertex
+ */
+ val fetcher: StorageReadable[Q]
+
+ /*
+ * Serialize Edge/Vertex, to common KeyValue, SKeyValue that
+ * can be stored aligned to backend storage's physical schema.
+ * Also Deserialize storage backend's KeyValue to SKeyValue.
+ */
+ val serDe: StorageSerDe
+
+ /*
+ * Common helper to translate SKeyValue to Edge/Vertex and vice versa.
+ * Note that it require storage backend specific implementation for serialize/deserialize.
+ */
+ lazy val io: StorageIO = new StorageIO(graph, serDe)
+
+ /*
+ * Common helper to resolve write-write conflict on snapshot edge with same EdgeId.
+ * Note that it require storage backend specific implementations for
+ * all of StorageWritable, StorageReadable, StorageSerDe, StorageIO
+ */
+ lazy val conflictResolver: WriteWriteConflictResolver = new WriteWriteConflictResolver(graph, serDe, io, mutator, fetcher)
+
+
+ /** IO **/
+ def snapshotEdgeSerializer(snapshotEdge: SnapshotEdge): serde.Serializable[SnapshotEdge] =
+ serDe.snapshotEdgeSerializer(snapshotEdge)
+
+ def indexEdgeSerializer(indexEdge: IndexEdge): serde.Serializable[IndexEdge] =
+ serDe.indexEdgeSerializer(indexEdge)
+
+ def vertexSerializer(vertex: S2Vertex): serde.Serializable[S2Vertex] =
+ serDe.vertexSerializer(vertex)
+ def snapshotEdgeDeserializer(schemaVer: String): Deserializable[SnapshotEdge] =
+ serDe.snapshotEdgeDeserializer(schemaVer)
- /** end of query */
+ def indexEdgeDeserializer(schemaVer: String): IndexEdgeDeserializable =
+ serDe.indexEdgeDeserializer(schemaVer)
+
+ def vertexDeserializer(schemaVer: String): Deserializable[S2Vertex] =
+ serDe.vertexDeserializer(schemaVer)
/** Mutation Builder */
+ def increments(edgeMutate: EdgeMutate): (Seq[SKeyValue], Seq[SKeyValue]) =
+ io.increments(edgeMutate)
+ def indexedEdgeMutations(edgeMutate: EdgeMutate): Seq[SKeyValue] =
+ io.indexedEdgeMutations(edgeMutate)
- /** EdgeMutate */
- def indexedEdgeMutations(edgeMutate: EdgeMutate): Seq[SKeyValue] = {
- // skip sampling for delete operation
- val deleteMutations = edgeMutate.edgesToDeleteWithIndexOpt.flatMap { indexEdge =>
- indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete, durability = indexEdge.label.durability))
- }
+ def buildIncrementsAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] =
+ io.buildIncrementsAsync(indexedEdge, amount)
- val insertMutations = edgeMutate.edgesToInsertWithIndexOpt.flatMap { indexEdge =>
- indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
- }
+ def buildIncrementsCountAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] =
+ io.buildIncrementsCountAsync(indexedEdge, amount)
- deleteMutations ++ insertMutations
- }
+ def buildVertexPutsAsync(edge: S2Edge): Seq[SKeyValue] =
+ io.buildVertexPutsAsync(edge)
def snapshotEdgeMutations(edgeMutate: EdgeMutate): Seq[SKeyValue] =
- edgeMutate.newSnapshotEdge.map(e => snapshotEdgeSerializer(e).toKeyValues.map(_.copy(durability = e.label.durability))).getOrElse(Nil)
+ io.snapshotEdgeMutations(edgeMutate)
- def increments(edgeMutate: EdgeMutate): (Seq[SKeyValue], Seq[SKeyValue]) = {
- (edgeMutate.edgesToDeleteWithIndexOptForDegree.isEmpty, edgeMutate.edgesToInsertWithIndexOptForDegree.isEmpty) match {
- case (true, true) =>
- /* when there is no need to update. shouldUpdate == false */
- Nil -> Nil
+ def buildDegreePuts(edge: S2Edge, degreeVal: Long): Seq[SKeyValue] =
+ io.buildDegreePuts(edge, degreeVal)
- case (true, false) =>
- /* no edges to delete but there is new edges to insert so increase degree by 1 */
- val (buffer, nonBuffer) = EdgeMutate.partitionBufferedIncrement(edgeMutate.edgesToInsertWithIndexOptForDegree)
- buffer.flatMap(buildIncrementsAsync(_)) -> nonBuffer.flatMap(buildIncrementsAsync(_))
+ def buildPutsAll(vertex: S2Vertex): Seq[SKeyValue] =
+ io.buildPutsAll(vertex)
- case (false, true) =>
- /* no edges to insert but there is old edges to delete so decrease degree by 1 */
- val (buffer, nonBuffer) = EdgeMutate.partitionBufferedIncrement(edgeMutate.edgesToDeleteWithIndexOptForDegree)
- buffer.flatMap(buildIncrementsAsync(_, -1)) -> nonBuffer.flatMap(buildIncrementsAsync(_, -1))
+ /** Mutation **/
- case (false, false) =>
- /* update on existing edges so no change on degree */
- Nil -> Nil
- }
- }
+ def writeToStorage(cluster: String, kvs: Seq[SKeyValue], withWait: Boolean)(implicit ec: ExecutionContext): Future[MutateResponse] =
+ mutator.writeToStorage(cluster, kvs, withWait)
- /** IndexEdge */
- def buildIncrementsAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] = {
- val newProps = indexedEdge.updatePropsWithTs()
- newProps.put(LabelMeta.degree.name, new S2Property(indexedEdge.toEdge, LabelMeta.degree, LabelMeta.degree.name, amount, indexedEdge.ts))
- val _indexedEdge = indexedEdge.copy(propsWithTs = newProps)
- indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment, durability = _indexedEdge.label.durability))
- }
+ def writeLock(requestKeyValue: SKeyValue, expectedOpt: Option[SKeyValue])(implicit ec: ExecutionContext): Future[MutateResponse] =
+ mutator.writeLock(requestKeyValue, expectedOpt)
- def buildIncrementsCountAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] = {
- val newProps = indexedEdge.updatePropsWithTs()
- newProps.put(LabelMeta.degree.name, new S2Property(indexedEdge.toEdge, LabelMeta.degree, LabelMeta.degree.name, amount, indexedEdge.ts))
- val _indexedEdge = indexedEdge.copy(propsWithTs = newProps)
- indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment, durability = _indexedEdge.label.durability))
- }
+ /** Fetch **/
- //TODO: ServiceColumn do not have durability property yet.
- def buildDeleteBelongsToId(vertex: S2Vertex): Seq[SKeyValue] = {
- val kvs = vertexSerializer(vertex).toKeyValues
- val kv = kvs.head
- vertex.belongLabelIds.map { id =>
- kv.copy(qualifier = Bytes.toBytes(S2Vertex.toPropKey(id)), operation = SKeyValue.Delete)
- }
- }
+ def buildRequest(queryRequest: QueryRequest, edge: S2Edge): Q = fetcher.buildRequest(queryRequest, edge)
- def buildVertexPutsAsync(edge: S2Edge): Seq[SKeyValue] = {
- val storeVertex = edge.innerLabel.extraOptions.get("storeVertex").map(_.as[Boolean]).getOrElse(false)
+ def buildRequest(queryRequest: QueryRequest, vertex: S2Vertex): Q = fetcher.buildRequest(queryRequest, vertex)
- if (storeVertex) {
- if (edge.op == GraphUtil.operations("delete"))
- buildDeleteBelongsToId(edge.srcForVertex) ++ buildDeleteBelongsToId(edge.tgtForVertex)
- else
- vertexSerializer(edge.srcForVertex).toKeyValues ++ vertexSerializer(edge.tgtForVertex).toKeyValues
- } else {
- Seq.empty
- }
- }
+ def fetches(queryRequests: Seq[QueryRequest],
+ prevStepEdges: Map[VertexId, Seq[EdgeWithScore]])(implicit ec: ExecutionContext): Future[Seq[StepResult]] =
+ fetcher.fetches(queryRequests, prevStepEdges)
- def buildDegreePuts(edge: S2Edge, degreeVal: Long): Seq[SKeyValue] = {
- edge.propertyInner(LabelMeta.degree.name, degreeVal, edge.ts)
- val kvs = edge.edgesWithIndexValid.flatMap { indexEdge =>
- indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
- }
+ def fetchKeyValues(rpc: Q)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]] = fetcher.fetchKeyValues(rpc)
- kvs
- }
+ def fetchEdgesAll()(implicit ec: ExecutionContext): Future[Seq[S2Edge]] = fetcher.fetchEdgesAll()
- 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))
- }
- }
+ def fetchVerticesAll()(implicit ec: ExecutionContext): Future[Seq[S2Vertex]] = fetcher.fetchVerticesAll()
- def info: Map[String, String] = Map("className" -> this.getClass.getSimpleName)
+ def fetchSnapshotEdgeInner(edge: S2Edge)(implicit ec: ExecutionContext): Future[(QueryParam, Option[S2Edge], Option[SKeyValue])] =
+ fetcher.fetchSnapshotEdgeInner(edge)
+
+ /** Conflict Resolver **/
+ def retry(tryNum: Int)(edges: Seq[S2Edge], statusCode: Byte, fetchedSnapshotEdgeOpt: Option[S2Edge])(implicit ec: ExecutionContext): Future[Boolean] =
+ conflictResolver.retry(tryNum)(edges, statusCode, fetchedSnapshotEdgeOpt)
+
+ /** Management **/
+ def flush(): Unit = management.flush()
+ def createTable(config: Config, tableNameStr: String): Unit = management.createTable(config, tableNameStr)
+
+ def truncateTable(config: Config, tableNameStr: String): Unit = management.truncateTable(config, tableNameStr)
+
+ def deleteTable(config: Config, tableNameStr: String): Unit = management.deleteTable(config, tableNameStr)
+
+ def shutdown(): Unit = management.shutdown()
+
+
+ def info: Map[String, String] = Map("className" -> this.getClass.getSimpleName)
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageDeserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageDeserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageDeserializable.scala
deleted file mode 100644
index 811cf62..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageDeserializable.scala
+++ /dev/null
@@ -1,120 +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
-
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.QueryParam
-import org.apache.s2graph.core.mysqls.{LabelMeta, Label}
-import org.apache.s2graph.core.types.{HBaseType, InnerVal, InnerValLike, InnerValLikeWithTs}
-import org.apache.s2graph.core.utils.logger
-
-object StorageDeserializable {
- /** Deserializer */
- def bytesToLabelIndexSeqWithIsInverted(bytes: Array[Byte], offset: Int): (Byte, Boolean) = {
- val byte = bytes(offset)
- val isInverted = if ((byte & 1) != 0) true else false
- val labelOrderSeq = byte >> 1
- (labelOrderSeq.toByte, isInverted)
- }
-
- def bytesToKeyValues(bytes: Array[Byte],
- offset: Int,
- length: Int,
- schemaVer: String,
- label: Label): (Array[(LabelMeta, InnerValLike)], Int) = {
- var pos = offset
- val len = bytes(pos)
- pos += 1
- val kvs = new Array[(LabelMeta, InnerValLike)](len)
- var i = 0
- while (i < len) {
- val k = label.labelMetaMap(bytes(pos))
- pos += 1
- val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, schemaVer)
- pos += numOfBytesUsed
- kvs(i) = (k -> v)
- i += 1
- }
- val ret = (kvs, pos)
- // logger.debug(s"bytesToProps: $ret")
- ret
- }
-
- def bytesToKeyValuesWithTs(bytes: Array[Byte],
- offset: Int,
- schemaVer: String,
- label: Label): (Array[(LabelMeta, InnerValLikeWithTs)], Int) = {
- var pos = offset
- val len = bytes(pos)
- pos += 1
- val kvs = new Array[(LabelMeta, InnerValLikeWithTs)](len)
- var i = 0
- while (i < len) {
- val k = label.labelMetaMap(bytes(pos))
- pos += 1
- val (v, numOfBytesUsed) = InnerValLikeWithTs.fromBytes(bytes, pos, 0, schemaVer)
- pos += numOfBytesUsed
- kvs(i) = (k -> v)
- i += 1
- }
- val ret = (kvs, pos)
- // logger.debug(s"bytesToProps: $ret")
- ret
- }
-
- def bytesToProps(bytes: Array[Byte],
- offset: Int,
- schemaVer: String): (Array[(LabelMeta, InnerValLike)], Int) = {
- var pos = offset
- val len = bytes(pos)
- pos += 1
- val kvs = new Array[(LabelMeta, InnerValLike)](len)
- var i = 0
- while (i < len) {
- val k = LabelMeta.empty
- val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, schemaVer)
- pos += numOfBytesUsed
- kvs(i) = (k -> v)
- i += 1
- }
- // logger.error(s"bytesToProps: $kvs")
- val ret = (kvs, pos)
-
- ret
- }
-
- def bytesToLong(bytes: Array[Byte], offset: Int): Long = Bytes.toLong(bytes, offset)
-
- def bytesToInt(bytes: Array[Byte], offset: Int): Int = Bytes.toInt(bytes, offset)
-}
-
-trait StorageDeserializable[E] {
- def fromKeyValues[T: CanSKeyValue](kvs: Seq[T], cacheElementOpt: Option[E]): Option[E]
-// = {
-// try {
-// Option(fromKeyValuesInner(kvs, cacheElementOpt))
-// } catch {
-// case e: Exception =>
-// logger.error(s"${this.getClass.getName} fromKeyValues failed.", e)
-// None
-// }
-// }
-// def fromKeyValuesInner[T: CanSKeyValue](kvs: Seq[T], cacheElementOpt: Option[E]): E
-}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala
new file mode 100644
index 0000000..2e11f0b
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageIO.scala
@@ -0,0 +1,241 @@
+package org.apache.s2graph.core.storage
+
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core.S2Graph.{convertEdges, normalize, processTimeDecay, sample}
+import org.apache.s2graph.core._
+import org.apache.s2graph.core.mysqls.LabelMeta
+import org.apache.s2graph.core.parsers.WhereParser
+import org.apache.s2graph.core.utils.logger
+
+class StorageIO(val graph: S2Graph, val serDe: StorageSerDe) {
+ val dummyCursor: Array[Byte] = Array.empty
+
+ /** Parsing Logic: parse from kv from Storage into Edge */
+ def toEdge[K: CanSKeyValue](kv: K,
+ queryRequest: QueryRequest,
+ cacheElementOpt: Option[S2Edge],
+ parentEdges: Seq[EdgeWithScore]): Option[S2Edge] = {
+ logger.debug(s"toEdge: $kv")
+
+ try {
+ val queryOption = queryRequest.query.queryOption
+ val queryParam = queryRequest.queryParam
+ val schemaVer = queryParam.label.schemaVersion
+ val indexEdgeOpt = serDe.indexEdgeDeserializer(schemaVer).fromKeyValues(Seq(kv), cacheElementOpt)
+ if (!queryOption.returnTree) indexEdgeOpt.map(indexEdge => indexEdge.copy(parentEdges = parentEdges))
+ else indexEdgeOpt
+ } catch {
+ case ex: Exception =>
+ logger.error(s"Fail on toEdge: ${kv.toString}, ${queryRequest}", ex)
+ None
+ }
+ }
+
+ def toSnapshotEdge[K: CanSKeyValue](kv: K,
+ queryRequest: QueryRequest,
+ cacheElementOpt: Option[SnapshotEdge] = None,
+ isInnerCall: Boolean,
+ parentEdges: Seq[EdgeWithScore]): Option[S2Edge] = {
+ // logger.debug(s"SnapshottoEdge: $kv")
+ val queryParam = queryRequest.queryParam
+ val schemaVer = queryParam.label.schemaVersion
+ val snapshotEdgeOpt = serDe.snapshotEdgeDeserializer(schemaVer).fromKeyValues(Seq(kv), cacheElementOpt)
+
+ if (isInnerCall) {
+ snapshotEdgeOpt.flatMap { snapshotEdge =>
+ val edge = snapshotEdge.toEdge.copy(parentEdges = parentEdges)
+ if (queryParam.where.map(_.filter(edge)).getOrElse(true)) Option(edge)
+ else None
+ }
+ } else {
+ snapshotEdgeOpt.flatMap { snapshotEdge =>
+ if (snapshotEdge.allPropsDeleted) None
+ else {
+ val edge = snapshotEdge.toEdge.copy(parentEdges = parentEdges)
+ if (queryParam.where.map(_.filter(edge)).getOrElse(true)) Option(edge)
+ else None
+ }
+ }
+ }
+ }
+
+ def toEdges[K: CanSKeyValue](kvs: Seq[K],
+ queryRequest: QueryRequest,
+ prevScore: Double = 1.0,
+ isInnerCall: Boolean,
+ parentEdges: Seq[EdgeWithScore],
+ startOffset: Int = 0,
+ len: Int = Int.MaxValue): StepResult = {
+
+ val toSKeyValue = implicitly[CanSKeyValue[K]].toSKeyValue _
+
+ if (kvs.isEmpty) StepResult.Empty.copy(cursors = Seq(dummyCursor))
+ else {
+ val queryOption = queryRequest.query.queryOption
+ val queryParam = queryRequest.queryParam
+ val labelWeight = queryRequest.labelWeight
+ val nextStepOpt = queryRequest.nextStepOpt
+ val where = queryParam.where.get
+ val label = queryParam.label
+ val isDefaultTransformer = queryParam.edgeTransformer.isDefault
+ val first = kvs.head
+ val kv = first
+ val schemaVer = queryParam.label.schemaVersion
+ val cacheElementOpt =
+ if (queryParam.isSnapshotEdge) None
+ else serDe.indexEdgeDeserializer(schemaVer).fromKeyValues(Seq(kv), None)
+
+ val (degreeEdges, keyValues) = cacheElementOpt match {
+ case None => (Nil, kvs)
+ case Some(cacheElement) =>
+ val head = cacheElement
+ if (!head.isDegree) (Nil, kvs)
+ else (Seq(EdgeWithScore(head, 1.0, label)), kvs.tail)
+ }
+
+ val lastCursor: Seq[Array[Byte]] = Seq(if (keyValues.nonEmpty) toSKeyValue(keyValues(keyValues.length - 1)).row else dummyCursor)
+
+ if (!queryOption.ignorePrevStepCache) {
+ val edgeWithScores = for {
+ (kv, idx) <- keyValues.zipWithIndex if idx >= startOffset && idx < startOffset + len
+ edge <- (if (queryParam.isSnapshotEdge) toSnapshotEdge(kv, queryRequest, None, isInnerCall, parentEdges) else toEdge(kv, queryRequest, cacheElementOpt, parentEdges)).toSeq
+ if where == WhereParser.success || where.filter(edge)
+ convertedEdge <- if (isDefaultTransformer) Seq(edge) else convertEdges(queryParam, edge, nextStepOpt)
+ } yield {
+ val score = edge.rank(queryParam.rank)
+ EdgeWithScore(convertedEdge, score, label)
+ }
+ StepResult(edgeWithScores = edgeWithScores, grouped = Nil, degreeEdges = degreeEdges, cursors = lastCursor)
+ } else {
+ val degreeScore = 0.0
+
+ val edgeWithScores = for {
+ (kv, idx) <- keyValues.zipWithIndex if idx >= startOffset && idx < startOffset + len
+ edge <- (if (queryParam.isSnapshotEdge) toSnapshotEdge(kv, queryRequest, None, isInnerCall, parentEdges) else toEdge(kv, queryRequest, cacheElementOpt, parentEdges)).toSeq
+ if where == WhereParser.success || where.filter(edge)
+ convertedEdge <- if (isDefaultTransformer) Seq(edge) else convertEdges(queryParam, edge, nextStepOpt)
+ } yield {
+ val edgeScore = edge.rank(queryParam.rank)
+ 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
+ EdgeWithScore(convertedEdge.copy(parentEdges = parentEdges), score = newScore * labelWeight * tsVal, label = label)
+ }
+
+ val sampled =
+ if (queryRequest.queryParam.sample >= 0) sample(queryRequest, edgeWithScores, queryRequest.queryParam.sample)
+ else edgeWithScores
+
+ val normalized = if (queryParam.shouldNormalize) normalize(sampled) else sampled
+
+ StepResult(edgeWithScores = normalized, grouped = Nil, degreeEdges = degreeEdges, cursors = lastCursor)
+ }
+ }
+ }
+
+ /** End Of Parse Logic */
+
+
+ /** end of query */
+
+ /** Mutation Builder */
+
+
+ /** EdgeMutate */
+ def indexedEdgeMutations(edgeMutate: EdgeMutate): Seq[SKeyValue] = {
+ // skip sampling for delete operation
+ val deleteMutations = edgeMutate.edgesToDeleteWithIndexOpt.flatMap { indexEdge =>
+ serDe.indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete, durability = indexEdge.label.durability))
+ }
+
+ val insertMutations = edgeMutate.edgesToInsertWithIndexOpt.flatMap { indexEdge =>
+ serDe.indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
+ }
+
+ deleteMutations ++ insertMutations
+ }
+
+ def snapshotEdgeMutations(edgeMutate: EdgeMutate): Seq[SKeyValue] =
+ edgeMutate.newSnapshotEdge.map(e => serDe.snapshotEdgeSerializer(e).toKeyValues.map(_.copy(durability = e.label.durability))).getOrElse(Nil)
+
+ def increments(edgeMutate: EdgeMutate): (Seq[SKeyValue], Seq[SKeyValue]) = {
+ (edgeMutate.edgesToDeleteWithIndexOptForDegree.isEmpty, edgeMutate.edgesToInsertWithIndexOptForDegree.isEmpty) match {
+ case (true, true) =>
+ /* when there is no need to update. shouldUpdate == false */
+ Nil -> Nil
+
+ case (true, false) =>
+ /* no edges to delete but there is new edges to insert so increase degree by 1 */
+ val (buffer, nonBuffer) = EdgeMutate.partitionBufferedIncrement(edgeMutate.edgesToInsertWithIndexOptForDegree)
+ buffer.flatMap(buildIncrementsAsync(_)) -> nonBuffer.flatMap(buildIncrementsAsync(_))
+
+ case (false, true) =>
+ /* no edges to insert but there is old edges to delete so decrease degree by 1 */
+ val (buffer, nonBuffer) = EdgeMutate.partitionBufferedIncrement(edgeMutate.edgesToDeleteWithIndexOptForDegree)
+ buffer.flatMap(buildIncrementsAsync(_, -1)) -> nonBuffer.flatMap(buildIncrementsAsync(_, -1))
+
+ case (false, false) =>
+ /* update on existing edges so no change on degree */
+ Nil -> Nil
+ }
+ }
+
+ /** IndexEdge */
+ def buildIncrementsAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] = {
+ val newProps = indexedEdge.updatePropsWithTs()
+ newProps.put(LabelMeta.degree.name, new S2Property(indexedEdge.toEdge, LabelMeta.degree, LabelMeta.degree.name, amount, indexedEdge.ts))
+ val _indexedEdge = indexedEdge.copy(propsWithTs = newProps)
+ serDe.indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment, durability = _indexedEdge.label.durability))
+ }
+
+ def buildIncrementsCountAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] = {
+ val newProps = indexedEdge.updatePropsWithTs()
+ newProps.put(LabelMeta.degree.name, new S2Property(indexedEdge.toEdge, LabelMeta.degree, LabelMeta.degree.name, amount, indexedEdge.ts))
+ val _indexedEdge = indexedEdge.copy(propsWithTs = newProps)
+ serDe.indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment, durability = _indexedEdge.label.durability))
+ }
+
+ //TODO: ServiceColumn do not have durability property yet.
+ def buildDeleteBelongsToId(vertex: S2Vertex): Seq[SKeyValue] = {
+ val kvs = serDe.vertexSerializer(vertex).toKeyValues
+ val kv = kvs.head
+ vertex.belongLabelIds.map { id =>
+ kv.copy(qualifier = Bytes.toBytes(S2Vertex.toPropKey(id)), operation = SKeyValue.Delete)
+ }
+ }
+
+ def buildVertexPutsAsync(edge: S2Edge): Seq[SKeyValue] = {
+ val storeVertex = edge.innerLabel.extraOptions.get("storeVertex").map(_.as[Boolean]).getOrElse(false)
+
+ if (storeVertex) {
+ if (edge.op == GraphUtil.operations("delete"))
+ buildDeleteBelongsToId(edge.srcForVertex) ++ buildDeleteBelongsToId(edge.tgtForVertex)
+ else
+ serDe.vertexSerializer(edge.srcForVertex).toKeyValues ++ serDe.vertexSerializer(edge.tgtForVertex).toKeyValues
+ } else {
+ Seq.empty
+ }
+ }
+
+ def buildDegreePuts(edge: S2Edge, degreeVal: Long): Seq[SKeyValue] = {
+ edge.propertyInner(LabelMeta.degree.name, degreeVal, edge.ts)
+ val kvs = edge.edgesWithIndexValid.flatMap { indexEdge =>
+ serDe.indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
+ }
+
+ kvs
+ }
+
+ def buildPutsAll(vertex: S2Vertex): Seq[SKeyValue] = {
+ vertex.op match {
+ case d: Byte if d == GraphUtil.operations("delete") => serDe.vertexSerializer(vertex).toKeyValues.map(_.copy(operation = SKeyValue.Delete))
+ case _ => serDe.vertexSerializer(vertex).toKeyValues.map(_.copy(operation = SKeyValue.Put))
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala
new file mode 100644
index 0000000..da94767
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageManagement.scala
@@ -0,0 +1,35 @@
+package org.apache.s2graph.core.storage
+
+import com.typesafe.config.Config
+
+trait StorageManagement {
+ /**
+ * this method need to be called when client shutdown. this is responsible to cleanUp the resources
+ * such as client into storage.
+ */
+ def flush(): Unit
+
+ /**
+ * create table on storage.
+ * if storage implementation does not support namespace or table, then there is nothing to be done
+ * @param config
+ */
+ def createTable(config: Config, tableNameStr: String): Unit
+ /**
+ *
+ * @param config
+ * @param tableNameStr
+ */
+ def truncateTable(config: Config, tableNameStr: String): Unit
+ /**
+ *
+ * @param config
+ * @param tableNameStr
+ */
+ def deleteTable(config: Config, tableNameStr: String): Unit
+
+ /**
+ *
+ */
+ def shutdown(): Unit
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
new file mode 100644
index 0000000..96669ca
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
@@ -0,0 +1,62 @@
+package org.apache.s2graph.core.storage
+
+import org.apache.s2graph.core.GraphExceptions.FetchTimeoutException
+import org.apache.s2graph.core._
+import org.apache.s2graph.core.types.VertexId
+import org.apache.s2graph.core.utils.logger
+
+import scala.concurrent.{ExecutionContext, Future}
+
+trait StorageReadable[Q] {
+ val io: StorageIO
+ /**
+ * build proper request which is specific into storage to call fetchIndexEdgeKeyValues or fetchSnapshotEdgeKeyValues.
+ * for example, Asynchbase use GetRequest, Scanner so this method is responsible to build
+ * client request(GetRequest, Scanner) based on user provided query.
+ *
+ * @param queryRequest
+ * @return
+ */
+ def buildRequest(queryRequest: QueryRequest, edge: S2Edge): Q
+
+ def buildRequest(queryRequest: QueryRequest, vertex: S2Vertex): Q
+
+ /**
+ * 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]])(implicit ec: ExecutionContext): Future[Seq[StepResult]]
+
+ def fetchKeyValues(rpc: Q)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]]
+
+ def fetchEdgesAll()(implicit ec: ExecutionContext): Future[Seq[S2Edge]]
+
+ def fetchVerticesAll()(implicit ec: ExecutionContext): Future[Seq[S2Vertex]]
+
+ def fetchSnapshotEdgeInner(edge: S2Edge)(implicit ec: ExecutionContext): Future[(QueryParam, Option[S2Edge], Option[SKeyValue])] = {
+ 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)
+
+ fetchKeyValues(buildRequest(queryRequest, edge)).map { kvs =>
+ val (edgeOpt, kvOpt) =
+ if (kvs.isEmpty) (None, None)
+ else {
+ val snapshotEdgeOpt = io.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 new FetchTimeoutException(s"${edge.toLogString}")
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala
new file mode 100644
index 0000000..f973e0f
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerDe.scala
@@ -0,0 +1,59 @@
+package org.apache.s2graph.core.storage
+
+import org.apache.s2graph.core.{IndexEdge, S2Graph, S2Vertex, SnapshotEdge}
+import org.apache.s2graph.core.storage.serde.Deserializable
+import org.apache.s2graph.core.storage.serde.indexedge.tall.IndexEdgeDeserializable
+
+trait StorageSerDe {
+ /**
+ * Compatibility table
+ * | label schema version | snapshot edge | index edge | vertex | note |
+ * | v1 | serde.snapshotedge.wide | serde.indexedge.wide | serde.vertex | do not use this. this exist only for backward compatibility issue |
+ * | v2 | serde.snapshotedge.wide | serde.indexedge.wide | serde.vertex | do not use this. this exist only for backward compatibility issue |
+ * | v3 | serde.snapshotedge.tall | serde.indexedge.wide | serde.vertex | recommended with HBase. current stable schema |
+ * | v4 | serde.snapshotedge.tall | serde.indexedge.tall | serde.vertex | experimental schema. use scanner instead of get |
+ *
+ */
+
+ /**
+ * create serializer that knows how to convert given snapshotEdge into kvs: Seq[SKeyValue]
+ * so we can store this kvs.
+ *
+ * @param snapshotEdge : snapshotEdge to serialize
+ * @return serializer implementation for StorageSerializable which has toKeyValues return Seq[SKeyValue]
+ */
+ def snapshotEdgeSerializer(snapshotEdge: SnapshotEdge): serde.Serializable[SnapshotEdge]
+
+ /**
+ * create serializer that knows how to convert given indexEdge into kvs: Seq[SKeyValue]
+ *
+ * @param indexEdge : indexEdge to serialize
+ * @return serializer implementation
+ */
+ def indexEdgeSerializer(indexEdge: IndexEdge): serde.Serializable[IndexEdge]
+
+ /**
+ * create serializer that knows how to convert given vertex into kvs: Seq[SKeyValue]
+ *
+ * @param vertex : vertex to serialize
+ * @return serializer implementation
+ */
+ def vertexSerializer(vertex: S2Vertex): serde.Serializable[S2Vertex]
+
+ /**
+ * create deserializer that can parse stored CanSKeyValue into snapshotEdge.
+ * note that each storage implementation should implement implicit type class
+ * to convert storage dependent dataType into common SKeyValue type by implementing CanSKeyValue
+ *
+ * ex) Asynchbase use it's KeyValue class and CanSKeyValue object has implicit type conversion method.
+ * if any storaage use different class to represent stored byte array,
+ * then that storage implementation is responsible to provide implicit type conversion method on CanSKeyValue.
+ **/
+ def snapshotEdgeDeserializer(schemaVer: String): Deserializable[SnapshotEdge]
+
+ def indexEdgeDeserializer(schemaVer: String): IndexEdgeDeserializable
+
+ def vertexDeserializer(schemaVer: String): Deserializable[S2Vertex]
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerializable.scala
deleted file mode 100644
index c1efe7b..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerializable.scala
+++ /dev/null
@@ -1,82 +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
-
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.mysqls.LabelMeta
-import org.apache.s2graph.core.types.{InnerValLike, InnerValLikeWithTs}
-import org.apache.s2graph.core.utils.logger
-
-object StorageSerializable {
- /** serializer */
- def propsToBytes(props: Seq[(LabelMeta, InnerValLike)]): Array[Byte] = {
- val len = props.length
- assert(len < Byte.MaxValue)
- var bytes = Array.fill(1)(len.toByte)
- for ((_, v) <- props) bytes = Bytes.add(bytes, v.bytes)
- bytes
- }
-
- def propsToKeyValues(props: Seq[(LabelMeta, InnerValLike)]): Array[Byte] = {
- val len = props.length
- assert(len < Byte.MaxValue)
- var bytes = Array.fill(1)(len.toByte)
- for ((k, v) <- props) bytes = Bytes.add(bytes, Array.fill(1)(k.seq), v.bytes)
- bytes
- }
-
- def propsToKeyValuesWithTs(props: Seq[(LabelMeta, InnerValLikeWithTs)]): Array[Byte] = {
- val len = props.length
- assert(len < Byte.MaxValue)
- var bytes = Array.fill(1)(len.toByte)
- for ((k, v) <- props) bytes = Bytes.add(bytes, Array.fill(1)(k.seq), v.bytes)
- bytes
- }
-
- def labelOrderSeqWithIsInverted(labelOrderSeq: Byte, isInverted: Boolean): Array[Byte] = {
- assert(labelOrderSeq < (1 << 6))
- val byte = labelOrderSeq << 1 | (if (isInverted) 1 else 0)
- Array.fill(1)(byte.toByte)
- }
-
- def intToBytes(value: Int): Array[Byte] = Bytes.toBytes(value)
-
- def longToBytes(value: Long): Array[Byte] = Bytes.toBytes(value)
-}
-
-trait StorageSerializable[E] {
- val cf = Serializable.edgeCf
-
- def table: Array[Byte]
- def ts: Long
-
- def toRowKey: Array[Byte]
- def toQualifier: Array[Byte]
- def toValue: Array[Byte]
-
- def toKeyValues: Seq[SKeyValue] = {
- val row = toRowKey
- val qualifier = toQualifier
- val value = toValue
- val kv = SKeyValue(table, row, cf, qualifier, value, ts)
-// logger.debug(s"[SER]: ${kv.toLogString}}")
- Seq(kv)
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala
new file mode 100644
index 0000000..216aece
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageWritable.scala
@@ -0,0 +1,45 @@
+package org.apache.s2graph.core.storage
+
+import scala.concurrent.{ExecutionContext, Future}
+
+trait StorageWritable {
+ /**
+ * decide how to store given key values Seq[SKeyValue] into storage using storage's client.
+ * note that this should be return true on all success.
+ * we assumes that each storage implementation has client as member variable.
+ *
+ *
+ * @param cluster: where this key values should be stored.
+ * @param kvs: sequence of SKeyValue that need to be stored in storage.
+ * @param withWait: flag to control wait ack from storage.
+ * note that in AsynchbaseStorage(which support asynchronous operations), even with true,
+ * it never block thread, but rather submit work and notified by event loop when storage send ack back.
+ * @return ack message from storage.
+ */
+ def writeToStorage(cluster: String, kvs: Seq[SKeyValue], withWait: Boolean)(implicit ec: ExecutionContext): Future[MutateResponse]
+
+ /**
+ * write requestKeyValue into storage if the current value in storage that is stored matches.
+ * note that we only use SnapshotEdge as place for lock, so this method only change SnapshotEdge.
+ *
+ * Most important thing is this have to be 'atomic' operation.
+ * When this operation is mutating requestKeyValue's snapshotEdge, then other thread need to be
+ * either blocked or failed on write-write conflict case.
+ *
+ * Also while this method is still running, then fetchSnapshotEdgeKeyValues should be synchronized to
+ * prevent wrong data for read.
+ *
+ * Best is use storage's concurrency control(either pessimistic or optimistic) such as transaction,
+ * compareAndSet to synchronize.
+ *
+ * for example, AsynchbaseStorage use HBase's CheckAndSet atomic operation to guarantee 'atomicity'.
+ * for storage that does not support concurrency control, then storage implementation
+ * itself can maintain manual locks that synchronize read(fetchSnapshotEdgeKeyValues)
+ * and write(writeLock).
+ * @param requestKeyValue
+ * @param expectedOpt
+ * @return
+ */
+ def writeLock(requestKeyValue: SKeyValue, expectedOpt: Option[SKeyValue])(implicit ec: ExecutionContext): Future[MutateResponse]
+
+}
[7/8] incubator-s2graph git commit: provide default implementation on
fetchVertices at StorageReadable.
Posted by st...@apache.org.
provide default implementation on fetchVertices at StorageReadable.
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/af9c1053
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/af9c1053
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/af9c1053
Branch: refs/heads/master
Commit: af9c1053fcf9f0392290636c78c014e009edc589
Parents: 55d194e
Author: DO YUNG YOON <st...@apache.org>
Authored: Mon Oct 30 18:28:51 2017 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Mon Oct 30 18:28:51 2017 +0900
----------------------------------------------------------------------
.../s2graph/core/storage/StorageReadable.scala | 26 +++++++++++++++++---
.../hbase/AsynchbaseStorageReadable.scala | 21 ----------------
2 files changed, 23 insertions(+), 24 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/af9c1053/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
index 03b01fd..052ca69 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
@@ -27,6 +27,7 @@ import org.apache.s2graph.core.utils.logger
import scala.concurrent.{ExecutionContext, Future}
trait StorageReadable {
+ val serDe: StorageSerDe
val io: StorageIO
/**
* responsible to fire parallel fetch call into storage and create future that will return merged result.
@@ -38,9 +39,6 @@ trait StorageReadable {
def fetches(queryRequests: Seq[QueryRequest],
prevStepEdges: Map[VertexId, Seq[EdgeWithScore]])(implicit ec: ExecutionContext): Future[Seq[StepResult]]
-// private def fetchKeyValues(rpc: Q)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]]
- def fetchVertices(vertices: Seq[S2Vertex])(implicit ec: ExecutionContext): Future[Seq[S2Vertex]]
-
def fetchEdgesAll()(implicit ec: ExecutionContext): Future[Seq[S2Edge]]
def fetchVerticesAll()(implicit ec: ExecutionContext): Future[Seq[S2Vertex]]
@@ -49,6 +47,7 @@ trait StorageReadable {
protected def fetchKeyValues(queryRequest: QueryRequest, vertex: S2Vertex)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]]
+
def fetchSnapshotEdgeInner(edge: S2Edge)(implicit ec: ExecutionContext): Future[(Option[S2Edge], Option[SKeyValue])] = {
val queryParam = QueryParam(labelName = edge.innerLabel.label,
direction = GraphUtil.fromDirection(edge.labelWithDir.dir),
@@ -72,4 +71,25 @@ trait StorageReadable {
throw new FetchTimeoutException(s"${edge.toLogString}")
}
}
+
+ def fetchVertices(vertices: Seq[S2Vertex])(implicit ec: ExecutionContext): Future[Seq[S2Vertex]] = {
+ def fromResult(kvs: Seq[SKeyValue], version: String): Seq[S2Vertex] = {
+ if (kvs.isEmpty) Nil
+ else serDe.vertexDeserializer(version).fromKeyValues(kvs, None).toSeq
+ }
+
+ val futures = vertices.map { vertex =>
+ val queryParam = QueryParam.Empty
+ val q = Query.toQuery(Seq(vertex), Seq(queryParam))
+ val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
+
+ fetchKeyValues(queryRequest, vertex).map { kvs =>
+ fromResult(kvs, vertex.serviceColumn.schemaVersion)
+ } recoverWith {
+ case ex: Throwable => Future.successful(Nil)
+ }
+ }
+
+ Future.sequence(futures).map(_.flatten)
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/af9c1053/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
index 0dc8491..8ff0ee0 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
@@ -246,27 +246,6 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
Future.sequence(futures).map(_.flatten)
}
- override def fetchVertices(vertices: Seq[S2Vertex])(implicit ec: ExecutionContext) = {
- def fromResult(kvs: Seq[SKeyValue], version: String): Seq[S2Vertex] = {
- if (kvs.isEmpty) Nil
- else serDe.vertexDeserializer(version).fromKeyValues(kvs, None).toSeq
- }
-
- val futures = vertices.map { vertex =>
- val queryParam = QueryParam.Empty
- val q = Query.toQuery(Seq(vertex), Seq(queryParam))
- val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
-
- fetchKeyValues(queryRequest, vertex).map { kvs =>
- fromResult(kvs, vertex.serviceColumn.schemaVersion)
- } recoverWith {
- case ex: Throwable => Future.successful(Nil)
- }
- }
-
- Future.sequence(futures).map(_.flatten)
- }
-
override def fetchVerticesAll()(implicit ec: ExecutionContext) = {
val futures = ServiceColumn.findAll().groupBy(_.service.hTableName).toSeq.map { case (hTableName, columns) =>
val distinctColumns = columns.toSet
[4/8] incubator-s2graph git commit: Separate interfaces from Storage.
Posted by st...@apache.org.
Separate interfaces from Storage.
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/39544dc5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/39544dc5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/39544dc5
Branch: refs/heads/master
Commit: 39544dc5debd4821c4f73db17e88bdbeb9f43b38
Parents: 3361320
Author: DO YUNG YOON <st...@apache.org>
Authored: Sat Oct 28 09:27:51 2017 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Sat Oct 28 09:27:51 2017 +0900
----------------------------------------------------------------------
.gitignore | 2 +-
s2core/build.sbt | 3 +-
.../org/apache/s2graph/core/Management.scala | 74 +-
.../org/apache/s2graph/core/QueryParam.scala | 2 +-
.../scala/org/apache/s2graph/core/S2Edge.scala | 2 +-
.../scala/org/apache/s2graph/core/S2Graph.scala | 275 +++--
.../org/apache/s2graph/core/S2Vertex.scala | 40 +-
.../s2graph/core/storage/Deserializable.scala | 43 -
.../s2graph/core/storage/MutateResponse.scala | 12 +
.../apache/s2graph/core/storage/SKeyValue.scala | 1 +
.../s2graph/core/storage/Serializable.scala | 27 -
.../apache/s2graph/core/storage/Storage.scala | 1146 ++----------------
.../core/storage/StorageDeserializable.scala | 120 --
.../apache/s2graph/core/storage/StorageIO.scala | 241 ++++
.../core/storage/StorageManagement.scala | 35 +
.../s2graph/core/storage/StorageReadable.scala | 62 +
.../s2graph/core/storage/StorageSerDe.scala | 59 +
.../core/storage/StorageSerializable.scala | 82 --
.../s2graph/core/storage/StorageWritable.scala | 45 +
.../storage/WriteWriteConflictResolver.scala | 438 +++++++
.../core/storage/hbase/AsynchbaseStorage.scala | 764 +-----------
.../hbase/AsynchbaseStorageManagement.scala | 263 ++++
.../hbase/AsynchbaseStorageReadable.scala | 335 +++++
.../storage/hbase/AsynchbaseStorageSerDe.scala | 68 ++
.../hbase/AsynchbaseStorageWritable.scala | 118 ++
.../core/storage/serde/Deserializable.scala | 41 +
.../core/storage/serde/Serializable.scala | 27 +
.../storage/serde/StorageDeserializable.scala | 144 +++
.../storage/serde/StorageSerializable.scala | 90 ++
.../tall/IndexEdgeDeserializable.scala | 12 +-
.../indexedge/tall/IndexEdgeSerializable.scala | 5 +-
.../wide/IndexEdgeDeserializable.scala | 4 +-
.../indexedge/wide/IndexEdgeSerializable.scala | 5 +-
.../tall/SnapshotEdgeDeserializable.scala | 7 +-
.../tall/SnapshotEdgeSerializable.scala | 4 +-
.../wide/SnapshotEdgeDeserializable.scala | 9 +-
.../wide/SnapshotEdgeSerializable.scala | 4 +-
.../serde/vertex/VertexDeserializable.scala | 146 +--
.../serde/vertex/VertexSerializable.scala | 114 +-
.../vertex/tall/VertexDeserializable.scala | 58 +
.../serde/vertex/tall/VertexSerializable.scala | 54 +
.../vertex/wide/VertexDeserializable.scala | 73 ++
.../serde/vertex/wide/VertexSerializable.scala | 52 +
.../apache/s2graph/core/utils/DeferCache.scala | 8 +-
.../s2graph/core/Integrate/CrudTest.scala | 8 +-
.../LabelLabelIndexMutateOptionTest.scala | 6 +-
.../s2graph/core/storage/StorageIOTest.scala | 59 +
.../core/storage/hbase/IndexEdgeTest.scala | 3 +-
.../core/storage/rocks/RocksStorageTest.scala | 33 +
.../rest/play/controllers/EdgeController.scala | 11 +-
.../play/controllers/VertexController.scala | 5 +-
51 files changed, 2913 insertions(+), 2326 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 9f295f2..ad5a5e9 100644
--- a/.gitignore
+++ b/.gitignore
@@ -107,4 +107,4 @@ server.pid
.cache
### Local Embedded HBase Data ###
-storage/
+#storage/
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/build.sbt
----------------------------------------------------------------------
diff --git a/s2core/build.sbt b/s2core/build.sbt
index 8033581..e7e602f 100644
--- a/s2core/build.sbt
+++ b/s2core/build.sbt
@@ -48,7 +48,8 @@ libraryDependencies ++= Seq(
"org.specs2" %% "specs2-core" % specs2Version % "test",
"org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion ,
"org.apache.lucene" % "lucene-core" % "6.6.0",
- "org.apache.lucene" % "lucene-queryparser" % "6.6.0"
+ "org.apache.lucene" % "lucene-queryparser" % "6.6.0",
+ "org.rocksdb" % "rocksdbjni" % "5.8.0"
)
libraryDependencies := {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 a9741d2..49d3c0e 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
@@ -21,6 +21,7 @@ package org.apache.s2graph.core
import java.util
+import com.typesafe.config.{Config, ConfigFactory}
import org.apache.s2graph.core.GraphExceptions.{InvalidHTableException, LabelAlreadyExistException, LabelNameTooLongException, LabelNotExistException}
import org.apache.s2graph.core.Management.JsonModel.{Index, Prop}
import org.apache.s2graph.core.mysqls._
@@ -36,9 +37,22 @@ import scala.util.Try
* s2core never use this for finding models.
*/
object Management {
-
+ import HBaseType._
import scala.collection.JavaConversions._
+ val ZookeeperQuorum = "hbase.zookeeper.quorum"
+ val ColumnFamilies = "hbase.table.column.family"
+ val RegionMultiplier = "hbase.table.region.multiplier"
+ val Ttl = "hbase.table.ttl"
+ val CompressionAlgorithm = "hbase.table.compression.algorithm"
+ val ReplicationScope = "hbase.table.replication.scope"
+ val TotalRegionCount = "hbase.table.total.region.count"
+
+ val DefaultColumnFamilies = Seq("e", "v")
+ val DefaultCompressionAlgorithm = "gz"
+ val LABEL_NAME_MAX_LENGTH = 100
+
+
def newProp(name: String, defaultValue: String, datatType: String): Prop = {
new Prop(name, defaultValue, datatType)
}
@@ -56,10 +70,6 @@ object Management {
case class Index(name: String, propNames: Seq[String], direction: Option[Int] = None, options: Option[String] = None)
}
- import HBaseType._
-
- val LABEL_NAME_MAX_LENGTH = 100
- val DefaultCompressionAlgorithm = "gz"
def findService(serviceName: String) = {
Service.findByName(serviceName, useCache = false)
@@ -263,6 +273,24 @@ object Management {
Label.updateName(tempLabel, rightLabel)
}
}
+ def toConfig(params: Map[String, Any]): Config = {
+ import scala.collection.JavaConversions._
+
+ val filtered = params.filter { case (k, v) =>
+ v match {
+ case None => false
+ case _ => true
+ }
+ }.map { case (k, v) =>
+ val newV = v match {
+ case Some(value) => value
+ case _ => v
+ }
+ k -> newV
+ }
+
+ ConfigFactory.parseMap(filtered)
+ }
}
class Management(graph: S2Graph) {
@@ -277,7 +305,15 @@ class Management(graph: S2Graph) {
compressionAlgorithm: String = DefaultCompressionAlgorithm,
replicationScopeOpt: Option[Int] = None,
totalRegionCount: Option[Int] = None): Unit = {
- graph.defaultStorage.createTable(zkAddr, tableName, cfs, regionMultiplier, ttl, compressionAlgorithm, replicationScopeOpt, totalRegionCount)
+ val config = toConfig(Map(
+ ZookeeperQuorum -> zkAddr,
+// ColumnFamilies -> cfs,
+ RegionMultiplier -> regionMultiplier,
+ Ttl -> ttl,
+ CompressionAlgorithm -> compressionAlgorithm,
+ TotalRegionCount -> totalRegionCount
+ ))
+ graph.defaultStorage.createTable(config, tableName)
}
@@ -299,8 +335,15 @@ class Management(graph: S2Graph) {
Model withTx { implicit session =>
val service = Service.findOrInsert(serviceName, cluster, hTableName, preSplitSize, hTableTTL.orElse(Some(Integer.MAX_VALUE)), compressionAlgorithm, useCache = false)
+ val config = toConfig(Map(
+ ZookeeperQuorum -> service.cluster,
+// ColumnFamilies -> List("e", "v"),
+ RegionMultiplier -> service.preSplitSize,
+ Ttl -> service.hTableTTL,
+ CompressionAlgorithm -> compressionAlgorithm
+ ))
/* create hbase table for service */
- graph.getStorage(service).createTable(service.cluster, service.hTableName, List("e", "v"), service.preSplitSize, service.hTableTTL, compressionAlgorithm)
+ graph.getStorage(service).createTable(config, service.hTableName)
service
}
}
@@ -390,7 +433,14 @@ class Management(graph: S2Graph) {
/* create hbase table */
val storage = graph.getStorage(newLabel)
val service = newLabel.service
- storage.createTable(service.cluster, newLabel.hbaseTableName, List("e", "v"), service.preSplitSize, newLabel.hTableTTL, newLabel.compressionAlgorithm)
+ val config = toConfig(Map(
+ ZookeeperQuorum -> service.cluster,
+// ColumnFamilies -> List("e", "v"),
+ RegionMultiplier -> service.preSplitSize,
+ Ttl -> newLabel.hTableTTL,
+ CompressionAlgorithm -> newLabel.compressionAlgorithm
+ ))
+ storage.createTable(config, newLabel.hbaseTableName)
newLabel
}
@@ -449,7 +499,9 @@ class Management(graph: S2Graph) {
labelOpt.map { label =>
val storage = graph.getStorage(label)
val zkAddr = label.service.cluster
- storage.truncateTable(zkAddr, label.hbaseTableName)
+
+ val config = toConfig(Map(ZookeeperQuorum -> zkAddr))
+ storage.truncateTable(config, label.hbaseTableName)
}
}
}
@@ -459,7 +511,9 @@ class Management(graph: S2Graph) {
labelOpt.map { label =>
val storage = graph.getStorage(label)
val zkAddr = label.service.cluster
- storage.deleteTable(zkAddr, label.hbaseTableName)
+
+ val config = toConfig(Map(ZookeeperQuorum -> zkAddr))
+ storage.deleteTable(config, label.hbaseTableName)
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 7e95f58..1100f6c 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
@@ -26,7 +26,7 @@ import org.apache.s2graph.core.GraphExceptions.LabelNotExistException
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.storage.serde.StorageSerializable._
import org.apache.s2graph.core.types._
import org.apache.tinkerpop.gremlin.process.traversal.step.util.HasContainer
import org.hbase.async.ColumnRangeFilter
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 7165579..51af831 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Edge.scala
@@ -684,7 +684,7 @@ case class S2Edge(innerGraph: S2Graph,
// should we delete related edges also?
val future = innerGraph.mutateEdges(Seq(edgeToDelete), withWait = true)
val mutateSuccess = Await.result(future, innerGraph.WaitTimeout)
- if (!mutateSuccess.forall(identity)) throw new RuntimeException("edge remove failed.")
+ if (!mutateSuccess.forall(_.isSuccess)) throw new RuntimeException("edge remove failed.")
} else {
throw Edge.Exceptions.edgeRemovalNotSupported()
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 d1eda5e..32724b4 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
@@ -22,18 +22,17 @@ package org.apache.s2graph.core
import java.util
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
import java.util.concurrent.{Executors, TimeUnit}
-import java.util.function.Consumer
-
import com.typesafe.config.{Config, ConfigFactory}
import org.apache.commons.configuration.{BaseConfiguration, Configuration}
-import org.apache.s2graph.core.GraphExceptions.{FetchTimeoutException, LabelNotExistException}
+import org.apache.s2graph.core.GraphExceptions.LabelNotExistException
import org.apache.s2graph.core.JSONParser._
import org.apache.s2graph.core.features.S2GraphVariables
-import org.apache.s2graph.core.index.{IndexProvider, LuceneIndexProvider}
+import org.apache.s2graph.core.index.IndexProvider
import org.apache.s2graph.core.io.tinkerpop.optimize.S2GraphStepStrategy
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.storage.rocks.RocksStorage
+import org.apache.s2graph.core.storage.{ MutateResponse, 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
@@ -44,8 +43,6 @@ import org.apache.tinkerpop.gremlin.structure.Graph.{Features, Variables}
import org.apache.tinkerpop.gremlin.structure.io.{GraphReader, GraphWriter, Io, Mapper}
import org.apache.tinkerpop.gremlin.structure.{Edge, Element, Graph, T, Transaction, Vertex}
import play.api.libs.json.{JsObject, Json}
-import scalikejdbc.DBSession
-
import scala.annotation.tailrec
import scala.collection.JavaConversions._
import scala.collection.mutable
@@ -137,12 +134,13 @@ object S2Graph {
new S2Graph(configuration)(ec)
}
- def initStorage(graph: S2Graph, config: Config)(ec: ExecutionContext): Storage[_, _] = {
+ 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 "hbase" => new AsynchbaseStorage(graph, config)
+ case "rocks" => new RocksStorage(graph, config)
case _ => throw new RuntimeException("not supported storage.")
}
}
@@ -912,7 +910,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
/**
* TODO: we need to some way to handle malformed configuration for storage.
*/
- val storagePool: scala.collection.mutable.Map[String, Storage[_, _]] = {
+ val storagePool: scala.collection.mutable.Map[String, Storage[_]] = {
val labels = Label.findAll()
val services = Service.findAll()
@@ -923,12 +921,12 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
confWithFallback(conf)
}.toSet
- val pools = new java.util.HashMap[Config, Storage[_, _]]()
+ 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[_, _]]()
+ val m = new java.util.concurrent.ConcurrentHashMap[String, Storage[_]]()
labels.foreach { label =>
if (label.storageConfigOpt.isDefined) {
@@ -945,7 +943,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
m
}
- val defaultStorage: Storage[_, _] = S2Graph.initStorage(this, config)(ec)
+ val defaultStorage: Storage[_] = S2Graph.initStorage(this, config)(ec)
/** QueryLevel FutureCache */
val queryFutureCache = new DeferCache[StepResult, Promise, Future](parseCacheConfig(config, "query."), empty = StepResult.Empty)
@@ -957,11 +955,11 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
val indexProvider = IndexProvider.apply(config)
- def getStorage(service: Service): Storage[_, _] = {
+ def getStorage(service: Service): Storage[_] = {
storagePool.getOrElse(s"service:${service.serviceName}", defaultStorage)
}
- def getStorage(label: Label): Storage[_, _] = {
+ def getStorage(label: Label): Storage[_] = {
storagePool.getOrElse(s"label:${label.label}", defaultStorage)
}
@@ -979,8 +977,8 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
val futures = for {
edge <- edges
} yield {
- fetchSnapshotEdge(edge).map { case (queryParam, edgeOpt, kvOpt) =>
- edgeOpt.toSeq.map(e => EdgeWithScore(e, 1.0, queryParam.label))
+ getStorage(edge.innerLabel).fetchSnapshotEdgeInner(edge).map { case (_, edgeOpt, _) =>
+ edgeOpt.toSeq.map(e => EdgeWithScore(e, 1.0, edge.innerLabel))
}
}
@@ -1147,39 +1145,33 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
fallback
} get
}
+
+ def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
+ def getVertices[Q](storage: Storage[Q])(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
+ def fromResult(kvs: Seq[SKeyValue],
+ version: String): Option[S2Vertex] = {
+ if (kvs.isEmpty) None
+ else storage.vertexDeserializer(version).fromKeyValues(kvs, None)
+ // .map(S2Vertex(graph, _))
+ }
-
- 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)
+ val futures = vertices.map { vertex =>
+ val queryParam = QueryParam.Empty
+ val q = Query.toQuery(Seq(vertex), Seq(queryParam))
+ val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
+ val rpc = storage.buildRequest(queryRequest, vertex)
+ storage.fetchKeyValues(rpc).map { kvs =>
+ fromResult(kvs, vertex.serviceColumn.schemaVersion)
+ } recoverWith { case ex: Throwable =>
+ Future.successful(None)
}
- (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]] = {
+ Future.sequence(futures).map { result => result.toList.flatten }
+ }
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)))
+ getVertices(getStorage(service))(vertexGroup.map(_._1)).map(_.zip(vertexGroup.map(_._2)))
}
Future.sequence(futures).map { ls =>
@@ -1221,8 +1213,9 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
def fetchAndDeleteAll(queries: Seq[Query], requestTs: Long): Future[(Boolean, Boolean)] = {
+ val futures = queries.map(getEdgesStepInner(_, true))
val future = for {
- stepInnerResultLs <- Future.sequence(queries.map(getEdgesStepInner(_, true)))
+ stepInnerResultLs <- Future.sequence(futures)
(allDeleted, ret) <- deleteAllFetchedEdgesLs(stepInnerResultLs, requestTs)
} yield {
// logger.debug(s"fetchAndDeleteAll: ${allDeleted}, ${ret}")
@@ -1241,6 +1234,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
def deleteAllFetchedEdgesLs(stepInnerResultLs: Seq[StepResult],
requestTs: Long): Future[(Boolean, Boolean)] = {
stepInnerResultLs.foreach { stepInnerResult =>
+ logger.error(s"[!!!!!!]: ${stepInnerResult.edgeWithScores.size}")
if (stepInnerResult.isFailure) throw new RuntimeException("fetched result is fallback.")
}
val futures = for {
@@ -1257,9 +1251,9 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
* 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))
+ mutateEdges(deleteStepInnerResult.edgeWithScores.map(_.edge), withWait = true).map(_.forall(_.isSuccess))
} else {
- getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
+ deleteAllFetchedEdgesAsyncOld(getStorage(label))(deleteStepInnerResult, requestTs, MaxRetryNum)
}
case _ =>
@@ -1267,7 +1261,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
* read: x
* write: N x ((1(snapshotEdge) + 2(1 for incr, 1 for delete) x indices)
*/
- getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
+ deleteAllFetchedEdgesAsyncOld(getStorage(label))(deleteStepInnerResult, requestTs, MaxRetryNum)
}
ret
}
@@ -1280,6 +1274,44 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
}
+ private def deleteAllFetchedEdgesAsyncOld(storage: Storage[_])(stepInnerResult: StepResult,
+ requestTs: Long,
+ retryNum: Int): Future[Boolean] = {
+ if (stepInnerResult.isEmpty) Future.successful(true)
+ else {
+ val head = stepInnerResult.edgeWithScores.head
+ val zkQuorum = head.edge.innerLabel.hbaseZkAddr
+ val futures = for {
+ edgeWithScore <- stepInnerResult.edgeWithScores
+ } yield {
+ val edge = edgeWithScore.edge
+ val score = edgeWithScore.score
+
+ val edgeSnapshot = edge.copyEdge(propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()))
+ val reversedSnapshotEdgeMutations = storage.snapshotEdgeSerializer(edgeSnapshot.toSnapshotEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put))
+
+ val edgeForward = edge.copyEdge(propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()))
+ val forwardIndexedEdgeMutations = edgeForward.edgesWithIndex.flatMap { indexEdge =>
+ storage.indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete)) ++
+ storage.buildIncrementsAsync(indexEdge, -1L)
+ }
+
+ /* reverted direction */
+ val edgeRevert = edge.copyEdge(propsWithTs = S2Edge.propsToState(edge.updatePropsWithTs()))
+ val reversedIndexedEdgesMutations = edgeRevert.duplicateEdge.edgesWithIndex.flatMap { indexEdge =>
+ storage.indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete)) ++
+ storage.buildIncrementsAsync(indexEdge, -1L)
+ }
+
+ val mutations = reversedIndexedEdgesMutations ++ reversedSnapshotEdgeMutations ++ forwardIndexedEdgeMutations
+
+ storage.writeToStorage(zkQuorum, mutations, withWait = true)
+ }
+
+ Future.sequence(futures).map { rets => rets.forall(_.isSuccess) }
+ }
+ }
+
def buildEdgesToDelete(stepInnerResult: StepResult, requestTs: Long): StepResult = {
val filtered = stepInnerResult.edgeWithScores.filter { edgeWithScore =>
(edgeWithScore.edge.ts < requestTs) && !edgeWithScore.edge.isDegree
@@ -1297,20 +1329,6 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
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}")
@@ -1321,11 +1339,9 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
}
- // 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]] = {
+ withWait: Boolean = false): Future[Seq[MutateResponse]] = {
val edgeBuffer = ArrayBuffer[(S2Edge, Int)]()
val vertexBuffer = ArrayBuffer[(S2Vertex, Int)]()
@@ -1355,7 +1371,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
// 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]] = {
+ def mutateEdges(edges: Seq[S2Edge], withWait: Boolean = false): Future[Seq[MutateResponse]] = {
val edgeWithIdxs = edges.zipWithIndex
val (strongEdges, weakEdges) =
@@ -1383,7 +1399,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
storage.writeToStorage(zkQuorum, mutations, withWait).map { ret =>
- idxs.map(idx => idx -> ret)
+ idxs.map(idx => idx -> ret.isSuccess)
}
}
Future.sequence(futures)
@@ -1398,7 +1414,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
val edges = edgeGroup.map(_._1)
val idxs = edgeGroup.map(_._2)
val storage = getStorage(label)
- storage.mutateStrongEdges(edges, withWait = true).map { rets =>
+ mutateStrongEdges(storage)(edges, withWait = true).map { rets =>
idxs.zip(rets)
}
}
@@ -1408,27 +1424,130 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
deleteAll <- Future.sequence(deleteAllFutures)
strong <- Future.sequence(strongEdgesFutures)
} yield {
- (deleteAll ++ weak.flatten.flatten ++ strong.flatten).sortBy(_._1).map(_._2)
+ (deleteAll ++ weak.flatten.flatten ++ strong.flatten).sortBy(_._1).map(r => new MutateResponse(r._2))
}
}
- def mutateVertices(vertices: Seq[S2Vertex], withWait: Boolean = false): Future[Seq[Boolean]] = {
+ private def mutateStrongEdges(storage: Storage[_])(_edges: Seq[S2Edge], withWait: Boolean): Future[Seq[Boolean]] = {
+
+ val edgeWithIdxs = _edges.zipWithIndex
+ val grouped = edgeWithIdxs.groupBy { case (edge, idx) =>
+ (edge.innerLabel, edge.srcVertex.innerId, edge.tgtVertex.innerId)
+ } toSeq
+
+ val mutateEdges = grouped.map { case ((_, _, _), edgeGroup) =>
+ val edges = edgeGroup.map(_._1)
+ val idxs = edgeGroup.map(_._2)
+ // After deleteAll, process others
+ val mutateEdgeFutures = edges.toList match {
+ case head :: tail =>
+ val edgeFuture = mutateEdgesInner(storage)(edges, checkConsistency = true , withWait)
+
+ //TODO: decide what we will do on failure on vertex put
+ val puts = storage.buildVertexPutsAsync(head)
+ val vertexFuture = storage.writeToStorage(head.innerLabel.hbaseZkAddr, puts, withWait)
+ Seq(edgeFuture, vertexFuture)
+ case Nil => Nil
+ }
+
+ val composed = for {
+ // deleteRet <- Future.sequence(deleteAllFutures)
+ mutateRet <- Future.sequence(mutateEdgeFutures)
+ } yield mutateRet
+
+ composed.map(_.forall(_.isSuccess)).map { ret => idxs.map( idx => idx -> ret) }
+ }
+
+ Future.sequence(mutateEdges).map { squashedRets =>
+ squashedRets.flatten.sortBy { case (idx, ret) => idx }.map(_._2)
+ }
+ }
+
+
+ private def mutateEdgesInner(storage: Storage[_])(edges: Seq[S2Edge],
+ checkConsistency: Boolean,
+ withWait: Boolean): Future[MutateResponse] = {
+ assert(edges.nonEmpty)
+ // TODO:: remove after code review: unreachable code
+ if (!checkConsistency) {
+
+ val zkQuorum = edges.head.innerLabel.hbaseZkAddr
+ val futures = edges.map { edge =>
+ val (_, edgeUpdate) = S2Edge.buildOperation(None, Seq(edge))
+
+ val (bufferIncr, nonBufferIncr) = storage.increments(edgeUpdate.deepCopy)
+ val mutations =
+ storage.indexedEdgeMutations(edgeUpdate.deepCopy) ++ storage.snapshotEdgeMutations(edgeUpdate.deepCopy) ++ nonBufferIncr
+
+ if (bufferIncr.nonEmpty) storage.writeToStorage(zkQuorum, bufferIncr, withWait = false)
+
+ storage.writeToStorage(zkQuorum, mutations, withWait)
+ }
+ Future.sequence(futures).map { rets => new MutateResponse(rets.forall(_.isSuccess)) }
+ } else {
+ storage.fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+ storage.retry(1)(edges, 0, snapshotEdgeOpt).map(new MutateResponse(_))
+ }
+ }
+ }
+
+ def mutateVertices(vertices: Seq[S2Vertex], withWait: Boolean = false): Future[Seq[MutateResponse]] = {
+ def mutateVertex(storage: Storage[_])(vertex: S2Vertex, withWait: Boolean): Future[MutateResponse] = {
+ if (vertex.op == GraphUtil.operations("delete")) {
+ storage.writeToStorage(vertex.hbaseZkAddr,
+ storage.vertexSerializer(vertex).toKeyValues.map(_.copy(operation = SKeyValue.Delete)), withWait)
+ } else if (vertex.op == GraphUtil.operations("deleteAll")) {
+ logger.info(s"deleteAll for vertex is truncated. $vertex")
+ Future.successful(MutateResponse.Success) // Ignore withWait parameter, because deleteAll operation may takes long time
+ } else {
+ storage.writeToStorage(vertex.hbaseZkAddr, storage.buildPutsAll(vertex), withWait)
+ }
+ }
+
+ def mutateVertices(storage: Storage[_])(vertices: Seq[S2Vertex],
+ withWait: Boolean = false): Future[Seq[MutateResponse]] = {
+ val futures = vertices.map { vertex => mutateVertex(storage)(vertex, withWait) }
+ Future.sequence(futures)
+ }
+
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)))
+ mutateVertices(getStorage(service))(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)]] = {
+
+
+ def incrementCounts(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[MutateResponse]] = {
+ def incrementCounts(storage: Storage[_])(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[MutateResponse]] = {
+ val futures = for {
+ edge <- edges
+ } yield {
+ val kvs = for {
+ relEdge <- edge.relatedEdges
+ edgeWithIndex <- EdgeMutate.filterIndexOption(relEdge.edgesWithIndexValid)
+ } yield {
+ val countWithTs = edge.propertyValueInner(LabelMeta.count)
+ val countVal = countWithTs.innerVal.toString().toLong
+ storage.buildIncrementsCountAsync(edgeWithIndex, countVal).head
+ }
+ storage.writeToStorage(edge.innerLabel.hbaseZkAddr, kvs, withWait = withWait)
+ }
+
+ Future.sequence(futures)
+ }
+
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)))
+ incrementCounts(getStorage(label))(edgeGroup.map(_._1), withWait).map(_.zip(edgeGroup.map(_._2)))
+ }
+ Future.sequence(futures).map { ls =>
+ ls.flatten.toSeq.sortBy(_._2).map(_._1)
}
- Future.sequence(futures).map { ls => ls.flatten.toSeq.sortBy(_._2).map(_._1) }
}
- def updateDegree(edge: S2Edge, degreeVal: Long = 0): Future[Boolean] = {
+ def updateDegree(edge: S2Edge, degreeVal: Long = 0): Future[MutateResponse] = {
val label = edge.innerLabel
val storage = getStorage(label)
@@ -1840,7 +1959,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
val vertex = newVertex(id, ts, props, op, belongLabelIds)
val future = mutateVertices(Seq(vertex), withWait = true).map { rets =>
- if (rets.forall(identity)) vertex
+ if (rets.forall(_.isSuccess)) vertex
else throw new RuntimeException("addVertex failed.")
}
Await.ready(future, WaitTimeout)
@@ -1850,7 +1969,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
def addVertexInner(vertex: S2Vertex): S2Vertex = {
val future = mutateVertices(Seq(vertex), withWait = true).flatMap { rets =>
- if (rets.forall(identity)) {
+ if (rets.forall(_.isSuccess)) {
indexProvider.mutateVerticesAsync(Seq(vertex))
} else throw new RuntimeException("addVertex failed.")
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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
index c0dc23b..177d859 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Vertex.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Vertex.scala
@@ -169,6 +169,29 @@ case class S2Vertex(graph: S2Graph,
graph.fetchEdges(this, labelNameWithDirs.distinct)
}
+ private def edgesAsync(direction: Direction, labelNames: String*): Future[util.Iterator[Edge]] = {
+ val labelNameWithDirs =
+ if (labelNames.isEmpty) {
+ // TODO: Let's clarify direction
+ if (direction == Direction.BOTH) {
+ Label.findBySrcColumnId(id.colId).map(l => l.label -> Direction.OUT.name) ++
+ Label.findByTgtColumnId(id.colId).map(l => l.label -> Direction.IN.name)
+ } else if (direction == Direction.IN) {
+ Label.findByTgtColumnId(id.colId).map(l => l.label -> direction.name)
+ } else {
+ Label.findBySrcColumnId(id.colId).map(l => l.label -> direction.name)
+ }
+ } else {
+ direction match {
+ case Direction.BOTH =>
+ Seq(Direction.OUT, Direction.IN).flatMap { dir => labelNames.map(_ -> dir.name()) }
+ case _ => labelNames.map(_ -> direction.name())
+ }
+ }
+
+ graph.fetchEdgesAsync(this, labelNameWithDirs.distinct)
+ }
+
// do no save to storage
def propertyInner[V](cardinality: Cardinality, key: String, value: V, objects: AnyRef*): VertexProperty[V] = {
S2Property.assertValidProp(key, value)
@@ -242,21 +265,18 @@ case class S2Vertex(graph: S2Graph,
// remove edge
// TODO: remove related edges also.
implicit val ec = graph.ec
- val ts = System.currentTimeMillis()
- val outLabels = Label.findBySrcColumnId(id.colId)
- val inLabels = Label.findByTgtColumnId(id.colId)
+
val verticesToDelete = Seq(this.copy(op = GraphUtil.operations("delete")))
- val outFuture = graph.deleteAllAdjacentEdges(verticesToDelete, outLabels, GraphUtil.directions("out"), ts)
- val inFuture = graph.deleteAllAdjacentEdges(verticesToDelete, inLabels, GraphUtil.directions("in"), ts)
+
val vertexFuture = graph.mutateVertices(verticesToDelete, withWait = true)
+
val future = for {
- outSuccess <- outFuture
- inSuccess <- inFuture
vertexSuccess <- vertexFuture
+ edges <- edgesAsync(Direction.BOTH)
} yield {
- if (!outSuccess) throw new RuntimeException("Vertex.remove out direction edge delete failed.")
- if (!inSuccess) throw new RuntimeException("Vertex.remove in direction edge delete failed.")
- if (!vertexSuccess.forall(identity)) throw new RuntimeException("Vertex.remove vertex delete failed.")
+ edges.asScala.toSeq.foreach { edge => edge.remove() }
+ if (!vertexSuccess.forall(_.isSuccess)) throw new RuntimeException("Vertex.remove vertex delete failed.")
+
true
}
Await.result(future, graph.WaitTimeout)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/Deserializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/Deserializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/Deserializable.scala
deleted file mode 100644
index af20483..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/Deserializable.scala
+++ /dev/null
@@ -1,43 +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
-
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.types.{HBaseType, LabelWithDirection, SourceVertexId, VertexId}
-
-
-trait Deserializable[E] extends StorageDeserializable[E] {
- import StorageDeserializable._
-
- type RowKeyRaw = (VertexId, LabelWithDirection, Byte, Boolean, Int)
-
-// /** version 1 and version 2 share same code for parsing row key part */
-// def parseRow(kv: SKeyValue, version: String = HBaseType.DEFAULT_VERSION): RowKeyRaw = {
-// var pos = 0
-// val (srcVertexId, srcIdLen) = SourceVertexId.fromBytes(kv.row, pos, kv.row.length, version)
-// pos += srcIdLen
-// val labelWithDir = LabelWithDirection(Bytes.toInt(kv.row, pos, 4))
-// pos += 4
-// val (labelIdxSeq, isInverted) = bytesToLabelIndexSeqWithIsInverted(kv.row, pos)
-//
-// val rowLen = srcIdLen + 4 + 1
-// (srcVertexId, labelWithDir, labelIdxSeq, isInverted, rowLen)
-// }
-}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala
new file mode 100644
index 0000000..bed1152
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/MutateResponse.scala
@@ -0,0 +1,12 @@
+package org.apache.s2graph.core.storage
+
+object MutateResponse {
+ val Success = new MutateResponse(isSuccess = true)
+ val Failure = new MutateResponse(isSuccess = false)
+ val IncrementFailure = new IncrementResponse(isSuccess = false, -1, -1)
+ val IncrementSuccess = new IncrementResponse(isSuccess = true, -1, -1)
+}
+
+class MutateResponse(val isSuccess: Boolean)
+
+case class IncrementResponse(override val isSuccess: Boolean, afterCount: Long, beforeCount: Long) extends MutateResponse(isSuccess)
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
index db9a9da..924d9a3 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
@@ -26,6 +26,7 @@ import org.hbase.async.KeyValue
object SKeyValue {
val EdgeCf = "e".getBytes("UTF-8")
+ val VertexCf = "v".getBytes("UTF-8")
val Put = 1
val Delete = 2
val Increment = 3
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/Serializable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/Serializable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/Serializable.scala
deleted file mode 100644
index 6de0b30..0000000
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/Serializable.scala
+++ /dev/null
@@ -1,27 +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
-
-object Serializable {
- val vertexCf = "v".getBytes("UTF-8")
- val edgeCf = "e".getBytes("UTF-8")
-}
-
-trait Serializable[E] extends StorageSerializable[E]
[2/8] incubator-s2graph git commit: Separate interfaces from Storage.
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
new file mode 100644
index 0000000..eab5cab
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
@@ -0,0 +1,438 @@
+package org.apache.s2graph.core.storage
+
+import java.util.concurrent.{Executors, TimeUnit}
+
+import org.apache.s2graph.core.GraphExceptions.{FetchTimeoutException, NoStackException}
+import org.apache.s2graph.core._
+import org.apache.s2graph.core.utils.logger
+
+import scala.concurrent.{ExecutionContext, Future, Promise}
+import scala.util.Random
+
+class WriteWriteConflictResolver(graph: S2Graph,
+ serDe: StorageSerDe,
+ io: StorageIO,
+ mutator: StorageWritable,
+ fetcher: StorageReadable[_]) {
+
+ val BackoffTimeout = graph.BackoffTimeout
+ val MaxRetryNum = graph.MaxRetryNum
+ val MaxBackOff = graph.MaxBackOff
+ val FailProb = graph.FailProb
+ val LockExpireDuration = graph.LockExpireDuration
+ val MaxSize = graph.MaxSize
+ val ExpireAfterWrite = graph.ExpireAfterWrite
+ val ExpireAfterAccess = graph.ExpireAfterAccess
+
+ /** retry scheduler */
+ val scheduledThreadPool = Executors.newSingleThreadScheduledExecutor()
+
+ protected def exponentialBackOff(tryNum: Int) = {
+ // time slot is divided by 10 ms
+ val slot = 10
+ Random.nextInt(Math.min(BackoffTimeout, slot * Math.pow(2, tryNum)).toInt)
+ }
+
+ def retry(tryNum: Int)(edges: Seq[S2Edge], statusCode: Byte, fetchedSnapshotEdgeOpt: Option[S2Edge])(implicit ec: ExecutionContext): Future[Boolean] = {
+ if (tryNum >= MaxRetryNum) {
+ edges.foreach { edge =>
+ logger.error(s"commit failed after $MaxRetryNum\n${edge.toLogString}")
+ }
+
+ Future.successful(false)
+ } else {
+ val future = commitUpdate(edges, statusCode, fetchedSnapshotEdgeOpt)
+ future.onSuccess {
+ case success =>
+ logger.debug(s"Finished. [$tryNum]\n${edges.head.toLogString}\n")
+ }
+ future recoverWith {
+ case FetchTimeoutException(retryEdge) =>
+ logger.info(s"[Try: $tryNum], Fetch fail.\n${retryEdge}")
+ /* fetch failed. re-fetch should be done */
+ fetcher.fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+ retry(tryNum + 1)(edges, statusCode, snapshotEdgeOpt)
+ }
+
+ case PartialFailureException(retryEdge, failedStatusCode, faileReason) =>
+ val status = failedStatusCode match {
+ case 0 => "AcquireLock failed."
+ case 1 => "Mutation failed."
+ case 2 => "Increment failed."
+ case 3 => "ReleaseLock failed."
+ case 4 => "Unknown"
+ }
+ logger.info(s"[Try: $tryNum], [Status: $status] partial fail.\n${retryEdge.toLogString}\nFailReason: ${faileReason}")
+
+ /* retry logic */
+ val promise = Promise[Boolean]
+ val backOff = exponentialBackOff(tryNum)
+ scheduledThreadPool.schedule(new Runnable {
+ override def run(): Unit = {
+ val future = if (failedStatusCode == 0) {
+ // acquire Lock failed. other is mutating so this thead need to re-fetch snapshotEdge.
+ /* fetch failed. re-fetch should be done */
+ fetcher.fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+ retry(tryNum + 1)(edges, statusCode, snapshotEdgeOpt)
+ }
+ } else {
+ // partial failure occur while self locked and mutating.
+ // assert(fetchedSnapshotEdgeOpt.nonEmpty)
+ retry(tryNum + 1)(edges, failedStatusCode, fetchedSnapshotEdgeOpt)
+ }
+ promise.completeWith(future)
+ }
+
+ }, backOff, TimeUnit.MILLISECONDS)
+ promise.future
+
+ case ex: Exception =>
+ logger.error("Unknown exception", ex)
+ Future.successful(false)
+ }
+ }
+ }
+
+ protected def commitUpdate(edges: Seq[S2Edge],
+ statusCode: Byte,
+ fetchedSnapshotEdgeOpt: Option[S2Edge])(implicit ec: ExecutionContext): Future[Boolean] = {
+ // Future.failed(new PartialFailureException(edges.head, 0, "ahahah"))
+ assert(edges.nonEmpty)
+ // assert(statusCode == 0 || fetchedSnapshotEdgeOpt.isDefined)
+
+ statusCode match {
+ case 0 =>
+ fetchedSnapshotEdgeOpt match {
+ case None =>
+ /*
+ * no one has never mutated this SN.
+ * (squashedEdge, edgeMutate) = Edge.buildOperation(None, edges)
+ * pendingE = squashedEdge.copy(statusCode = 1, lockTs = now, version = squashedEdge.ts + 1)
+ * lock = (squashedEdge, pendingE)
+ * releaseLock = (edgeMutate.newSnapshotEdge, None)
+ */
+ val (squashedEdge, edgeMutate) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
+
+ assert(edgeMutate.newSnapshotEdge.isDefined)
+
+ val lockTs = Option(System.currentTimeMillis())
+ val pendingEdge = squashedEdge.copy(statusCode = 1, lockTs = lockTs, version = squashedEdge.ts + 1)
+ val lockSnapshotEdge = squashedEdge.toSnapshotEdge.copy(pendingEdgeOpt = Option(pendingEdge))
+ val releaseLockSnapshotEdge = edgeMutate.newSnapshotEdge.get.copy(statusCode = 0,
+ pendingEdgeOpt = None, version = lockSnapshotEdge.version + 1)
+
+ commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
+
+ case Some(snapshotEdge) =>
+ snapshotEdge.pendingEdgeOpt match {
+ case None =>
+ /*
+ * others finished commit on this SN. but there is no contention.
+ * (squashedEdge, edgeMutate) = Edge.buildOperation(snapshotEdgeOpt, edges)
+ * pendingE = squashedEdge.copy(statusCode = 1, lockTs = now, version = snapshotEdge.version + 1) ?
+ * lock = (snapshotEdge, pendingE)
+ * releaseLock = (edgeMutate.newSnapshotEdge, None)
+ */
+ 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)
+ } else {
+ val lockTs = Option(System.currentTimeMillis())
+ val pendingEdge = squashedEdge.copy(statusCode = 1, lockTs = lockTs, version = snapshotEdge.version + 1)
+ val lockSnapshotEdge = snapshotEdge.toSnapshotEdge.copy(pendingEdgeOpt = Option(pendingEdge))
+ val releaseLockSnapshotEdge = edgeMutate.newSnapshotEdge.get.copy(statusCode = 0,
+ pendingEdgeOpt = None, version = lockSnapshotEdge.version + 1)
+ commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
+ }
+ case Some(pendingEdge) =>
+ val isLockExpired = pendingEdge.lockTs.get + LockExpireDuration < System.currentTimeMillis()
+ if (isLockExpired) {
+ /*
+ * if pendingEdge.ts == snapshotEdge.ts =>
+ * (squashedEdge, edgeMutate) = Edge.buildOperation(None, Seq(pendingEdge))
+ * else =>
+ * (squashedEdge, edgeMutate) = Edge.buildOperation(snapshotEdgeOpt, Seq(pendingEdge))
+ * pendingE = squashedEdge.copy(statusCode = 1, lockTs = now, version = snapshotEdge.version + 1)
+ * lock = (snapshotEdge, pendingE)
+ * releaseLock = (edgeMutate.newSnapshotEdge, None)
+ */
+ logger.debug(s"${pendingEdge.toLogString} has been expired.")
+ val (squashedEdge, edgeMutate) =
+ 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)
+ val lockSnapshotEdge = snapshotEdge.toSnapshotEdge.copy(pendingEdgeOpt = Option(newPendingEdge))
+ val releaseLockSnapshotEdge = edgeMutate.newSnapshotEdge.get.copy(statusCode = 0,
+ pendingEdgeOpt = None, version = lockSnapshotEdge.version + 1)
+
+ commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
+ } else {
+ /*
+ * others finished commit on this SN and there is currently contention.
+ * this can't be proceed so retry from re-fetch.
+ * throw EX
+ */
+ val (squashedEdge, _) = S2Edge.buildOperation(fetchedSnapshotEdgeOpt, edges)
+ Future.failed(new PartialFailureException(squashedEdge, 0, s"others[${pendingEdge.ts}] is mutating. me[${squashedEdge.ts}]"))
+ }
+ }
+
+ }
+ case _ =>
+
+ /*
+ * statusCode > 0 which means self locked and there has been partial failure either on mutate, increment, releaseLock
+ */
+
+ /*
+ * this succeed to lock this SN. keep doing on commit process.
+ * if SN.isEmpty =>
+ * no one never succed to commit on this SN.
+ * this is first mutation try on this SN.
+ * (squashedEdge, edgeMutate) = Edge.buildOperation(SN, edges)
+ * else =>
+ * assert(SN.pengingEdgeOpt.isEmpty) no-fetch after acquire lock when self retrying.
+ * there has been success commit on this SN.
+ * (squashedEdge, edgeMutate) = Edge.buildOperation(SN, edges)
+ * releaseLock = (edgeMutate.newSnapshotEdge, None)
+ */
+ val _edges =
+ if (fetchedSnapshotEdgeOpt.isDefined && fetchedSnapshotEdgeOpt.get.pendingEdgeOpt.isDefined) fetchedSnapshotEdgeOpt.get.pendingEdgeOpt.get +: edges
+ else 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)
+ case Some(newSnapshotEdge) => newSnapshotEdge.copy(statusCode = 0, pendingEdgeOpt = None, version = newVersion)
+ }
+ // lockSnapshotEdge will be ignored.
+ commitProcess(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, releaseLockSnapshotEdge, releaseLockSnapshotEdge, edgeMutate)
+ }
+ }
+
+ /**
+ * orchestrate commit process.
+ * we separate into 4 step to avoid duplicating each step over and over.
+ *
+ * @param statusCode : current statusCode of this thread to process edges.
+ * @param squashedEdge : squashed(in memory) final edge from input edges on same snapshotEdge.
+ * @param fetchedSnapshotEdgeOpt : fetched snapshotEdge from storage before commit process begin.
+ * @param lockSnapshotEdge : lockEdge that hold necessary data to lock this snapshotEdge for this thread.
+ * @param releaseLockSnapshotEdge : releaseLockEdge that will remove lock by storing new final merged states
+ * all from current request edges and fetched snapshotEdge.
+ * @param edgeMutate : mutations for indexEdge and snapshotEdge.
+ * @return
+ */
+ protected def commitProcess(statusCode: Byte,
+ squashedEdge: S2Edge,
+ fetchedSnapshotEdgeOpt: Option[S2Edge],
+ lockSnapshotEdge: SnapshotEdge,
+ releaseLockSnapshotEdge: SnapshotEdge,
+ edgeMutate: EdgeMutate)(implicit ec: ExecutionContext): Future[Boolean] = {
+ for {
+ locked <- acquireLock(statusCode, squashedEdge, fetchedSnapshotEdgeOpt, lockSnapshotEdge)
+ mutated <- commitIndexEdgeMutations(locked, statusCode, squashedEdge, edgeMutate)
+ incremented <- commitIndexEdgeDegreeMutations(mutated, statusCode, squashedEdge, edgeMutate)
+ lockReleased <- releaseLock(incremented, statusCode, squashedEdge, releaseLockSnapshotEdge)
+ } yield lockReleased
+ }
+
+ 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")
+ logger.debug(msg)
+ }
+
+ protected def debug(ret: Boolean, phase: String, snapshotEdge: SnapshotEdge, edgeMutate: EdgeMutate) = {
+ val msg = Seq(s"[$ret] [$phase]", s"${snapshotEdge.toLogString()}",
+ s"${edgeMutate.toLogString}").mkString("\n")
+ logger.debug(msg)
+ }
+
+ /**
+ * try to acquire lock on storage for this given snapshotEdge(lockEdge).
+ *
+ * @param statusCode : current statusCode of this thread to process edges.
+ * @param squashedEdge : squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
+ * @param fetchedSnapshotEdgeOpt : fetched snapshot edge from storage.
+ * @param lockEdge : lockEdge to build RPC request(compareAndSet) into Storage.
+ * @return
+ */
+ protected def acquireLock(statusCode: Byte,
+ squashedEdge: S2Edge,
+ fetchedSnapshotEdgeOpt: Option[S2Edge],
+ lockEdge: SnapshotEdge)(implicit ec: ExecutionContext): Future[Boolean] = {
+ if (statusCode >= 1) {
+ logger.debug(s"skip acquireLock: [$statusCode]\n${squashedEdge.toLogString}")
+ Future.successful(true)
+ } else {
+ val p = Random.nextDouble()
+ if (p < FailProb) {
+ Future.failed(new PartialFailureException(squashedEdge, 0, s"$p"))
+ } else {
+ val lockEdgePut = serDe.snapshotEdgeSerializer(lockEdge).toKeyValues.head
+ val oldPut = fetchedSnapshotEdgeOpt.map(e => serDe.snapshotEdgeSerializer(e.toSnapshotEdge).toKeyValues.head)
+ mutator.writeLock(lockEdgePut, oldPut).recoverWith { case ex: Exception =>
+ logger.error(s"AcquireLock RPC Failed.")
+ throw new PartialFailureException(squashedEdge, 0, "AcquireLock RPC Failed")
+ }.map { ret =>
+ if (ret.isSuccess) {
+ val log = Seq(
+ "\n",
+ "=" * 50,
+ s"[Success]: acquireLock",
+ s"[RequestEdge]: ${squashedEdge.toLogString}",
+ s"[LockEdge]: ${lockEdge.toLogString()}",
+ s"[PendingEdge]: ${lockEdge.pendingEdgeOpt.map(_.toLogString).getOrElse("")}",
+ "=" * 50, "\n").mkString("\n")
+
+ logger.debug(log)
+ // debug(ret, "acquireLock", edge.toSnapshotEdge)
+ } else {
+ throw new PartialFailureException(squashedEdge, 0, "hbase fail.")
+ }
+ true
+ }
+ }
+ }
+ }
+
+
+ /**
+ * change this snapshot's state on storage from locked into committed by
+ * storing new merged states on storage. merge state come from releaseLockEdge.
+ * note that releaseLock return Future.failed on predicate failure.
+ *
+ * @param predicate : indicate if this releaseLock phase should be proceed or not.
+ * @param statusCode : releaseLock do not use statusCode, only for debug.
+ * @param squashedEdge : squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
+ * @param releaseLockEdge : final merged states if all process goes well.
+ * @return
+ */
+ protected def releaseLock(predicate: Boolean,
+ statusCode: Byte,
+ squashedEdge: S2Edge,
+ releaseLockEdge: SnapshotEdge)(implicit ec: ExecutionContext): Future[Boolean] = {
+ if (!predicate) {
+ Future.failed(new PartialFailureException(squashedEdge, 3, "predicate failed."))
+ } else {
+ val p = Random.nextDouble()
+ if (p < FailProb) Future.failed(new PartialFailureException(squashedEdge, 3, s"$p"))
+ else {
+ val releaseLockEdgePuts = serDe.snapshotEdgeSerializer(releaseLockEdge).toKeyValues
+ mutator.writeToStorage(squashedEdge.innerLabel.hbaseZkAddr, releaseLockEdgePuts, withWait = true).recoverWith {
+ case ex: Exception =>
+ logger.error(s"ReleaseLock RPC Failed.")
+ throw new PartialFailureException(squashedEdge, 3, "ReleaseLock RPC Failed")
+ }.map { ret =>
+ if (ret.isSuccess) {
+ debug(ret.isSuccess, "releaseLock", squashedEdge.toSnapshotEdge)
+ } else {
+ val msg = Seq("\nFATAL ERROR\n",
+ "=" * 50,
+ squashedEdge.toLogString,
+ releaseLockEdgePuts,
+ "=" * 50,
+ "\n"
+ )
+ logger.error(msg.mkString("\n"))
+ // error(ret, "releaseLock", edge.toSnapshotEdge)
+ throw new PartialFailureException(squashedEdge, 3, "hbase fail.")
+ }
+ true
+ }
+ }
+ }
+ }
+
+ /**
+ *
+ * @param predicate : indicate if this commitIndexEdgeMutations phase should be proceed or not.
+ * @param statusCode : current statusCode of this thread to process edges.
+ * @param squashedEdge : squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
+ * @param edgeMutate : actual collection of mutations. note that edgeMutate contains snapshotEdge mutations,
+ * but in here, we only use indexEdge's mutations.
+ * @return
+ */
+ protected def commitIndexEdgeMutations(predicate: Boolean,
+ statusCode: Byte,
+ squashedEdge: S2Edge,
+ edgeMutate: EdgeMutate)(implicit ec: ExecutionContext): Future[Boolean] = {
+ if (!predicate) Future.failed(new PartialFailureException(squashedEdge, 1, "predicate failed."))
+ else {
+ if (statusCode >= 2) {
+ logger.debug(s"skip mutate: [$statusCode]\n${squashedEdge.toLogString}")
+ Future.successful(true)
+ } else {
+ val p = Random.nextDouble()
+ if (p < FailProb) Future.failed(new PartialFailureException(squashedEdge, 1, s"$p"))
+ else
+ mutator.writeToStorage(squashedEdge.innerLabel.hbaseZkAddr, io.indexedEdgeMutations(edgeMutate), withWait = true).map { ret =>
+ if (ret.isSuccess) {
+ debug(ret.isSuccess, "mutate", squashedEdge.toSnapshotEdge, edgeMutate)
+ } else {
+ throw new PartialFailureException(squashedEdge, 1, "hbase fail.")
+ }
+ true
+ }
+ }
+ }
+ }
+
+ /**
+ *
+ * @param predicate : indicate if this commitIndexEdgeMutations phase should be proceed or not.
+ * @param statusCode : current statusCode of this thread to process edges.
+ * @param squashedEdge : squashed(in memory) final edge from input edges on same snapshotEdge. only for debug
+ * @param edgeMutate : actual collection of mutations. note that edgeMutate contains snapshotEdge mutations,
+ * but in here, we only use indexEdge's degree mutations.
+ * @return
+ */
+ protected def commitIndexEdgeDegreeMutations(predicate: Boolean,
+ statusCode: Byte,
+ squashedEdge: S2Edge,
+ edgeMutate: EdgeMutate)(implicit ec: ExecutionContext): Future[Boolean] = {
+
+ def _write(kvs: Seq[SKeyValue], withWait: Boolean): Future[Boolean] = {
+ mutator.writeToStorage(squashedEdge.innerLabel.hbaseZkAddr, kvs, withWait = withWait).map { ret =>
+ if (ret.isSuccess) {
+ debug(ret.isSuccess, "increment", squashedEdge.toSnapshotEdge, edgeMutate)
+ } else {
+ throw new PartialFailureException(squashedEdge, 2, "hbase fail.")
+ }
+ true
+ }
+ }
+
+ if (!predicate) Future.failed(new PartialFailureException(squashedEdge, 2, "predicate failed."))
+ if (statusCode >= 3) {
+ logger.debug(s"skip increment: [$statusCode]\n${squashedEdge.toLogString}")
+ Future.successful(true)
+ } else {
+ val p = Random.nextDouble()
+ if (p < FailProb) Future.failed(new PartialFailureException(squashedEdge, 2, s"$p"))
+ else {
+ val (bufferIncr, nonBufferIncr) = io.increments(edgeMutate.deepCopy)
+
+ if (bufferIncr.nonEmpty) _write(bufferIncr, withWait = false)
+ _write(nonBufferIncr, withWait = true)
+ }
+ }
+ }
+
+ /** end of methods for consistency */
+
+ 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")}",
+ s"newEdge: ${newEdge.toLogString}",
+ s"mutation: \n${edgeMutate.toLogString}",
+ "----------------------------------------------").mkString("\n")
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/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 4fb2240..54007d5 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
@@ -20,38 +20,18 @@
package org.apache.s2graph.core.storage.hbase
-
import java.util
-import java.util.Base64
-import java.util.concurrent.{ExecutorService, Executors, TimeUnit}
+import java.util.concurrent.{ExecutorService, Executors}
-import com.stumbleupon.async.{Callback, Deferred}
import com.typesafe.config.Config
import org.apache.commons.io.FileUtils
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.hbase.client.{Admin, ConnectionFactory, Durability}
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
-import org.apache.hadoop.hbase.regionserver.BloomType
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.hadoop.hbase.{HBaseConfiguration, HColumnDescriptor, HTableDescriptor, TableName}
-import org.apache.hadoop.security.UserGroupInformation
import org.apache.s2graph.core._
-import org.apache.s2graph.core.mysqls.{Label, LabelMeta, ServiceColumn}
import org.apache.s2graph.core.storage._
-import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage.{AsyncRPC, ScanWithRange}
-import org.apache.s2graph.core.types.{HBaseType, VertexId}
+import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage.AsyncRPC
import org.apache.s2graph.core.utils._
-import org.hbase.async.FilterList.Operator.MUST_PASS_ALL
import org.hbase.async._
-
+import org.apache.s2graph.core.storage.serde._
import scala.collection.JavaConversions._
-import scala.collection.mutable.ArrayBuffer
-import scala.concurrent._
-import scala.concurrent.duration.Duration
-import scala.util.Try
-import scala.util.control.NonFatal
-import scala.util.hashing.MurmurHash3
object AsynchbaseStorage {
@@ -92,6 +72,7 @@ object AsynchbaseStorage {
}
case class ScanWithRange(scan: Scanner, offset: Int, limit: Int)
+
type AsyncRPC = Either[GetRequest, ScanWithRange]
def initLocalHBase(config: Config,
@@ -159,734 +140,29 @@ object AsynchbaseStorage {
class AsynchbaseStorage(override val graph: S2Graph,
- override val config: Config)(implicit ec: ExecutionContext)
- extends Storage[AsyncRPC, Deferred[StepResult]](graph, config) {
-
- import Extensions.DeferOps
-
- val hbaseExecutor: ExecutorService =
- if (config.getString("hbase.zookeeper.quorum") == "localhost")
- AsynchbaseStorage.initLocalHBase(config)
- else
- null
-
- /**
- * Asynchbase client setup.
- * note that we need two client, one for bulk(withWait=false) and another for withWait=true
- */
- private val clientFlushInterval = config.getInt("hbase.rpcs.buffered_flush_interval").toString().toShort
-
- /**
- * since some runtime environment such as spark cluster has issue with guava version, that is used in Asynchbase.
- * to fix version conflict, make this as lazy val for clients that don't require hbase client.
- */
- lazy val client = AsynchbaseStorage.makeClient(config)
- lazy val clientWithFlush = AsynchbaseStorage.makeClient(config, "hbase.rpcs.buffered_flush_interval" -> "0")
- lazy val clients = Seq(client, clientWithFlush)
-
- private val emptyKeyValues = new util.ArrayList[KeyValue]()
- private val emptyKeyValuesLs = new util.ArrayList[util.ArrayList[KeyValue]]()
- private val emptyStepResult = new util.ArrayList[StepResult]()
-
- private def client(withWait: Boolean): HBaseClient = if (withWait) clientWithFlush else client
-
- import CanDefer._
-
- /** Future Cache to squash request */
- lazy private val futureCache = new DeferCache[StepResult, Deferred, Deferred](config, StepResult.Empty, "AsyncHbaseFutureCache", useMetric = true)
-
- /** Simple Vertex Cache */
- lazy private val vertexCache = new DeferCache[Seq[SKeyValue], Promise, Future](config, Seq.empty[SKeyValue])
-
- private val zkQuorum = config.getString("hbase.zookeeper.quorum")
- private val zkQuorumSlave =
- if (config.hasPath("hbase.slave.zookeeper.quorum")) Option(config.getString("hbase.slave.zookeeper.quorum"))
- else None
-
- /** v4 max next row size */
- private val v4_max_num_rows = 10000
- private def getV4MaxNumRows(limit : Int): Int = {
- if (limit < v4_max_num_rows) limit
- else v4_max_num_rows
- }
-
- /**
- * fire rpcs into proper hbase cluster using client and
- * return true on all mutation success. otherwise return false.
- */
- override def writeToStorage(cluster: String, kvs: Seq[SKeyValue], withWait: Boolean): Future[Boolean] = {
- if (kvs.isEmpty) Future.successful(true)
- else {
- val _client = client(withWait)
- val (increments, putAndDeletes) = kvs.partition(_.operation == SKeyValue.Increment)
-
- /* Asynchbase IncrementRequest does not implement HasQualifiers */
- val incrementsFutures = increments.map { kv =>
- val inc = new AtomicIncrementRequest(kv.table, kv.row, kv.cf, kv.qualifier, Bytes.toLong(kv.value))
- val defer = _client.atomicIncrement(inc)
- val future = defer.toFuture(Long.box(0)).map(_ => true).recover { case ex: Exception =>
- logger.error(s"mutation failed. $kv", ex)
- false
- }
- if (withWait) future else Future.successful(true)
- }
-
- /* PutRequest and DeleteRequest accept byte[][] qualifiers/values. */
- val othersFutures = putAndDeletes.groupBy { kv =>
- (kv.table.toSeq, kv.row.toSeq, kv.cf.toSeq, kv.operation, kv.timestamp)
- }.map { case ((table, row, cf, operation, timestamp), groupedKeyValues) =>
-
- val durability = groupedKeyValues.head.durability
- val qualifiers = new ArrayBuffer[Array[Byte]]()
- val values = new ArrayBuffer[Array[Byte]]()
-
- groupedKeyValues.foreach { kv =>
- if (kv.qualifier != null) qualifiers += kv.qualifier
- if (kv.value != null) values += kv.value
- }
- val defer = operation match {
- case SKeyValue.Put =>
- val put = new PutRequest(table.toArray, row.toArray, cf.toArray, qualifiers.toArray, values.toArray, timestamp)
- put.setDurable(durability)
- _client.put(put)
- case SKeyValue.Delete =>
- val delete =
- if (qualifiers.isEmpty)
- new DeleteRequest(table.toArray, row.toArray, cf.toArray, timestamp)
- else
- new DeleteRequest(table.toArray, row.toArray, cf.toArray, qualifiers.toArray, timestamp)
- delete.setDurable(durability)
- _client.delete(delete)
- }
- if (withWait) {
- defer.toFuture(new AnyRef()).map(_ => true).recover { case ex: Exception =>
- groupedKeyValues.foreach { kv => logger.error(s"mutation failed. $kv", ex) }
- false
- }
- } else Future.successful(true)
- }
- for {
- incrementRets <- Future.sequence(incrementsFutures)
- otherRets <- Future.sequence(othersFutures)
- } yield (incrementRets ++ otherRets).forall(identity)
- }
- }
-
- private def fetchKeyValues(rpc: AsyncRPC): Future[Seq[SKeyValue]] = {
- val defer = fetchKeyValuesInner(rpc)
- defer.toFuture(emptyKeyValues).map { kvsArr =>
- kvsArr.map { kv =>
- implicitly[CanSKeyValue[KeyValue]].toSKeyValue(kv)
- }
- }
- }
-
- override def fetchSnapshotEdgeKeyValues(queryRequest: QueryRequest): Future[Seq[SKeyValue]] = {
- val edge = toRequestEdge(queryRequest, Nil)
- val rpc = buildRequest(queryRequest, edge)
-
- fetchKeyValues(rpc)
- }
-
- /**
- * since HBase natively provide CheckAndSet on storage level, implementation becomes simple.
- * @param rpc: key value that is need to be stored on storage.
- * @param expectedOpt: last valid value for rpc's KeyValue.value from fetching.
- * @return return true if expected value matches and our rpc is successfully applied, otherwise false.
- * note that when some other thread modified same cell and have different value on this KeyValue,
- * then HBase atomically return false.
- */
- override def writeLock(rpc: SKeyValue, expectedOpt: Option[SKeyValue]): Future[Boolean] = {
- val put = new PutRequest(rpc.table, rpc.row, rpc.cf, rpc.qualifier, rpc.value, rpc.timestamp)
- val expected = expectedOpt.map(_.value).getOrElse(Array.empty)
- client(withWait = true).compareAndSet(put, expected).map(true.booleanValue())(ret => ret.booleanValue()).toFuture(true)
- }
-
-
- /**
- * given queryRequest, build storage specific RPC Request.
- * In HBase case, we either build Scanner or GetRequest.
- *
- * IndexEdge layer:
- * Tall schema(v4): use scanner.
- * Wide schema(label's schema version in v1, v2, v3): use GetRequest with columnRangeFilter
- * when query is given with itnerval option.
- * SnapshotEdge layer:
- * Tall schema(v3, v4): use GetRequest without column filter.
- * Wide schema(label's schema version in v1, v2): use GetRequest with columnRangeFilter.
- * Vertex layer:
- * all version: use GetRequest without column filter.
- * @param queryRequest
- * @return Scanner or GetRequest with proper setup with StartKey, EndKey, RangeFilter.
- */
- override def buildRequest(queryRequest: QueryRequest, edge: S2Edge): AsyncRPC = {
- import Serializable._
- val queryParam = queryRequest.queryParam
- val label = queryParam.label
-
- val serializer = if (queryParam.tgtVertexInnerIdOpt.isDefined) {
- val snapshotEdge = edge.toSnapshotEdge
- snapshotEdgeSerializer(snapshotEdge)
- } else {
- val indexEdge = edge.toIndexEdge(queryParam.labelOrderSeq)
- indexEdgeSerializer(indexEdge)
- }
-
- val rowKey = serializer.toRowKey
- val (minTs, maxTs) = queryParam.durationOpt.getOrElse((0L, Long.MaxValue))
-
- val (intervalMaxBytes, intervalMinBytes) = queryParam.buildInterval(Option(edge))
-
- label.schemaVersion match {
- case HBaseType.VERSION4 if queryParam.tgtVertexInnerIdOpt.isEmpty =>
- val scanner = AsynchbasePatcher.newScanner(client, label.hbaseTableName)
- scanner.setFamily(edgeCf)
-
- /*
- * TODO: remove this part.
- */
- val indexEdgeOpt = edge.edgesWithIndex.find(edgeWithIndex => edgeWithIndex.labelIndex.seq == queryParam.labelOrderSeq)
- val indexEdge = indexEdgeOpt.getOrElse(throw new RuntimeException(s"Can`t find index for query $queryParam"))
-
- val srcIdBytes = VertexId.toSourceVertexId(indexEdge.srcVertex.id).bytes
- val labelWithDirBytes = indexEdge.labelWithDir.bytes
- val labelIndexSeqWithIsInvertedBytes = StorageSerializable.labelOrderSeqWithIsInverted(indexEdge.labelIndexSeq, isInverted = false)
- val baseKey = Bytes.add(srcIdBytes, labelWithDirBytes, labelIndexSeqWithIsInvertedBytes)
-
- val (startKey, stopKey) =
- if (queryParam.intervalOpt.isDefined) {
- // interval is set.
- val _startKey = queryParam.cursorOpt match {
- case Some(cursor) => Base64.getDecoder.decode(cursor)
- case None => Bytes.add(baseKey, intervalMaxBytes)
- }
- (_startKey , Bytes.add(baseKey, intervalMinBytes))
- } else {
- /*
- * note: since propsToBytes encode size of property map at first byte, we are sure about max value here
- */
- val _startKey = queryParam.cursorOpt match {
- case Some(cursor) => Base64.getDecoder.decode(cursor)
- case None => baseKey
- }
- (_startKey, Bytes.add(baseKey, Array.fill(1)(-1)))
- }
-
- scanner.setStartKey(startKey)
- scanner.setStopKey(stopKey)
-
- if (queryParam.limit == Int.MinValue) logger.debug(s"MinValue: $queryParam")
-
- scanner.setMaxVersions(1)
- // TODO: exclusive condition innerOffset with cursorOpt
- if (queryParam.cursorOpt.isDefined) {
- scanner.setMaxNumRows(getV4MaxNumRows(queryParam.limit))
- } else {
- scanner.setMaxNumRows(getV4MaxNumRows(queryParam.innerOffset + queryParam.innerLimit))
- }
- scanner.setMaxTimestamp(maxTs)
- scanner.setMinTimestamp(minTs)
- scanner.setRpcTimeout(queryParam.rpcTimeout)
-
- // SET option for this rpc properly.
- if (queryParam.cursorOpt.isDefined) Right(ScanWithRange(scanner, 0, queryParam.limit))
- else Right(ScanWithRange(scanner, 0, queryParam.innerOffset + queryParam.innerLimit))
-
- case _ =>
- val get = if (queryParam.tgtVertexInnerIdOpt.isDefined) {
- new GetRequest(label.hbaseTableName.getBytes, rowKey, edgeCf, serializer.toQualifier)
- } else {
- new GetRequest(label.hbaseTableName.getBytes, rowKey, edgeCf)
- }
-
- get.maxVersions(1)
- get.setFailfast(true)
- get.setMinTimestamp(minTs)
- get.setMaxTimestamp(maxTs)
- get.setTimeout(queryParam.rpcTimeout)
-
- val pagination = new ColumnPaginationFilter(queryParam.limit, queryParam.offset)
- val columnRangeFilterOpt = queryParam.intervalOpt.map { interval =>
- new ColumnRangeFilter(intervalMaxBytes, true, intervalMinBytes, true)
- }
- get.setFilter(new FilterList(pagination +: columnRangeFilterOpt.toSeq, MUST_PASS_ALL))
- Left(get)
- }
- }
+ override val config: Config) extends Storage[AsyncRPC](graph, config) {
/**
- * we are using future cache to squash requests into same key on storage.
- *
- * @param queryRequest
- * @param isInnerCall
- * @param parentEdges
- * @return we use Deferred here since it has much better performrance compared to scala.concurrent.Future.
- * seems like map, flatMap on scala.concurrent.Future is slower than Deferred's addCallback
- */
- override def fetch(queryRequest: QueryRequest,
- isInnerCall: Boolean,
- parentEdges: Seq[EdgeWithScore]): Deferred[StepResult] = {
-
- def fetchInner(hbaseRpc: AsyncRPC): Deferred[StepResult] = {
- val prevStepScore = queryRequest.prevStepScore
- val fallbackFn: (Exception => StepResult) = { ex =>
- logger.error(s"fetchInner failed. fallback return. $hbaseRpc}", ex)
- StepResult.Failure
- }
-
- val queryParam = queryRequest.queryParam
- fetchKeyValuesInner(hbaseRpc).mapWithFallback(emptyKeyValues)(fallbackFn) { kvs =>
- val (startOffset, len) = queryParam.label.schemaVersion match {
- case HBaseType.VERSION4 =>
- val offset = if (queryParam.cursorOpt.isDefined) 0 else queryParam.offset
- (offset, queryParam.limit)
- case _ => (0, kvs.length)
- }
-
- toEdges(kvs, queryRequest, prevStepScore, isInnerCall, parentEdges, startOffset, len)
- }
- }
-
- val queryParam = queryRequest.queryParam
- val cacheTTL = queryParam.cacheTTLInMillis
- /* with version 4, request's type is (Scanner, (Int, Int)). otherwise GetRequest. */
+ * since some runtime environment such as spark cluster has issue with guava version, that is used in Asynchbase.
+ * to fix version conflict, make this as lazy val for clients that don't require hbase client.
+ */
+ val client = AsynchbaseStorage.makeClient(config)
+ val clientWithFlush = AsynchbaseStorage.makeClient(config, "hbase.rpcs.buffered_flush_interval" -> "0")
+ val clients = Seq(client, clientWithFlush)
- val edge = toRequestEdge(queryRequest, parentEdges)
- val request = buildRequest(queryRequest, edge)
+ override val management: StorageManagement = new AsynchbaseStorageManagement(config, clients)
- val (intervalMaxBytes, intervalMinBytes) = queryParam.buildInterval(Option(edge))
- val requestCacheKey = Bytes.add(toCacheKeyBytes(request), intervalMaxBytes, intervalMinBytes)
+ override val mutator: StorageWritable = new AsynchbaseStorageWritable(client, clientWithFlush)
- if (cacheTTL <= 0) fetchInner(request)
- else {
- val cacheKeyBytes = Bytes.add(queryRequest.query.queryOption.cacheKeyBytes, requestCacheKey)
+ override val serDe: StorageSerDe = new AsynchbaseStorageSerDe(graph)
-// val cacheKeyBytes = toCacheKeyBytes(request)
- val cacheKey = queryParam.toCacheKey(cacheKeyBytes)
- futureCache.getOrElseUpdate(cacheKey, cacheTTL)(fetchInner(request))
- }
- }
-
- override def fetches(queryRequests: Seq[QueryRequest],
- prevStepEdges: Map[VertexId, Seq[EdgeWithScore]]): Future[Seq[StepResult]] = {
- val defers: Seq[Deferred[StepResult]] = for {
- queryRequest <- queryRequests
- } yield {
- val queryOption = queryRequest.query.queryOption
- val queryParam = queryRequest.queryParam
- val shouldBuildParents = queryOption.returnTree || queryParam.whereHasParent
- val parentEdges = if (shouldBuildParents) prevStepEdges.getOrElse(queryRequest.vertex.id, Nil) else Nil
- fetch(queryRequest, isInnerCall = false, parentEdges)
- }
-
- val grouped: Deferred[util.ArrayList[StepResult]] = Deferred.groupInOrder(defers)
- grouped.map(emptyStepResult) { queryResults: util.ArrayList[StepResult] =>
- queryResults.toSeq
- }.toFuture(emptyStepResult)
- }
-
-
- def fetchVertexKeyValues(request: QueryRequest): Future[Seq[SKeyValue]] = {
- val edge = toRequestEdge(request, Nil)
- fetchKeyValues(buildRequest(request, edge))
- }
-
-
- def fetchVertexKeyValues(request: AsyncRPC): Future[Seq[SKeyValue]] = fetchKeyValues(request)
+ override val fetcher: StorageReadable[AsyncRPC] = new AsynchbaseStorageReadable(graph, config, client, serDe, io)
- /**
- * when withWait is given, we use client with flushInterval set to 0.
- * if we are not using this, then we are adding extra wait time as much as flushInterval in worst case.
- *
- * @param edges
- * @param withWait
- * @return
- */
- 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 {
- edge <- edges
- } yield {
- val futures: List[Deferred[(Boolean, Long, Long)]] = for {
- relEdge <- edge.relatedEdges
- edgeWithIndex <- EdgeMutate.filterIndexOption(relEdge.edgesWithIndexValid)
- } yield {
- val countWithTs = edge.propertyValueInner(LabelMeta.count)
- val countVal = countWithTs.innerVal.toString().toLong
- val kv = buildIncrementsCountAsync(edgeWithIndex, countVal).head
- val request = new AtomicIncrementRequest(kv.table, kv.row, kv.cf, kv.qualifier, Bytes.toLong(kv.value))
- val fallbackFn: (Exception => (Boolean, Long, Long)) = { ex =>
- logger.error(s"mutation failed. $request", ex)
- (false, -1L, -1L)
- }
- val defer = _client.bufferAtomicIncrement(request).mapWithFallback(0L)(fallbackFn) { resultCount: java.lang.Long =>
- (true, resultCount.longValue(), countVal)
- }
- if (withWait) defer
- else Deferred.fromResult((true, -1L, -1L))
- }
-
- val grouped: Deferred[util.ArrayList[(Boolean, Long, Long)]] = Deferred.group(futures)
- grouped.map(new util.ArrayList[(Boolean, Long, Long)]()) { resultLs => resultLs.head }
- }
-
- val grouped: Deferred[util.ArrayList[(Boolean, Long, Long)]] = Deferred.groupInOrder(defers)
- grouped.toFuture(new util.ArrayList[(Boolean, Long, Long)]()).map(_.toSeq)
- }
-
-
- override def flush(): Unit = clients.foreach { client =>
- super.flush()
- val timeout = Duration((clientFlushInterval + 10) * 20, duration.MILLISECONDS)
- Await.result(client.flush().toFuture(new AnyRef), timeout)
- }
+ // val hbaseExecutor: ExecutorService =
+ // if (config.getString("hbase.zookeeper.quorum") == "localhost")
+ // AsynchbaseStorage.initLocalHBase(config)
+ // else
+ // null
- override def shutdown(): Unit = {
- flush()
- clients.foreach { client =>
- AsynchbaseStorage.shutdown(client)
- }
- if (hbaseExecutor != null) {
- hbaseExecutor.shutdown()
- hbaseExecutor.awaitTermination(1, TimeUnit.MINUTES)
- }
- }
-
- override def createTable(_zkAddr: String,
- tableName: String,
- cfs: List[String],
- regionMultiplier: Int,
- ttl: Option[Int],
- compressionAlgorithm: String,
- replicationScopeOpt: Option[Int] = None,
- totalRegionCount: Option[Int] = None): Unit = {
- /* TODO: Decide if we will allow each app server to connect to multiple hbase cluster */
- for {
- zkAddr <- Seq(zkQuorum) ++ zkQuorumSlave.toSeq
- } {
- logger.info(s"create table: $tableName on $zkAddr, $cfs, $regionMultiplier, $compressionAlgorithm")
- withAdmin(zkAddr) { admin =>
- val regionCount = totalRegionCount.getOrElse(admin.getClusterStatus.getServersSize * regionMultiplier)
- try {
- if (!admin.tableExists(TableName.valueOf(tableName))) {
- val desc = new HTableDescriptor(TableName.valueOf(tableName))
- desc.setDurability(Durability.ASYNC_WAL)
- for (cf <- cfs) {
- val columnDesc = new HColumnDescriptor(cf)
- .setCompressionType(Algorithm.valueOf(compressionAlgorithm.toUpperCase))
- .setBloomFilterType(BloomType.ROW)
- .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)
- .setMaxVersions(1)
- .setTimeToLive(2147483647)
- .setMinVersions(0)
- .setBlocksize(32768)
- .setBlockCacheEnabled(true)
- // FIXME: For test!!
- .setInMemory(true)
- if (ttl.isDefined) columnDesc.setTimeToLive(ttl.get)
- if (replicationScopeOpt.isDefined) columnDesc.setScope(replicationScopeOpt.get)
- desc.addFamily(columnDesc)
- }
-
- if (regionCount <= 1) admin.createTable(desc)
- else admin.createTable(desc, getStartKey(regionCount), getEndKey(regionCount), regionCount)
- } else {
- logger.info(s"$zkAddr, $tableName, $cfs already exist.")
- }
- } catch {
- case e: Throwable =>
- logger.error(s"$zkAddr, $tableName failed with $e", e)
- throw e
- }
- }
- }
- }
-
- override def truncateTable(zkAddr: String, tableNameStr: String): Unit = {
- withAdmin(zkAddr) { admin =>
- val tableName = TableName.valueOf(tableNameStr)
- if (!Try(admin.tableExists(tableName)).getOrElse(false)) {
- logger.info(s"No table to truncate ${tableNameStr}")
- return
- }
-
- Try(admin.isTableDisabled(tableName)).map {
- case true =>
- logger.info(s"${tableNameStr} is already disabled.")
-
- case false =>
- logger.info(s"Before disabling to trucate ${tableNameStr}")
- Try(admin.disableTable(tableName)).recover {
- case NonFatal(e) =>
- logger.info(s"Failed to disable ${tableNameStr}: ${e}")
- }
- logger.info(s"After disabling to trucate ${tableNameStr}")
- }
-
- logger.info(s"Before truncating ${tableNameStr}")
- Try(admin.truncateTable(tableName, true)).recover {
- case NonFatal(e) =>
- logger.info(s"Failed to truncate ${tableNameStr}: ${e}")
- }
- logger.info(s"After truncating ${tableNameStr}")
- Try(admin.close()).recover {
- case NonFatal(e) =>
- logger.info(s"Failed to close admin ${tableNameStr}: ${e}")
- }
- Try(admin.getConnection.close()).recover {
- case NonFatal(e) =>
- logger.info(s"Failed to close connection ${tableNameStr}: ${e}")
- }
- }
- }
-
- override def deleteTable(zkAddr: String, tableNameStr: String): Unit = {
- withAdmin(zkAddr) { admin =>
- val tableName = TableName.valueOf(tableNameStr)
- if (!admin.tableExists(tableName)) {
- return
- }
- if (admin.isTableEnabled(tableName)) {
- admin.disableTable(tableName)
- }
- admin.deleteTable(tableName)
- }
- }
-
- /** Asynchbase implementation override default getVertices to use future Cache */
- override def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
- def fromResult(kvs: Seq[SKeyValue],
- version: String): Option[S2Vertex] = {
- if (kvs.isEmpty) None
- else vertexDeserializer.fromKeyValues(kvs, None)
- }
-
- val futures = vertices.map { vertex =>
- val kvs = vertexSerializer(vertex).toKeyValues
- val get = new GetRequest(vertex.hbaseTableName.getBytes, kvs.head.row, Serializable.vertexCf)
- // get.setTimeout(this.singleGetTimeout.toShort)
- get.setFailfast(true)
- get.maxVersions(1)
-
- fetchVertexKeyValues(Left(get)).map { kvs =>
- fromResult(kvs, vertex.serviceColumn.schemaVersion)
- }
-// val cacheKey = MurmurHash3.stringHash(get.toString)
-// vertexCache.getOrElseUpdate(cacheKey, cacheTTL = -1)(fetchVertexKeyValues(Left(get))).map { kvs =>
-// fromResult(kvs, vertex.serviceColumn.schemaVersion)
-// }
- }
-
- Future.sequence(futures).map { result => result.toList.flatten }
- }
-
- //TODO: Limited to 100000 edges per hbase table. fix this later.
- override def fetchEdgesAll(): Future[Seq[S2Edge]] = {
- val futures = Label.findAll().groupBy(_.hbaseTableName).toSeq.map { case (hTableName, labels) =>
- val distinctLabels = labels.toSet
- val scan = AsynchbasePatcher.newScanner(client, hTableName)
- scan.setFamily(Serializable.edgeCf)
- scan.setMaxVersions(1)
-
- scan.nextRows(S2Graph.FetchAllLimit).toFuture(emptyKeyValuesLs).map {
- case null => Seq.empty
- case kvsLs =>
- kvsLs.flatMap { kvs =>
- kvs.flatMap { kv =>
- val sKV = implicitly[CanSKeyValue[KeyValue]].toSKeyValue(kv)
-
- indexEdgeDeserializer.fromKeyValues(Seq(kv), None)
- .filter(e => distinctLabels(e.innerLabel) && e.direction == "out" && !e.isDegree)
- }
- }
- }
- }
-
- Future.sequence(futures).map(_.flatten)
- }
-
- override def fetchVerticesAll(): Future[Seq[S2Vertex]] = {
- val futures = ServiceColumn.findAll().groupBy(_.service.hTableName).toSeq.map { case (hTableName, columns) =>
- val distinctColumns = columns.toSet
- val scan = AsynchbasePatcher.newScanner(client, hTableName)
- scan.setFamily(Serializable.vertexCf)
- scan.setMaxVersions(1)
-
- scan.nextRows(S2Graph.FetchAllLimit).toFuture(emptyKeyValuesLs).map {
- case null => Seq.empty
- case kvsLs =>
- kvsLs.flatMap { kvs =>
- vertexDeserializer.fromKeyValues(kvs, None)
- .filter(v => distinctColumns(v.serviceColumn))
- }
- }
- }
- Future.sequence(futures).map(_.flatten)
- }
-
- class V4ResultHandler(scanner: Scanner, defer: Deferred[util.ArrayList[KeyValue]], offset: Int, limit : Int) extends Callback[Object, util.ArrayList[util.ArrayList[KeyValue]]] {
- val results = new util.ArrayList[KeyValue]()
- var offsetCount = 0
-
- override def call(kvsLs: util.ArrayList[util.ArrayList[KeyValue]]): Object = {
- try {
- if (kvsLs == null) {
- defer.callback(results)
- Try(scanner.close())
- } else {
- val curRet = new util.ArrayList[KeyValue]()
- kvsLs.foreach(curRet.addAll(_))
- val prevOffset = offsetCount
- offsetCount += curRet.size()
-
- val nextRet = if(offsetCount > offset){
- if(prevOffset < offset ) {
- curRet.subList(offset - prevOffset, curRet.size())
- } else{
- curRet
- }
- } else{
- emptyKeyValues
- }
-
- val needCount = limit - results.size()
- if (needCount >= nextRet.size()) {
- results.addAll(nextRet)
- } else {
- results.addAll(nextRet.subList(0, needCount))
- }
-
- if (results.size() < limit) {
- scanner.nextRows().addCallback(this)
- } else {
- defer.callback(results)
- Try(scanner.close())
- }
- }
- } catch{
- case ex: Exception =>
- logger.error(s"fetchKeyValuesInner failed.", ex)
- defer.callback(ex)
- Try(scanner.close())
- }
- }
- }
-
- /**
- * Private Methods which is specific to Asynchbase implementation.
- */
- private def fetchKeyValuesInner(rpc: AsyncRPC): Deferred[util.ArrayList[KeyValue]] = {
- rpc match {
- case Left(get) => client.get(get)
- case Right(ScanWithRange(scanner, offset, limit)) =>
- val deferred = new Deferred[util.ArrayList[KeyValue]]()
- scanner.nextRows().addCallback(new V4ResultHandler(scanner, deferred, offset, limit))
- deferred
- case _ => Deferred.fromError(new RuntimeException(s"fetchKeyValues failed. $rpc"))
- }
- }
-
- private def toCacheKeyBytes(hbaseRpc: AsyncRPC): Array[Byte] = {
- /* with version 4, request's type is (Scanner, (Int, Int)). otherwise GetRequest. */
- hbaseRpc match {
- case Left(getRequest) => getRequest.key
- case Right(ScanWithRange(scanner, offset, limit)) =>
- Bytes.add(scanner.getCurrentKey, Bytes.add(Bytes.toBytes(offset), Bytes.toBytes(limit)))
- case _ =>
- logger.error(s"toCacheKeyBytes failed. not supported class type. $hbaseRpc")
- throw new RuntimeException(s"toCacheKeyBytes: $hbaseRpc")
- }
- }
-
- private def getSecureClusterAdmin(zkAddr: String) = {
- val jaas = config.getString("java.security.auth.login.config")
- val krb5Conf = config.getString("java.security.krb5.conf")
- val realm = config.getString("realm")
- val principal = config.getString("principal")
- val keytab = config.getString("keytab")
-
- System.setProperty("java.security.auth.login.config", jaas)
- System.setProperty("java.security.krb5.conf", krb5Conf)
- // System.setProperty("sun.security.krb5.debug", "true")
- // System.setProperty("sun.security.spnego.debug", "true")
- val conf = new Configuration(true)
- val hConf = HBaseConfiguration.create(conf)
-
- hConf.set("hbase.zookeeper.quorum", zkAddr)
-
- hConf.set("hadoop.security.authentication", "Kerberos")
- hConf.set("hbase.security.authentication", "Kerberos")
- hConf.set("hbase.master.kerberos.principal", "hbase/_HOST@" + realm)
- hConf.set("hbase.regionserver.kerberos.principal", "hbase/_HOST@" + realm)
-
- System.out.println("Connecting secure cluster, using keytab\n")
- UserGroupInformation.setConfiguration(hConf)
- UserGroupInformation.loginUserFromKeytab(principal, keytab)
- val currentUser = UserGroupInformation.getCurrentUser()
- System.out.println("current user : " + currentUser + "\n")
-
- // get table list
- val conn = ConnectionFactory.createConnection(hConf)
- conn.getAdmin
- }
-
- private def withAdmin(zkAddr: String)(op: Admin => Unit): Unit = {
- val admin = getAdmin(zkAddr)
- try {
- op(admin)
- } finally {
- admin.close()
- admin.getConnection.close()
- }
- }
- /**
- * following configuration need to come together to use secured hbase cluster.
- * 1. set hbase.security.auth.enable = true
- * 2. set file path to jaas file java.security.auth.login.config
- * 3. set file path to kerberos file java.security.krb5.conf
- * 4. set realm
- * 5. set principal
- * 6. set file path to keytab
- * @param zkAddr
- * @return
- */
- private def getAdmin(zkAddr: String) = {
- if (config.hasPath("hbase.security.auth.enable") && config.getBoolean("hbase.security.auth.enable")) {
- getSecureClusterAdmin(zkAddr)
- } else {
- val conf = HBaseConfiguration.create()
- conf.set("hbase.zookeeper.quorum", zkAddr)
- val conn = ConnectionFactory.createConnection(conf)
- conn.getAdmin
- }
- }
-
- private def enableTable(zkAddr: String, tableName: String) = {
- withAdmin(zkAddr) { admin =>
- admin.enableTable(TableName.valueOf(tableName))
- }
- }
-
- private def disableTable(zkAddr: String, tableName: String) = {
- withAdmin(zkAddr) { admin =>
- admin.disableTable(TableName.valueOf(tableName))
- }
- }
-
- private def dropTable(zkAddr: String, tableName: String) = {
- withAdmin(zkAddr) { admin =>
- admin.disableTable(TableName.valueOf(tableName))
- admin.deleteTable(TableName.valueOf(tableName))
- }
- }
-
- private def getStartKey(regionCount: Int): Array[Byte] = {
- Bytes.toBytes((Int.MaxValue / regionCount))
- }
-
- private def getEndKey(regionCount: Int): Array[Byte] = {
- Bytes.toBytes((Int.MaxValue / regionCount * (regionCount - 1)))
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala
new file mode 100644
index 0000000..c55c6c7
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageManagement.scala
@@ -0,0 +1,263 @@
+package org.apache.s2graph.core.storage.hbase
+
+import java.util.concurrent.{Executors, TimeUnit}
+
+import com.typesafe.config.Config
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hbase.{HBaseConfiguration, HColumnDescriptor, HTableDescriptor, TableName}
+import org.apache.hadoop.hbase.client.{Admin, ConnectionFactory, Durability}
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
+import org.apache.hadoop.hbase.regionserver.BloomType
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.s2graph.core.storage.StorageManagement
+import org.apache.s2graph.core.utils.{Extensions, logger}
+import org.hbase.async.HBaseClient
+
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, ExecutionContext, duration}
+import scala.util.Try
+import scala.util.control.NonFatal
+
+object AsynchbaseStorageManagement {
+ /* Secure cluster */
+ val SecurityAuthEnabled = "hbase.security.auth.enable"
+ val Jaas = "java.security.auth.login.config"
+ val Krb5Conf = "java.security.krb5.conf"
+ val Realm = "realm"
+ val Principal = "principal"
+ val Keytab = "keytab"
+ val HadoopAuthentication = "hadoop.security.authentication"
+ val HBaseAuthentication = "hbase.security.authentication"
+ val MasterKerberosPrincipal = "hbase.master.kerberos.principal"
+ val RegionServerKerberosPrincipal = "hbase.regionserver.kerberos.principal"
+
+
+ val DefaultCreateTableOptions = Map(
+ "hbase.zookeeper.quorum" -> "localhost"
+ )
+}
+
+class AsynchbaseStorageManagement(val config: Config, val clients: Seq[HBaseClient]) extends StorageManagement {
+ import org.apache.s2graph.core.Management._
+ import AsynchbaseStorageManagement._
+ import Extensions.DeferOps
+
+ /**
+ * Asynchbase client setup.
+ * note that we need two client, one for bulk(withWait=false) and another for withWait=true
+ */
+ private val clientFlushInterval = config.getInt("hbase.rpcs.buffered_flush_interval").toString().toShort
+
+ /**
+ * this method need to be called when client shutdown. this is responsible to cleanUp the resources
+ * such as client into storage.
+ */
+ override def flush(): Unit = clients.foreach { client =>
+ implicit val ec = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor())
+
+ val timeout = Duration((clientFlushInterval + 10) * 20, duration.MILLISECONDS)
+ Await.result(client.flush().toFuture(new AnyRef), timeout)
+ }
+
+ def getOption[T](config: Config, key: String): Option[T] = {
+ import scala.util._
+ Try { config.getAnyRef(key).asInstanceOf[T] }.toOption
+ }
+ /**
+ * create table on storage.
+ * if storage implementation does not support namespace or table, then there is nothing to be done
+ *
+ * @param config
+ */
+ override def createTable(config: Config, tableNameStr: String): Unit = {
+ val zkAddr = config.getString(ZookeeperQuorum)
+
+ withAdmin(config) { admin =>
+ val regionMultiplier = getOption[Int](config, RegionMultiplier).getOrElse(0)
+ val regionCount = getOption[Int](config, TotalRegionCount).getOrElse(admin.getClusterStatus.getServersSize * regionMultiplier)
+ val cfs = getOption[Seq[String]](config, ColumnFamilies).getOrElse(DefaultColumnFamilies)
+ val compressionAlgorithm = getOption[String](config, CompressionAlgorithm).getOrElse(DefaultCompressionAlgorithm)
+ val ttl = getOption[Int](config, Ttl)
+ val replicationScoreOpt = getOption[Int](config, ReplicationScope)
+
+ val tableName = TableName.valueOf(tableNameStr)
+ try {
+ if (!admin.tableExists(tableName)) {
+ val desc = new HTableDescriptor(tableName)
+ desc.setDurability(Durability.ASYNC_WAL)
+ for (cf <- cfs) {
+ val columnDesc = new HColumnDescriptor(cf)
+ .setCompressionType(Algorithm.valueOf(compressionAlgorithm.toUpperCase))
+ .setBloomFilterType(BloomType.ROW)
+ .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)
+ .setMaxVersions(1)
+ .setTimeToLive(2147483647)
+ .setMinVersions(0)
+ .setBlocksize(32768)
+ .setBlockCacheEnabled(true)
+ // FIXME: For test!!
+ .setInMemory(true)
+
+ ttl.foreach(columnDesc.setTimeToLive(_))
+ replicationScoreOpt.foreach(columnDesc.setScope(_))
+
+ desc.addFamily(columnDesc)
+ }
+
+ if (regionCount <= 1) admin.createTable(desc)
+ else admin.createTable(desc, getStartKey(regionCount), getEndKey(regionCount), regionCount)
+ } else {
+ logger.info(s"$zkAddr, $tableName, $cfs already exist.")
+ }
+ } catch {
+ case e: Throwable =>
+ logger.error(s"$zkAddr, $tableName failed with $e", e)
+ throw e
+ }
+ }
+ }
+
+ /**
+ *
+ * @param config
+ * @param tableNameStr
+ */
+ override def truncateTable(config: Config, tableNameStr: String): Unit = {
+ withAdmin(config) { admin =>
+ val tableName = TableName.valueOf(tableNameStr)
+ if (!Try(admin.tableExists(tableName)).getOrElse(false)) {
+ logger.info(s"No table to truncate ${tableNameStr}")
+ return
+ }
+
+ Try(admin.isTableDisabled(tableName)).map {
+ case true =>
+ logger.info(s"${tableNameStr} is already disabled.")
+
+ case false =>
+ logger.info(s"Before disabling to trucate ${tableNameStr}")
+ Try(admin.disableTable(tableName)).recover {
+ case NonFatal(e) =>
+ logger.info(s"Failed to disable ${tableNameStr}: ${e}")
+ }
+ logger.info(s"After disabling to trucate ${tableNameStr}")
+ }
+
+ logger.info(s"Before truncating ${tableNameStr}")
+ Try(admin.truncateTable(tableName, true)).recover {
+ case NonFatal(e) =>
+ logger.info(s"Failed to truncate ${tableNameStr}: ${e}")
+ }
+ logger.info(s"After truncating ${tableNameStr}")
+ Try(admin.close()).recover {
+ case NonFatal(e) =>
+ logger.info(s"Failed to close admin ${tableNameStr}: ${e}")
+ }
+ Try(admin.getConnection.close()).recover {
+ case NonFatal(e) =>
+ logger.info(s"Failed to close connection ${tableNameStr}: ${e}")
+ }
+ }
+ }
+
+ /**
+ *
+ * @param config
+ * @param tableNameStr
+ */
+ override def deleteTable(config: Config, tableNameStr: String): Unit = {
+ withAdmin(config) { admin =>
+ val tableName = TableName.valueOf(tableNameStr)
+ if (!admin.tableExists(tableName)) {
+ return
+ }
+ if (admin.isTableEnabled(tableName)) {
+ admin.disableTable(tableName)
+ }
+ admin.deleteTable(tableName)
+ }
+ }
+
+ /**
+ *
+ */
+ override def shutdown(): Unit = {
+ flush()
+ clients.foreach { client =>
+ AsynchbaseStorage.shutdown(client)
+ }
+ }
+
+
+
+ private def getSecureClusterAdmin(config: Config) = {
+ val zkAddr = config.getString(ZookeeperQuorum)
+ val realm = config.getString(Realm)
+ val principal = config.getString(Principal)
+ val keytab = config.getString(Keytab)
+
+ System.setProperty(Jaas, config.getString(Jaas))
+ System.setProperty(Krb5Conf, config.getString(Krb5Conf))
+
+
+ val conf = new Configuration(true)
+ val hConf = HBaseConfiguration.create(conf)
+
+ hConf.set(ZookeeperQuorum, zkAddr)
+
+ hConf.set(HadoopAuthentication, "Kerberos")
+ hConf.set(HBaseAuthentication, "Kerberos")
+ hConf.set(MasterKerberosPrincipal, "hbase/_HOST@" + realm)
+ hConf.set(RegionServerKerberosPrincipal, "hbase/_HOST@" + realm)
+
+ System.out.println("Connecting secure cluster, using keytab\n")
+ UserGroupInformation.setConfiguration(hConf)
+ UserGroupInformation.loginUserFromKeytab(principal, keytab)
+ val currentUser = UserGroupInformation.getCurrentUser()
+ System.out.println("current user : " + currentUser + "\n")
+
+ // get table list
+ val conn = ConnectionFactory.createConnection(hConf)
+ conn.getAdmin
+ }
+
+ private def withAdmin(config: Config)(op: Admin => Unit): Unit = {
+ val admin = getAdmin(config)
+ try {
+ op(admin)
+ } finally {
+ admin.close()
+ admin.getConnection.close()
+ }
+ }
+ /**
+ * following configuration need to come together to use secured hbase cluster.
+ * 1. set hbase.security.auth.enable = true
+ * 2. set file path to jaas file java.security.auth.login.config
+ * 3. set file path to kerberos file java.security.krb5.conf
+ * 4. set realm
+ * 5. set principal
+ * 6. set file path to keytab
+ */
+ private def getAdmin(config: Config) = {
+ if (config.hasPath(SecurityAuthEnabled) && config.getBoolean(SecurityAuthEnabled)) {
+ getSecureClusterAdmin(config)
+ } else {
+ val zkAddr = config.getString(ZookeeperQuorum)
+ val conf = HBaseConfiguration.create()
+ conf.set(ZookeeperQuorum, zkAddr)
+ val conn = ConnectionFactory.createConnection(conf)
+ conn.getAdmin
+ }
+ }
+
+ private def getStartKey(regionCount: Int): Array[Byte] = {
+ Bytes.toBytes((Int.MaxValue / regionCount))
+ }
+
+ private def getEndKey(regionCount: Int): Array[Byte] = {
+ Bytes.toBytes((Int.MaxValue / regionCount * (regionCount - 1)))
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
new file mode 100644
index 0000000..4ef95b8
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
@@ -0,0 +1,335 @@
+package org.apache.s2graph.core.storage.hbase
+
+import java.util
+import java.util.Base64
+
+import com.stumbleupon.async.Deferred
+import com.typesafe.config.Config
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core._
+import org.apache.s2graph.core.mysqls.{Label, ServiceColumn}
+import org.apache.s2graph.core.storage._
+import org.apache.s2graph.core.storage.serde._
+import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage.{AsyncRPC, ScanWithRange}
+import org.apache.s2graph.core.types.{HBaseType, VertexId}
+import org.apache.s2graph.core.utils.{CanDefer, DeferCache, Extensions, logger}
+import org.hbase.async.FilterList.Operator.MUST_PASS_ALL
+import org.hbase.async._
+
+import scala.collection.JavaConversions._
+import scala.concurrent.{ExecutionContext, Future}
+
+class AsynchbaseStorageReadable(val graph: S2Graph,
+ val config: Config,
+ val client: HBaseClient,
+ val serDe: StorageSerDe,
+ override val io: StorageIO) extends StorageReadable[AsyncRPC] {
+ import Extensions.DeferOps
+ import CanDefer._
+
+ private val emptyKeyValues = new util.ArrayList[KeyValue]()
+ private val emptyKeyValuesLs = new util.ArrayList[util.ArrayList[KeyValue]]()
+ private val emptyStepResult = new util.ArrayList[StepResult]()
+
+ /** Future Cache to squash request */
+ lazy private val futureCache = new DeferCache[StepResult, Deferred, Deferred](config, StepResult.Empty, "AsyncHbaseFutureCache", useMetric = true)
+ /** v4 max next row size */
+ private val v4_max_num_rows = 10000
+ private def getV4MaxNumRows(limit : Int): Int = {
+ if (limit < v4_max_num_rows) limit
+ else v4_max_num_rows
+ }
+
+ /**
+ * build proper request which is specific into storage to call fetchIndexEdgeKeyValues or fetchSnapshotEdgeKeyValues.
+ * for example, Asynchbase use GetRequest, Scanner so this method is responsible to build
+ * client request(GetRequest, Scanner) based on user provided query.
+ *
+ * @param queryRequest
+ * @return
+ */
+ override def buildRequest(queryRequest: QueryRequest, edge: S2Edge) = {
+ import Serializable._
+ val queryParam = queryRequest.queryParam
+ val label = queryParam.label
+
+ val serializer = if (queryParam.tgtVertexInnerIdOpt.isDefined) {
+ val snapshotEdge = edge.toSnapshotEdge
+ serDe.snapshotEdgeSerializer(snapshotEdge)
+ } else {
+ val indexEdge = edge.toIndexEdge(queryParam.labelOrderSeq)
+ serDe.indexEdgeSerializer(indexEdge)
+ }
+
+ val rowKey = serializer.toRowKey
+ val (minTs, maxTs) = queryParam.durationOpt.getOrElse((0L, Long.MaxValue))
+
+ val (intervalMaxBytes, intervalMinBytes) = queryParam.buildInterval(Option(edge))
+
+ label.schemaVersion match {
+ case HBaseType.VERSION4 if queryParam.tgtVertexInnerIdOpt.isEmpty =>
+ val scanner = AsynchbasePatcher.newScanner(client, label.hbaseTableName)
+ scanner.setFamily(edgeCf)
+
+ /*
+ * TODO: remove this part.
+ */
+ val indexEdgeOpt = edge.edgesWithIndex.find(edgeWithIndex => edgeWithIndex.labelIndex.seq == queryParam.labelOrderSeq)
+ val indexEdge = indexEdgeOpt.getOrElse(throw new RuntimeException(s"Can`t find index for query $queryParam"))
+
+ val srcIdBytes = VertexId.toSourceVertexId(indexEdge.srcVertex.id).bytes
+ val labelWithDirBytes = indexEdge.labelWithDir.bytes
+ val labelIndexSeqWithIsInvertedBytes = StorageSerializable.labelOrderSeqWithIsInverted(indexEdge.labelIndexSeq, isInverted = false)
+ val baseKey = Bytes.add(srcIdBytes, labelWithDirBytes, labelIndexSeqWithIsInvertedBytes)
+
+ val (startKey, stopKey) =
+ if (queryParam.intervalOpt.isDefined) {
+ // interval is set.
+ val _startKey = queryParam.cursorOpt match {
+ case Some(cursor) => Base64.getDecoder.decode(cursor)
+ case None => Bytes.add(baseKey, intervalMaxBytes)
+ }
+ (_startKey , Bytes.add(baseKey, intervalMinBytes))
+ } else {
+ /*
+ * note: since propsToBytes encode size of property map at first byte, we are sure about max value here
+ */
+ val _startKey = queryParam.cursorOpt match {
+ case Some(cursor) => Base64.getDecoder.decode(cursor)
+ case None => baseKey
+ }
+ (_startKey, Bytes.add(baseKey, Array.fill(1)(-1)))
+ }
+
+ scanner.setStartKey(startKey)
+ scanner.setStopKey(stopKey)
+
+ if (queryParam.limit == Int.MinValue) logger.debug(s"MinValue: $queryParam")
+
+ scanner.setMaxVersions(1)
+ // TODO: exclusive condition innerOffset with cursorOpt
+ if (queryParam.cursorOpt.isDefined) {
+ scanner.setMaxNumRows(getV4MaxNumRows(queryParam.limit))
+ } else {
+ scanner.setMaxNumRows(getV4MaxNumRows(queryParam.innerOffset + queryParam.innerLimit))
+ }
+ scanner.setMaxTimestamp(maxTs)
+ scanner.setMinTimestamp(minTs)
+ scanner.setRpcTimeout(queryParam.rpcTimeout)
+
+ // SET option for this rpc properly.
+ if (queryParam.cursorOpt.isDefined) Right(ScanWithRange(scanner, 0, queryParam.limit))
+ else Right(ScanWithRange(scanner, 0, queryParam.innerOffset + queryParam.innerLimit))
+
+ case _ =>
+ val get = if (queryParam.tgtVertexInnerIdOpt.isDefined) {
+ new GetRequest(label.hbaseTableName.getBytes, rowKey, edgeCf, serializer.toQualifier)
+ } else {
+ new GetRequest(label.hbaseTableName.getBytes, rowKey, edgeCf)
+ }
+
+ get.maxVersions(1)
+ get.setFailfast(true)
+ get.setMinTimestamp(minTs)
+ get.setMaxTimestamp(maxTs)
+ get.setTimeout(queryParam.rpcTimeout)
+
+ val pagination = new ColumnPaginationFilter(queryParam.limit, queryParam.offset)
+ val columnRangeFilterOpt = queryParam.intervalOpt.map { interval =>
+ new ColumnRangeFilter(intervalMaxBytes, true, intervalMinBytes, true)
+ }
+ get.setFilter(new FilterList(pagination +: columnRangeFilterOpt.toSeq, MUST_PASS_ALL))
+ Left(get)
+ }
+ }
+
+ /**
+ *
+ * @param queryRequest
+ * @param vertex
+ * @return
+ */
+ override def buildRequest(queryRequest: QueryRequest, vertex: S2Vertex) = {
+ val kvs = serDe.vertexSerializer(vertex).toKeyValues
+ val get = new GetRequest(vertex.hbaseTableName.getBytes, kvs.head.row, Serializable.vertexCf)
+ // get.setTimeout(this.singleGetTimeout.toShort)
+ get.setFailfast(true)
+ get.maxVersions(1)
+ Left(get)
+ }
+
+ /**
+ * responsible to fire parallel fetch call into storage and create future that will return merged result.
+ *
+ * @param queryRequests
+ * @param prevStepEdges
+ * @return
+ */
+ override def fetches(queryRequests: Seq[QueryRequest], prevStepEdges: Map[VertexId, Seq[EdgeWithScore]])(implicit ec: ExecutionContext) = {
+ val defers: Seq[Deferred[StepResult]] = for {
+ queryRequest <- queryRequests
+ } yield {
+ val queryOption = queryRequest.query.queryOption
+ val queryParam = queryRequest.queryParam
+ val shouldBuildParents = queryOption.returnTree || queryParam.whereHasParent
+ val parentEdges = if (shouldBuildParents) prevStepEdges.getOrElse(queryRequest.vertex.id, Nil) else Nil
+ fetch(queryRequest, isInnerCall = false, parentEdges)
+ }
+
+ val grouped: Deferred[util.ArrayList[StepResult]] = Deferred.groupInOrder(defers)
+ grouped.map(emptyStepResult) { queryResults: util.ArrayList[StepResult] =>
+ queryResults.toSeq
+ }.toFuture(emptyStepResult)
+ }
+
+ override def fetchKeyValues(rpc: AsyncRPC)(implicit ec: ExecutionContext) = {
+ val defer = fetchKeyValuesInner(rpc)
+ defer.toFuture(emptyKeyValues).map { kvsArr =>
+ kvsArr.map { kv =>
+ implicitly[CanSKeyValue[KeyValue]].toSKeyValue(kv)
+ }
+ }
+ }
+
+ override def fetchEdgesAll()(implicit ec: ExecutionContext): Future[Seq[S2Edge]] = {
+ val futures = Label.findAll().groupBy(_.hbaseTableName).toSeq.map { case (hTableName, labels) =>
+ val distinctLabels = labels.toSet
+ val scan = AsynchbasePatcher.newScanner(client, hTableName)
+ scan.setFamily(Serializable.edgeCf)
+ scan.setMaxVersions(1)
+
+ scan.nextRows(S2Graph.FetchAllLimit).toFuture(emptyKeyValuesLs).map {
+ case null => Seq.empty
+ case kvsLs =>
+ kvsLs.flatMap { kvs =>
+ kvs.flatMap { kv =>
+ val sKV = implicitly[CanSKeyValue[KeyValue]].toSKeyValue(kv)
+
+ serDe.indexEdgeDeserializer(schemaVer = HBaseType.DEFAULT_VERSION).fromKeyValues(Seq(kv), None)
+ .filter(e => distinctLabels(e.innerLabel) && e.direction == "out" && !e.isDegree)
+ }
+ }
+ }
+ }
+
+ Future.sequence(futures).map(_.flatten)
+ }
+
+ override def fetchVerticesAll()(implicit ec: ExecutionContext) = {
+ val futures = ServiceColumn.findAll().groupBy(_.service.hTableName).toSeq.map { case (hTableName, columns) =>
+ val distinctColumns = columns.toSet
+ val scan = AsynchbasePatcher.newScanner(client, hTableName)
+ scan.setFamily(Serializable.vertexCf)
+ scan.setMaxVersions(1)
+
+ scan.nextRows(S2Graph.FetchAllLimit).toFuture(emptyKeyValuesLs).map {
+ case null => Seq.empty
+ case kvsLs =>
+ kvsLs.flatMap { kvs =>
+ serDe.vertexDeserializer(schemaVer = HBaseType.DEFAULT_VERSION).fromKeyValues(kvs, None)
+ .filter(v => distinctColumns(v.serviceColumn))
+ }
+ }
+ }
+ Future.sequence(futures).map(_.flatten)
+ }
+
+
+ /**
+ * we are using future cache to squash requests into same key on storage.
+ *
+ * @param queryRequest
+ * @param isInnerCall
+ * @param parentEdges
+ * @return we use Deferred here since it has much better performrance compared to scala.concurrent.Future.
+ * seems like map, flatMap on scala.concurrent.Future is slower than Deferred's addCallback
+ */
+ private def fetch(queryRequest: QueryRequest,
+ isInnerCall: Boolean,
+ parentEdges: Seq[EdgeWithScore])(implicit ec: ExecutionContext): Deferred[StepResult] = {
+
+ def fetchInner(hbaseRpc: AsyncRPC): Deferred[StepResult] = {
+ val prevStepScore = queryRequest.prevStepScore
+ val fallbackFn: (Exception => StepResult) = { ex =>
+ logger.error(s"fetchInner failed. fallback return. $hbaseRpc}", ex)
+ StepResult.Failure
+ }
+
+ val queryParam = queryRequest.queryParam
+ fetchKeyValuesInner(hbaseRpc).mapWithFallback(emptyKeyValues)(fallbackFn) { kvs =>
+ val (startOffset, len) = queryParam.label.schemaVersion match {
+ case HBaseType.VERSION4 =>
+ val offset = if (queryParam.cursorOpt.isDefined) 0 else queryParam.offset
+ (offset, queryParam.limit)
+ case _ => (0, kvs.length)
+ }
+
+ io.toEdges(kvs, queryRequest, prevStepScore, isInnerCall, parentEdges, startOffset, len)
+ }
+ }
+
+ val queryParam = queryRequest.queryParam
+ val cacheTTL = queryParam.cacheTTLInMillis
+ /* with version 4, request's type is (Scanner, (Int, Int)). otherwise GetRequest. */
+
+ val edge = Storage.toRequestEdge(graph)(queryRequest, parentEdges)
+ val request = buildRequest(queryRequest, edge)
+
+ val (intervalMaxBytes, intervalMinBytes) = queryParam.buildInterval(Option(edge))
+ val requestCacheKey = Bytes.add(toCacheKeyBytes(request), intervalMaxBytes, intervalMinBytes)
+
+ if (cacheTTL <= 0) fetchInner(request)
+ else {
+ val cacheKeyBytes = Bytes.add(queryRequest.query.queryOption.cacheKeyBytes, requestCacheKey)
+
+ // val cacheKeyBytes = toCacheKeyBytes(request)
+ val cacheKey = queryParam.toCacheKey(cacheKeyBytes)
+ futureCache.getOrElseUpdate(cacheKey, cacheTTL)(fetchInner(request))
+ }
+ }
+
+ /**
+ * Private Methods which is specific to Asynchbase implementation.
+ */
+ private def fetchKeyValuesInner(rpc: AsyncRPC)(implicit ec: ExecutionContext): Deferred[util.ArrayList[KeyValue]] = {
+ rpc match {
+ case Left(get) => client.get(get)
+ case Right(ScanWithRange(scanner, offset, limit)) =>
+ val fallbackFn: (Exception => util.ArrayList[KeyValue]) = { ex =>
+ logger.error(s"fetchKeyValuesInner failed.", ex)
+ scanner.close()
+ emptyKeyValues
+ }
+
+ scanner.nextRows().mapWithFallback(new util.ArrayList[util.ArrayList[KeyValue]]())(fallbackFn) { kvsLs =>
+ val ls = new util.ArrayList[KeyValue]
+ if (kvsLs == null) {
+ } else {
+ kvsLs.foreach { kvs =>
+ if (kvs != null) kvs.foreach { kv => ls.add(kv) }
+ else {
+
+ }
+ }
+ }
+
+ scanner.close()
+ val toIndex = Math.min(ls.size(), offset + limit)
+ new util.ArrayList[KeyValue](ls.subList(offset, toIndex))
+ }
+ case _ => Deferred.fromError(new RuntimeException(s"fetchKeyValues failed. $rpc"))
+ }
+ }
+
+ private def toCacheKeyBytes(hbaseRpc: AsyncRPC): Array[Byte] = {
+ /* with version 4, request's type is (Scanner, (Int, Int)). otherwise GetRequest. */
+ hbaseRpc match {
+ case Left(getRequest) => getRequest.key
+ case Right(ScanWithRange(scanner, offset, limit)) =>
+ Bytes.add(scanner.getCurrentKey, Bytes.add(Bytes.toBytes(offset), Bytes.toBytes(limit)))
+ case _ =>
+ logger.error(s"toCacheKeyBytes failed. not supported class type. $hbaseRpc")
+ throw new RuntimeException(s"toCacheKeyBytes: $hbaseRpc")
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/39544dc5/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala
new file mode 100644
index 0000000..ab1ff19
--- /dev/null
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageSerDe.scala
@@ -0,0 +1,68 @@
+package org.apache.s2graph.core.storage.hbase
+
+import org.apache.s2graph.core.storage.serde.Deserializable
+import org.apache.s2graph.core.{IndexEdge, S2Graph, S2Vertex, SnapshotEdge}
+import org.apache.s2graph.core.storage.{StorageIO, StorageSerDe, serde}
+
+class AsynchbaseStorageSerDe(val graph: S2Graph) extends StorageSerDe {
+ import org.apache.s2graph.core.types.HBaseType._
+
+ /**
+ * create serializer that knows how to convert given snapshotEdge into kvs: Seq[SKeyValue]
+ * so we can store this kvs.
+ *
+ * @param snapshotEdge : snapshotEdge to serialize
+ * @return serializer implementation for StorageSerializable which has toKeyValues return Seq[SKeyValue]
+ */
+ override def snapshotEdgeSerializer(snapshotEdge: SnapshotEdge) = {
+ snapshotEdge.schemaVer match {
+ // case VERSION1 |
+ case VERSION2 => new serde.snapshotedge.wide.SnapshotEdgeSerializable(snapshotEdge)
+ case VERSION3 | VERSION4 => new serde.snapshotedge.tall.SnapshotEdgeSerializable(snapshotEdge)
+ case _ => throw new RuntimeException(s"not supported version: ${snapshotEdge.schemaVer}")
+ }
+ }
+
+ /**
+ * create serializer that knows how to convert given indexEdge into kvs: Seq[SKeyValue]
+ *
+ * @param indexEdge : indexEdge to serialize
+ * @return serializer implementation
+ */
+ override def indexEdgeSerializer(indexEdge: IndexEdge) = {
+ indexEdge.schemaVer match {
+ // case VERSION1
+ case VERSION2 | VERSION3 => new serde.indexedge.wide.IndexEdgeSerializable(indexEdge)
+ case VERSION4 => new serde.indexedge.tall.IndexEdgeSerializable(indexEdge)
+ case _ => throw new RuntimeException(s"not supported version: ${indexEdge.schemaVer}")
+ }
+ }
+
+ /**
+ * create serializer that knows how to convert given vertex into kvs: Seq[SKeyValue]
+ *
+ * @param vertex : vertex to serialize
+ * @return serializer implementation
+ */
+ override def vertexSerializer(vertex: S2Vertex) = new serde.vertex.wide.VertexSerializable(vertex)
+
+ /**
+ * create deserializer that can parse stored CanSKeyValue into snapshotEdge.
+ * note that each storage implementation should implement implicit type class
+ * to convert storage dependent dataType into common SKeyValue type by implementing CanSKeyValue
+ *
+ * ex) Asynchbase use it's KeyValue class and CanSKeyValue object has implicit type conversion method.
+ * if any storaage use different class to represent stored byte array,
+ * then that storage implementation is responsible to provide implicit type conversion method on CanSKeyValue.
+ **/
+ private val snapshotEdgeDeserializable = new serde.snapshotedge.tall.SnapshotEdgeDeserializable(graph)
+ override def snapshotEdgeDeserializer(schemaVer: String) = snapshotEdgeDeserializable
+
+ /** create deserializer that can parse stored CanSKeyValue into indexEdge. */
+ private val indexEdgeDeserializer = new serde.indexedge.tall.IndexEdgeDeserializable(graph)
+ override def indexEdgeDeserializer(schemaVer: String) = indexEdgeDeserializer
+
+ /** create deserializer that can parser stored CanSKeyValue into vertex. */
+ private val vertexDeserializer = new serde.vertex.wide.VertexDeserializable(graph)
+ override def vertexDeserializer(schemaVer: String): Deserializable[S2Vertex] = vertexDeserializer
+}
[8/8] incubator-s2graph git commit: [S2GRAPH-169]: Separate multiple
functionalities on Storage class into multiple Interface.
Posted by st...@apache.org.
[S2GRAPH-169]: Separate multiple functionalities on Storage class into multiple Interface.
JIRA:
[S2GRAPH-169] https://issues.apache.org/jira/browse/S2GRAPH-169
Pull Request:
Closes #126
Author
DO YUNG YOON <st...@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/128d67c0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/128d67c0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/128d67c0
Branch: refs/heads/master
Commit: 128d67c06a3406963b102912cb353a3f25f69beb
Parents: ed9bedf af9c105
Author: DO YUNG YOON <st...@apache.org>
Authored: Mon Oct 30 19:23:00 2017 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Mon Oct 30 19:24:56 2017 +0900
----------------------------------------------------------------------
.gitignore | 2 +-
CHANGES | 1 +
.../org/apache/s2graph/core/Management.scala | 74 +-
.../org/apache/s2graph/core/QueryParam.scala | 2 +-
.../scala/org/apache/s2graph/core/S2Edge.scala | 2 +-
.../scala/org/apache/s2graph/core/S2Graph.scala | 282 +++--
.../org/apache/s2graph/core/S2Vertex.scala | 40 +-
.../s2graph/core/features/S2GraphFeatures.scala | 3 +-
.../s2graph/core/storage/Deserializable.scala | 43 -
.../s2graph/core/storage/MutateResponse.scala | 31 +
.../apache/s2graph/core/storage/SKeyValue.scala | 14 +-
.../s2graph/core/storage/Serializable.scala | 27 -
.../apache/s2graph/core/storage/Storage.scala | 1136 ++----------------
.../core/storage/StorageDeserializable.scala | 120 --
.../apache/s2graph/core/storage/StorageIO.scala | 260 ++++
.../core/storage/StorageManagement.scala | 54 +
.../s2graph/core/storage/StorageReadable.scala | 95 ++
.../s2graph/core/storage/StorageSerDe.scala | 78 ++
.../core/storage/StorageSerializable.scala | 82 --
.../s2graph/core/storage/StorageWritable.scala | 64 +
.../storage/WriteWriteConflictResolver.scala | 457 +++++++
.../core/storage/hbase/AsynchbaseStorage.scala | 763 +-----------
.../hbase/AsynchbaseStorageManagement.scala | 282 +++++
.../hbase/AsynchbaseStorageReadable.scala | 367 ++++++
.../storage/hbase/AsynchbaseStorageSerDe.scala | 87 ++
.../hbase/AsynchbaseStorageWritable.scala | 137 +++
.../core/storage/serde/Deserializable.scala | 41 +
.../core/storage/serde/Serializable.scala | 27 +
.../storage/serde/StorageDeserializable.scala | 144 +++
.../storage/serde/StorageSerializable.scala | 90 ++
.../tall/IndexEdgeDeserializable.scala | 12 +-
.../indexedge/tall/IndexEdgeSerializable.scala | 5 +-
.../wide/IndexEdgeDeserializable.scala | 4 +-
.../indexedge/wide/IndexEdgeSerializable.scala | 5 +-
.../tall/SnapshotEdgeDeserializable.scala | 7 +-
.../tall/SnapshotEdgeSerializable.scala | 4 +-
.../wide/SnapshotEdgeDeserializable.scala | 9 +-
.../wide/SnapshotEdgeSerializable.scala | 4 +-
.../serde/vertex/VertexDeserializable.scala | 146 +--
.../serde/vertex/VertexSerializable.scala | 114 +-
.../vertex/tall/VertexDeserializable.scala | 58 +
.../serde/vertex/tall/VertexSerializable.scala | 54 +
.../vertex/wide/VertexDeserializable.scala | 73 ++
.../serde/vertex/wide/VertexSerializable.scala | 52 +
.../apache/s2graph/core/utils/DeferCache.scala | 8 +-
.../s2graph/core/Integrate/CrudTest.scala | 8 +-
.../LabelLabelIndexMutateOptionTest.scala | 6 +-
.../s2graph/core/storage/StorageIOTest.scala | 55 +
.../core/storage/hbase/IndexEdgeTest.scala | 3 +-
.../rest/play/controllers/EdgeController.scala | 11 +-
.../play/controllers/VertexController.scala | 5 +-
51 files changed, 3099 insertions(+), 2349 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/128d67c0/CHANGES
----------------------------------------------------------------------
diff --cc CHANGES
index 0a523b3,7f37f00..6e685ae
--- a/CHANGES
+++ b/CHANGES
@@@ -45,8 -45,7 +45,9 @@@ Release Notes - S2Graph - Version 0.2.
* [S2GRAPH-133] - Deprecating Vagrant environment.
* [S2GRAPH-135] - Change the way LabelIndexOption is implemented and improve it
* [S2GRAPH-165] - Add TinkerPop support documentation on README
-
+ * [S2GRAPH-167] - Provide Vertex CSV File BulkLoad
++ * [S2GRAPH-169] - Separate multiple functionalities on Storage class into multiple Interface.
+
** New Feature
* [S2GRAPH-123] - Support different index on out/in direction.
* [S2GRAPH-125] - Add options field on Label model for controlling advanced options.
[6/8] incubator-s2graph git commit: remove type parameter on Storage
Trait (#13)
Posted by st...@apache.org.
remove type parameter on Storage Trait (#13)
Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/55d194ed
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/55d194ed
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/55d194ed
Branch: refs/heads/master
Commit: 55d194ed8edfb0ec78ce5723c379c20fa7462632
Parents: f416194
Author: daewon <da...@apache.org>
Authored: Mon Oct 30 17:07:48 2017 +0900
Committer: Doyung Yoon <st...@apache.org>
Committed: Mon Oct 30 10:07:48 2017 +0200
----------------------------------------------------------------------
.../scala/org/apache/s2graph/core/S2Graph.scala | 54 ++++++--------------
.../apache/s2graph/core/storage/SKeyValue.scala | 13 +++--
.../apache/s2graph/core/storage/Storage.scala | 16 ++----
.../s2graph/core/storage/StorageReadable.scala | 30 +++++------
.../storage/WriteWriteConflictResolver.scala | 6 +--
.../core/storage/hbase/AsynchbaseStorage.scala | 5 +-
.../hbase/AsynchbaseStorageReadable.scala | 46 ++++++++++++++---
7 files changed, 84 insertions(+), 86 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55d194ed/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 92f68dc..34db9e4 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
@@ -133,7 +133,7 @@ object S2Graph {
new S2Graph(configuration)(ec)
}
- def initStorage(graph: S2Graph, config: Config)(ec: ExecutionContext): Storage[_] = {
+ def initStorage(graph: S2Graph, config: Config)(ec: ExecutionContext): Storage = {
val storageBackend = config.getString("s2graph.storage.backend")
logger.info(s"[InitStorage]: $storageBackend")
@@ -908,7 +908,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
/**
* TODO: we need to some way to handle malformed configuration for storage.
*/
- val storagePool: scala.collection.mutable.Map[String, Storage[_]] = {
+ val storagePool: scala.collection.mutable.Map[String, Storage] = {
val labels = Label.findAll()
val services = Service.findAll()
@@ -919,12 +919,12 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
confWithFallback(conf)
}.toSet
- val pools = new java.util.HashMap[Config, Storage[_]]()
+ 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[_]]()
+ val m = new java.util.concurrent.ConcurrentHashMap[String, Storage]()
labels.foreach { label =>
if (label.storageConfigOpt.isDefined) {
@@ -941,7 +941,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
m
}
- val defaultStorage: Storage[_] = S2Graph.initStorage(this, config)(ec)
+ val defaultStorage: Storage = S2Graph.initStorage(this, config)(ec)
/** QueryLevel FutureCache */
val queryFutureCache = new DeferCache[StepResult, Promise, Future](parseCacheConfig(config, "query."), empty = StepResult.Empty)
@@ -953,11 +953,11 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
val indexProvider = IndexProvider.apply(config)
- def getStorage(service: Service): Storage[_] = {
+ def getStorage(service: Service): Storage = {
storagePool.getOrElse(s"service:${service.serviceName}", defaultStorage)
}
- def getStorage(label: Label): Storage[_] = {
+ def getStorage(label: Label): Storage = {
storagePool.getOrElse(s"label:${label.label}", defaultStorage)
}
@@ -975,7 +975,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
val futures = for {
edge <- edges
} yield {
- getStorage(edge.innerLabel).fetchSnapshotEdgeInner(edge).map { case (_, edgeOpt, _) =>
+ getStorage(edge.innerLabel).fetchSnapshotEdgeInner(edge).map { case (edgeOpt, _) =>
edgeOpt.toSeq.map(e => EdgeWithScore(e, 1.0, edge.innerLabel))
}
}
@@ -1145,31 +1145,9 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
def getVertices(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
- def getVertices[Q](storage: Storage[Q])(vertices: Seq[S2Vertex]): Future[Seq[S2Vertex]] = {
- def fromResult(kvs: Seq[SKeyValue],
- version: String): Option[S2Vertex] = {
- if (kvs.isEmpty) None
- else storage.vertexDeserializer(version).fromKeyValues(kvs, None)
- // .map(S2Vertex(graph, _))
- }
-
- val futures = vertices.map { vertex =>
- val queryParam = QueryParam.Empty
- val q = Query.toQuery(Seq(vertex), Seq(queryParam))
- val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
- val rpc = storage.buildRequest(queryRequest, vertex)
- storage.fetchKeyValues(rpc).map { kvs =>
- fromResult(kvs, vertex.serviceColumn.schemaVersion)
- } recoverWith { case ex: Throwable =>
- Future.successful(None)
- }
- }
-
- Future.sequence(futures).map { result => result.toList.flatten }
- }
val verticesWithIdx = vertices.zipWithIndex
val futures = verticesWithIdx.groupBy { case (v, idx) => v.service }.map { case (service, vertexGroup) =>
- getVertices(getStorage(service))(vertexGroup.map(_._1)).map(_.zip(vertexGroup.map(_._2)))
+ getStorage(service).fetchVertices(vertices).map(_.zip(vertexGroup.map(_._2)))
}
Future.sequence(futures).map { ls =>
@@ -1272,7 +1250,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
}
- private def deleteAllFetchedEdgesAsyncOld(storage: Storage[_])(stepInnerResult: StepResult,
+ private def deleteAllFetchedEdgesAsyncOld(storage: Storage)(stepInnerResult: StepResult,
requestTs: Long,
retryNum: Int): Future[Boolean] = {
if (stepInnerResult.isEmpty) Future.successful(true)
@@ -1426,7 +1404,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
}
- private def mutateStrongEdges(storage: Storage[_])(_edges: Seq[S2Edge], withWait: Boolean): Future[Seq[Boolean]] = {
+ private def mutateStrongEdges(storage: Storage)(_edges: Seq[S2Edge], withWait: Boolean): Future[Seq[Boolean]] = {
val edgeWithIdxs = _edges.zipWithIndex
val grouped = edgeWithIdxs.groupBy { case (edge, idx) =>
@@ -1462,7 +1440,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
- private def mutateEdgesInner(storage: Storage[_])(edges: Seq[S2Edge],
+ private def mutateEdgesInner(storage: Storage)(edges: Seq[S2Edge],
checkConsistency: Boolean,
withWait: Boolean): Future[MutateResponse] = {
assert(edges.nonEmpty)
@@ -1483,14 +1461,14 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
Future.sequence(futures).map { rets => new MutateResponse(rets.forall(_.isSuccess)) }
} else {
- storage.fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+ storage.fetchSnapshotEdgeInner(edges.head).flatMap { case (snapshotEdgeOpt, kvOpt) =>
storage.retry(1)(edges, 0, snapshotEdgeOpt).map(new MutateResponse(_))
}
}
}
def mutateVertices(vertices: Seq[S2Vertex], withWait: Boolean = false): Future[Seq[MutateResponse]] = {
- def mutateVertex(storage: Storage[_])(vertex: S2Vertex, withWait: Boolean): Future[MutateResponse] = {
+ def mutateVertex(storage: Storage)(vertex: S2Vertex, withWait: Boolean): Future[MutateResponse] = {
if (vertex.op == GraphUtil.operations("delete")) {
storage.writeToStorage(vertex.hbaseZkAddr,
storage.vertexSerializer(vertex).toKeyValues.map(_.copy(operation = SKeyValue.Delete)), withWait)
@@ -1502,7 +1480,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
}
}
- def mutateVertices(storage: Storage[_])(vertices: Seq[S2Vertex],
+ def mutateVertices(storage: Storage)(vertices: Seq[S2Vertex],
withWait: Boolean = false): Future[Seq[MutateResponse]] = {
val futures = vertices.map { vertex => mutateVertex(storage)(vertex, withWait) }
Future.sequence(futures)
@@ -1518,7 +1496,7 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends Graph
def incrementCounts(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[MutateResponse]] = {
- def incrementCounts(storage: Storage[_])(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[MutateResponse]] = {
+ def incrementCounts(storage: Storage)(edges: Seq[S2Edge], withWait: Boolean): Future[Seq[MutateResponse]] = {
val futures = for {
edge <- edges
} yield {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55d194ed/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
index 924d9a3..775afda 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/SKeyValue.scala
@@ -56,18 +56,17 @@ trait CanSKeyValue[T] {
}
object CanSKeyValue {
+ def instance[T](f: T => SKeyValue): CanSKeyValue[T] = new CanSKeyValue[T] {
+ override def toSKeyValue(from: T): SKeyValue = f.apply(from)
+ }
// For asyncbase KeyValues
- implicit val asyncKeyValue = new CanSKeyValue[KeyValue] {
- def toSKeyValue(kv: KeyValue): SKeyValue = {
- SKeyValue(Array.empty[Byte], kv.key(), kv.family(), kv.qualifier(), kv.value(), kv.timestamp())
- }
+ implicit val asyncKeyValue = instance[KeyValue] { kv =>
+ SKeyValue(Array.empty[Byte], kv.key(), kv.family(), kv.qualifier(), kv.value(), kv.timestamp())
}
// For asyncbase KeyValues
- implicit val sKeyValue = new CanSKeyValue[SKeyValue] {
- def toSKeyValue(kv: SKeyValue): SKeyValue = kv
- }
+ implicit val sKeyValue = instance[SKeyValue](identity)
// For hbase KeyValues
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55d194ed/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 2fe6e42..e4eafbf 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
@@ -27,7 +27,7 @@ import org.apache.s2graph.core.storage.serde.indexedge.tall.IndexEdgeDeserializa
import org.apache.s2graph.core.types._
import scala.concurrent.{ExecutionContext, Future}
-abstract class Storage[Q](val graph: S2Graph,
+abstract class Storage(val graph: S2Graph,
val config: Config) {
/* Storage backend specific resource management */
val management: StorageManagement
@@ -39,7 +39,7 @@ abstract class Storage[Q](val graph: S2Graph,
* Given QueryRequest/Vertex/Edge, fetch KeyValue from storage
* then convert them into Edge/Vertex
*/
- val fetcher: StorageReadable[Q]
+ val fetcher: StorageReadable
/*
* Serialize Edge/Vertex, to common KeyValue, SKeyValue that
@@ -61,7 +61,6 @@ abstract class Storage[Q](val graph: S2Graph,
*/
lazy val conflictResolver: WriteWriteConflictResolver = new WriteWriteConflictResolver(graph, serDe, io, mutator, fetcher)
-
/** IO **/
def snapshotEdgeSerializer(snapshotEdge: SnapshotEdge): serde.Serializable[SnapshotEdge] =
serDe.snapshotEdgeSerializer(snapshotEdge)
@@ -115,22 +114,18 @@ abstract class Storage[Q](val graph: S2Graph,
mutator.writeLock(requestKeyValue, expectedOpt)
/** Fetch **/
-
- def buildRequest(queryRequest: QueryRequest, edge: S2Edge): Q = fetcher.buildRequest(queryRequest, edge)
-
- def buildRequest(queryRequest: QueryRequest, vertex: S2Vertex): Q = fetcher.buildRequest(queryRequest, vertex)
-
def fetches(queryRequests: Seq[QueryRequest],
prevStepEdges: Map[VertexId, Seq[EdgeWithScore]])(implicit ec: ExecutionContext): Future[Seq[StepResult]] =
fetcher.fetches(queryRequests, prevStepEdges)
- def fetchKeyValues(rpc: Q)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]] = fetcher.fetchKeyValues(rpc)
+ def fetchVertices(vertices: Seq[S2Vertex])(implicit ec: ExecutionContext): Future[Seq[S2Vertex]] =
+ fetcher.fetchVertices(vertices)
def fetchEdgesAll()(implicit ec: ExecutionContext): Future[Seq[S2Edge]] = fetcher.fetchEdgesAll()
def fetchVerticesAll()(implicit ec: ExecutionContext): Future[Seq[S2Vertex]] = fetcher.fetchVerticesAll()
- def fetchSnapshotEdgeInner(edge: S2Edge)(implicit ec: ExecutionContext): Future[(QueryParam, Option[S2Edge], Option[SKeyValue])] =
+ def fetchSnapshotEdgeInner(edge: S2Edge)(implicit ec: ExecutionContext): Future[(Option[S2Edge], Option[SKeyValue])] =
fetcher.fetchSnapshotEdgeInner(edge)
/** Conflict Resolver **/
@@ -149,6 +144,5 @@ abstract class Storage[Q](val graph: S2Graph,
def shutdown(): Unit = management.shutdown()
-
def info: Map[String, String] = Map("className" -> this.getClass.getSimpleName)
}
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55d194ed/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
index 7a0d8ef..03b01fd 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageReadable.scala
@@ -26,21 +26,9 @@ import org.apache.s2graph.core.utils.logger
import scala.concurrent.{ExecutionContext, Future}
-trait StorageReadable[Q] {
+trait StorageReadable {
val io: StorageIO
- /**
- * build proper request which is specific into storage to call fetchIndexEdgeKeyValues or fetchSnapshotEdgeKeyValues.
- * for example, Asynchbase use GetRequest, Scanner so this method is responsible to build
- * client request(GetRequest, Scanner) based on user provided query.
- *
- * @param queryRequest
- * @return
- */
- def buildRequest(queryRequest: QueryRequest, edge: S2Edge): Q
-
- def buildRequest(queryRequest: QueryRequest, vertex: S2Vertex): Q
-
- /**
+ /**
* responsible to fire parallel fetch call into storage and create future that will return merged result.
*
* @param queryRequests
@@ -50,13 +38,18 @@ trait StorageReadable[Q] {
def fetches(queryRequests: Seq[QueryRequest],
prevStepEdges: Map[VertexId, Seq[EdgeWithScore]])(implicit ec: ExecutionContext): Future[Seq[StepResult]]
- def fetchKeyValues(rpc: Q)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]]
+// private def fetchKeyValues(rpc: Q)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]]
+ def fetchVertices(vertices: Seq[S2Vertex])(implicit ec: ExecutionContext): Future[Seq[S2Vertex]]
def fetchEdgesAll()(implicit ec: ExecutionContext): Future[Seq[S2Edge]]
def fetchVerticesAll()(implicit ec: ExecutionContext): Future[Seq[S2Vertex]]
- def fetchSnapshotEdgeInner(edge: S2Edge)(implicit ec: ExecutionContext): Future[(QueryParam, Option[S2Edge], Option[SKeyValue])] = {
+ protected def fetchKeyValues(queryRequest: QueryRequest, edge: S2Edge)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]]
+
+ protected def fetchKeyValues(queryRequest: QueryRequest, vertex: S2Vertex)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]]
+
+ def fetchSnapshotEdgeInner(edge: S2Edge)(implicit ec: ExecutionContext): Future[(Option[S2Edge], Option[SKeyValue])] = {
val queryParam = QueryParam(labelName = edge.innerLabel.label,
direction = GraphUtil.fromDirection(edge.labelWithDir.dir),
tgtVertexIdOpt = Option(edge.tgtVertex.innerIdVal),
@@ -64,15 +57,16 @@ trait StorageReadable[Q] {
val q = Query.toQuery(Seq(edge.srcVertex), Seq(queryParam))
val queryRequest = QueryRequest(q, 0, edge.srcVertex, queryParam)
- fetchKeyValues(buildRequest(queryRequest, edge)).map { kvs =>
+ fetchKeyValues(queryRequest, edge).map { kvs =>
val (edgeOpt, kvOpt) =
if (kvs.isEmpty) (None, None)
else {
+ import CanSKeyValue._
val snapshotEdgeOpt = io.toSnapshotEdge(kvs.head, queryRequest, isInnerCall = true, parentEdges = Nil)
val _kvOpt = kvs.headOption
(snapshotEdgeOpt, _kvOpt)
}
- (queryParam, edgeOpt, kvOpt)
+ (edgeOpt, kvOpt)
} recoverWith { case ex: Throwable =>
logger.error(s"fetchQueryParam failed. fallback return.", ex)
throw new FetchTimeoutException(s"${edge.toLogString}")
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55d194ed/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
index 79b764d..227cfa7 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/WriteWriteConflictResolver.scala
@@ -32,7 +32,7 @@ class WriteWriteConflictResolver(graph: S2Graph,
serDe: StorageSerDe,
io: StorageIO,
mutator: StorageWritable,
- fetcher: StorageReadable[_]) {
+ fetcher: StorageReadable) {
val BackoffTimeout = graph.BackoffTimeout
val MaxRetryNum = graph.MaxRetryNum
@@ -69,7 +69,7 @@ class WriteWriteConflictResolver(graph: S2Graph,
case FetchTimeoutException(retryEdge) =>
logger.info(s"[Try: $tryNum], Fetch fail.\n${retryEdge}")
/* fetch failed. re-fetch should be done */
- fetcher.fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+ fetcher.fetchSnapshotEdgeInner(edges.head).flatMap { case (snapshotEdgeOpt, kvOpt) =>
retry(tryNum + 1)(edges, statusCode, snapshotEdgeOpt)
}
@@ -91,7 +91,7 @@ class WriteWriteConflictResolver(graph: S2Graph,
val future = if (failedStatusCode == 0) {
// acquire Lock failed. other is mutating so this thead need to re-fetch snapshotEdge.
/* fetch failed. re-fetch should be done */
- fetcher.fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+ fetcher.fetchSnapshotEdgeInner(edges.head).flatMap { case (snapshotEdgeOpt, kvOpt) =>
retry(tryNum + 1)(edges, statusCode, snapshotEdgeOpt)
}
} else {
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55d194ed/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 54007d5..ef1350a 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
@@ -27,7 +27,6 @@ import com.typesafe.config.Config
import org.apache.commons.io.FileUtils
import org.apache.s2graph.core._
import org.apache.s2graph.core.storage._
-import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage.AsyncRPC
import org.apache.s2graph.core.utils._
import org.hbase.async._
import org.apache.s2graph.core.storage.serde._
@@ -140,7 +139,7 @@ object AsynchbaseStorage {
class AsynchbaseStorage(override val graph: S2Graph,
- override val config: Config) extends Storage[AsyncRPC](graph, config) {
+ override val config: Config) extends Storage(graph, config) {
/**
* since some runtime environment such as spark cluster has issue with guava version, that is used in Asynchbase.
@@ -156,7 +155,7 @@ class AsynchbaseStorage(override val graph: S2Graph,
override val serDe: StorageSerDe = new AsynchbaseStorageSerDe(graph)
- override val fetcher: StorageReadable[AsyncRPC] = new AsynchbaseStorageReadable(graph, config, client, serDe, io)
+ override val fetcher: StorageReadable = new AsynchbaseStorageReadable(graph, config, client, serDe, io)
// val hbaseExecutor: ExecutorService =
// if (config.getString("hbase.zookeeper.quorum") == "localhost")
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55d194ed/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
index 1cb6109..0dc8491 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseStorageReadable.scala
@@ -14,7 +14,7 @@
* "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
@@ -42,7 +42,7 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
val config: Config,
val client: HBaseClient,
val serDe: StorageSerDe,
- override val io: StorageIO) extends StorageReadable[AsyncRPC] {
+ override val io: StorageIO) extends StorageReadable {
import Extensions.DeferOps
import CanDefer._
@@ -67,7 +67,7 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
* @param queryRequest
* @return
*/
- override def buildRequest(queryRequest: QueryRequest, edge: S2Edge) = {
+ private def buildRequest(queryRequest: QueryRequest, edge: S2Edge) = {
import Serializable._
val queryParam = queryRequest.queryParam
val label = queryParam.label
@@ -168,15 +168,26 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
* @param vertex
* @return
*/
- override def buildRequest(queryRequest: QueryRequest, vertex: S2Vertex) = {
+ private def buildRequest(queryRequest: QueryRequest, vertex: S2Vertex) = {
val kvs = serDe.vertexSerializer(vertex).toKeyValues
val get = new GetRequest(vertex.hbaseTableName.getBytes, kvs.head.row, Serializable.vertexCf)
// get.setTimeout(this.singleGetTimeout.toShort)
get.setFailfast(true)
get.maxVersions(1)
+
Left(get)
}
+ override def fetchKeyValues(queryRequest: QueryRequest, edge: S2Edge)(implicit ec: ExecutionContext) = {
+ val rpc = buildRequest(queryRequest, edge)
+ fetchKeyValues(rpc)
+ }
+
+ override def fetchKeyValues(queryRequest: QueryRequest, vertex: S2Vertex)(implicit ec: ExecutionContext) = {
+ val rpc = buildRequest(queryRequest, vertex)
+ fetchKeyValues(rpc)
+ }
+
/**
* responsible to fire parallel fetch call into storage and create future that will return merged result.
*
@@ -201,7 +212,7 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
}.toFuture(emptyStepResult)
}
- override def fetchKeyValues(rpc: AsyncRPC)(implicit ec: ExecutionContext) = {
+ def fetchKeyValues(rpc: AsyncRPC)(implicit ec: ExecutionContext) = {
val defer = fetchKeyValuesInner(rpc)
defer.toFuture(emptyKeyValues).map { kvsArr =>
kvsArr.map { kv =>
@@ -224,7 +235,8 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
kvs.flatMap { kv =>
val sKV = implicitly[CanSKeyValue[KeyValue]].toSKeyValue(kv)
- serDe.indexEdgeDeserializer(schemaVer = HBaseType.DEFAULT_VERSION).fromKeyValues(Seq(kv), None)
+ serDe.indexEdgeDeserializer(schemaVer = HBaseType.DEFAULT_VERSION)
+ .fromKeyValues(Seq(kv), None)
.filter(e => distinctLabels(e.innerLabel) && e.direction == "out" && !e.isDegree)
}
}
@@ -234,6 +246,27 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
Future.sequence(futures).map(_.flatten)
}
+ override def fetchVertices(vertices: Seq[S2Vertex])(implicit ec: ExecutionContext) = {
+ def fromResult(kvs: Seq[SKeyValue], version: String): Seq[S2Vertex] = {
+ if (kvs.isEmpty) Nil
+ else serDe.vertexDeserializer(version).fromKeyValues(kvs, None).toSeq
+ }
+
+ val futures = vertices.map { vertex =>
+ val queryParam = QueryParam.Empty
+ val q = Query.toQuery(Seq(vertex), Seq(queryParam))
+ val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
+
+ fetchKeyValues(queryRequest, vertex).map { kvs =>
+ fromResult(kvs, vertex.serviceColumn.schemaVersion)
+ } recoverWith {
+ case ex: Throwable => Future.successful(Nil)
+ }
+ }
+
+ Future.sequence(futures).map(_.flatten)
+ }
+
override def fetchVerticesAll()(implicit ec: ExecutionContext) = {
val futures = ServiceColumn.findAll().groupBy(_.service.hTableName).toSeq.map { case (hTableName, columns) =>
val distinctColumns = columns.toSet
@@ -351,4 +384,5 @@ class AsynchbaseStorageReadable(val graph: S2Graph,
throw new RuntimeException(s"toCacheKeyBytes: $hbaseRpc")
}
}
+
}