You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@s2graph.apache.org by st...@apache.org on 2016/11/16 16:33:34 UTC

[1/7] incubator-s2graph git commit: [S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

Repository: incubator-s2graph
Updated Branches:
  refs/heads/master 8dbb9a3ee -> 66bdf1bc0


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 ea90061..8e80fd9 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
@@ -19,15 +19,16 @@
 
 package org.apache.s2graph.core.storage.hbase
 
-import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
+import org.apache.s2graph.core.mysqls.{Model, Label, LabelIndex, LabelMeta}
 import org.apache.s2graph.core.types._
-import org.apache.s2graph.core.{IndexEdge, TestCommonWithModels, Vertex}
+import org.apache.s2graph.core.{QueryParam, IndexEdge, TestCommonWithModels, Vertex}
 import org.scalatest.{FunSuite, Matchers}
 
 
 class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
   initTests()
 
+  val testLabelMeta = LabelMeta(Option(-1), labelV2.id.get, "test", 1.toByte, "0.0", "double")
   /**
    * check if storage serializer/deserializer can translate from/to bytes array.
    * @param l: label for edge.
@@ -35,17 +36,19 @@ class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
    * @param to: to VertexId for edge.
    * @param props: expected props of edge.
    */
-  def check(l: Label, ts: Long, to: InnerValLike, props: Map[Byte, InnerValLikeWithTs]): Unit = {
+  def check(l: Label, ts: Long, to: InnerValLike, props: Map[LabelMeta, InnerValLikeWithTs]): Unit = {
     val from = InnerVal.withLong(1, l.schemaVersion)
     val vertexId = SourceVertexId(HBaseType.DEFAULT_COL_ID, from)
     val tgtVertexId = TargetVertexId(HBaseType.DEFAULT_COL_ID, to)
     val vertex = Vertex(vertexId, ts)
     val tgtVertex = Vertex(tgtVertexId, ts)
     val labelWithDir = LabelWithDirection(l.id.get, 0)
+    val labelOpt = Option(l)
+
+    val indexEdge = IndexEdge(vertex, tgtVertex, l, labelWithDir.dir, 0, ts, LabelIndex.DefaultSeq, props, tsInnerValOpt = Option(InnerVal.withLong(ts, l.schemaVersion)))
+    val _indexEdgeOpt = graph.getStorage(l).indexEdgeDeserializer(l.schemaVersion).fromKeyValues(labelOpt,
+      graph.getStorage(l).indexEdgeSerializer(indexEdge).toKeyValues, l.schemaVersion, None)
 
-    val indexEdge = IndexEdge(vertex, tgtVertex, labelWithDir, 0, ts, LabelIndex.DefaultSeq, props)
-    val _indexEdgeOpt = graph.storage.indexEdgeDeserializer(l.schemaVersion).fromKeyValues(queryParam,
-      graph.storage.indexEdgeSerializer(indexEdge).toKeyValues, l.schemaVersion, None)
 
     _indexEdgeOpt should not be empty
     indexEdge should be(_indexEdgeOpt.get)
@@ -60,8 +63,8 @@ class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
     } {
       val to = InnerVal.withLong(101, l.schemaVersion)
       val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
-      val props = Map(LabelMeta.timeStampSeq -> tsInnerValWithTs,
-        1.toByte -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion))
+      val props = Map(LabelMeta.timestamp -> tsInnerValWithTs,
+        testLabelMeta -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion))
 
       check(l, ts, to, props)
     }
@@ -75,8 +78,8 @@ class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
       val to = InnerVal.withStr("0", l.schemaVersion)
       val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
       val props = Map(
-        LabelMeta.degreeSeq -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion),
-        LabelMeta.timeStampSeq -> tsInnerValWithTs)
+        LabelMeta.degree -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion),
+        LabelMeta.timestamp -> tsInnerValWithTs)
 
       check(l, ts, to, props)
     }
@@ -89,12 +92,12 @@ class IndexEdgeTest extends FunSuite with Matchers with TestCommonWithModels {
     } {
       val to = InnerVal.withLong(101, l.schemaVersion)
       val tsInnerValWithTs = InnerValLikeWithTs.withLong(ts, ts, l.schemaVersion)
-      val props = Map(LabelMeta.timeStampSeq -> tsInnerValWithTs,
-        1.toByte -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion),
-        LabelMeta.countSeq -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion))
+      val props = Map(LabelMeta.timestamp -> tsInnerValWithTs,
+        testLabelMeta -> InnerValLikeWithTs.withDouble(2.1, ts, l.schemaVersion),
+        LabelMeta.count -> InnerValLikeWithTs.withLong(10, ts, l.schemaVersion))
 
 
       check(l, ts, to, props)
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
----------------------------------------------------------------------
diff --git a/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala b/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
index c8f65bf..a6f8f5c 100644
--- a/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
+++ b/s2rest_netty/src/main/scala/org/apache/s2graph/rest/netty/Server.scala
@@ -33,20 +33,19 @@ import io.netty.handler.codec.http.HttpHeaders._
 import io.netty.handler.codec.http._
 import io.netty.handler.logging.{LogLevel, LoggingHandler}
 import io.netty.util.CharsetUtil
-import org.apache.s2graph.core.GraphExceptions.BadQueryException
+import org.apache.s2graph.core.GraphExceptions.{BadQueryException}
 import org.apache.s2graph.core.mysqls.Experiment
 import org.apache.s2graph.core.rest.RestHandler
 import org.apache.s2graph.core.rest.RestHandler.{CanLookup, HandlerResult}
 import org.apache.s2graph.core.utils.Extensions._
 import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{Graph, JSONParser, PostProcess}
+import org.apache.s2graph.core.{Graph, PostProcess}
 import play.api.libs.json._
 
 import scala.collection.mutable
 import scala.concurrent.{ExecutionContext, Future}
 import scala.io.Source
 import scala.util.{Failure, Success, Try}
-import scala.language.existentials
 
 class S2RestHandler(s2rest: RestHandler)(implicit ec: ExecutionContext) extends SimpleChannelInboundHandler[FullHttpRequest] {
   val ApplicationJson = "application/json"
@@ -101,8 +100,9 @@ class S2RestHandler(s2rest: RestHandler)(implicit ec: ExecutionContext) extends
     result.body onComplete {
       case Success(json) =>
         val duration = System.currentTimeMillis() - startedAt
+        val bucketName = result.headers.toMap.get(Experiment.ImpressionKey).getOrElse("")
 
-        val log = s"${req.getMethod} ${req.getUri} took ${duration} ms 200 ${s2rest.calcSize(json)} ${requestBody}"
+        val log = s"${req.getMethod} ${req.getUri} took ${duration} ms 200 ${s2rest.calcSize(json)} ${requestBody} ${bucketName}"
         logger.info(log)
 
         val buf: ByteBuf = Unpooled.copiedBuffer(json.toString, CharsetUtil.UTF_8)
@@ -114,7 +114,7 @@ class S2RestHandler(s2rest: RestHandler)(implicit ec: ExecutionContext) extends
         case e: BadQueryException =>
           logger.error(s"{$requestBody}, ${e.getMessage}", e)
           val buf: ByteBuf = Unpooled.copiedBuffer(PostProcess.badRequestResults(e).toString, CharsetUtil.UTF_8)
-          simpleResponse(ctx, Ok, byteBufOpt = Option(buf), channelFutureListenerOpt = CloseOpt, headers = headers.result())
+          simpleResponse(ctx, BadRequest, byteBufOpt = Option(buf), channelFutureListenerOpt = CloseOpt, headers = headers.result())
         case e: Exception =>
           logger.error(s"${requestBody}, ${e.getMessage}", e)
           val buf: ByteBuf = Unpooled.copiedBuffer(PostProcess.emptyResults.toString, CharsetUtil.UTF_8)
@@ -204,7 +204,9 @@ class S2RestHandler(s2rest: RestHandler)(implicit ec: ExecutionContext) extends
 }
 
 // Simple http server
-object NettyServer {
+object NettyServer extends App {
+  /** should be same with Boostrap.onStart on play */
+
   val numOfThread = Runtime.getRuntime.availableProcessors()
   val threadPool = Executors.newFixedThreadPool(numOfThread)
   val ec = ExecutionContext.fromExecutor(threadPool)
@@ -234,31 +236,30 @@ object NettyServer {
       (new NioEventLoopGroup(1), new NioEventLoopGroup(), classOf[NioServerSocketChannel])
   }
 
-  def main(args : scala.Array[scala.Predef.String]) : scala.Unit = {
-    try {
-      val b: ServerBootstrap = new ServerBootstrap()
-        .option(ChannelOption.SO_BACKLOG, Int.box(2048))
-      b.group(bossGroup, workerGroup).channel(channelClass)
-        .handler(new LoggingHandler(LogLevel.INFO))
-        .childHandler(new ChannelInitializer[SocketChannel] {
-          override def initChannel(ch: SocketChannel) {
-            val p = ch.pipeline()
-            p.addLast(new HttpServerCodec())
-            p.addLast(new HttpObjectAggregator(maxBodySize))
-            p.addLast(new S2RestHandler(rest)(ec))
-          }
-        })
-
-      // for check server is started
-      logger.info(s"Listening for HTTP on /0.0.0.0:$port")
-      val ch: Channel = b.bind(port).sync().channel()
-      ch.closeFuture().sync()
-
-    } finally {
-      bossGroup.shutdownGracefully()
-      workerGroup.shutdownGracefully()
-      s2graph.shutdown()
-    }
+  try {
+    val b: ServerBootstrap = new ServerBootstrap()
+      .option(ChannelOption.SO_BACKLOG, Int.box(2048))
+
+    b.group(bossGroup, workerGroup).channel(channelClass)
+      .handler(new LoggingHandler(LogLevel.INFO))
+      .childHandler(new ChannelInitializer[SocketChannel] {
+        override def initChannel(ch: SocketChannel) {
+          val p = ch.pipeline()
+          p.addLast(new HttpServerCodec())
+          p.addLast(new HttpObjectAggregator(maxBodySize))
+          p.addLast(new S2RestHandler(rest)(ec))
+        }
+      })
+
+    // for check server is started
+    logger.info(s"Listening for HTTP on /0.0.0.0:$port")
+    val ch: Channel = b.bind(port).sync().channel()
+    ch.closeFuture().sync()
+
+  } finally {
+    bossGroup.shutdownGracefully()
+    workerGroup.shutdownGracefully()
+    s2graph.shutdown()
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2rest_play/app/org/apache/s2graph/rest/play/controllers/AdminController.scala
----------------------------------------------------------------------
diff --git a/s2rest_play/app/org/apache/s2graph/rest/play/controllers/AdminController.scala b/s2rest_play/app/org/apache/s2graph/rest/play/controllers/AdminController.scala
index 36610e0..087e12e 100644
--- a/s2rest_play/app/org/apache/s2graph/rest/play/controllers/AdminController.scala
+++ b/s2rest_play/app/org/apache/s2graph/rest/play/controllers/AdminController.scala
@@ -448,7 +448,7 @@ object AdminController extends Controller {
     //    Management.createTable(cluster, hTableName, List("e", "v"), preSplitSize, hTableTTL, compressionAlgorithm)
     request.body.asJson.map(_.validate[HTableParams] match {
       case JsSuccess(hTableParams, _) => {
-        management.createTable(hTableParams.cluster, hTableParams.hTableName, List("e", "v"),
+        management.createStorageTable(hTableParams.cluster, hTableParams.hTableName, List("e", "v"),
           hTableParams.preSplitSize, hTableParams.hTableTTL,
           hTableParams.compressionAlgorithm.getOrElse(Management.DefaultCompressionAlgorithm))
         logger.info(hTableParams.toString())

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 8000cf8..835cc72 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
@@ -20,6 +20,7 @@
 package org.apache.s2graph.rest.play.controllers
 
 import com.fasterxml.jackson.databind.JsonMappingException
+import org.apache.s2graph.core.ExceptionHandler.KafkaMessage
 import org.apache.s2graph.core._
 import org.apache.s2graph.core.mysqls.Label
 import org.apache.s2graph.core.rest.RequestParser
@@ -35,7 +36,6 @@ import scala.concurrent.Future
 object EdgeController extends Controller {
 
   import ApplicationController._
-  import ExceptionHandler._
   import play.api.libs.concurrent.Execution.Implicits._
 
   private val s2: Graph = org.apache.s2graph.rest.play.Global.s2graph
@@ -232,8 +232,8 @@ object EdgeController extends Controller {
     else {
 
       s2.incrementCounts(edges, withWait = true).map { results =>
-        val json = results.map { case (isSuccess, resultCount) =>
-          Json.obj("success" -> isSuccess, "result" -> resultCount)
+        val json = results.map { case (isSuccess, resultCount, count) =>
+          Json.obj("success" -> isSuccess, "result" -> resultCount, "_count" -> count)
         }
 
         jsonResponse(Json.toJson(json))


[2/7] incubator-s2graph git commit: [S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala b/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
index d60641f..1c58086 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/types/InnerValLike.scala
@@ -20,6 +20,7 @@
 package org.apache.s2graph.core.types
 
 import org.apache.hadoop.hbase.util._
+import org.apache.s2graph.core.utils.logger
 
 object InnerVal extends HBaseDeserializableWithIsVertexId {
   import HBaseType._
@@ -39,7 +40,7 @@ object InnerVal extends HBaseDeserializableWithIsVertexId {
   val INT = "integer"
   val SHORT = "short"
   val BYTE = "byte"
-  val NUMERICS = List(DOUBLE, FLOAT, LONG, INT, SHORT, BYTE)
+  val NUMERICS = Set(DOUBLE, FLOAT, LONG, INT, SHORT, BYTE)
   val BOOLEAN = "boolean"
 
   def isNumericType(dataType: String): Boolean = {
@@ -218,15 +219,11 @@ trait InnerValLike extends HBaseSerializable {
 
   override def hashCode(): Int = value.hashCode()
 
-  override def equals(obj: Any): Boolean = {
-    obj match {
-      case other: InnerValLike =>
-        val ret = toString == obj.toString
-//        logger.debug(s"InnerValLike.equals($this, $obj) => $ret")
-        ret
-      case _ => false
-    }
+  override def equals(obj: Any): Boolean = obj match {
+    case other: InnerValLike => value == other.value
+    case _ => false
   }
+
   def hashKey(dataType: String): Int
 
   def toIdString(): String

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 96f87ed..9bb99ed 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
@@ -117,7 +117,7 @@ class DeferCache[A, M[_], C[_]](config: Config, empty: => A, name: String = "",
       DeferCache.addScheduleJob(delay = metricInterval) { logger.metric(s"${name}: ${cache.stats()}") }
       cache
     } else {
-      builder.build[java.lang.Long, (Long, M[A])]()
+      builder.recordStats().build[java.lang.Long, (Long, M[A])]()
     }
   }
 
@@ -194,4 +194,7 @@ class DeferCache[A, M[_], C[_]](config: Config, empty: => A, name: String = "",
         checkAndExpire(cacheKey, cacheTTL, cachedAt, canDefer.future(cachedPromise))(op)
     }
   }
+
+  def stats = futureCache.stats()
+  def size = futureCache.size()
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/utils/Extentions.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/utils/Extentions.scala b/s2core/src/main/scala/org/apache/s2graph/core/utils/Extentions.scala
index 8ffd1dc..da5f635 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/utils/Extentions.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/utils/Extentions.scala
@@ -26,7 +26,6 @@ import scala.concurrent.{ExecutionContext, Future, Promise}
 
 object Extensions {
 
-
   def retryOnSuccess[T](maxRetryNum: Int, n: Int = 1)(fn: => Future[T])(shouldStop: T => Boolean)(implicit ex: ExecutionContext): Future[T] = n match {
     case i if n <= maxRetryNum =>
       fn.flatMap { result =>
@@ -52,37 +51,67 @@ object Extensions {
 
 
   implicit class DeferOps[T](d: Deferred[T])(implicit ex: ExecutionContext) {
+    def map[R](dummy: => T)(op: T => R): Deferred[R] = {
+      val newDefer = new Deferred[R]
 
-    def withCallback[R](op: T => R): Deferred[R] = {
-      d.addCallback(new Callback[R, T] {
-        override def call(arg: T): R = op(arg)
+      d.addCallback(new Callback[T, T] {
+        override def call(arg: T): T = {
+          newDefer.callback(op(arg))
+          arg
+        }
       })
-    }
 
-    def recoverWith(op: Exception => T): Deferred[T] = {
-      d.addErrback(new Callback[Deferred[T], Exception] {
-        override def call(e: Exception): Deferred[T] = Deferred.fromResult(op(e))
+      d.addErrback(new Callback[T, Exception] {
+        override def call(e: Exception): T = {
+          newDefer.callback(e)
+          dummy
+        }
       })
+
+      newDefer
     }
 
+    def mapWithFallback[R](dummy: => T)(fallback: Exception => R)(op: T => R): Deferred[R] = {
+      val newDefer = new Deferred[R]
 
-    def toFuture: Future[T] = {
-      val promise = Promise[T]
+      d.addCallback(new Callback[T, T] {
+        override def call(arg: T): T = {
+          newDefer.callback(op(arg))
+          arg
+        }
+      })
 
-      d.addBoth(new Callback[Unit, T] {
-        def call(arg: T) = arg match {
-          case e: Exception => promise.failure(e)
-          case _ => promise.success(arg)
+      d.addErrback(new Callback[T, Exception] {
+        override def call(e: Exception): T = {
+          newDefer.callback(fallback(e))
+          dummy
         }
       })
 
-      promise.future
+      newDefer
     }
 
-    def toFutureWith(fallback: => T): Future[T] = {
-      toFuture recoverWith { case t: Throwable => Future.successful(fallback) }
-    }
+    def toFuture(dummy: => T): Future[T] = {
+      val promise = Promise[T]
+
+      val cb = new Callback[T, T] {
+        override def call(arg: T): T = {
+          promise.success(arg)
+          arg
+        }
+      }
+
+      val eb = new Callback[T, Exception] {
+        override def call(e: Exception): T = {
+          promise.failure(e)
+          dummy
+        }
+      }
+
+      d.addCallbacks(cb, eb)
 
+      promise.future
+    }
   }
 
   implicit class ConfigOps(config: Config) {

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/utils/SafeUpdateCache.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/utils/SafeUpdateCache.scala b/s2core/src/main/scala/org/apache/s2graph/core/utils/SafeUpdateCache.scala
index 0bc4554..13eb1a3 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/utils/SafeUpdateCache.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/utils/SafeUpdateCache.scala
@@ -20,11 +20,10 @@
 package org.apache.s2graph.core.utils
 
 import java.util.concurrent.atomic.AtomicBoolean
-
 import com.google.common.cache.CacheBuilder
-
 import scala.concurrent.{ExecutionContext, Future}
 import scala.util.{Failure, Success}
+import scala.collection.JavaConversions._
 
 object SafeUpdateCache {
 
@@ -77,5 +76,11 @@ class SafeUpdateCache[T](prefix: String, maxSize: Int, ttl: Int)(implicit execut
       }
     }
   }
+
+  def getAllData() : List[(String, T)] = {
+    cache.asMap().map { case (key, value) =>
+      (key.key.substring(prefix.size + 1), value._1)
+    }.toList
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala
index 6933320..6321fef 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/EdgeTest.scala
@@ -29,7 +29,11 @@ import play.api.libs.json.{JsObject, Json}
 class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
   initTests()
 
+  val testLabelMeta1 = LabelMeta(Option(-1), labelV2.id.get, "test", 1.toByte, "true", "boolean")
+  val testLabelMeta3 = LabelMeta(Option(-1), labelV2.id.get, "test", 3.toByte, "-1", "long")
+
   test("toLogString") {
+    val testServiceName = serviceNameV2
     val testLabelName = labelNameV2
     val bulkQueries = List(
       ("1445240543366", "update", "{\"is_blocked\":true}"),
@@ -45,12 +49,14 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
       Edge.toEdge(srcId, tgtId, labelName, "out", properties, ts.toLong, op).toLogString
     }).mkString("\n")
 
+    val attachedProps = "\"from\":\"1\",\"to\":\"2\",\"label\":\"" + testLabelName +
+      "\",\"service\":\"" + testServiceName + "\""
     val expected = Seq(
-      Seq("1445240543366", "update", "e", "1", "2", testLabelName, "{\"is_blocked\":true}"),
-      Seq("1445240543362", "insert", "e", "1", "2", testLabelName, "{\"is_hidden\":false}"),
-      Seq("1445240543364", "insert", "e", "1", "2", testLabelName, "{\"is_hidden\":false,\"weight\":10}"),
-      Seq("1445240543363", "delete", "e", "1", "2", testLabelName),
-      Seq("1445240543365", "update", "e", "1", "2", testLabelName, "{\"time\":1,\"weight\":-10}")
+      Seq("1445240543366", "update", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_blocked\":true}"),
+      Seq("1445240543362", "insert", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_hidden\":false}"),
+      Seq("1445240543364", "insert", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"is_hidden\":false,\"weight\":10}"),
+      Seq("1445240543363", "delete", "e", "1", "2", testLabelName, "{" + attachedProps + "}"),
+      Seq("1445240543365", "update", "e", "1", "2", testLabelName, "{" + attachedProps + ",\"time\":1,\"weight\":-10}")
     ).map(_.mkString("\t")).mkString("\n")
 
     assert(bulkEdge === expected)
@@ -62,16 +68,16 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
     val srcVertex = Vertex(vertexId)
     val tgtVertex = srcVertex
 
-    val timestampProp = LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+    val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
 
     val snapshotEdge = None
     val propsWithTs = Map(timestampProp)
-    val requestEdge = Edge(srcVertex, tgtVertex, labelWithDirV2, propsWithTs = propsWithTs)
+    val requestEdge = Edge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
     val newVersion = 0L
 
     val newPropsWithTs = Map(
       timestampProp,
-      1.toByte -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
+      testLabelMeta1 -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
     )
 
     val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
@@ -88,16 +94,16 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
     val srcVertex = Vertex(vertexId)
     val tgtVertex = srcVertex
 
-    val timestampProp = LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+    val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
 
     val snapshotEdge = None
     val propsWithTs = Map(timestampProp)
-    val requestEdge = Edge(srcVertex, tgtVertex, labelWithDirV2, propsWithTs = propsWithTs)
+    val requestEdge = Edge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
     val newVersion = 0L
 
     val newPropsWithTs = Map(
       timestampProp,
-      1.toByte -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
+      testLabelMeta1 -> InnerValLikeWithTs(InnerVal.withBoolean(false, schemaVersion), 1)
     )
 
     val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, propsWithTs, newPropsWithTs)
@@ -114,11 +120,11 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
     val srcVertex = Vertex(vertexId)
     val tgtVertex = srcVertex
 
-    val timestampProp = LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+    val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
 
     val snapshotEdge = None
     val propsWithTs = Map(timestampProp)
-    val requestEdge = Edge(srcVertex, tgtVertex, labelWithDirV2, propsWithTs = propsWithTs)
+    val requestEdge = Edge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
     val newVersion = 0L
 
     val newPropsWithTs = propsWithTs
@@ -137,7 +143,7 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
     val srcVertex = Vertex(vertexId)
     val tgtVertex = srcVertex
 
-    val timestampProp = LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+    val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
     val oldPropsWithTs = Map(
       timestampProp,
       LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
@@ -145,14 +151,14 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
 
     val propsWithTs = Map(
       timestampProp,
-      3.toByte -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 2),
+      testLabelMeta3 -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 2),
       LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
     )
 
     val snapshotEdge =
-      Option(Edge(srcVertex, tgtVertex, labelWithDirV2, op = GraphUtil.operations("delete"), propsWithTs = oldPropsWithTs))
+      Option(Edge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, op = GraphUtil.operations("delete"), propsWithTs = oldPropsWithTs))
 
-    val requestEdge = Edge(srcVertex, tgtVertex, labelWithDirV2, propsWithTs = propsWithTs)
+    val requestEdge = Edge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
     val newVersion = 0L
     val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
     logger.info(edgeMutate.toLogString)
@@ -168,7 +174,7 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
     val srcVertex = Vertex(vertexId)
     val tgtVertex = srcVertex
 
-    val timestampProp = LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
+    val timestampProp = LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 1)
     val oldPropsWithTs = Map(
       timestampProp,
       LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
@@ -176,14 +182,14 @@ class EdgeTest extends FunSuite with TestCommon with TestCommonWithModels {
 
     val propsWithTs = Map(
       timestampProp,
-      3.toByte -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 4),
+      testLabelMeta3 -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 4),
       LabelMeta.lastDeletedAt -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 3)
     )
 
     val snapshotEdge =
-      Option(Edge(srcVertex, tgtVertex, labelWithDirV2, op = GraphUtil.operations("delete"), propsWithTs = oldPropsWithTs))
+      Option(Edge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, op = GraphUtil.operations("delete"), propsWithTs = oldPropsWithTs))
 
-    val requestEdge = Edge(srcVertex, tgtVertex, labelWithDirV2, propsWithTs = propsWithTs)
+    val requestEdge = Edge(srcVertex, tgtVertex, labelV2, labelWithDirV2.dir, propsWithTs = propsWithTs)
     val newVersion = 0L
     val edgeMutate = Edge.buildMutation(snapshotEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
     logger.info(edgeMutate.toLogString)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 6054d67..a41152c 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
@@ -199,7 +199,7 @@ class CrudTest extends IntegrateCommon {
   object CrudHelper {
 
     class CrudTestRunner {
-      var seed = 0
+      var seed = System.currentTimeMillis()
 
       def run(tcNum: Int, tcString: String, opWithProps: List[(Long, String, String)], expected: Map[String, String]) = {
         for {
@@ -271,7 +271,7 @@ class CrudTest extends IntegrateCommon {
           val queryJson = querySnapshotEdgeJson(serviceName, columnName, labelName, id)
 
           if (!rets.forall(identity)) {
-            Thread.sleep(graph.storage.LockExpireDuration + 100)
+            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: _*)
@@ -296,7 +296,7 @@ class CrudTest extends IntegrateCommon {
           val queryJson = querySnapshotEdgeJson(serviceName, columnName, labelName, id)
 
           if (!rets.forall(identity)) {
-            Thread.sleep(graph.storage.LockExpireDuration + 100)
+            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: _*)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
index b341ec5..c84ad6e 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/IntegrateCommon.scala
@@ -23,7 +23,7 @@ import com.typesafe.config._
 import org.apache.s2graph.core.mysqls.Label
 import org.apache.s2graph.core.rest.{RequestParser, RestHandler}
 import org.apache.s2graph.core.utils.logger
-import org.apache.s2graph.core.{Graph, GraphUtil, Management, PostProcess}
+import org.apache.s2graph.core._
 import org.scalatest._
 import play.api.libs.json.{JsValue, Json}
 
@@ -90,7 +90,7 @@ trait IntegrateCommon extends FunSuite with Matchers with BeforeAndAfterAll {
       }
     }
 
-    val vertexPropsKeys = List("age" -> "int")
+    val vertexPropsKeys = List("age" -> "int", "im" -> "string")
 
     vertexPropsKeys.map { case (key, keyType) =>
       Management.addVertexProp(testServiceName, testColumnName, key, keyType)
@@ -129,10 +129,19 @@ trait IntegrateCommon extends FunSuite with Matchers with BeforeAndAfterAll {
       Await.result(future, HttpRequestWaitingTime)
     }
 
+    def getEdgesSync(s2Query: Query): JsValue = {
+      logger.info(s2Query.toString)
+      val stepResult = Await.result(graph.getEdges(s2Query), HttpRequestWaitingTime)
+      val result = PostProcess.toJson(Option(s2Query.jsonQuery))(graph, s2Query.queryOption, stepResult)
+//      val result = Await.result(graph.getEdges(s2Query).(PostProcess.toJson), HttpRequestWaitingTime)
+      logger.debug(s"${Json.prettyPrint(result)}")
+      result
+    }
+
     def getEdgesSync(queryJson: JsValue): JsValue = {
       logger.info(Json.prettyPrint(queryJson))
       val restHandler = new RestHandler(graph)
-      val result = Await.result(restHandler.getEdgesAsync(queryJson)(PostProcess.toJson), HttpRequestWaitingTime)
+      val result = Await.result(restHandler.getEdgesAsync(queryJson)(PostProcess.toJson(Option(queryJson))), HttpRequestWaitingTime)
       logger.debug(s"${Json.prettyPrint(result)}")
       result
     }
@@ -301,7 +310,10 @@ trait IntegrateCommon extends FunSuite with Matchers with BeforeAndAfterAll {
     "tgtServiceName": "$testServiceName",
     "tgtColumnName": "$testTgtColumnName",
     "tgtColumnType": "string",
-    "indices": [{"name": "$index1", "propNames": ["time", "weight", "is_hidden", "is_blocked"]}],
+    "indices": [
+      {"name": "$index1", "propNames": ["time", "weight", "is_hidden", "is_blocked"]},
+      {"name": "$index2", "propNames": ["time"]}
+    ],
     "props": [
     {
       "name": "time",

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/Integrate/QueryTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/QueryTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/QueryTest.scala
index 54bb12c..34f4d2c 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/QueryTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/QueryTest.scala
@@ -19,9 +19,11 @@
 
 package org.apache.s2graph.core.Integrate
 
+import org.apache.s2graph.core.parsers.Where
 import org.apache.s2graph.core.utils.logger
+import org.apache.s2graph.core._
 import org.scalatest.BeforeAndAfterEach
-import play.api.libs.json.{JsNumber, JsValue, Json}
+import play.api.libs.json._
 
 class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
 
@@ -32,27 +34,178 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
   val weight = "weight"
   val is_hidden = "is_hidden"
 
-  test("interval") {
-    def queryWithInterval(id: Int, index: String, prop: String, fromVal: Int, toVal: Int) = Json.parse(
-      s"""
-        { "srcVertices": [
-          { "serviceName": "$testServiceName",
-            "columnName": "$testColumnName",
-            "id": $id
-           }],
-          "steps": [
-          [ {
-              "label": "$testLabelName",
-              "index": "$index",
-              "interval": {
-                  "from": [ { "$prop": $fromVal } ],
-                  "to": [ { "$prop": $toVal } ]
+  def querySingle(id: Int, offset: Int = 0, limit: Int = 100) = Json.parse(
+    s"""
+          { "srcVertices": [
+            { "serviceName": "$testServiceName",
+              "columnName": "$testColumnName",
+              "id": $id
+             }],
+            "steps": [
+            [ {
+                "label": "$testLabelName",
+                "direction": "out",
+                "offset": $offset,
+                "limit": $limit
               }
-            }
-          ]]
-        }
-        """)
+            ]]
+          }
+          """)
+
+  def queryGlobalLimit(id: Int, limit: Int): JsValue = Json.obj(
+    "limit" -> limit,
+    "srcVertices" -> Json.arr(
+      Json.obj("serviceName" -> testServiceName, "columnName" -> testColumnName, "id" -> id)
+    ),
+    "steps" -> Json.arr(
+      Json.obj(
+        "step" -> Json.arr(
+          Json.obj(
+            "label" -> testLabelName
+          )
+        )
+      )
+    )
+  )
+
+  def getQuery(id: Int, where: String): Query =
+    Query(
+      vertices = Seq(Vertex.toVertex(testServiceName, testColumnName, id)),
+      steps = Vector(
+        Step(Seq(QueryParam(testLabelName, where = Where(testLabelName, where))))
+      )
+    )
+
+  def queryIntervalWithParent(id: Int, index: String, prop: String, value: String) =
+    Query(
+      vertices = Seq(Vertex.toVertex(testServiceName, testColumnName, id)),
+      steps = Vector(
+        Step(Seq(QueryParam(testLabelName, indexName = index))),
+        Step(Seq(QueryParam(testLabelName, indexName = index,
+          intervalOpt = Option(Seq(prop -> JsString(value)), Seq(prop -> JsString(value)))))
+        )
+      )
+    )
+
+  def queryIntervalWithParentRange(id: Int, index: String,
+                                   prop: String, value: String,
+                                   toProp: String, toValue: String) =
+    Query(
+      vertices = Seq(Vertex.toVertex(testServiceName, testColumnName, id)),
+      steps = Vector(
+        Step(Seq(QueryParam(testLabelName, indexName = index))),
+        Step(Seq(QueryParam(testLabelName, indexName = index,
+          intervalOpt = Option(Seq(prop -> JsString(value)), Seq(toProp -> JsString(toValue)))))
+        )
+      )
+    )
+
+  def queryWithInterval(id: Int, index: String, prop: String, fromVal: Int, toVal: Int) =
+    Query(
+      vertices = Seq(Vertex.toVertex(testServiceName, testColumnName, id)),
+      steps = Vector(
+        Step(Seq(QueryParam(testLabelName, indexName = index,
+          intervalOpt = Option(Seq(prop -> JsNumber(fromVal)), Seq(prop -> JsNumber(toVal))))))
+      )
+    )
+
+  def queryExclude(id: Int) =
+    Query(
+      vertices = Seq(Vertex.toVertex(testServiceName, testColumnName, id)),
+      steps = Vector(
+        Step(
+          Seq(
+            QueryParam(testLabelName, limit = 2),
+            QueryParam(testLabelName, direction = "in", limit = 2, exclude = true)
+          )
+        )
+      )
+    )
+
+  def queryGroupBy(id: Int, props: Seq[String]) =
+    Query(
+      vertices = Seq(Vertex.toVertex(testServiceName, testColumnName, id)),
+      steps = Vector(
+        Step(
+          Seq(QueryParam(testLabelName))
+        )
+      ),
+      queryOption = QueryOption(groupBy = GroupBy(props, 100))
+    )
+
+  test("query with defaultValue") {
+    // ref: edges from initTestData()
+
+    // no default value
+    var edges = getEdgesSync(getQuery(0, "_to = 1"))
+    (edges \\ "is_hidden").head.as[Boolean] should be(true)
+
+    // default value(weight, is_hidden)
+    edges = getEdgesSync(getQuery(-1, "_to = 1000"))
+    (edges \\ "is_hidden").head.as[Boolean] should be(false)
+    (edges \\ "weight").head.as[Long] should be(0)
+
+    // default value(is_hidden)
+    edges = getEdgesSync(getQuery(10, "_to = 20"))
+    (edges \\ "is_hidden").head.as[Boolean] should be(false)
+  }
+
+  test("degree with `Where clause") {
+    val edges = getEdgesSync(getQuery(2, "_from != 2"))
+    (edges \ "degrees").as[Seq[JsValue]].nonEmpty should be(true)
+  }
+
+  test("interval parent") {
+    val baseId = 1024
 
+    insertEdgesSync(
+      toEdge(20, insert, e, baseId, baseId + 1, testLabelName, Json.obj(weight -> 30, is_hidden -> true)),
+
+      toEdge(10, insert, e, baseId + 1, baseId + 10, testLabelName, Json.obj(weight -> 30, is_hidden -> true)),
+      toEdge(20, insert, e, baseId + 1, baseId + 20, testLabelName, Json.obj(weight -> 30, is_hidden -> true)),
+      toEdge(30, insert, e, baseId + 1, baseId + 30, testLabelName, Json.obj(weight -> 30, is_hidden -> true))
+    )
+
+    val edges = getEdgesSync(queryIntervalWithParent(baseId, index2, "_timestamp", "_parent._timestamp"))
+    (edges \ "size").get.toString should be("1")
+
+    val to = (edges \\ "to").head.as[Long]
+    to should be (baseId + 20)
+  }
+
+  test("interval parent with range") {
+    val baseId = 9876
+
+    val minute: Long = 60 * 1000L
+    val hour = 60 * minute
+
+    insertEdgesSync(
+      toEdge(1, insert, e, baseId, baseId + 1, testLabelName, Json.obj(weight -> 30, is_hidden -> true)),
+      toEdge(1 + hour * 2, insert, e, baseId + 1, baseId + 10, testLabelName, Json.obj(weight -> 30, is_hidden -> true)),
+      toEdge(1 + hour * 3, insert, e, baseId + 1, baseId + 20, testLabelName, Json.obj(weight -> 30, is_hidden -> true)),
+      toEdge(1 + hour * 4, insert, e, baseId + 1, baseId + 30, testLabelName, Json.obj(weight -> 30, is_hidden -> true))
+    )
+
+    val edges = getEdgesSync(queryIntervalWithParentRange(baseId, index2,
+      "_timestamp", "${_parent._timestamp}",
+      "_timestamp", "${_parent._timestamp + 3 hour}"))
+
+    (edges \ "size").get.toString should be("2")
+
+    val edges2 = getEdgesSync(queryIntervalWithParentRange(baseId, index2,
+      "_timestamp", "${_parent._timestamp}",
+      "_timestamp", "${_parent._timestamp + 2 hour}"))
+
+    (edges2 \ "size").get.toString should be("1")
+
+    val edges3 = getEdgesSync(queryIntervalWithParentRange(baseId, index2,
+      "_timestamp", "${_parent._timestamp + 130 minute}",
+      "_timestamp", "${_parent._timestamp + 4 hour}"))
+
+    (edges3 \ "size").get.toString should be("2")
+  }
+
+  test("interval") {
     var edges = getEdgesSync(queryWithInterval(0, index2, "_timestamp", 1000, 1001)) // test interval on timestamp index
     (edges \ "size").get.toString should be("1")
 
@@ -67,88 +220,29 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
   }
 
   test("get edge with where condition") {
-    def queryWhere(id: Int, where: String) = Json.parse(
-      s"""
-        { "srcVertices": [
-          { "serviceName": "${testServiceName}",
-            "columnName": "${testColumnName}",
-            "id": ${id}
-           }],
-          "steps": [
-          [ {
-              "label": "${testLabelName}",
-              "direction": "out",
-              "offset": 0,
-              "limit": 100,
-              "where": "${where}"
-            }
-          ]]
-        }""")
 
-    var result = getEdgesSync(queryWhere(0, "is_hidden=false and _from in (-1, 0)"))
+    var result = getEdgesSync(getQuery(0, "is_hidden=false and _from in (-1, 0)"))
     (result \ "results").as[List[JsValue]].size should be(1)
 
-    result = getEdgesSync(queryWhere(0, "is_hidden=true and _to in (1)"))
+    result = getEdgesSync(getQuery(0, "is_hidden=true and _to in (1)"))
     (result \ "results").as[List[JsValue]].size should be(1)
 
-    result = getEdgesSync(queryWhere(0, "_from=0"))
+    result = getEdgesSync(getQuery(0, "_from=0"))
     (result \ "results").as[List[JsValue]].size should be(2)
 
-    result = getEdgesSync(queryWhere(2, "_from=2 or weight in (-1)"))
+    result = getEdgesSync(getQuery(2, "_from=2 or weight in (-1)"))
     (result \ "results").as[List[JsValue]].size should be(2)
 
-    result = getEdgesSync(queryWhere(2, "_from=2 and weight in (10, 20)"))
+    result = getEdgesSync(getQuery(2, "_from=2 and weight in (10, 20)"))
     (result \ "results").as[List[JsValue]].size should be(2)
   }
 
   test("get edge exclude") {
-    def queryExclude(id: Int) = Json.parse(
-      s"""
-        { "srcVertices": [
-          { "serviceName": "${testServiceName}",
-            "columnName": "${testColumnName}",
-            "id": ${id}
-           }],
-          "steps": [
-          [ {
-              "label": "${testLabelName}",
-              "direction": "out",
-              "offset": 0,
-              "limit": 2
-            },
-            {
-              "label": "${testLabelName}",
-              "direction": "in",
-              "offset": 0,
-              "limit": 2,
-              "exclude": true
-            }
-          ]]
-        }""")
-
     val result = getEdgesSync(queryExclude(0))
     (result \ "results").as[List[JsValue]].size should be(1)
   }
 
   test("get edge groupBy property") {
-    def queryGroupBy(id: Int, props: Seq[String]): JsValue = {
-      Json.obj(
-        "groupBy" -> props,
-        "srcVertices" -> Json.arr(
-          Json.obj("serviceName" -> testServiceName, "columnName" -> testColumnName, "id" -> id)
-        ),
-        "steps" -> Json.arr(
-          Json.obj(
-            "step" -> Json.arr(
-              Json.obj(
-                "label" -> testLabelName
-              )
-            )
-          )
-        )
-      )
-    }
-
     val result = getEdgesSync(queryGroupBy(0, Seq("weight")))
     (result \ "size").as[Int] should be(2)
     val weights = (result \ "results" \\ "groupBy").map { js =>
@@ -235,8 +329,6 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
   //      }
   //    }
 
-
-
   test("duration") {
     def queryDuration(ids: Seq[Int], from: Int, to: Int) = {
       val $from = Json.arr(
@@ -285,11 +377,36 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
 
   }
 
-
   test("return tree") {
+    def queryParentsWithoutSelect(id: Long) = Json.parse(
+      s"""
+        {
+          "returnTree": true,
+          "srcVertices": [
+          { "serviceName": "$testServiceName",
+            "columnName": "$testColumnName",
+            "id": $id
+           }],
+          "steps": [
+          [ {
+              "label": "$testLabelName",
+              "direction": "out",
+              "offset": 0,
+              "limit": 2
+            }
+          ],[{
+              "label": "$testLabelName",
+              "direction": "in",
+              "offset": 0,
+              "limit": 1000
+            }
+          ]]
+        }""".stripMargin)
+
     def queryParents(id: Long) = Json.parse(
       s"""
         {
+          "select": ["weight"],
           "returnTree": true,
           "srcVertices": [
           { "serviceName": "$testServiceName",
@@ -317,15 +434,45 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
 
     insertEdgesSync(toEdge(1001, "insert", "e", src, tgt, testLabelName))
 
-    val result = TestUtil.getEdgesSync(queryParents(src))
-    val parents = (result \ "results").as[Seq[JsValue]]
-    val ret = parents.forall {
-      edge => (edge \ "parents").as[Seq[JsValue]].size == 1
+    // test parent With select fields
+    var result = TestUtil.getEdgesSync(queryParents(src))
+    var parents = (result \ "results").as[Seq[JsValue]]
+    var ret = parents.forall { edge =>
+      val parentEdges = (edge \ "parents").as[Seq[JsValue]]
+      val assertSize = parentEdges.size == 1
+      val parentProps = (parentEdges.head \ "props").as[JsObject]
+      val parentWeight = (parentProps \ "weight").as[Long]
+      val parentIsHidden = (parentProps \ "is_hidden").asOpt[Boolean]
+
+      val assertProp = parentWeight == 0 && parentIsHidden.isEmpty // select only "weight"
+
+      assertSize && assertProp
     }
 
     ret should be(true)
-  }
 
+    // test parent With select fields: check default Prop
+    result = TestUtil.getEdgesSync(queryParentsWithoutSelect(src))
+    parents = (result \ "results").as[Seq[JsValue]]
+    ret = parents.forall { edge =>
+      val parentEdges = (edge \ "parents").as[Seq[JsValue]]
+      val assertSize = parentEdges.size == 1
+
+      val parentProps = (parentEdges.head \ "props").as[JsObject]
+
+      val parentWeight = (parentProps \ "weight").as[Int]
+      val parentIsHidden = (parentProps \ "is_hidden").as[Boolean]
+      val parentIsBlocked = (parentProps \ "is_blocked").as[Boolean]
+      val parentTime = (parentProps \ "time").as[Long]
+
+      val assertProp =
+        parentWeight == 0 && parentIsHidden == false && parentIsBlocked == false && parentTime == 0
+
+      assertSize && assertProp
+    }
+
+    ret should be(true)
+  }
 
 
   test("pagination and _to") {
@@ -438,7 +585,6 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
     edgesTo.reverse should be(ascOrderByTo)
   }
 
-
   test("query with sampling") {
     def queryWithSampling(id: Int, sample: Int) = Json.parse(
       s"""
@@ -757,7 +903,7 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
          |        {
          |          "step": [
          |            {
-         |              "label": "$testLabelName2",
+         |              "label": "$testLabelName",
          |              "direction": "out",
          |              "offset": 0,
          |              "limit": 5
@@ -778,9 +924,9 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
       toEdge(1, insert, e, testId1, 111, testLabelName, Json.obj(weight -> 10)),
       toEdge(2, insert, e, testId1, 222, testLabelName, Json.obj(weight -> 10)),
       toEdge(3, insert, e, testId1, 333, testLabelName, Json.obj(weight -> 10)),
-      toEdge(4, insert, e, testId2, 111, testLabelName2, Json.obj(weight -> 1)),
-      toEdge(5, insert, e, testId2, 333, testLabelName2, Json.obj(weight -> 1)),
-      toEdge(6, insert, e, testId2, 555, testLabelName2, Json.obj(weight -> 1))
+      toEdge(4, insert, e, testId2, 111, testLabelName, Json.obj(weight -> 1)),
+      toEdge(5, insert, e, testId2, 333, testLabelName, Json.obj(weight -> 1)),
+      toEdge(6, insert, e, testId2, 555, testLabelName, Json.obj(weight -> 1))
     )
 
     insertEdgesSync(bulkEdges: _*)
@@ -859,196 +1005,6 @@ class QueryTest extends IntegrateCommon with BeforeAndAfterEach {
     results.size should be(4)
   }
 
-  test("scorePropagateOp test") {
-    def queryWithPropertyOp(id: String, op: String, shrinkageVal: Long) = Json.parse(
-      s"""{
-         |  "limit": 10,
-         |  "groupBy": ["from"],
-         |  "duplicate": "sum",
-         |  "srcVertices": [
-         |    {
-         |      "serviceName": "$testServiceName",
-         |      "columnName": "$testColumnName",
-         |      "id": $id
-         |    }
-         |  ],
-         |  "steps": [
-         |    {
-         |      "step": [
-         |        {
-         |          "label": "$testLabelName",
-         |          "direction": "out",
-         |          "offset": 0,
-         |          "limit": 10,
-         |          "groupBy": ["from"],
-         |          "duplicate": "sum",
-         |          "index": "idx_1",
-         |          "scoring": {
-         |            "weight":1,
-         |            "time": 0
-         |          },
-         |          "transform": [["_from"]]
-         |        }
-         |      ]
-         |    }, {
-         |      "step": [
-         |        {
-         |          "label": "$testLabelName2",
-         |          "direction": "out",
-         |          "offset": 0,
-         |          "limit": 10,
-         |          "scorePropagateOp": "$op",
-         |          "scorePropagateShrinkage": $shrinkageVal
-         |        }
-         |      ]
-         |    }
-         |  ]
-         |}
-       """.stripMargin
-    )
-
-    def queryWithOp(ids: Seq[String], op: String, shrinkageVal: Long) = Json.parse(
-      s"""{
-         |  "limit": 10,
-         |  "groupBy": ["from"],
-         |  "duplicate": "sum",
-         |  "srcVertices": [
-         |    {
-         |      "serviceName": "$testServiceName",
-         |      "columnName": "$testColumnName",
-         |      "ids": [${ids.mkString(",")}]
-         |    }
-         |  ],
-         |  "steps": [
-         |    {
-         |      "step": [
-         |        {
-         |          "label": "$testLabelName",
-         |          "direction": "out",
-         |          "offset": 0,
-         |          "limit": 10,
-         |          "groupBy": ["from"],
-         |          "duplicate": "countSum",
-         |          "transform": [["_from"]]
-         |        }
-         |      ]
-         |    }, {
-         |      "step": [
-         |        {
-         |          "label": "$testLabelName2",
-         |          "direction": "out",
-         |          "offset": 0,
-         |          "limit": 10,
-         |          "scorePropagateOp": "$op",
-         |          "scorePropagateShrinkage": $shrinkageVal
-         |        }
-         |      ]
-         |    }
-         |  ]
-         |}
-       """.stripMargin
-    )
-
-    val testId = "-30000"
-    val testId2 = "-4000"
-
-    val bulkEdges = Seq(
-      toEdge(1, insert, e, testId, 101, testLabelName, Json.obj(weight -> -10)),
-      toEdge(1, insert, e, testId, 102, testLabelName, Json.obj(weight -> -10)),
-      toEdge(1, insert, e, testId, 103, testLabelName, Json.obj(weight -> -10)),
-      toEdge(1, insert, e, testId, 102, testLabelName2, Json.obj(weight -> 10)),
-      toEdge(1, insert, e, testId, 103, testLabelName2, Json.obj(weight -> 10)),
-      toEdge(1, insert, e, testId, 104, testLabelName2, Json.obj(weight -> 10)),
-      toEdge(1, insert, e, testId, 105, testLabelName2, Json.obj(weight -> 10)),
-
-      toEdge(1, insert, e, testId2, 101, testLabelName, Json.obj(weight -> -10)),
-      toEdge(1, insert, e, testId2, 102, testLabelName, Json.obj(weight -> -10)),
-      toEdge(1, insert, e, testId2, 103, testLabelName, Json.obj(weight -> -10)),
-      toEdge(1, insert, e, testId2, 102, testLabelName2, Json.obj(weight -> 10)),
-      toEdge(1, insert, e, testId2, 105, testLabelName2, Json.obj(weight -> 10))
-    )
-    insertEdgesSync(bulkEdges: _*)
-
-    val firstStepEdgeCount = 3l
-    val secondStepEdgeCount = 4l
-
-    var shrinkageVal = 10l
-    var rs = getEdgesSync(queryWithOp(Seq(testId), "divide", shrinkageVal))
-
-    var results = (rs \ "results").as[List[JsValue]]
-    results.size should be(1)
-    var scoreSum = secondStepEdgeCount.toDouble / (firstStepEdgeCount.toDouble + shrinkageVal)
-    (results(0) \ "scoreSum").as[Double] should be(scoreSum)
-
-    rs = getEdgesSync(queryWithOp(Seq(testId, testId2), "divide", shrinkageVal))
-
-    results = (rs \ "results").as[List[JsValue]]
-    results.size should be(2)
-    scoreSum = secondStepEdgeCount.toDouble / (firstStepEdgeCount.toDouble + shrinkageVal)
-    (results(0) \ "scoreSum").as[Double] should be(scoreSum)
-    scoreSum = 2.toDouble / (3.toDouble + shrinkageVal)
-    (results(1) \ "scoreSum").as[Double] should be(scoreSum)
-
-    // check for divide zero case
-    shrinkageVal = 30l
-    rs = getEdgesSync(queryWithPropertyOp(testId, "divide", shrinkageVal))
-
-    results = (rs \ "results").as[List[JsValue]]
-    results.size should be(1)
-    (results(0) \ "scoreSum").as[Double] should be(0)
-
-    // "plus" operation
-    rs = getEdgesSync(queryWithOp(Seq(testId), "plus", shrinkageVal))
-
-    results = (rs \ "results").as[List[JsValue]]
-    results.size should be(1)
-    scoreSum = (firstStepEdgeCount + 1) * secondStepEdgeCount
-    (results(0) \ "scoreSum").as[Long] should be(scoreSum)
-
-    // "multiply" operation
-    rs = getEdgesSync(queryWithOp(Seq(testId), "multiply", shrinkageVal))
-    logger.debug(Json.prettyPrint(rs))
-    results = (rs \ "results").as[List[JsValue]]
-    results.size should be(1)
-    scoreSum = (firstStepEdgeCount * 1) * secondStepEdgeCount
-    (results(0) \ "scoreSum").as[Long] should be(scoreSum)
-  }
-
-  def querySingle(id: Int, offset: Int = 0, limit: Int = 100) = Json.parse(
-    s"""
-          { "srcVertices": [
-            { "serviceName": "$testServiceName",
-              "columnName": "$testColumnName",
-              "id": $id
-             }],
-            "steps": [
-            [ {
-                "label": "$testLabelName",
-                "direction": "out",
-                "offset": $offset,
-                "limit": $limit
-              }
-            ]]
-          }
-          """)
-
-  def queryGlobalLimit(id: Int, limit: Int): JsValue = Json.obj(
-    "limit" -> limit,
-    "srcVertices" -> Json.arr(
-      Json.obj("serviceName" -> testServiceName, "columnName" -> testColumnName, "id" -> id)
-    ),
-    "steps" -> Json.arr(
-      Json.obj(
-        "step" -> Json.arr(
-          Json.obj(
-            "label" -> testLabelName
-          )
-        )
-      )
-    )
-  )
-
-
   // called by each test, each
   override def beforeEach = initTestData()
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/Integrate/WeakLabelDeleteTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/WeakLabelDeleteTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/WeakLabelDeleteTest.scala
index d62dee8..dc5dc2e 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/WeakLabelDeleteTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/WeakLabelDeleteTest.scala
@@ -22,7 +22,7 @@ package org.apache.s2graph.core.Integrate
 import java.util.concurrent.TimeUnit
 
 import org.scalatest.BeforeAndAfterEach
-import play.api.libs.json.{JsObject, JsValue, Json}
+import play.api.libs.json._
 
 import scala.concurrent.Await
 import scala.concurrent.duration.Duration
@@ -130,7 +130,9 @@ class WeakLabelDeleteTest extends IntegrateCommon with BeforeAndAfterEach {
       toEdge(startTs + 5, "insert", "e", "10", "20", testLabelNameWeak, s"""{"time": 10}"""),
       toEdge(startTs + 6, "insert", "e", "10", "21", testLabelNameWeak, s"""{"time": 11}"""),
       toEdge(startTs + 7, "insert", "e", "11", "20", testLabelNameWeak, s"""{"time": 12}"""),
-      toEdge(startTs + 8, "insert", "e", "12", "20", testLabelNameWeak, s"""{"time": 13}""")
+      toEdge(startTs + 8, "insert", "e", "12", "20", testLabelNameWeak, s"""{"time": 13}"""),
+      toEdge(startTs + 9, "insert", "e", "10000", "20000", testLabelNameWeak,
+        s"""{"time": 1, "weight": 0.1, "is_hidden": true, "is_blocked": false}""")
     )
 
     def query(id: Int, direction: String = "out", columnName: String = testColumnName) = Json.parse(

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/ManagementTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/ManagementTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/ManagementTest.scala
index e21f0e7..afc9aea 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/ManagementTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/ManagementTest.scala
@@ -20,9 +20,73 @@
 package org.apache.s2graph.core
 
 import org.apache.s2graph.core.Integrate.IntegrateCommon
-import org.apache.s2graph.core.mysqls.Label
+import org.apache.s2graph.core.mysqls.{Model, Label, Service}
+
+import scala.util.{Failure, Success}
+import play.api.libs.json.{JsValue, Json}
 
 class ManagementTest extends IntegrateCommon {
+
+
+  def checkCopyLabel(originalLabelName: String, newLabelName: String) = {
+    val originalLabelOpt = Label.findByName(originalLabelName, useCache = true)
+    originalLabelOpt.isDefined should be(true)
+    val originalLabel = originalLabelOpt.get
+
+    val labelTry = management.copyLabel(originalLabelName, newLabelName, hTableName = Option(newLabelName))
+    labelTry.isSuccess should be(true)
+    val copiedLabel = labelTry.get
+    copiedLabel.label should be(newLabelName)
+    copiedLabel.id.get != originalLabel.id.get should be(true)
+    copiedLabel.hTableTTL should equal(originalLabel.hTableTTL)
+
+    val copiedLabelMetaMap = copiedLabel.metas(useCache = false).map(m => m.seq -> m.name).toMap
+    val copiedLabelIndiceMap = copiedLabel.indices(useCache = false).map(m => m.seq -> m.metaSeqs).toMap
+    val originalLabelMetaMap = originalLabel.metas(useCache = false).map(m => m.seq -> m.name).toMap
+    val originalLabelIndiceMap = originalLabel.indices(useCache = false).map(m => m.seq -> m.metaSeqs).toMap
+
+    copiedLabelMetaMap should be(originalLabelMetaMap)
+    copiedLabelIndiceMap should be(originalLabelIndiceMap)
+
+    copiedLabel.metas().sortBy(m => m.id.get).map(m => m.name) should be(originalLabel.metas().sortBy(m => m.id.get).map(m => m.name))
+    copiedLabel.indices().sortBy(m => m.id.get).map(m => m.metaSeqs) should be(originalLabel.indices().sortBy(m => m.id.get).map(m => m.metaSeqs))
+  }
+
+  def checkLabelTTL(labelName:String, serviceName:String, setTTL:Option[Int], checkTTL:Option[Int]) = {
+    Management.deleteLabel(labelName)
+    val ttlOption = if(setTTL.isDefined) s""", "hTableTTL": ${setTTL.get}""" else ""
+    val createLabelJson = s"""{
+      "label": "$labelName",
+      "srcServiceName": "$serviceName",
+      "srcColumnName": "id",
+      "srcColumnType": "long",
+      "tgtServiceName": "$serviceName",
+      "tgtColumnName": "id",
+      "tgtColumnType": "long",
+      "indices":[],
+      "props":[],
+      "hTableName": "$labelName"
+      $ttlOption
+    }"""
+    val labelOpts = parser.toLabelElements(Json.parse(createLabelJson))
+    val tryLabel = (management.createLabel _).tupled(labelOpts.get)
+    assert(tryLabel.isSuccess)
+    val label = tryLabel.get
+    label.hTableTTL should be(checkTTL)
+  }
+
+  test("copy label test") {
+    val labelToCopy = s"${TestUtil.testLabelName}_copied"
+    Label.findByName(labelToCopy) match {
+      case None =>
+        //
+      case Some(oldLabel) =>
+        Label.delete(oldLabel.id.get)
+
+    }
+    checkCopyLabel(TestUtil.testLabelName, labelToCopy)
+  }
+
   test("swap label test") {
     val labelLeft = TestUtil.testLabelName
     val labelRight = TestUtil.testLabelName2
@@ -33,4 +97,53 @@ class ManagementTest extends IntegrateCommon {
     Label.findByName(labelLeft, false).get.schemaVersion should be("v4")
     Label.findByName(labelRight, false).get.schemaVersion should be("v3")
   }
-}
\ No newline at end of file
+
+  test("check created service without ttl") {
+    // createService
+    val svc_without_ttl = "s2graph_without_ttl"
+    val createServiceJson = s"""{"serviceName" : "$svc_without_ttl"}"""
+    val (serviceName, cluster, tableName, preSplitSize, ttl, compressionAlgorithm) =
+      parser.toServiceElements(Json.parse(createServiceJson))
+
+    val tryService = management.createService(serviceName, cluster, tableName, preSplitSize, ttl, compressionAlgorithm)
+    assert(tryService.isSuccess)
+    val service = tryService.get
+    assert(service.hTableTTL.isDefined)
+    service.hTableTTL.get should be(Integer.MAX_VALUE)
+
+    // check labels
+    checkLabelTTL("label_without_ttl", svc_without_ttl, None, service.hTableTTL)
+    checkLabelTTL("label_with_ttl", svc_without_ttl, Some(86400), Some(86400))
+
+    // check copied labels
+    Management.deleteLabel("label_without_ttl_copied")
+    checkCopyLabel("label_without_ttl", "label_without_ttl_copied")
+    Management.deleteLabel("label_with_ttl_copied")
+    checkCopyLabel("label_with_ttl", "label_with_ttl_copied")
+  }
+
+  test("check created service with ttl") {
+    // createService
+    val svc_with_ttl = "s2graph_with_ttl"
+    val ttl_val = 86400
+    val (serviceName, cluster, tableName, preSplitSize, ttl, compressionAlgorithm) =
+      parser.toServiceElements(Json.parse(s"""{"serviceName" : "$svc_with_ttl", "hTableTTL":$ttl_val}"""))
+
+    val tryService = management.createService(serviceName, cluster, tableName, preSplitSize, ttl, compressionAlgorithm)
+    assert(tryService.isSuccess)
+    val service = tryService.get
+    assert(service.hTableTTL.isDefined)
+    service.hTableTTL.get should be(ttl_val)
+
+    // check labels
+    checkLabelTTL("label_without_ttl", svc_with_ttl, None, service.hTableTTL)
+    checkLabelTTL("label_with_ttl", svc_with_ttl, Some(Integer.MAX_VALUE), Some(Integer.MAX_VALUE))
+
+    // check copied labels
+    Management.deleteLabel("label_without_ttl_copied")
+    checkCopyLabel("label_without_ttl", "label_without_ttl_copied")
+    Management.deleteLabel("label_with_ttl_copied")
+    checkCopyLabel("label_with_ttl", "label_with_ttl_copied")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/QueryParamTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/QueryParamTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/QueryParamTest.scala
index 61d1096..772fcd8 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/QueryParamTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/QueryParamTest.scala
@@ -1,159 +1,159 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.s2graph.core
-
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.types.{InnerVal, InnerValLike, HBaseSerializable, LabelWithDirection}
-import org.scalatest.{FunSuite, Matchers}
-
-class QueryParamTest extends FunSuite with Matchers with TestCommon {
-//  val version = HBaseType.VERSION2
-//  val testEdge = Management.toEdge(ts, "insert", "1", "10", labelNameV2, "out", Json.obj("is_blocked" -> true, "phone_number" -> "xxxx", "age" -> 20).toString)
-//  test("EdgeTransformer toInnerValOpt") {
-//
-//    /** only labelNameV2 has string type output */
-//    val jsVal = Json.arr(Json.arr("_to"), Json.arr("phone_number.$", "phone_number"), Json.arr("age.$", "age"))
-//    val transformer = EdgeTransformer(queryParamV2, jsVal)
-//    val convertedLs = transformer.transform(testEdge, None)
-//
-//    convertedLs(0).tgtVertex.innerId.toString == "10" shouldBe true
-//    convertedLs(1).tgtVertex.innerId.toString == "phone_number.xxxx" shouldBe true
-//    convertedLs(2).tgtVertex.innerId.toString == "age.20" shouldBe true
-//    true
+///*
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *   http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing,
+// * software distributed under the License is distributed on an
+// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// * KIND, either express or implied.  See the License for the
+// * specific language governing permissions and limitations
+// * under the License.
+// */
+//
+//package org.apache.s2graph.core
+//
+//import org.apache.hadoop.hbase.util.Bytes
+//import org.apache.s2graph.core.types.{InnerVal, InnerValLike, HBaseSerializable, LabelWithDirection}
+//import org.scalatest.{FunSuite, Matchers}
+//
+//class QueryParamTest extends FunSuite with Matchers with TestCommon {
+////  val version = HBaseType.VERSION2
+////  val testEdge = Management.toEdge(ts, "insert", "1", "10", labelNameV2, "out", Json.obj("is_blocked" -> true, "phone_number" -> "xxxx", "age" -> 20).toString)
+////  test("EdgeTransformer toInnerValOpt") {
+////
+////    /** only labelNameV2 has string type output */
+////    val jsVal = Json.arr(Json.arr("_to"), Json.arr("phone_number.$", "phone_number"), Json.arr("age.$", "age"))
+////    val transformer = EdgeTransformer(queryParamV2, jsVal)
+////    val convertedLs = transformer.transform(testEdge, None)
+////
+////    convertedLs(0).tgtVertex.innerId.toString == "10" shouldBe true
+////    convertedLs(1).tgtVertex.innerId.toString == "phone_number.xxxx" shouldBe true
+////    convertedLs(2).tgtVertex.innerId.toString == "age.20" shouldBe true
+////    true
+////  }
+//
+//  val dummyRequests = {
+//    for {
+//      id <- 0 until 1000
+//    } yield {
+//      Bytes.toBytes(id)
+//    }
+//  }
+//
+//  test("QueryParam toCacheKey bytes") {
+//    val startedAt = System.nanoTime()
+//    val queryParam = S2QueryParam(LabelWithDirection(1, 0))
+//
+//    for {
+//      i <- dummyRequests.indices
+//      x = queryParam.toCacheKey(dummyRequests(i))
+//    } {
+//      for {
+//        j <- dummyRequests.indices if i != j
+//        y = queryParam.toCacheKey(dummyRequests(j))
+//      } {
+//        x should not equal y
+//      }
+//    }
+//
+//    dummyRequests.zip(dummyRequests).foreach { case (x, y) =>
+//      val xHash = queryParam.toCacheKey(x)
+//      val yHash = queryParam.toCacheKey(y)
+////      println(xHash, yHash)
+//      xHash should be(yHash)
+//    }
+//    val duration = System.nanoTime() - startedAt
+//
+//    println(s">> bytes: $duration")
+//  }
+//
+//  test("QueryParam toCacheKey with variable params") {
+//    val startedAt = System.nanoTime()
+//    val queryParam = S2QueryParam(LabelWithDirection(1, 0))
+//
+//    dummyRequests.zip(dummyRequests).foreach { case (x, y) =>
+//      x shouldBe y
+//      queryParam.limit(0, 10)
+//      var xHash = queryParam.toCacheKey(x)
+//      xHash shouldBe queryParam.toCacheKey(y)
+//      queryParam.limit(1, 10)
+//      var yHash = queryParam.toCacheKey(y)
+//      queryParam.toCacheKey(x) shouldBe yHash
+////      println(xHash, yHash)
+//      xHash should not be yHash
+//
+//      queryParam.limit(0, 10)
+//      xHash = queryParam.toCacheKey(x)
+//      queryParam.limit(0, 11)
+//      yHash = queryParam.toCacheKey(y)
+//
+//      xHash should not be yHash
+//    }
+//
+//    val duration = System.nanoTime() - startedAt
+//
+//    println(s">> diff: $duration")
+//  }
+//  test("QueryParam interval min/max bytes padding test") {
+//    import HBaseSerializable._
+//    val queryParam = QueryParam.Empty
+//    def compare(_from: Seq[InnerValLike], _to: Seq[InnerValLike], _value: Seq[InnerValLike]): Boolean = {
+//      val len = _from.length.toByte
+//
+//      val from = _from.zipWithIndex map { case (innerVal: InnerValLike, idx: Int) => idx.toByte -> innerVal }
+//      val to = _to.zipWithIndex map { case (innerVal: InnerValLike, idx: Int) => idx.toByte -> innerVal }
+//      val value = _value.zipWithIndex map { case (innerVal: InnerValLike, idx: Int) => idx.toByte -> innerVal }
+//
+//      val (fromBytes, toBytes) = queryParam.paddingInterval(len, from, to)
+//      val valueBytes = propsToBytes(value)
+//
+//      val validFrom = Bytes.compareTo(fromBytes, valueBytes) <= 0
+//      val validTo = Bytes.compareTo(toBytes, valueBytes) >= 0
+//
+//      val res = validFrom && validTo
+//      //      if (!res) logger.error(s"from: $validFrom, to: $validTo, from: ${_from} to: ${_to} value: ${_value}")
+//      res
+//    }
+//
+//    val v = "v3"
+//    compare(
+//      Seq(InnerVal.withLong(0L, v)),
+//      Seq(InnerVal.withLong(0L, v)),
+//      Seq(InnerVal.withLong(0L, v))) shouldBe true
+//
+//    compare(
+//      Seq(InnerVal.withLong(0L, v)),
+//      Seq(InnerVal.withLong(0L, v)),
+//      Seq(InnerVal.withLong(1L, v))) shouldBe false
+//
+//    compare(
+//      Seq(InnerVal.withLong(1L, v)),
+//      Seq(InnerVal.withLong(1L, v)),
+//      Seq(InnerVal.withLong(0L, v))) shouldBe false
+//
+//    compare(
+//      Seq(InnerVal.withLong(0L, v)),
+//      Seq(InnerVal.withLong(1L, v)),
+//      Seq(InnerVal.withLong(2L, v))) shouldBe false
+//
+//    val testNum = 100000
+//    val tests = for {
+//      n <- 0 to testNum
+//      min = scala.util.Random.nextInt(Int.MaxValue / 2) + 1
+//      max = min + scala.util.Random.nextInt(min)
+//      value = min + scala.util.Random.nextInt(max - min + 1)
+//    } yield compare(
+//        Seq(InnerVal.withLong(min, v)),
+//        Seq(InnerVal.withLong(max, v)),
+//        Seq(InnerVal.withLong(value, v)))
+//
+//    tests.forall(identity) shouldBe true
 //  }
-
-  val dummyRequests = {
-    for {
-      id <- 0 until 1000
-    } yield {
-      Bytes.toBytes(id)
-    }
-  }
-
-  test("QueryParam toCacheKey bytes") {
-    val startedAt = System.nanoTime()
-    val queryParam = QueryParam(LabelWithDirection(1, 0))
-
-    for {
-      i <- dummyRequests.indices
-      x = queryParam.toCacheKey(dummyRequests(i))
-    } {
-      for {
-        j <- dummyRequests.indices if i != j
-        y = queryParam.toCacheKey(dummyRequests(j))
-      } {
-        x should not equal y
-      }
-    }
-
-    dummyRequests.zip(dummyRequests).foreach { case (x, y) =>
-      val xHash = queryParam.toCacheKey(x)
-      val yHash = queryParam.toCacheKey(y)
-//      println(xHash, yHash)
-      xHash should be(yHash)
-    }
-    val duration = System.nanoTime() - startedAt
-
-    println(s">> bytes: $duration")
-  }
-
-  test("QueryParam toCacheKey with variable params") {
-    val startedAt = System.nanoTime()
-    val queryParam = QueryParam(LabelWithDirection(1, 0))
-
-    dummyRequests.zip(dummyRequests).foreach { case (x, y) =>
-      x shouldBe y
-      queryParam.limit(0, 10)
-      var xHash = queryParam.toCacheKey(x)
-      xHash shouldBe queryParam.toCacheKey(y)
-      queryParam.limit(1, 10)
-      var yHash = queryParam.toCacheKey(y)
-      queryParam.toCacheKey(x) shouldBe yHash
-//      println(xHash, yHash)
-      xHash should not be yHash
-
-      queryParam.limit(0, 10)
-      xHash = queryParam.toCacheKey(x)
-      queryParam.limit(0, 11)
-      yHash = queryParam.toCacheKey(y)
-
-      xHash should not be yHash
-    }
-
-    val duration = System.nanoTime() - startedAt
-
-    println(s">> diff: $duration")
-  }
-  test("QueryParam interval min/max bytes padding test") {
-    import HBaseSerializable._
-    val queryParam = QueryParam.Empty
-    def compare(_from: Seq[InnerValLike], _to: Seq[InnerValLike], _value: Seq[InnerValLike]): Boolean = {
-      val len = _from.length.toByte
-
-      val from = _from.zipWithIndex map { case (innerVal: InnerValLike, idx: Int) => idx.toByte -> innerVal }
-      val to = _to.zipWithIndex map { case (innerVal: InnerValLike, idx: Int) => idx.toByte -> innerVal }
-      val value = _value.zipWithIndex map { case (innerVal: InnerValLike, idx: Int) => idx.toByte -> innerVal }
-
-      val (fromBytes, toBytes) = queryParam.paddingInterval(len, from, to)
-      val valueBytes = propsToBytes(value)
-
-      val validFrom = Bytes.compareTo(fromBytes, valueBytes) <= 0
-      val validTo = Bytes.compareTo(toBytes, valueBytes) >= 0
-
-      val res = validFrom && validTo
-      //      if (!res) logger.error(s"from: $validFrom, to: $validTo, from: ${_from} to: ${_to} value: ${_value}")
-      res
-    }
-
-    val v = "v3"
-    compare(
-      Seq(InnerVal.withLong(0L, v)),
-      Seq(InnerVal.withLong(0L, v)),
-      Seq(InnerVal.withLong(0L, v))) shouldBe true
-
-    compare(
-      Seq(InnerVal.withLong(0L, v)),
-      Seq(InnerVal.withLong(0L, v)),
-      Seq(InnerVal.withLong(1L, v))) shouldBe false
-
-    compare(
-      Seq(InnerVal.withLong(1L, v)),
-      Seq(InnerVal.withLong(1L, v)),
-      Seq(InnerVal.withLong(0L, v))) shouldBe false
-
-    compare(
-      Seq(InnerVal.withLong(0L, v)),
-      Seq(InnerVal.withLong(1L, v)),
-      Seq(InnerVal.withLong(2L, v))) shouldBe false
-
-    val testNum = 100000
-    val tests = for {
-      n <- 0 to testNum
-      min = scala.util.Random.nextInt(Int.MaxValue / 2) + 1
-      max = min + scala.util.Random.nextInt(min)
-      value = min + scala.util.Random.nextInt(max - min + 1)
-    } yield compare(
-        Seq(InnerVal.withLong(min, v)),
-        Seq(InnerVal.withLong(max, v)),
-        Seq(InnerVal.withLong(value, v)))
-
-    tests.forall(identity) shouldBe true
-  }
-}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala b/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
index 584a641..12eae77 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/TestCommonWithModels.scala
@@ -85,6 +85,7 @@ trait TestCommonWithModels {
   val preSplitSize = 0
 
   val labelName = "_test_label"
+  val labelNameSecure = "_test_label_secure"
   val labelNameV2 = "_test_label_v2"
   val labelNameV3 = "_test_label_v3"
   val labelNameV4 = "_test_label_v4"
@@ -102,6 +103,7 @@ trait TestCommonWithModels {
     Prop("score", "0.1", FLOAT),
     Prop("age", "10", INT)
   )
+
   val testIdxProps = Seq(Index("_PK", Seq("_timestamp", "affinity_score")))
   val consistencyLevel = "strong"
   val hTableTTL = None
@@ -129,10 +131,12 @@ trait TestCommonWithModels {
     Management.deleteLabel(labelNameV2)
     Management.deleteLabel(undirectedLabelName)
     Management.deleteLabel(undirectedLabelNameV2)
+    Management.deleteLabel(labelNameSecure)
   }
 
   def createTestLabel() = {
     implicit val session = AutoSession
+
     management.createLabel(labelName, serviceName, columnName, columnType, serviceName, columnName, columnType,
       isDirected = true, serviceName, testIdxProps, testProps, consistencyLevel, Some(hTableName), hTableTTL, VERSION1, false, "lg4", None)
 
@@ -150,6 +154,10 @@ trait TestCommonWithModels {
 
     management.createLabel(undirectedLabelNameV2, serviceNameV2, columnNameV2, columnTypeV2, serviceNameV2, tgtColumnNameV2, tgtColumnTypeV2,
       isDirected = false, serviceName, testIdxProps, testProps, consistencyLevel, Some(hTableName), hTableTTL, VERSION2, false, "lg4", None)
+
+    management.createLabel(labelNameSecure, serviceName, columnName, columnType, serviceName, tgtColumnName, tgtColumnType,
+      isDirected = false, serviceName, testIdxProps, testProps, consistencyLevel, Some(hTableName), hTableTTL, VERSION3, false, "lg4",
+      Option("""{ "tokens": ["xxx-yyy", "aaa-bbb"] }"""))
   }
 
   def service = Service.findByName(serviceName, useCache = false).get

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/models/ModelTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/models/ModelTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/models/ModelTest.scala
index 66d84e4..87a84ae 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/models/ModelTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/models/ModelTest.scala
@@ -32,34 +32,6 @@ class ModelTest extends FunSuite with Matchers with TestCommonWithModels with Be
     graph.shutdown()
   }
 
-  //  val serviceName = "testService"
-  //  val newServiceName = "newTestService"
-  //  val cluster = "localhost"
-  //  val hbaseTableName = "s2graph-dev"
-  //  val columnName = "user_id"
-  //  val columnType = "long"
-  //  val labelName = "model_test_label"
-  //  val newLabelName = "new_model_test_label"
-  //  val columnMetaName = "is_valid_user"
-  //  val labelMetaName = "is_hidden"
-  //  val hbaseTableTTL = -1
-  //  val id = 1
-  //
-  //  val service = HService(Map("id" -> id, "serviceName" -> serviceName, "cluster" -> cluster,
-  //    "hbaseTableName" -> hbaseTableName, "preSplitSize" -> 0, "hbaseTableTTL" -> -1))
-  //  val serviceColumn = HServiceColumn(Map("id" -> id, "serviceId" -> service.id.get,
-  //    "columnName" -> columnName, "columnType" -> columnType))
-  //  val columnMeta = HColumnMeta(Map("id" -> id, "columnId" -> serviceColumn.id.get, "name" -> columnMetaName,  "seq" -> 1.toByte))
-  //  val label = HLabel(Map("id" -> id, "label" -> labelName,
-  //    "srcServiceId" -> service.id.get, "srcColumnName" -> columnName, "srcColumnType" -> columnType,
-  //    "tgtServiceId" -> service.id.get, "tgtColumnName" -> columnName, "tgtColumnType" -> columnType,
-  //    "isDirected" -> true, "serviceName" -> service.serviceName, "serviceId" -> service.id.get,
-  //    "consistencyLevel" -> "weak", "hTableName" -> hbaseTableName, "hTableTTL" -> -1
-  //  ))
-  //  val labelMeta = HLabelMeta(Map("id" -> id, "labelId" -> label.id.get, "name" -> labelMetaName, "seq" -> 1.toByte,
-  //    "defaultValue" -> false, "dataType" -> "boolean", "usedInIndex" -> false))
-  //  val labelIndex = HLabelIndex(Map("id" -> id, "labelId" -> label.id.get, "seq" -> 1.toByte,
-  //    "metaSeqs" -> "0", "formular" -> "none"))
   test("test Label.findByName") {
     val labelOpt = Label.findByName(labelName, useCache = false)
     println(labelOpt)
@@ -84,66 +56,4 @@ class ModelTest extends FunSuite with Matchers with TestCommonWithModels with Be
     val tgtColumn = labelOpt.get.tgtService
     println(tgtColumn)
   }
-  //  test("test create") {
-  //    service.create()
-  //    HService.findByName(serviceName, useCache = false) == Some(service)
-  //
-  //    serviceColumn.create()
-  //    HServiceColumn.findsByServiceId(service.id.get, useCache = false).headOption == Some(serviceColumn)
-  //
-  //    columnMeta.create()
-  //    HColumnMeta.findByName(serviceColumn.id.get, columnMetaName, useCache = false) == Some(columnMeta)
-  //
-  //    label.create()
-  //    HLabel.findByName(labelName, useCache = false) == Some(label)
-  //
-  //    labelMeta.create()
-  //    HLabelMeta.findByName(label.id.get, labelMetaName, useCache = false) == Some(labelMeta)
-  //
-  //    labelIndex.create()
-  //    HLabelIndex.findByLabelIdAll(label.id.get, useCache = false).headOption == Some(labelIndex)
-  //  }
-  //
-  //  test("test update") {
-  //    service.update("cluster", "...")
-  //    HService.findById(service.id.get, useCache = false).cluster == "..."
-  //
-  //    service.update("serviceName", newServiceName)
-  //    assert(HService.findByName(serviceName, useCache = false) == None)
-  //    HService.findByName(newServiceName, useCache = false).map { service => service.id.get == service.id.get}
-  //
-  //    label.update("label", newLabelName)
-  //    HLabel.findById(label.id.get, useCache = false).label == "newLabelName"
-  //
-  //    label.update("consistencyLevel", "strong")
-  //    HLabel.findById(label.id.get, useCache = false).consistencyLevel == "strong" &&
-  //    HLabel.findByName(newLabelName).isDefined &&
-  //    HLabel.findByName(labelName) == None
-  //
-  //  }
-  //  test("test read by index") {
-  //    val labels = HLabel.findBySrcServiceId(service.id.get, useCache = false)
-  //    val idxs = HLabelIndex.findByLabelIdAll(label.id.get, useCache = false)
-  //    labels.length == 1 &&
-  //    labels.head == label
-  //    idxs.length == 1 &&
-  //    idxs.head == labelIndex
-  //  }
-  //  test("test delete") {
-  ////    HLabel.findByName(labelName).foreach { label =>
-  ////      label.deleteAll()
-  ////    }
-  //    HLabel.findByName(newLabelName).foreach { label =>
-  //      label.deleteAll()
-  //    }
-  //    HLabelMeta.findAllByLabelId(label.id.get, useCache = false).isEmpty &&
-  //    HLabelIndex.findByLabelIdAll(label.id.get, useCache = false).isEmpty
-  //
-  //    service.deleteAll()
-  //  }
-
-  //  test("test labelIndex") {
-  //    println(HLabelIndex.findByLabelIdAll(1))
-  //  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/mysqls/ExperimentSpec.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/mysqls/ExperimentSpec.scala b/s2core/src/test/scala/org/apache/s2graph/core/mysqls/ExperimentSpec.scala
deleted file mode 100644
index 229d9bd..0000000
--- a/s2core/src/test/scala/org/apache/s2graph/core/mysqls/ExperimentSpec.scala
+++ /dev/null
@@ -1,83 +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.mysqls
-
-import java.util.Properties
-
-import com.typesafe.config.ConfigFactory
-import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers}
-import scalikejdbc._
-
-class ExperimentSpec extends FlatSpec with Matchers with BeforeAndAfterAll {
-  val Ttl = 2
-  override def beforeAll(): Unit = {
-    /*
-    maxSize = config.getInt("cache.max.size")
-    ttl = config.getInt("cache.ttl.seconds")
-     */
-    val props = new Properties()
-    props.setProperty("cache.ttl.seconds", Ttl.toString)
-    Model.apply(ConfigFactory.load(ConfigFactory.parseProperties(props)))
-
-    implicit val session = AutoSession
-    sql"""DELETE FROM buckets""".update().apply()
-    sql"""DELETE FROM experiments""".update().apply()
-
-    val expId = sql"""INSERT INTO experiments(service_id, service_name, name, description) VALUES(1, 's1', 'exp1', '')""".updateAndReturnGeneratedKey().apply()
-    sql"""INSERT INTO
-           buckets(experiment_id, modular, http_verb, api_path, request_body, impression_id)
-           VALUES($expId, '1~100', 'POST', '/a/b/c', 'None', 'imp1')""".update().apply()
-
-  }
-
-  "Experiment" should "find bucket list" in {
-    Experiment.findBy(1, "exp1") should not be empty
-
-    Experiment.findBy(1, "exp1").foreach { exp =>
-      val bucket = exp.buckets.head
-      bucket.impressionId should equal("imp1")
-    }
-  }
-
-  it should "update bucket list after cache ttl time" in {
-    Experiment.findBy(1, "exp1").foreach { exp =>
-      val bucket = exp.buckets.head
-      bucket.impressionId should equal("imp1")
-
-      implicit val session = AutoSession
-
-      sql"""UPDATE buckets SET impression_id = 'imp2' WHERE id = ${bucket.id}""".update().apply()
-    }
-
-    // sleep ttl time
-    Thread.sleep((Ttl + 1) * 1000)
-
-    // update experiment and bucket
-    Experiment.findBy(1, "exp1").foreach(exp => exp.buckets)
-
-    // wait for cache updating
-    Thread.sleep(1 * 1000)
-
-    // should be updated
-    Experiment.findBy(1, "exp1").foreach { exp =>
-      exp.buckets.head.impressionId should equal("imp2")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
index 40166eb..042dce2 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
@@ -23,81 +23,109 @@ import org.apache.s2graph.core._
 import org.apache.s2graph.core.mysqls.{Label, LabelMeta}
 import org.apache.s2graph.core.rest.TemplateHelper
 import org.apache.s2graph.core.types._
+import org.apache.s2graph.core.utils.logger
 import org.scalatest.{FunSuite, Matchers}
 import play.api.libs.json.Json
 
+import scala.util.{Random, Try}
+
 class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
   initTests()
 
-  // dummy data for dummy edge
-  initTests()
-  
   import HBaseType.{VERSION1, VERSION2}
 
   val ts = System.currentTimeMillis()
-  val dummyTs = (LabelMeta.timeStampSeq -> InnerValLikeWithTs.withLong(ts, ts, label.schemaVersion))
-
-  def ids(version: String) = {
-    val colId = if (version == VERSION2) columnV2.id.get else column.id.get
-    val srcId = SourceVertexId(colId, InnerVal.withLong(1, version))
-    val tgtId = TargetVertexId(colId, InnerVal.withLong(2, version))
-
-    val srcIdStr = SourceVertexId(colId, InnerVal.withStr("abc", version))
-    val tgtIdStr = TargetVertexId(colId, InnerVal.withStr("def", version))
-
-    val srcVertex = Vertex(srcId, ts)
-    val tgtVertex = Vertex(tgtId, ts)
-    val srcVertexStr = Vertex(srcIdStr, ts)
-    val tgtVertexStr = Vertex(tgtIdStr, ts)
-    (srcId, tgtId, srcIdStr, tgtIdStr, srcVertex, tgtVertex, srcVertexStr, tgtVertexStr, version)
-  }
-
+  val dummyTs = LabelMeta.timestamp -> InnerValLikeWithTs.withLong(ts, ts, label.schemaVersion)
 
   def validate(label: Label)(edge: Edge)(sql: String)(expected: Boolean) = {
-    val whereOpt = WhereParser(label).parse(sql)
-    whereOpt.isSuccess shouldBe true
-
-    println("=================================================================")
-    println(sql)
-    println(whereOpt.get)
-
-    val ret = whereOpt.get.filter(edge)
-    if (ret != expected) {
+    def debug(whereOpt: Try[Where]) = {
       println("==================")
       println(s"$whereOpt")
       println(s"$edge")
       println("==================")
     }
-    ret shouldBe expected
+
+    val whereOpt = WhereParser(label).parse(sql)
+    if (whereOpt.isFailure) {
+      debug(whereOpt)
+      whereOpt.get // touch exception
+    } else {
+      val ret = whereOpt.get.filter(edge)
+      if (ret != expected) {
+        debug(whereOpt)
+      }
+
+      ret shouldBe expected
+    }
+  }
+
+  def ids = for {
+    version <- Seq(VERSION1, VERSION2)
+  } yield {
+    val srcId = SourceVertexId(0, InnerVal.withLong(1, version))
+    val tgtId =
+      if (version == VERSION2) TargetVertexId(0, InnerVal.withStr("2", version))
+      else TargetVertexId(0, InnerVal.withLong(2, version))
+
+    val srcVertex = Vertex(srcId, ts)
+    val tgtVertex = Vertex(tgtId, ts)
+    val (_label, dir) = if (version == VERSION2) (labelV2, labelWithDirV2.dir) else (label, labelWithDir.dir)
+
+    (srcVertex, tgtVertex, _label, dir)
   }
 
   test("check where clause not nested") {
     for {
-      (srcId, tgtId, srcIdStr, tgtIdStr, srcVertex, tgtVertex, srcVertexStr, tgtVertexStr, schemaVer) <- List(ids(VERSION1), ids(VERSION2))
+      (srcVertex, tgtVertex, label, dir) <- ids
     } {
       /** test for each version */
-      val js = Json.obj("is_hidden" -> true, "is_blocked" -> false, "weight" -> 10, "time" -> 3, "name" -> "abc")
+      val js = Json.obj("is_hidden" -> true, "is_blocked" -> false, "weight" -> 10, "time" -> 3, "phone_number" -> "1234")
       val propsInner = Management.toProps(label, js.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
-      val edge = Edge(srcVertex, tgtVertex, labelWithDir, 0.toByte, ts, propsInner)
+      val edge = Edge(srcVertex, tgtVertex, label, dir, 0.toByte, ts, propsInner)
       val f = validate(label)(edge) _
 
       /** labelName label is long-long relation */
-      f(s"_to=${tgtVertex.innerId.toString}")(true)
-
-      // currently this throw exception since label`s _to is long type.
+      f(s"_to=${tgtVertex.innerId}")(true)
       f(s"_to=19230495")(false)
       f(s"_to!=19230495")(true)
+      f(s"phone_number=1234")(true)
+    }
+  }
+
+  test("check where clause with string literal") {
+    for {
+      (srcVertex, tgtVertex,
+      label, dir) <- ids
+    } {
+      /** test for each version */
+      var js = Json.obj("phone_number" -> "")
+      var propsInner = Management.toProps(label, js.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
+      var edge = Edge(srcVertex, tgtVertex, label, labelWithDir.dir, 0.toByte, ts, propsInner)
+      var f = validate(label)(edge) _
+      f(s"phone_number = '' ")(true)
+
+      js = Json.obj("phone_number" -> "010 3167 1897")
+      propsInner = Management.toProps(label, js.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
+      edge = Edge(srcVertex, tgtVertex, label, labelWithDir.dir, 0.toByte, ts, propsInner)
+      f = validate(label)(edge) _
+      f(s"phone_number = '010 3167 1897' ")(true)
+
+      js = Json.obj("phone_number" -> "010' 3167 1897")
+      propsInner = Management.toProps(label, js.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
+      edge = Edge(srcVertex, tgtVertex, label, labelWithDir.dir, 0.toByte, ts, propsInner)
+      f = validate(label)(edge) _
+      f(s"phone_number = '010\\' 3167 1897' ")(true)
     }
   }
 
   test("check where clause nested") {
     for {
-      (srcId, tgtId, srcIdStr, tgtIdStr, srcVertex, tgtVertex, srcVertexStr, tgtVertexStr, schemaVer) <- List(ids(VERSION1), ids(VERSION2))
+      (srcVertex, tgtVertex, label, dir) <- ids
     } {
       /** test for each version */
       val js = Json.obj("is_hidden" -> true, "is_blocked" -> false, "weight" -> 10, "time" -> 3, "name" -> "abc")
       val propsInner = Management.toProps(label, js.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
-      val edge = Edge(srcVertex, tgtVertex, labelWithDir, 0.toByte, ts, propsInner)
+      val edge = Edge(srcVertex, tgtVertex, label, labelWithDir.dir, 0.toByte, ts, propsInner)
 
       val f = validate(label)(edge) _
 
@@ -119,19 +147,19 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
     }
   }
 
+
   test("check where clause with from/to long") {
     for {
-      (srcId, tgtId, srcIdStr, tgtIdStr, srcVertex, tgtVertex, srcVertexStr, tgtVertexStr, schemaVer) <- List(ids(VERSION1), ids(VERSION2))
+      (srcVertex, tgtVertex, label, dir) <- ids
     } {
       /** test for each version */
       val js = Json.obj("is_hidden" -> true, "is_blocked" -> false, "weight" -> 10, "time" -> 3, "name" -> "abc")
       val propsInner = Management.toProps(label, js.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
-      val labelWithDirection = if (schemaVer == VERSION2) labelWithDirV2 else labelWithDir
-      val edge = Edge(srcVertex, tgtVertex, labelWithDirection, 0.toByte, ts, propsInner)
-      val lname = if (schemaVer == VERSION2) labelNameV2 else labelName
+      val edge = Edge(srcVertex, tgtVertex, label, dir, 0.toByte, ts, propsInner)
       val f = validate(label)(edge) _
 
       f(s"_from = -1 or _to = ${tgtVertex.innerId.value}")(true)
+      f(s"_to = 2")(true)
       f(s"_from = ${srcVertex.innerId.value} and _to = ${tgtVertex.innerId.value}")(true)
       f(s"_from = ${tgtVertex.innerId.value} and _to = 102934")(false)
       f(s"_from = -1")(false)
@@ -139,10 +167,9 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
     }
   }
 
-
   test("check where clause with parent") {
     for {
-      (srcId, tgtId, srcIdStr, tgtIdStr, srcVertex, tgtVertex, srcVertexStr, tgtVertexStr, schemaVer) <- List(ids(VERSION1), ids(VERSION2))
+      (srcVertex, tgtVertex, label, dir) <- ids
     } {
       /** test for each version */
       val js = Json.obj("is_hidden" -> true, "is_blocked" -> false, "weight" -> 10, "time" -> 1, "name" -> "abc")
@@ -151,11 +178,11 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
       val propsInner = Management.toProps(label, js.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
       val parentPropsInner = Management.toProps(label, parentJs.fields).map { case (k, v) => k -> InnerValLikeWithTs(v, ts) }.toMap + dummyTs
 
-      val grandParentEdge = Edge(srcVertex, tgtVertex, labelWithDir, 0.toByte, ts, parentPropsInner)
-      val parentEdge = Edge(srcVertex, tgtVertex, labelWithDir, 0.toByte, ts, parentPropsInner,
-        parentEdges = Seq(EdgeWithScore(grandParentEdge, 1.0)))
-      val edge = Edge(srcVertex, tgtVertex, labelWithDir, 0.toByte, ts, propsInner,
-        parentEdges = Seq(EdgeWithScore(parentEdge, 1.0)))
+      val grandParentEdge = Edge(srcVertex, tgtVertex, label, labelWithDir.dir, 0.toByte, ts, parentPropsInner)
+      val parentEdge = Edge(srcVertex, tgtVertex, label, labelWithDir.dir, 0.toByte, ts, parentPropsInner,
+        parentEdges = Seq(EdgeWithScore(grandParentEdge, 1.0, grandParentEdge.label)))
+      val edge = Edge(srcVertex, tgtVertex, label, labelWithDir.dir, 0.toByte, ts, propsInner,
+        parentEdges = Seq(EdgeWithScore(parentEdge, 1.0, grandParentEdge.label)))
 
       println(edge.toString)
       println(parentEdge.toString)
@@ -190,13 +217,16 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
     val ts = 0
     import TemplateHelper._
 
+    calculate(ts, 1, "minute") should be(minute + ts)
     calculate(ts, 1, "hour") should be(hour + ts)
     calculate(ts, 1, "day") should be(day + ts)
 
     calculate(ts + 10, 1, "HOUR") should be(hour + ts + 10)
     calculate(ts + 10, 1, "DAY") should be(day + ts + 10)
 
-    val body = """{
+    val body =
+      """{
+        	"minute": ${1 minute},
         	"day": ${1day},
           "hour": ${1hour},
           "-day": "${-10 day}",
@@ -208,21 +238,26 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
     val parsed = replaceVariable(ts, body)
     val json = Json.parse(parsed)
 
-    (json \ "day").as[Long] should be (1 * day + ts)
-    (json \ "hour").as[Long] should be (1 * hour + ts)
+    (json \ "minute").as[Long] should be(1 * minute + ts)
 
-    (json \ "-day").as[Long] should be (-10 * day + ts)
-    (json \ "-hour").as[Long] should be (-10 * hour + ts)
+    (json \ "day").as[Long] should be(1 * day + ts)
+    (json \ "hour").as[Long] should be(1 * hour + ts)
 
-    (json \ "now").as[Long] should be (ts)
+    (json \ "-day").as[Long] should be(-10 * day + ts)
+    (json \ "-hour").as[Long] should be(-10 * hour + ts)
 
-    val otherBody = """{
+    (json \ "now").as[Long] should be(ts)
+
+    val otherBody =
+      """{
+          "nextminute": "${next_minute}",
           "nextday": "${next_day}",
           "3dayago": "${next_day - 3 day}",
           "nexthour": "${next_hour}"
         }"""
 
-    val currentTs = System.currentTimeMillis()
+    val currentTs = 1474422964000l
+    val expectedMinuteTs = currentTs / minute * minute + minute
     val expectedDayTs = currentTs / day * day + day
     val expectedHourTs = currentTs / hour * hour + hour
     val threeDayAgo = expectedDayTs - 3 * day
@@ -232,22 +267,15 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
       val parsed = replaceVariable(ts, otherBody)
       val json = Json.parse(parsed)
 
+      (json \ "nextminute").as[Long] should be(expectedMinuteTs)
       (json \ "nextday").as[Long] should be(expectedDayTs)
       (json \ "nexthour").as[Long] should be(expectedHourTs)
       (json \ "3dayago").as[Long] should be(threeDayAgo)
     }
-  }
 
-  //  test("time decay") {
-  //    val ts = System.currentTimeMillis()
-  //
-  //    for {
-  //      i <- (0 until 10)
-  //    } {
-  //      val timeUnit = 60 * 60
-  //      val diff = i * timeUnit
-  //      val x = TimeDecay(1.0, 0.05, timeUnit)
-  //      println(x.decay(diff))
-  //    }
-  //  }
+    (0 until 1000).forall { ith =>
+      val r = replaceVariable(ts, "${randint( 10,  30 )}").toInt
+      r >= 10 && r < 30
+    }
+  }
 }


[4/7] incubator-s2graph git commit: [S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala b/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
index d77ac7d..5466a9a 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/rest/RequestParser.scala
@@ -19,57 +19,82 @@
 
 package org.apache.s2graph.core.rest
 
+
 import java.util.concurrent.{Callable, TimeUnit}
+
 import com.google.common.cache.CacheBuilder
-import com.typesafe.config.Config
 import org.apache.s2graph.core.GraphExceptions.{BadQueryException, ModelNotFoundException}
+import org.apache.s2graph.core.JSONParser._
 import org.apache.s2graph.core._
 import org.apache.s2graph.core.mysqls._
 import org.apache.s2graph.core.parsers.{Where, WhereParser}
 import org.apache.s2graph.core.types._
-import org.apache.s2graph.core.JSONParser._
 import play.api.libs.json._
-import play.api.libs.json.Reads._
 
-import scala.util.{Failure, Success, Try}
+
+import scala.util.{Random, Failure, Success, Try}
 
 object TemplateHelper {
   val findVar = """\"?\$\{(.*?)\}\"?""".r
-  val num = """(next_day|next_hour|next_week|now)?\s*(-?\s*[0-9]+)?\s*(hour|day|week)?""".r
+  val num = """(next_minute|next_day|next_hour|next_week|now)?\s*(-?\s*[0-9]+)?\s*(minute|hour|day|week)?""".r
+  val randIntRegex = """randint\((.*,.*)\)""".r
 
-  val hour = 60 * 60 * 1000L
-  val day = hour * 24L
-  val week = day * 7L
+  val minute: Long = 60 * 1000L
+  val hour = 60 * minute
+  val day = 24 * hour
+  val week = 7 * day
 
   def calculate(now: Long, n: Int, unit: String): Long = {
     val duration = unit match {
+      case "minute" | "MINUTE" => n * minute
       case "hour" | "HOUR" => n * hour
       case "day" | "DAY" => n * day
       case "week" | "WEEK" => n * week
-      case _ => n * day
+      case _ => n
     }
 
     duration + now
   }
 
+  def randInt(s: String): Long = {
+    val tokens = s.split(",").map(_.trim)
+    if (tokens.length != 2) throw new RuntimeException(s"TemplateHelper.randint has wrong format. $s")
+    val (from, to) = try {
+      (tokens.head.toInt, tokens.last.toInt)
+    } catch {
+      case e: Exception => throw new RuntimeException(s"TemplateHelper.randint has wrong format. $s")
+    }
+    if (from > to) throw new RuntimeException(s"TemplateHelper.randint has wrong format. $s")
+    val diff = to - from
+    val r = Random.nextInt(diff + 1)
+    assert(diff >= 0 && diff < Int.MaxValue && from + r < Int.MaxValue)
+    from + r
+  }
+
   def replaceVariable(now: Long, body: String): String = {
     findVar.replaceAllIn(body, m => {
       val matched = m group 1
+      randIntRegex.findFirstMatchIn(matched) match {
+        case None =>
+          num.replaceSomeIn(matched, m => {
+            val (_pivot, n, unit) = (m.group(1), m.group(2), m.group(3))
+            val ts = _pivot match {
+              case null => now
+              case "now" | "NOW" => now
+              case "next_minute" | "NEXT_MINUTE" => now / minute * minute + minute
+              case "next_week" | "NEXT_WEEK" => now / week * week + week
+              case "next_day" | "NEXT_DAY" => now / day * day + day
+              case "next_hour" | "NEXT_HOUR" => now / hour * hour + hour
+            }
 
-      num.replaceSomeIn(matched, m => {
-        val (_pivot, n, unit) = (m.group(1), m.group(2), m.group(3))
-        val ts = _pivot match {
-          case null => now
-          case "now" | "NOW" => now
-          case "next_week" | "NEXT_WEEK" => now / week * week + week
-          case "next_day" | "NEXT_DAY" => now / day * day + day
-          case "next_hour" | "NEXT_HOUR" => now / hour * hour + hour
-        }
-
-        if (_pivot == null && n == null && unit == null) None
-        else if (n == null || unit == null) Option(ts.toString)
-        else Option(calculate(ts, n.replaceAll(" ", "").toInt, unit).toString)
-      })
+            if (_pivot == null && n == null && unit == null) None
+            else if (n == null || unit == null) Option(ts.toString)
+            else Option(calculate(ts, n.replaceAll(" ", "").toInt, unit).toString)
+          })
+        case Some(m) =>
+          val range = m group 1
+          randInt(range).toString
+      }
     })
   }
 }
@@ -107,37 +132,37 @@ class RequestParser(graph: Graph) {
   val DefaultCompressionAlgorithm = config.getString("hbase.table.compression.algorithm")
   val DefaultPhase = config.getString("phase")
   val parserCache = CacheBuilder.newBuilder()
-    .expireAfterAccess(10000, TimeUnit.MILLISECONDS)
-    .expireAfterWrite(10000, TimeUnit.MILLISECONDS)
-    .maximumSize(10000)
-    .initialCapacity(1000)
-    .build[String, Try[Where]]
+      .expireAfterAccess(10000, TimeUnit.MILLISECONDS)
+      .expireAfterWrite(10000, TimeUnit.MILLISECONDS)
+      .maximumSize(10000)
+      .initialCapacity(1000)
+      .build[String, Try[Where]]
 
-  private def extractScoring(labelId: Int, value: JsValue) = {
+  private def extractScoring(label: Label, value: JsValue): Option[Seq[(LabelMeta, Double)]] = {
     val ret = for {
       js <- parseOption[JsObject](value, "scoring")
     } yield {
       for {
         (k, v) <- js.fields
-        labelOrderType <- LabelMeta.findByName(labelId, k)
+        labelMata <- label.metaPropsInvMap.get(k)
       } yield {
         val value = v match {
           case n: JsNumber => n.as[Double]
           case _ => throw new Exception("scoring weight should be double.")
         }
-        (labelOrderType.seq, value)
+        (labelMata, value)
       }
     }
 
     ret
   }
 
-  def extractInterval(label: Label, jsValue: JsValue) = {
-    def extractKv(js: JsValue) = js match {
-      case JsObject(map) => map.toSeq
+  def extractInterval(label: Label, jsValue: JsValue): Option[(Seq[(String, JsValue)], Seq[(String, JsValue)])] = {
+    def extractKv(js: JsValue): Seq[(String, JsValue)] = js match {
+      case JsObject(obj) => obj.toSeq
       case JsArray(arr) => arr.flatMap {
-        case JsObject(map) => map.toSeq
-        case _ => throw new RuntimeException(s"cannot support json type: $js")
+        case JsObject(obj) => obj.toSeq
+        case _ => throw new RuntimeException(s"cannot support json type $js")
       }
       case _ => throw new RuntimeException(s"cannot support json type: $js")
     }
@@ -147,8 +172,8 @@ class RequestParser(graph: Graph) {
       fromJs <- (js \ "from").asOpt[JsValue]
       toJs <- (js \ "to").asOpt[JsValue]
     } yield {
-      val from = Management.toProps(label, extractKv(fromJs))
-      val to = Management.toProps(label, extractKv(toJs))
+      val from = extractKv(fromJs)
+      val to = extractKv(toJs)
       (from, to)
     }
 
@@ -188,10 +213,10 @@ class RequestParser(graph: Graph) {
         labelMeta <- LabelMeta.findByName(label.id.get, k)
         value <- jsValueToInnerVal(v, labelMeta.dataType, label.schemaVersion)
       } yield {
-        labelMeta.seq -> value
+        labelMeta.name -> value
       }
     }
-    ret.map(_.toMap).getOrElse(Map.empty[Byte, InnerValLike])
+    ret.map(_.toMap).getOrElse(Map.empty[String, InnerValLike])
   }
 
   def extractWhere(label: Label, whereClauseOpt: Option[String]): Try[Where] = {
@@ -214,6 +239,28 @@ class RequestParser(graph: Graph) {
     }
   }
 
+  def extractGroupBy(value: Option[JsValue]): GroupBy = value.map {
+    case obj: JsObject =>
+      val keys = (obj \ "keys").asOpt[Seq[String]].getOrElse(Nil)
+      val groupByLimit = (obj \ "limit").asOpt[Int].getOrElse(hardLimit)
+      val minShouldMatchOpt = (obj \ "minimumShouldMatch").asOpt[JsObject].map { o =>
+        val prop = (o \ "prop").asOpt[String].getOrElse("to")
+        val count = (o \ "count").asOpt[Int].getOrElse(0)
+        val terms = (o \ "terms").asOpt[Set[JsValue]].getOrElse(Set.empty).map {
+          case JsString(s) => s
+          case JsNumber(n) => n
+          case _ => throw new RuntimeException("not supported data type")
+        }.map(_.asInstanceOf[Any])
+
+        MinShouldMatchParam(prop, count, terms)
+      }
+
+      GroupBy(keys, groupByLimit, minShouldMatch = minShouldMatchOpt)
+    case arr: JsArray =>
+      val keys = arr.asOpt[Seq[String]].getOrElse(Nil)
+      GroupBy(keys)
+  }.getOrElse(GroupBy.Empty)
+
   def toVertices(labelName: String, direction: String, ids: Seq[JsValue]): Seq[Vertex] = {
     val vertices = for {
       label <- Label.findByName(labelName).toSeq
@@ -228,33 +275,40 @@ class RequestParser(graph: Graph) {
   }
 
   def toMultiQuery(jsValue: JsValue, impIdOpt: Option[String]): MultiQuery = {
+    val globalQueryOption = toQueryOption(jsValue, impIdOpt)
     val queries = for {
       queryJson <- (jsValue \ "queries").asOpt[Seq[JsValue]].getOrElse(Seq.empty)
     } yield {
-      toQuery(queryJson, impIdOpt = impIdOpt)
+      val innerQuery = toQuery(queryJson, impIdOpt = impIdOpt)
+      val queryOption = innerQuery.queryOption
+
+      if (queryOption.groupBy.keys.nonEmpty) throw new BadQueryException("Group by option is not allowed in multiple queries.")
+      if (queryOption.orderByKeys.nonEmpty) throw new BadQueryException("Order by option is not allowed in multiple queries.")
+
+      if (globalQueryOption.withScore) innerQuery.copy(queryOption = innerQuery.queryOption.copy(withScore = false))
+      else innerQuery
+      //        val innerQuery3 =
+      //          if (globalQueryOption.groupBy.keys.nonEmpty) innerQuery2.copy(queryOption = innerQuery2.queryOption.copy(groupBy = GroupBy.Empty))
+      //          else innerQuery2
+
     }
     val weights = (jsValue \ "weights").asOpt[Seq[Double]].getOrElse(queries.map(_ => 1.0))
-    MultiQuery(queries = queries, weights = weights, queryOption = toQueryOption(jsValue, impIdOpt))
+    MultiQuery(queries = queries, weights = weights, queryOption = globalQueryOption)
   }
 
+
   def toQueryOption(jsValue: JsValue, impIdOpt: Option[String]): QueryOption = {
     val filterOutFields = (jsValue \ "filterOutFields").asOpt[List[String]].getOrElse(List(LabelMeta.to.name))
-    val filterOutQuery = (jsValue \ "filterOut").asOpt[JsValue].map { v => toQuery(v, impIdOpt = impIdOpt) }.map { q =>
-      q.copy(queryOption = q.queryOption.copy(filterOutFields = filterOutFields))
+    val filterOutQuery = (jsValue \ "filterOut").asOpt[JsValue].map { v =>
+      toQuery(v, impIdOpt = impIdOpt)
+    }.map { q =>
+      q.copy(queryOption = q.queryOption.copy(filterOutFields = filterOutFields, selectColumns = filterOutFields))
     }
     val removeCycle = (jsValue \ "removeCycle").asOpt[Boolean].getOrElse(true)
     val selectColumns = (jsValue \ "select").asOpt[List[String]].getOrElse(List.empty)
-//    val groupByColumns = (jsValue \ "groupBy").asOpt[List[String]].getOrElse(List.empty)
-    val groupBy = (jsValue \ "groupBy").asOpt[JsValue].getOrElse(JsNull) match {
-      case obj: JsObject =>
-        val keys = (obj \ "key").asOpt[Seq[String]].getOrElse(Nil)
-        val groupByLimit = (obj \ "limit").asOpt[Int].getOrElse(hardLimit)
-        GroupBy(keys, groupByLimit)
-      case arr: JsArray =>
-        val keys = arr.asOpt[Seq[String]].getOrElse(Nil)
-        GroupBy(keys)
-      case _ => GroupBy.Empty
-    }
+
+    val groupBy = extractGroupBy((jsValue \ "groupBy").asOpt[JsValue])
+
     val orderByColumns: List[(String, Boolean)] = (jsValue \ "orderBy").asOpt[List[JsObject]].map { jsLs =>
       for {
         js <- jsLs
@@ -266,7 +320,7 @@ class RequestParser(graph: Graph) {
         }
         column -> ascending
       }
-    }.getOrElse(List("score" -> false, "timestamp" -> false))
+    }.getOrElse(Nil)
     val withScore = (jsValue \ "withScore").asOpt[Boolean].getOrElse(true)
     val returnTree = (jsValue \ "returnTree").asOpt[Boolean].getOrElse(false)
     //TODO: Refactor this
@@ -274,6 +328,8 @@ class RequestParser(graph: Graph) {
     val returnAgg = (jsValue \ "returnAgg").asOpt[Boolean].getOrElse(true)
     val scoreThreshold = (jsValue \ "scoreThreshold").asOpt[Double].getOrElse(Double.MinValue)
     val returnDegree = (jsValue \ "returnDegree").asOpt[Boolean].getOrElse(true)
+    val ignorePrevStepCache = (jsValue \ "ignorePrevStepCache").asOpt[Boolean].getOrElse(false)
+    val shouldPropagateScore = (jsValue \ "shouldPropagateScore").asOpt[Boolean].getOrElse(true)
 
     QueryOption(removeCycle = removeCycle,
       selectColumns = selectColumns,
@@ -287,34 +343,25 @@ class RequestParser(graph: Graph) {
       returnAgg = returnAgg,
       scoreThreshold = scoreThreshold,
       returnDegree = returnDegree,
-      impIdOpt = impIdOpt
+      impIdOpt = impIdOpt,
+      ignorePrevStepCache,
+      shouldPropagateScore
     )
   }
 
   def toQuery(jsValue: JsValue, impIdOpt: Option[String]): Query = {
     try {
-      val vertices =
-        (for {
-          value <- parse[List[JsValue]](jsValue, "srcVertices")
-          serviceName = parse[String](value, "serviceName")
-          column = parse[String](value, "columnName")
-        } yield {
-          val service = Service.findByName(serviceName).getOrElse(throw BadQueryException("service not found"))
-          val col = ServiceColumn.find(service.id.get, column).getOrElse(throw BadQueryException("bad column name"))
-          val (idOpt, idsOpt) = ((value \ "id").asOpt[JsValue], (value \ "ids").asOpt[List[JsValue]])
-          for {
-            idVal <- idOpt ++ idsOpt.toSeq.flatten
-
-            /* bug, need to use labels schemaVersion  */
-            innerVal <- jsValueToInnerVal(idVal, col.columnType, col.schemaVersion)
-          } yield {
-            Vertex(SourceVertexId(col.id.get, innerVal), System.currentTimeMillis())
-          }
-        }).flatten
+      val vertices = for {
+        value <- (jsValue \ "srcVertices").asOpt[Seq[JsValue]].getOrElse(Nil)
+        serviceName <- (value \ "serviceName").asOpt[String].toSeq
+        columnName <- (value \ "columnName").asOpt[String].toSeq
+        idJson = (value \ "id").asOpt[JsValue].map(Seq(_)).getOrElse(Nil)
+        idsJson = (value \ "ids").asOpt[Seq[JsValue]].getOrElse(Nil)
+        id <- (idJson ++ idsJson).flatMap(jsValueToAny(_).toSeq).distinct
+      } yield Vertex.toVertex(serviceName, columnName, id)
 
       if (vertices.isEmpty) throw BadQueryException("srcVertices`s id is empty")
       val steps = parse[Vector[JsValue]](jsValue, "steps")
-
       val queryOption = toQueryOption(jsValue, impIdOpt)
 
       val querySteps =
@@ -350,32 +397,35 @@ class RequestParser(graph: Graph) {
           val queryParams =
             for {
               labelGroup <- queryParamJsVals
-              queryParam <- parseQueryParam(labelGroup)
+              queryParam <- parseQueryParam(labelGroup, queryOption)
             } yield {
               val (_, columnName) =
-                if (queryParam.labelWithDir.dir == GraphUtil.directions("out")) {
+                if (queryParam.dir == GraphUtil.directions("out")) {
                   (queryParam.label.srcService.serviceName, queryParam.label.srcColumnName)
                 } else {
                   (queryParam.label.tgtService.serviceName, queryParam.label.tgtColumnName)
                 }
               //FIXME:
-              if (stepIdx == 0 && vertices.nonEmpty && !vertices.exists(v => v.serviceColumn.columnName == columnName)) {
+              if (stepIdx == 0 && vertices.nonEmpty && !vertices.exists(v => v.columnName == columnName)) {
                 throw BadQueryException("srcVertices contains incompatiable serviceName or columnName with first step.")
               }
 
               queryParam
             }
-          Step(queryParams.toList, labelWeights = labelWeights,
-            //            scoreThreshold = stepThreshold,
+
+
+          val groupBy = extractGroupBy((step \ "groupBy").asOpt[JsValue])
+
+          Step(queryParams = queryParams,
+            labelWeights = labelWeights,
             nextStepScoreThreshold = nextStepScoreThreshold,
             nextStepLimit = nextStepLimit,
-            cacheTTL = cacheTTL)
+            cacheTTL = cacheTTL,
+            groupBy = groupBy)
 
         }
 
-      val ret = Query(vertices, querySteps, queryOption)
-      //      logger.debug(ret.toString)
-      ret
+      Query(vertices, querySteps, queryOption)
     } catch {
       case e: BadQueryException =>
         throw e
@@ -386,12 +436,12 @@ class RequestParser(graph: Graph) {
     }
   }
 
-  private def parseQueryParam(labelGroup: JsValue): Option[QueryParam] = {
+  private def parseQueryParam(labelGroup: JsValue, queryOption: QueryOption): Option[QueryParam] = {
     for {
       labelName <- parseOption[String](labelGroup, "label")
     } yield {
       val label = Label.findByName(labelName).getOrElse(throw BadQueryException(s"$labelName not found"))
-      val direction = parseOption[String](labelGroup, "direction").map(GraphUtil.toDirection(_)).getOrElse(0)
+      val direction = parseOption[String](labelGroup, "direction").getOrElse("out")
       val limit = {
         parseOption[Int](labelGroup, "limit") match {
           case None => defaultLimit
@@ -402,28 +452,24 @@ class RequestParser(graph: Graph) {
       val offset = parseOption[Int](labelGroup, "offset").getOrElse(0)
       val interval = extractInterval(label, labelGroup)
       val duration = extractDuration(label, labelGroup)
-      val scoring = extractScoring(label.id.get, labelGroup).getOrElse(List.empty[(Byte, Double)]).toList
+      val scoring = extractScoring(label, labelGroup).getOrElse(Nil).toList
       val exclude = parseOption[Boolean](labelGroup, "exclude").getOrElse(false)
       val include = parseOption[Boolean](labelGroup, "include").getOrElse(false)
       val hasFilter = extractHas(label, labelGroup)
-      val labelWithDir = LabelWithDirection(label.id.get, direction)
-      val indexNameOpt = (labelGroup \ "index").asOpt[String]
-      val indexSeq = indexNameOpt match {
-        case None => label.indexSeqsMap.get(scoring.map(kv => kv._1)).map(_.seq).getOrElse(LabelIndex.DefaultSeq)
-        case Some(indexName) => label.indexNameMap.get(indexName).map(_.seq).getOrElse(throw new RuntimeException("cannot find index"))
-      }
+
+      val indexName = (labelGroup \ "index").asOpt[String].getOrElse(LabelIndex.DefaultName)
       val whereClauseOpt = (labelGroup \ "where").asOpt[String]
       val where = extractWhere(label, whereClauseOpt)
       val includeDegree = (labelGroup \ "includeDegree").asOpt[Boolean].getOrElse(true)
       val rpcTimeout = (labelGroup \ "rpcTimeout").asOpt[Int].getOrElse(DefaultRpcTimeout)
       val maxAttempt = (labelGroup \ "maxAttempt").asOpt[Int].getOrElse(DefaultMaxAttempt)
-      val tgtVertexInnerIdOpt = (labelGroup \ "_to").asOpt[JsValue].flatMap { jsVal =>
-        jsValueToInnerVal(jsVal, label.tgtColumnWithDir(direction).columnType, label.schemaVersion)
-      }
+
+      val tgtVertexInnerIdOpt = (labelGroup \ "_to").asOpt[JsValue].filterNot(_ == JsNull).flatMap(jsValueToAny)
+
       val cacheTTL = (labelGroup \ "cacheTTL").asOpt[Long].getOrElse(-1L)
       val timeDecayFactor = (labelGroup \ "timeDecay").asOpt[JsObject].map { jsVal =>
         val propName = (jsVal \ "propName").asOpt[String].getOrElse(LabelMeta.timestamp.name)
-        val propNameSeq = label.metaPropsInvMap.get(propName).map(_.seq).getOrElse(LabelMeta.timeStampSeq)
+        val propNameSeq = label.metaPropsInvMap.get(propName).getOrElse(LabelMeta.timestamp)
         val initial = (jsVal \ "initial").asOpt[Double].getOrElse(1.0)
         val decayRate = (jsVal \ "decayRate").asOpt[Double].getOrElse(0.1)
         if (decayRate >= 1.0 || decayRate <= 0.0) throw new BadQueryException("decay rate should be 0.0 ~ 1.0")
@@ -432,40 +478,36 @@ class RequestParser(graph: Graph) {
       }
       val threshold = (labelGroup \ "threshold").asOpt[Double].getOrElse(QueryParam.DefaultThreshold)
       // TODO: refactor this. dirty
-      val duplicate = parseOption[String](labelGroup, "duplicate").map(s => Query.DuplicatePolicy(s))
+      val duplicate = parseOption[String](labelGroup, "duplicate").map(s => DuplicatePolicy(s)).getOrElse(DuplicatePolicy.First)
 
       val outputField = (labelGroup \ "outputField").asOpt[String].map(s => Json.arr(Json.arr(s)))
-      val transformer = if (outputField.isDefined) outputField else (labelGroup \ "transform").asOpt[JsValue]
+      val transformer = (if (outputField.isDefined) outputField else (labelGroup \ "transform").asOpt[JsValue]) match {
+        case None => EdgeTransformer(EdgeTransformer.DefaultJson)
+        case Some(json) => EdgeTransformer(json)
+      }
       val scorePropagateOp = (labelGroup \ "scorePropagateOp").asOpt[String].getOrElse("multiply")
       val scorePropagateShrinkage = (labelGroup \ "scorePropagateShrinkage").asOpt[Long].getOrElse(500l)
       val sample = (labelGroup \ "sample").asOpt[Int].getOrElse(-1)
       val shouldNormalize = (labelGroup \ "normalize").asOpt[Boolean].getOrElse(false)
       val cursorOpt = (labelGroup \ "cursor").asOpt[String]
       // FIXME: Order of command matter
-      QueryParam(labelWithDir)
-        .sample(sample)
-        .rank(RankParam(label.id.get, scoring))
-        .exclude(exclude)
-        .include(include)
-        .duration(duration)
-        .has(hasFilter)
-        .labelOrderSeq(indexSeq)
-        .interval(interval)
-        .limit(offset, limit)
-        .where(where)
-        .duplicatePolicy(duplicate)
-        .includeDegree(includeDegree)
-        .rpcTimeout(rpcTimeout)
-        .maxAttempt(maxAttempt)
-        .tgtVertexInnerIdOpt(tgtVertexInnerIdOpt)
-        .cacheTTLInMillis(cacheTTL)
-        .timeDecay(timeDecayFactor)
-        .threshold(threshold)
-        .transformer(transformer)
-        .scorePropagateOp(scorePropagateOp)
-        .scorePropagateShrinkage(scorePropagateShrinkage)
-        .shouldNormalize(shouldNormalize)
-        .cursorOpt(cursorOpt)
+      QueryParam(labelName = labelName,
+        direction = direction,
+        offset = offset,
+        limit = limit,
+        sample = sample,
+        maxAttempt = maxAttempt,
+        rpcTimeout = rpcTimeout,
+        cacheTTLInMillis = cacheTTL,
+        indexName = indexName, where = where, threshold = threshold,
+        rank = RankParam(scoring), intervalOpt = interval, durationOpt = duration,
+        exclude = exclude, include = include, has = hasFilter, duplicatePolicy = duplicate,
+        includeDegree = includeDegree, scorePropagateShrinkage = scorePropagateShrinkage,
+        scorePropagateOp = scorePropagateOp, shouldNormalize = shouldNormalize,
+        whereRawOpt = whereClauseOpt, cursorOpt = cursorOpt,
+        tgtVertexIdOpt = tgtVertexInnerIdOpt,
+        edgeTransformer = transformer, timeDecay = timeDecayFactor
+      )
     }
   }
 
@@ -489,6 +531,11 @@ class RequestParser(graph: Graph) {
     }
   }
 
+  def jsToStr(js: JsValue): String = js match {
+    case JsString(s) => s
+    case _ => js.toString()
+  }
+
   def parseBulkFormat(str: String): Seq[(GraphElement, String)] = {
     val edgeStrs = str.split("\\n").filterNot(_.isEmpty)
     val elementsWithTsv = for {
@@ -624,30 +671,15 @@ class RequestParser(graph: Graph) {
   }
 
   def toCheckEdgeParam(jsValue: JsValue) = {
-    val params = jsValue.as[List[JsValue]]
-    var isReverted = false
-    val labelWithDirs = scala.collection.mutable.HashSet[LabelWithDirection]()
-    val quads = for {
-      param <- params
-      labelName <- (param \ "label").asOpt[String]
-      direction <- GraphUtil.toDir((param \ "direction").asOpt[String].getOrElse("out"))
-      label <- Label.findByName(labelName)
-      srcId <- jsValueToInnerVal((param \ "from").as[JsValue], label.srcColumnWithDir(direction.toInt).columnType, label.schemaVersion)
-      tgtId <- jsValueToInnerVal((param \ "to").as[JsValue], label.tgtColumnWithDir(direction.toInt).columnType, label.schemaVersion)
+    for {
+      json <- jsValue.asOpt[Seq[JsValue]].getOrElse(Nil)
+      from <- (json \ "from").asOpt[JsValue].flatMap(jsValueToAny(_))
+      to <- (json \ "to").asOpt[JsValue].flatMap(jsValueToAny(_))
+      labelName <- (json \ "label").asOpt[String]
+      direction = (json \ "direction").asOpt[String].getOrElse("out")
     } yield {
-      val labelWithDir = LabelWithDirection(label.id.get, direction)
-      labelWithDirs += labelWithDir
-      val (src, tgt, dir) = if (direction == 1) {
-        isReverted = true
-        (Vertex(VertexId(label.tgtColumnWithDir(direction.toInt).id.get, tgtId)),
-          Vertex(VertexId(label.srcColumnWithDir(direction.toInt).id.get, srcId)), 0)
-      } else {
-        (Vertex(VertexId(label.srcColumnWithDir(direction.toInt).id.get, srcId)),
-          Vertex(VertexId(label.tgtColumnWithDir(direction.toInt).id.get, tgtId)), 0)
-      }
-      (src, tgt, QueryParam(LabelWithDirection(label.id.get, dir)))
+      Edge.toEdge(from, to, labelName, direction, Map.empty)
     }
-    (quads, isReverted)
   }
 
   def toGraphElements(str: String): Seq[GraphElement] = {
@@ -671,16 +703,6 @@ class RequestParser(graph: Graph) {
     (labels, direction, ids, ts, vertices)
   }
 
-  def toFetchAndDeleteParam(json: JsValue) = {
-    val labelName = (json \ "label").as[String]
-    val fromOpt = (json \ "from").asOpt[JsValue]
-    val toOpt = (json \ "to").asOpt[JsValue]
-    val direction = (json \ "direction").asOpt[String].getOrElse("out")
-    val indexOpt = (json \ "index").asOpt[String]
-    val propsOpt = (json \ "props").asOpt[JsObject]
-    (labelName, fromOpt, toOpt, direction, indexOpt, propsOpt)
-  }
-
   def parseExperiment(jsQuery: JsValue): Seq[ExperimentParam] = jsQuery.as[Seq[JsObject]].map { obj =>
     def _require(field: String) = throw new RuntimeException(s"${field} not found")
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala b/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
index 4c77ad6..099a7f9 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/rest/RestHandler.scala
@@ -29,6 +29,8 @@ import org.apache.s2graph.core.utils.logger
 import play.api.libs.json._
 
 import scala.concurrent.{ExecutionContext, Future}
+import scala.util.Try
+import scala.util.control.NonFatal
 
 object RestHandler {
   trait CanLookup[A] {
@@ -56,7 +58,7 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
 
   import RestHandler._
   val requestParser = new RequestParser(graph)
-
+  val querySampleRate: Double = graph.config.getDouble("query.log.sample.rate")
 
   /**
     * Public APIS
@@ -69,15 +71,8 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
       val jsQuery = Json.parse(body)
 
       uri match {
-//        case "/graphs/getEdges" => HandlerResult(getEdgesAsync(jsQuery, impIdOpt)(PostProcess.toSimpleVertexArrJson))
-        case "/graphs/getEdges" => HandlerResult(getEdgesAsync(jsQuery, impIdOpt)(PostProcess.toJson))
-//        case "/graphs/getEdges/grouped" => HandlerResult(getEdgesAsync(jsQuery)(PostProcess.summarizeWithListFormatted))
-//        case "/graphs/getEdgesExcluded" => HandlerResult(getEdgesExcludedAsync(jsQuery)(PostProcess.toSimpleVertexArrJson))
-//        case "/graphs/getEdgesExcluded/grouped" => HandlerResult(getEdgesExcludedAsync(jsQuery)(PostProcess.summarizeWithListExcludeFormatted))
+        case "/graphs/getEdges" => HandlerResult(getEdgesAsync(jsQuery, impIdOpt)(PostProcess.toJson(Option(jsQuery))))
         case "/graphs/checkEdges" => checkEdges(jsQuery)
-//        case "/graphs/getEdgesGrouped" => HandlerResult(getEdgesAsync(jsQuery)(PostProcess.summarizeWithList))
-//        case "/graphs/getEdgesGroupedExcluded" => HandlerResult(getEdgesExcludedAsync(jsQuery)(PostProcess.summarizeWithListExclude))
-//        case "/graphs/getEdgesGroupedExcludedFormatted" => HandlerResult(getEdgesExcludedAsync(jsQuery)(PostProcess.summarizeWithListExcludeFormatted))
         case "/graphs/getVertices" => HandlerResult(getVertices(jsQuery))
         case "/graphs/experiments" => experiments(jsQuery)
         case uri if uri.startsWith("/graphs/experiment") =>
@@ -93,17 +88,27 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
   // TODO: Refactor to doGet
   def checkEdges(jsValue: JsValue): HandlerResult = {
     try {
-      val (quads, isReverted) = requestParser.toCheckEdgeParam(jsValue)
-
-      HandlerResult(graph.checkEdges(quads).map { case stepResult =>
-        PostProcess.toJson(graph, QueryOption(), stepResult)
+      val edges = requestParser.toCheckEdgeParam(jsValue)
+
+      HandlerResult(graph.checkEdges(edges).map { case stepResult =>
+        val jsArray = for {
+          s2EdgeWithScore <- stepResult.edgeWithScores
+//          json <- PostProcess.s2EdgeToJsValue(QueryOption(), s2EdgeWithScore)
+          json = PostProcess.s2EdgeToJsValue(QueryOption(), s2EdgeWithScore)
+        } yield json
+        Json.toJson(jsArray)
       })
     } catch {
-      case e: Exception => HandlerResult(Future.failed(e))
+      case e: Exception =>
+        logger.error(s"RestHandler#checkEdges error: $e")
+        HandlerResult(Future.failed(e))
     }
   }
 
 
+  /**
+    * Private APIS
+    */
   private def experiments(jsQuery: JsValue): HandlerResult = {
     val params: Seq[RequestParser.ExperimentParam] = requestParser.parseExperiment(jsQuery)
 
@@ -120,7 +125,7 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
     HandlerResult(body = result)
   }
 
-  private def experiment(contentsBody: JsValue, accessToken: String, experimentName: String, uuid: String, impKeyOpt: => Option[String] = None): HandlerResult = {
+  def experiment(contentsBody: JsValue, accessToken: String, experimentName: String, uuid: String, impKeyOpt: => Option[String] = None): HandlerResult = {
     try {
       val bucketOpt = for {
         service <- Service.findByAccessToken(accessToken)
@@ -128,9 +133,21 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
         bucket <- experiment.findBucket(uuid, impKeyOpt)
       } yield bucket
 
-      val bucket = bucketOpt.getOrElse(throw new RuntimeException("bucket is not found"))
+      val bucket = bucketOpt.getOrElse(throw new RuntimeException(s"bucket is not found. $accessToken, $experimentName, $uuid, $impKeyOpt"))
       if (bucket.isGraphQuery) {
         val ret = buildRequestInner(contentsBody, bucket, uuid)
+
+        logQuery(Json.obj(
+          "type" -> "experiment",
+          "time" -> System.currentTimeMillis(),
+          "body" -> contentsBody,
+          "uri" -> Seq("graphs", "experiment", accessToken, experimentName, uuid).mkString("/"),
+          "accessToken" -> accessToken,
+          "experimentName" -> experimentName,
+          "uuid" -> uuid,
+          "impressionId" -> bucket.impressionId
+        ))
+
         HandlerResult(ret.body, Experiment.ImpressionKey -> bucket.impressionId)
       }
       else throw new RuntimeException("not supported yet")
@@ -156,24 +173,39 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
 
   def getEdgesAsync(jsonQuery: JsValue, impIdOpt: Option[String] = None)
                    (post: (Graph, QueryOption, StepResult) => JsValue): Future[JsValue] = {
-    jsonQuery match {
-      case obj@JsObject(_) =>
-        (obj \ "queries").asOpt[JsValue] match {
-          case None =>
-            val query = requestParser.toQuery(obj, impIdOpt)
-            graph.getEdges(query).map(post(graph, query.queryOption, _))
-          case _ =>
-            val multiQuery = requestParser.toMultiQuery(obj, impIdOpt)
-            graph.getEdgesMultiQuery(multiQuery).map(post(graph, multiQuery.queryOption, _))
-        }
-
-      case JsArray(arr) =>
-        val queries = arr.map(requestParser.toQuery(_, impIdOpt))
-        val weights = queries.map(_ => 1.0)
-        val multiQuery = MultiQuery(queries, weights, QueryOption(), jsonQuery)
-        graph.getEdgesMultiQuery(multiQuery).map(post(graph, multiQuery.queryOption, _))
-
-      case _ => throw BadQueryException("Cannot support")
+
+    def query(obj: JsValue): Future[JsValue] = {
+      (obj \ "queries").asOpt[JsValue] match {
+        case None =>
+          val s2Query = requestParser.toQuery(obj, impIdOpt)
+          graph.getEdges(s2Query).map(post(graph, s2Query.queryOption, _))
+        case _ =>
+          val multiQuery = requestParser.toMultiQuery(obj, impIdOpt)
+          graph.getEdgesMultiQuery(multiQuery).map(post(graph, multiQuery.queryOption, _))
+      }
+    }
+
+    logQuery(Json.obj(
+      "type" -> "getEdges",
+      "time" -> System.currentTimeMillis(),
+      "body" -> jsonQuery,
+      "uri" -> "graphs/getEdges"
+    ))
+
+    val unionQuery = (jsonQuery \ "union").asOpt[JsObject]
+    unionQuery match {
+      case None => jsonQuery match {
+        case obj@JsObject(_) => query(obj)
+        case JsArray(arr) =>
+          val res = arr.map(js => query(js.as[JsObject]))
+          Future.sequence(res).map(JsArray)
+        case _ => throw BadQueryException("Cannot support")
+      }
+
+      case Some(jsUnion) =>
+        val (keys, queries) = jsUnion.value.unzip
+        val futures = queries.map(query)
+        Future.sequence(futures).map(res => JsObject(keys.zip(res).toSeq))
     }
   }
 
@@ -194,7 +226,6 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
     graph.getVertices(vertices) map { vertices => PostProcess.verticesToJson(vertices) }
   }
 
-
   private def buildRequestBody(requestKeyJsonOpt: Option[JsValue], bucket: Bucket, uuid: String): String = {
     var body = bucket.requestBody.replace("#uuid", uuid)
 
@@ -203,19 +234,24 @@ class RestHandler(graph: Graph)(implicit ec: ExecutionContext) {
       jsObj <- requestKeyJson.asOpt[JsObject]
       (key, value) <- jsObj.fieldSet
     } {
+      val escaped = Json.stringify(value)
       val replacement = value match {
-        case JsString(s) => s
-        case _ => value.toString
+        case _: JsString => escaped.slice(1, escaped.length - 1)
+        case _ => escaped
       }
+
       body = body.replace(key, replacement)
     }
 
     body
   }
 
-  def calcSize(js: JsValue): Int = js match {
-    case JsObject(obj) => (js \ "size").asOpt[Int].getOrElse(0)
-    case JsArray(seq) => seq.map(js => (js \ "size").asOpt[Int].getOrElse(0)).sum
-    case _ => 0
+  def calcSize(js: JsValue): Int =
+    (js \\ "size") map { sizeJs => sizeJs.asOpt[Int].getOrElse(0) } sum
+
+  def logQuery(queryJson: => JsObject): Unit = {
+    if (scala.util.Random.nextDouble() < querySampleRate) {
+      logger.query(queryJson.toString)
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 c76c25c..403ceeb 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
@@ -24,24 +24,29 @@ import org.hbase.async.KeyValue
 
 
 object SKeyValue {
+  val EdgeCf = "e".getBytes()
   val Put = 1
   val Delete = 2
   val Increment = 3
   val Default = Put
 }
+
 case class SKeyValue(table: Array[Byte],
                      row: Array[Byte],
                      cf: Array[Byte],
                      qualifier: Array[Byte],
                      value: Array[Byte],
                      timestamp: Long,
-                     operation: Int = SKeyValue.Default) {
+                     operation: Int = SKeyValue.Default,
+                     durability: Boolean = true) {
   def toLogString = {
-    Map("table" -> table.toList, "row" -> row.toList, "cf" -> Bytes.toString(cf),
+    Map("table" -> Bytes.toString(table), "row" -> row.toList, "cf" -> Bytes.toString(cf),
       "qualifier" -> qualifier.toList, "value" -> value.toList, "timestamp" -> timestamp,
-      "operation" -> operation).mapValues(_.toString).toString
+      "operation" -> operation, "durability" -> durability).toString
   }
   override def toString(): String = toLogString
+
+  def toKeyValue: KeyValue = new KeyValue(row, cf, qualifier, timestamp, value)
 }
 
 trait CanSKeyValue[T] {
@@ -64,3 +69,4 @@ object CanSKeyValue {
 
   // For hbase KeyValues
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 a6e81b4..07e39aa 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
@@ -19,53 +19,44 @@
 
 package org.apache.s2graph.core.storage
 
-import java.util.concurrent.{Executors, TimeUnit}
 
-import com.typesafe.config.Config
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.GraphExceptions.FetchTimeoutException
+import org.apache.s2graph.core.GraphExceptions.{NoStackException, FetchTimeoutException}
 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.types._
-import org.apache.s2graph.core.utils.{Extensions, logger}
+import org.apache.s2graph.core.utils.{DeferCache, Extensions, logger}
 
-import scala.annotation.tailrec
-import scala.collection.Seq
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.concurrent.{ExecutionContext, Future, Promise}
 import scala.util.{Random, Try}
+import java.util.concurrent.{Executors, TimeUnit}
+
+import com.typesafe.config.{Config, ConfigFactory}
+import org.apache.hadoop.hbase.util.Bytes
+
 
-abstract class Storage[R](val graph: Graph,
+abstract class Storage[Q, R](val graph: Graph,
                           val config: Config)(implicit ec: ExecutionContext) {
   import HBaseType._
+  import Graph._
 
-  /** storage dependent configurations */
-  val DeleteAllFetchCount = config.getInt("delete.all.fetch.count")
-  val MaxRetryNum = config.getInt("max.retry.number")
-  val MaxBackOff = config.getInt("max.back.off")
-  val BackoffTimeout = config.getInt("back.off.timeout")
-  val DeleteAllFetchSize = config.getInt("delete.all.fetch.size")
-  val FailProb = config.getDouble("hbase.fail.prob")
-  val LockExpireDuration = config.getInt("lock.expire.time")
-  val MaxSize = config.getInt("future.cache.max.size")
-  val ExpireAfterWrite = config.getInt("future.cache.expire.after.write")
-  val ExpireAfterAccess = config.getInt("future.cache.expire.after.access")
+  val 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()
 
 
-  /** handle mutate failed */
-  val exceptionHandler = new ExceptionHandler(config)
-  val failTopic = s"mutateFailed_${config.getString("phase")}"
-
-  /** fallback */
-  val fallback = Future.successful(StepResult.Empty)
-  val innerFallback = Future.successful(StepInnerResult.Empty)
-
   /**
    * Compatibility table
    * | label schema version | snapshot edge | index edge | vertex | note |
@@ -109,7 +100,7 @@ abstract class Storage[R](val graph: Graph,
    * @param vertex: vertex to serialize
    * @return serializer implementation
    */
-  def vertexSerializer(vertex: Vertex) = new VertexSerializable(vertex)
+  def vertexSerializer(vertex: Vertex): Serializable[Vertex] = new VertexSerializable(vertex)
 
   /**
    * create deserializer that can parse stored CanSKeyValue into snapshotEdge.
@@ -142,7 +133,7 @@ abstract class Storage[R](val graph: Graph,
     indexEdgeDeserializers.get(schemaVer).getOrElse(throw new RuntimeException(s"not supported version: ${schemaVer}"))
 
   /** create deserializer that can parser stored CanSKeyValue into vertex. */
-  val vertexDeserializer = new VertexDeserializable
+  val vertexDeserializer: Deserializable[Vertex] = new VertexDeserializable
 
 
   /**
@@ -170,7 +161,7 @@ abstract class Storage[R](val graph: Graph,
    * @param request
    * @return
    */
-  def fetchSnapshotEdgeKeyValues(request: AnyRef): Future[Seq[SKeyValue]]
+  def fetchSnapshotEdgeKeyValues(request: QueryRequest): Future[Seq[SKeyValue]]
 
   /**
    * write requestKeyValue into storage if the current value in storage that is stored matches.
@@ -200,10 +191,11 @@ abstract class Storage[R](val graph: Graph,
    * 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
+    *
+    * @param queryRequest
    * @return
-   */
-  def buildRequest(queryRequest: QueryRequest): AnyRef
+    */
+  protected def buildRequest(queryRequest: QueryRequest, edge: Edge): Q
 
   /**
    * fetch IndexEdges for given queryParam in queryRequest.
@@ -215,53 +207,55 @@ abstract class Storage[R](val graph: Graph,
    * 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 prevStepScore
+    *
+    * @param queryRequest
    * @param isInnerCall
    * @param parentEdges
    * @return
    */
   def fetch(queryRequest: QueryRequest,
-            prevStepScore: Double,
             isInnerCall: Boolean,
             parentEdges: Seq[EdgeWithScore]): R
 
   /**
    * responsible to fire parallel fetch call into storage and create future that will return merged result.
-   * @param queryRequestWithScoreLs
+   *
+   * @param queryRequests
    * @param prevStepEdges
    * @return
    */
-  def fetches(queryRequestWithScoreLs: Seq[(QueryRequest, Double)],
-              prevStepEdges: Map[VertexId, Seq[EdgeWithScore]]): Future[Seq[StepInnerResult]]
+  def fetches(queryRequests: Seq[QueryRequest],
+              prevStepEdges: Map[VertexId, Seq[EdgeWithScore]]): Future[Seq[StepResult]]
 
   /**
    * fetch Vertex for given request from storage.
-   * @param request
+    *
+    * @param request
    * @return
    */
-  def fetchVertexKeyValues(request: AnyRef): Future[Seq[SKeyValue]]
+  def fetchVertexKeyValues(request: QueryRequest): Future[Seq[SKeyValue]]
 
   /**
    * decide how to apply given edges(indexProps values + Map(_count -> countVal)) into storage.
-   * @param edges
+    *
+    * @param edges
    * @param withWait
    * @return
    */
-  def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long)]]
+  def incrementCounts(edges: Seq[Edge], 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 = {
-    exceptionHandler.shutdown()
   }
 
   /**
    * create table on storage.
    * if storage implementation does not support namespace or table, then there is nothing to be done
-   * @param zkAddr
+    *
+    * @param zkAddr
    * @param tableName
    * @param cfs
    * @param regionMultiplier
@@ -273,28 +267,29 @@ abstract class Storage[R](val graph: Graph,
                   cfs: List[String],
                   regionMultiplier: Int,
                   ttl: Option[Int],
-                  compressionAlgorithm: String): Unit
+                  compressionAlgorithm: String,
+                  replicationScopeOpt: Option[Int] = None,
+                  totalRegionCount: Option[Int] = None): Unit
 
 
 
 
 
   /** Public Interface */
-
   def getVertices(vertices: Seq[Vertex]): Future[Seq[Vertex]] = {
-    def fromResult(queryParam: QueryParam,
-                   kvs: Seq[SKeyValue],
+    def fromResult(kvs: Seq[SKeyValue],
                    version: String): Option[Vertex] = {
       if (kvs.isEmpty) None
-      else vertexDeserializer.fromKeyValues(queryParam, kvs, version, None)
+      else vertexDeserializer.fromKeyValues(None, kvs, version, None)
+//        .map(S2Vertex(graph, _))
     }
 
     val futures = vertices.map { vertex =>
       val queryParam = QueryParam.Empty
       val q = Query.toQuery(Seq(vertex), queryParam)
       val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
-      fetchVertexKeyValues(buildRequest(queryRequest)).map { kvs =>
-        fromResult(queryParam, kvs, vertex.serviceColumn.schemaVersion)
+      fetchVertexKeyValues(queryRequest).map { kvs =>
+        fromResult(kvs, vertex.serviceColumn.schemaVersion)
       } recoverWith { case ex: Throwable =>
         Future.successful(None)
       }
@@ -302,92 +297,39 @@ abstract class Storage[R](val graph: Graph,
 
     Future.sequence(futures).map { result => result.toList.flatten }
   }
-
-  def mutateElements(elements: Seq[GraphElement],
-                     withWait: Boolean = false): Future[Seq[Boolean]] = {
-
-    val edgeBuffer = ArrayBuffer[Edge]()
-    val vertexBuffer = ArrayBuffer[Vertex]()
-
-    elements.foreach {
-      case e: Edge => edgeBuffer += e
-      case v: Vertex => vertexBuffer += v
-      case any@_ => logger.error(s"Unknown type: ${any}")
-    }
-
-    val edgeFuture = mutateEdges(edgeBuffer, withWait)
-    val vertexFuture = mutateVertices(vertexBuffer, withWait)
-
-    val graphFuture = for {
-      edgesMutated <- edgeFuture
-      verticesMutated <- vertexFuture
-    } yield edgesMutated ++ verticesMutated
-
-    graphFuture
-  }
-
-  def mutateEdges(edges: Seq[Edge], withWait: Boolean): Future[Seq[Boolean]] = {
-    val (strongEdges, weakEdges) =
-      (edges.partition(e => e.label.consistencyLevel == "strong" && e.op != GraphUtil.operations("insertBulk")))
-
-    val weakEdgesFutures = weakEdges.groupBy { e => e.label.hbaseZkAddr }.map { case (zkQuorum, edges) =>
-      val mutations = edges.flatMap { edge =>
-        val (_, edgeUpdate) =
-          if (edge.op == GraphUtil.operations("delete")) Edge.buildDeleteBulk(None, edge)
-          else Edge.buildOperation(None, Seq(edge))
-
-        buildVertexPutsAsync(edge) ++ indexedEdgeMutations(edgeUpdate) ++
-          snapshotEdgeMutations(edgeUpdate) ++ increments(edgeUpdate)
-      }
-      writeToStorage(zkQuorum, mutations, withWait)
-    }
-    val strongEdgesFutures = mutateStrongEdges(strongEdges, withWait)
-    for {
-      weak <- Future.sequence(weakEdgesFutures)
-      strong <- strongEdgesFutures
-    } yield {
-      strong ++ weak
-    }
-  }
-
   def mutateStrongEdges(_edges: Seq[Edge], withWait: Boolean): Future[Seq[Boolean]] = {
 
-    val grouped = _edges.groupBy { edge => (edge.label, edge.srcVertex.innerId, edge.tgtVertex.innerId) } toSeq
+    val edgeWithIdxs = _edges.zipWithIndex
+    val grouped = edgeWithIdxs.groupBy { case (edge, idx) =>
+      (edge.label, edge.srcVertex.innerId, edge.tgtVertex.innerId)
+    } toSeq
 
     val mutateEdges = grouped.map { case ((_, _, _), edgeGroup) =>
-      val (deleteAllEdges, edges) = edgeGroup.partition(_.op == GraphUtil.operations("deleteAll"))
-
-      // DeleteAll first
-      val deleteAllFutures = deleteAllEdges.map { edge =>
-        deleteAllAdjacentEdges(Seq(edge.srcVertex), Seq(edge.label), edge.labelWithDir.dir, edge.ts)
-      }
-
+      val edges = edgeGroup.map(_._1)
+      val idxs = edgeGroup.map(_._2)
       // After deleteAll, process others
-      lazy val mutateEdgeFutures = edges.toList match {
+      val mutateEdgeFutures = edges.toList match {
         case head :: tail =>
-          //          val strongConsistency = edges.head.label.consistencyLevel == "strong"
-          //          if (strongConsistency) {
           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.label.hbaseZkAddr, puts, withWait)
           Seq(edgeFuture, vertexFuture)
-        //          } else {
-        //            edges.map { edge => mutateEdge(edge, withWait = withWait) }
-        //          }
         case Nil => Nil
       }
 
       val composed = for {
-        deleteRet <- Future.sequence(deleteAllFutures)
+//        deleteRet <- Future.sequence(deleteAllFutures)
         mutateRet <- Future.sequence(mutateEdgeFutures)
-      } yield deleteRet ++ mutateRet
+      } yield mutateRet
 
-      composed.map(_.forall(identity))
+      composed.map(_.forall(identity)).map { ret => idxs.map(idx => idx -> ret) }
     }
 
-    Future.sequence(mutateEdges)
+    Future.sequence(mutateEdges).map { squashedRets =>
+      squashedRets.flatten.sortBy { case (idx, ret) => idx }.map(_._2)
+    }
   }
 
   def mutateVertex(vertex: Vertex, withWait: Boolean): Future[Boolean] = {
@@ -413,16 +355,22 @@ abstract class Storage[R](val graph: Graph,
                        checkConsistency: Boolean,
                        withWait: Boolean): Future[Boolean] = {
     assert(edges.nonEmpty)
+    // TODO:: remove after code review: unreachable code
     if (!checkConsistency) {
+
       val zkQuorum = edges.head.label.hbaseZkAddr
       val futures = edges.map { edge =>
         val (_, edgeUpdate) = Edge.buildOperation(None, Seq(edge))
-        val mutations = indexedEdgeMutations(edgeUpdate) ++ snapshotEdgeMutations(edgeUpdate) ++ increments(edgeUpdate)
+
+        val mutations =
+          indexedEdgeMutations(edgeUpdate) ++ snapshotEdgeMutations(edgeUpdate) ++ increments(edgeUpdate)
+
+
         writeToStorage(zkQuorum, mutations, withWait)
       }
       Future.sequence(futures).map { rets => rets.forall(identity) }
     } else {
-      fetchSnapshotEdge(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+      fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
         retry(1)(edges, 0, snapshotEdgeOpt)
       }
     }
@@ -438,9 +386,6 @@ abstract class Storage[R](val graph: Graph,
     if (tryNum >= MaxRetryNum) {
       edges.foreach { edge =>
         logger.error(s"commit failed after $MaxRetryNum\n${edge.toLogString}")
-
-        val kafkaMessage = ExceptionHandler.toKafkaMessage(failTopic, element = edge)
-        exceptionHandler.enqueue(kafkaMessage)
       }
 
       Future.successful(false)
@@ -454,11 +399,10 @@ abstract class Storage[R](val graph: Graph,
         case FetchTimeoutException(retryEdge) =>
           logger.info(s"[Try: $tryNum], Fetch fail.\n${retryEdge}")
           /** fetch failed. re-fetch should be done */
-          fetchSnapshotEdge(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+          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."
@@ -477,7 +421,7 @@ abstract class Storage[R](val graph: Graph,
               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 */
-                fetchSnapshotEdge(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
+                fetchSnapshotEdgeInner(edges.head).flatMap { case (queryParam, snapshotEdgeOpt, kvOpt) =>
                   retry(tryNum + 1)(edges, statusCode, snapshotEdgeOpt)
                 }
               } else {
@@ -620,7 +564,8 @@ abstract class Storage[R](val graph: Graph,
   /**
    * 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 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.
@@ -643,7 +588,7 @@ abstract class Storage[R](val graph: Graph,
     } yield lockReleased
   }
 
-  case class PartialFailureException(edge: Edge, statusCode: Byte, failReason: String) extends Exception
+  case class PartialFailureException(edge: Edge, 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")
@@ -658,7 +603,8 @@ abstract class Storage[R](val graph: Graph,
 
   /**
    * try to acquire lock on storage for this given snapshotEdge(lockEdge).
-   * @param statusCode: current statusCode of this thread to process edges.
+    *
+    * @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.
@@ -708,7 +654,8 @@ abstract class Storage[R](val graph: Graph,
    * 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 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.
@@ -797,6 +744,18 @@ abstract class Storage[R](val graph: Graph,
                           statusCode: Byte,
                           squashedEdge: Edge,
                           edgeMutate: EdgeMutate): Future[Boolean] = {
+
+    def _write(kvs: Seq[SKeyValue], withWait: Boolean): Future[Boolean] = {
+      writeToStorage(squashedEdge.label.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}")
@@ -804,21 +763,13 @@ abstract class Storage[R](val graph: Graph,
     } else {
       val p = Random.nextDouble()
       if (p < FailProb) Future.failed(new PartialFailureException(squashedEdge, 2, s"$p"))
-      else
-        writeToStorage(squashedEdge.label.hbaseZkAddr, increments(edgeMutate), withWait = true).map { ret =>
-          if (ret) {
-            debug(ret, "increment", squashedEdge.toSnapshotEdge, edgeMutate)
-          } else {
-            throw new PartialFailureException(squashedEdge, 2, "hbase fail.")
-          }
-          true
-        }
+      else {
+        val incrs = increments(edgeMutate)
+        _write(incrs, true)
+      }
     }
   }
 
-
-
-
   /** end of methods for consistency */
 
   def mutateLog(snapshotEdgeOpt: Option[Edge], edges: Seq[Edge],
@@ -832,17 +783,18 @@ abstract class Storage[R](val graph: Graph,
 
 
   /** Delete All */
-  protected def deleteAllFetchedEdgesAsyncOld(stepInnerResult: StepInnerResult,
+  def deleteAllFetchedEdgesAsyncOld(stepInnerResult: StepResult,
                                               requestTs: Long,
                                               retryNum: Int): Future[Boolean] = {
     if (stepInnerResult.isEmpty) Future.successful(true)
     else {
-      val head = stepInnerResult.edgesWithScoreLs.head
+      val head = stepInnerResult.edgeWithScores.head
       val zkQuorum = head.edge.label.hbaseZkAddr
       val futures = for {
-        edgeWithScore <- stepInnerResult.edgesWithScoreLs
-        (edge, score) = EdgeWithScore.unapply(edgeWithScore).get
+        edgeWithScore <- stepInnerResult.edgeWithScores
       } yield {
+          val edge = edgeWithScore.edge
+          val score = edgeWithScore.score
           /** reverted direction */
           val reversedIndexedEdgesMutations = edge.duplicateEdge.edgesWithIndex.flatMap { indexEdge =>
             indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete)) ++
@@ -861,141 +813,6 @@ abstract class Storage[R](val graph: Graph,
     }
   }
 
-  protected def buildEdgesToDelete(stepInnerResult: StepInnerResult, requestTs: Long): StepInnerResult = {
-    val filtered = stepInnerResult.edgesWithScoreLs.filter { edgeWithScore =>
-      (edgeWithScore.edge.ts < requestTs) && !edgeWithScore.edge.isDegree
-    }
-    if (filtered.isEmpty) StepInnerResult.Empty
-    else {
-      val head = filtered.head
-      val label = head.edge.label
-      val edgeWithScoreLs = filtered.map { edgeWithScore =>
-        val (newOp, newVersion, newPropsWithTs) = label.consistencyLevel match {
-          case "strong" =>
-            val _newPropsWithTs = edgeWithScore.edge.propsWithTs ++
-              Map(LabelMeta.timeStampSeq -> InnerValLikeWithTs.withLong(requestTs, requestTs, label.schemaVersion))
-            (GraphUtil.operations("delete"), requestTs, _newPropsWithTs)
-          case _ =>
-            val oldEdge = edgeWithScore.edge
-            (oldEdge.op, oldEdge.version, oldEdge.propsWithTs)
-        }
-
-        val copiedEdge =
-          edgeWithScore.edge.copy(op = newOp, version = newVersion, propsWithTs = newPropsWithTs)
-
-        val edgeToDelete = edgeWithScore.copy(edge = copiedEdge)
-        //      logger.debug(s"delete edge from deleteAll: ${edgeToDelete.edge.toLogString}")
-        edgeToDelete
-      }
-      //Degree edge?
-      StepInnerResult(edgeWithScoreLs, Nil, false)
-    }
-  }
-
-  protected def deleteAllFetchedEdgesLs(stepInnerResultLs: Seq[StepInnerResult],
-                                        requestTs: Long): Future[(Boolean, Boolean)] = {
-    stepInnerResultLs.foreach { stepInnerResult =>
-      if (stepInnerResult.isFailure) throw new RuntimeException("fetched result is fallback.")
-    }
-    val futures = for {
-      stepInnerResult <- stepInnerResultLs
-      deleteStepInnerResult = buildEdgesToDelete(stepInnerResult, requestTs)
-      if deleteStepInnerResult.edgesWithScoreLs.nonEmpty
-    } yield {
-        val head = deleteStepInnerResult.edgesWithScoreLs.head
-        val label = head.edge.label
-        label.schemaVersion match {
-          case HBaseType.VERSION3 | HBaseType.VERSION4 =>
-            if (label.consistencyLevel == "strong") {
-              /**
-               * read: snapshotEdge on queryResult = O(N)
-               * write: N x (relatedEdges x indices(indexedEdge) + 1(snapshotEdge))
-               */
-              mutateEdges(deleteStepInnerResult.edgesWithScoreLs.map(_.edge), withWait = true).map(_.forall(identity))
-            } else {
-              deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
-            }
-          case _ =>
-
-            /**
-             * read: x
-             * write: N x ((1(snapshotEdge) + 2(1 for incr, 1 for delete) x indices)
-             */
-            deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
-        }
-      }
-
-    if (futures.isEmpty) {
-      // all deleted.
-      Future.successful(true -> true)
-    } else {
-      Future.sequence(futures).map { rets => false -> rets.forall(identity) }
-    }
-  }
-
-  protected def fetchAndDeleteAll(queries: Seq[Query], requestTs: Long): Future[(Boolean, Boolean)] = {
-    val future = for {
-      stepInnerResultLs <- Future.sequence(queries.map(getEdgesStepInner(_)))
-      (allDeleted, ret) <- deleteAllFetchedEdgesLs(stepInnerResultLs, requestTs)
-    } yield {
-//        logger.debug(s"fetchAndDeleteAll: ${allDeleted}, ${ret}")
-        (allDeleted, ret)
-      }
-
-    Extensions.retryOnFailure(MaxRetryNum) {
-      future
-    } {
-      logger.error(s"fetch and deleteAll failed.")
-      (true, false)
-    }
-
-  }
-
-  def deleteAllAdjacentEdges(srcVertices: Seq[Vertex],
-                             labels: Seq[Label],
-                             dir: Int,
-                             ts: Long): Future[Boolean] = {
-
-    def enqueueLogMessage() = {
-      val kafkaMessages = for {
-        vertice <- srcVertices
-        id = vertice.innerId.toIdString()
-        label <- labels
-      } yield {
-          val tsv = Seq(ts, "deleteAll", "e", id, id, label.label, "{}", GraphUtil.fromOp(dir.toByte)).mkString("\t")
-          ExceptionHandler.toKafkaMessage(failTopic, tsv)
-        }
-
-      kafkaMessages.foreach(exceptionHandler.enqueue)
-    }
-
-    val requestTs = ts
-    /** create query per label */
-    val queries = for {
-      label <- labels
-    } yield {
-        val labelWithDir = LabelWithDirection(label.id.get, dir)
-        val queryParam = QueryParam(labelWithDir).limit(0, DeleteAllFetchSize).duplicatePolicy(Option(Query.DuplicatePolicy.Raw))
-        val step = Step(List(queryParam))
-        Query(srcVertices, Vector(step))
-      }
-
-    //    Extensions.retryOnSuccessWithBackoff(MaxRetryNum, Random.nextInt(MaxBackOff) + 1) {
-        val retryFuture = Extensions.retryOnSuccess(DeleteAllFetchCount) {
-      fetchAndDeleteAll(queries, requestTs)
-    } { case (allDeleted, deleteSuccess) =>
-      allDeleted && deleteSuccess
-    }.map { case (allDeleted, deleteSuccess) => allDeleted && deleteSuccess }
-
-    retryFuture onFailure {
-      case ex =>
-        logger.error(s"[Error]: deleteAllAdjacentEdges failed.")
-        enqueueLogMessage()
-    }
-
-    retryFuture
-  }
-
   /** End Of Delete All */
 
 
@@ -1003,29 +820,34 @@ abstract class Storage[R](val graph: Graph,
 
   /** Parsing Logic: parse from kv from Storage into Edge */
   def toEdge[K: CanSKeyValue](kv: K,
-                              queryParam: QueryParam,
+                              queryRequest: QueryRequest,
                               cacheElementOpt: Option[IndexEdge],
                               parentEdges: Seq[EdgeWithScore]): Option[Edge] = {
-//        logger.debug(s"toEdge: $kv")
+    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(queryParam, Seq(kv), queryParam.label.schemaVersion, cacheElementOpt)
-      indexEdgeOpt.map(indexEdge => indexEdge.toEdge.copy(parentEdges = parentEdges))
+      val indexEdgeOpt = indexEdgeDeserializer(schemaVer).fromKeyValues(Option(queryParam.label), Seq(kv), queryParam.label.schemaVersion, cacheElementOpt)
+      if (!queryOption.returnTree) indexEdgeOpt.map(indexEdge => indexEdge.toEdge.copy(parentEdges = parentEdges))
+      else indexEdgeOpt.map(indexEdge => indexEdge.toEdge)
     } catch {
       case ex: Exception =>
-        logger.error(s"Fail on toEdge: ${kv.toString}, ${queryParam}", ex)
+        logger.error(s"Fail on toEdge: ${kv.toString}, ${queryRequest}", ex)
         None
     }
   }
 
   def toSnapshotEdge[K: CanSKeyValue](kv: K,
-                                      queryParam: QueryParam,
+                                      queryRequest: QueryRequest,
                                       cacheElementOpt: Option[SnapshotEdge] = None,
                                       isInnerCall: Boolean,
                                       parentEdges: Seq[EdgeWithScore]): Option[Edge] = {
 //        logger.debug(s"SnapshottoEdge: $kv")
+    val queryParam = queryRequest.queryParam
     val schemaVer = queryParam.label.schemaVersion
-    val snapshotEdgeOpt = snapshotEdgeDeserializer(schemaVer).fromKeyValues(queryParam, Seq(kv), queryParam.label.schemaVersion, cacheElementOpt)
+    val snapshotEdgeOpt = snapshotEdgeDeserializer(schemaVer).fromKeyValues(Option(queryParam.label), Seq(kv), queryParam.label.schemaVersion, cacheElementOpt)
 
     if (isInnerCall) {
       snapshotEdgeOpt.flatMap { snapshotEdge =>
@@ -1045,87 +867,137 @@ abstract class Storage[R](val graph: Graph,
     }
   }
 
+  val dummyCursor: Array[Byte] = Array.empty
+
   def toEdges[K: CanSKeyValue](kvs: Seq[K],
-                               queryParam: QueryParam,
+                               queryRequest: QueryRequest,
                                prevScore: Double = 1.0,
                                isInnerCall: Boolean,
                                parentEdges: Seq[EdgeWithScore],
                                startOffset: Int = 0,
-                               len: Int = Int.MaxValue): Seq[EdgeWithScore] = {
-    if (kvs.isEmpty) Seq.empty
+                               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(queryParam, Seq(kv), queryParam.label.schemaVersion, None)
+        else indexEdgeDeserializer(schemaVer).fromKeyValues(Option(queryParam.label), Seq(kv), queryParam.label.schemaVersion, None)
+
+      val (degreeEdges, keyValues) = cacheElementOpt match {
+        case None => (Nil, kvs)
+        case Some(cacheElement) =>
+          val head = cacheElement.toEdge
+          if (!head.isDegree) (Nil, kvs)
+          else (Seq(EdgeWithScore(head, 1.0, label)), kvs.tail)
+      }
 
-      for {
-        (kv, idx) <- kvs.zipWithIndex if idx >= startOffset && idx < startOffset + len
-        edge <-
-        if (queryParam.isSnapshotEdge) toSnapshotEdge(kv, queryParam, None, isInnerCall, parentEdges)
-        else toEdge(kv, queryParam, cacheElementOpt, parentEdges)
-      } yield {
-        //TODO: Refactor this.
-        val currentScore =
-          queryParam.scorePropagateOp match {
-            case "plus" => edge.rank(queryParam.rank) + prevScore
-            case "divide" =>
-              if ((prevScore + queryParam.scorePropagateShrinkage) == 0) 0
-              else edge.rank(queryParam.rank) / (prevScore + queryParam.scorePropagateShrinkage)
-            case _ => edge.rank(queryParam.rank) * prevScore
+      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)
           }
-        EdgeWithScore(edge, currentScore)
+
+        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): Edge = {
+  protected def toRequestEdge(queryRequest: QueryRequest, parentEdges: Seq[EdgeWithScore]): Edge = {
     val srcVertex = queryRequest.vertex
-    //    val tgtVertexOpt = queryRequest.tgtVertexOpt
-    val edgeCf = Serializable.edgeCf
     val queryParam = queryRequest.queryParam
     val tgtVertexIdOpt = queryParam.tgtVertexInnerIdOpt
     val label = queryParam.label
     val labelWithDir = queryParam.labelWithDir
     val (srcColumn, tgtColumn) = label.srcTgtColumn(labelWithDir.dir)
-    val (srcInnerId, tgtInnerId) = tgtVertexIdOpt match {
+    val propsWithTs = label.EmptyPropsWithTs
+
+    tgtVertexIdOpt match {
       case Some(tgtVertexId) => // _to is given.
         /** we use toSnapshotEdge so dont need to swap src, tgt */
         val src = InnerVal.convertVersion(srcVertex.innerId, srcColumn.columnType, label.schemaVersion)
         val tgt = InnerVal.convertVersion(tgtVertexId, tgtColumn.columnType, label.schemaVersion)
-        (src, tgt)
+        val (srcVId, tgtVId) = (SourceVertexId(srcColumn.id.get, src), TargetVertexId(tgtColumn.id.get, tgt))
+        val (srcV, tgtV) = (Vertex(srcVId), Vertex(tgtVId))
+
+        Edge(srcV, tgtV, label, labelWithDir.dir, propsWithTs = propsWithTs)
       case None =>
         val src = InnerVal.convertVersion(srcVertex.innerId, srcColumn.columnType, label.schemaVersion)
-        (src, src)
-    }
+        val srcVId = SourceVertexId(srcColumn.id.get, src)
+        val srcV = Vertex(srcVId)
 
-    val (srcVId, tgtVId) = (SourceVertexId(srcColumn.id.get, srcInnerId), TargetVertexId(tgtColumn.id.get, tgtInnerId))
-    val (srcV, tgtV) = (Vertex(srcVId), Vertex(tgtVId))
-    val currentTs = System.currentTimeMillis()
-    val propsWithTs = Map(LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(currentTs, label.schemaVersion), currentTs)).toMap
-    Edge(srcV, tgtV, labelWithDir, propsWithTs = propsWithTs)
+        Edge(srcV, srcV, label, labelWithDir.dir, propsWithTs = propsWithTs, parentEdges = parentEdges)
+    }
   }
 
+  protected def fetchSnapshotEdgeInner(edge: Edge): Future[(QueryParam, Option[Edge], Option[SKeyValue])] = {
+    /** TODO: Fix this. currently fetchSnapshotEdge should not use future cache
+      * so use empty cacheKey.
+      * */
+    val queryParam = QueryParam(labelName = edge.label.label,
+      direction = GraphUtil.fromDirection(edge.labelWithDir.dir),
+      tgtVertexIdOpt = Option(edge.tgtVertex.innerIdVal),
+      cacheTTLInMillis = -1)
+    val q = Query.toQuery(Seq(edge.srcVertex), queryParam)
+    val queryRequest = QueryRequest(q, 0, edge.srcVertex, queryParam)
+    //    val q = Query.toQuery(Seq(edge.srcVertex), queryParam)
 
 
-  protected def fetchSnapshotEdge(edge: Edge): Future[(QueryParam, Option[Edge], Option[SKeyValue])] = {
-    val labelWithDir = edge.labelWithDir
-    val queryParam = QueryParam(labelWithDir)
-    val _queryParam = queryParam.tgtVertexInnerIdOpt(Option(edge.tgtVertex.innerId))
-    val q = Query.toQuery(Seq(edge.srcVertex), _queryParam)
-    val queryRequest = QueryRequest(q, 0, edge.srcVertex, _queryParam)
-
-    fetchSnapshotEdgeKeyValues(buildRequest(queryRequest)).map { kvs =>
+    fetchSnapshotEdgeKeyValues(queryRequest).map { kvs =>
       val (edgeOpt, kvOpt) =
         if (kvs.isEmpty) (None, None)
         else {
-          val _edgeOpt = toEdges(kvs, queryParam, 1.0, isInnerCall = true, parentEdges = Nil).headOption.map(_.edge)
+          val snapshotEdgeOpt = toSnapshotEdge(kvs.head, queryRequest, isInnerCall = true, parentEdges = Nil)
           val _kvOpt = kvs.headOption
-          (_edgeOpt, _kvOpt)
+          (snapshotEdgeOpt, _kvOpt)
         }
       (queryParam, edgeOpt, kvOpt)
     } recoverWith { case ex: Throwable =>
@@ -1134,182 +1006,7 @@ abstract class Storage[R](val graph: Graph,
     }
   }
 
-  protected def fetchStep(orgQuery: Query,
-                          stepIdx: Int,
-                          stepInnerResult: StepInnerResult): Future[StepInnerResult] = {
-    if (stepInnerResult.isEmpty) Future.successful(StepInnerResult.Empty)
-    else {
-      val edgeWithScoreLs = stepInnerResult.edgesWithScoreLs
-
-      val q = orgQuery
-
-      val prevStepOpt = if (stepIdx > 0) Option(q.steps(stepIdx - 1)) else None
-      val prevStepThreshold = prevStepOpt.map(_.nextStepScoreThreshold).getOrElse(QueryParam.DefaultThreshold)
-      val prevStepLimit = prevStepOpt.map(_.nextStepLimit).getOrElse(-1)
-      val step = q.steps(stepIdx)
-
-      val alreadyVisited =
-        if (stepIdx == 0) Map.empty[(LabelWithDirection, Vertex), Boolean]
-        else Graph.alreadyVisitedVertices(stepInnerResult.edgesWithScoreLs)
-
-      val groupedBy = edgeWithScoreLs.map { case edgeWithScore =>
-        edgeWithScore.edge.tgtVertex -> edgeWithScore
-      }.groupBy { case (vertex, edgeWithScore) => vertex }
-
-      val groupedByFiltered = for {
-        (vertex, edgesWithScore) <- groupedBy
-        aggregatedScore = edgesWithScore.map(_._2.score).sum if aggregatedScore >= prevStepThreshold
-      } yield vertex -> aggregatedScore
-
-      val prevStepTgtVertexIdEdges = for {
-        (vertex, edgesWithScore) <- groupedBy
-      } yield vertex.id -> edgesWithScore.map { case (vertex, edgeWithScore) => edgeWithScore }
-
-      val nextStepSrcVertices = if (prevStepLimit >= 0) {
-        groupedByFiltered.toSeq.sortBy(-1 * _._2).take(prevStepLimit)
-      } else {
-        groupedByFiltered.toSeq
-      }
-
-      val queryRequests = for {
-        (vertex, prevStepScore) <- nextStepSrcVertices
-        queryParam <- step.queryParams
-      } yield (QueryRequest(q, stepIdx, vertex, queryParam), prevStepScore)
-
-      val fetchedLs = fetches(queryRequests, prevStepTgtVertexIdEdges)
-      Graph.filterEdges(orgQuery, stepIdx, queryRequests.map(_._1), fetchedLs, orgQuery.steps(stepIdx).queryParams, alreadyVisited)(ec)
-    }
-  }
-  private def getEdgesStepInner(q: Query): Future[StepInnerResult] = {
-    Try {
-      if (q.steps.isEmpty) innerFallback
-      else {
-        // current stepIdx = -1
-        val startStepInnerResult = QueryResult.fromVertices(q)
-        q.steps.zipWithIndex.foldLeft(Future.successful(startStepInnerResult)) { case (prevStepInnerResultFuture, (step, stepIdx)) =>
-          for {
-            prevStepInnerResult <- prevStepInnerResultFuture
-            currentStepInnerResult <- fetchStep(q, stepIdx, prevStepInnerResult)
-          } yield currentStepInnerResult
-        }
-      }
-    } recover {
-      case e: Exception =>
-        logger.error(s"getEdgesAsync: $e", e)
-        innerFallback
-    } get
-  }
-  def getEdges(q: Query): Future[StepResult] = {
-    Try {
-      if (q.steps.isEmpty) {
-        // TODO: this should be get vertex query.
-        fallback
-      } else {
-        val filterOutFuture = q.queryOption.filterOutQuery match {
-          case None => innerFallback
-          case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
-        }
-        for {
-          innerResult <- getEdgesStepInner(q)
-          filterOutInnerResult <- filterOutFuture
-        } yield {
-          val result = StepResult(graph, q.queryOption, innerResult)
-          if (filterOutInnerResult.isEmpty) result
-          else {
-            StepResult.filterOut(graph, q.queryOption, result, filterOutInnerResult)
-          }
-        }
-      }
-    } recover {
-      case e: Exception =>
-        logger.error(s"getEdgesAsync: $e", e)
-        fallback
-    } get
-  }
 
-  def getEdgesMultiQuery(mq: MultiQuery): Future[StepResult] = {
-    val fallback = Future.successful(StepResult.Empty)
-
-    Try {
-      if (mq.queries.isEmpty) fallback
-      else {
-        val filterOutFuture = mq.queryOption.filterOutQuery match {
-          case None => innerFallback
-          case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
-        }
-
-        val multiQueryFutures = Future.sequence(mq.queries.map { query => getEdges(query) })
-        for {
-          multiQueryResults <- multiQueryFutures
-          filterOutInnerResult <- filterOutFuture
-        } yield {
-          val merged = StepResult.merges(mq.queryOption, multiQueryResults, mq.weights)
-          StepResult.filterOut(graph, mq.queryOption, merged, filterOutInnerResult)
-        }
-      }
-    } recover {
-      case e: Exception =>
-        logger.error(s"getEdgesAsync: $e", e)
-        fallback
-    } get
-  }
-
-  def checkEdges(params: Seq[(Vertex, Vertex, QueryParam)]): Future[StepResult] = {
-    val ts = System.currentTimeMillis()
-    val futures = for {
-      (srcVertex, tgtVertex, queryParam) <- params
-      propsWithTs = Map(LabelMeta.timeStampSeq -> InnerValLikeWithTs.withLong(ts, ts, queryParam.label.schemaVersion))
-      edge = Edge(srcVertex, tgtVertex, queryParam.labelWithDir, propsWithTs = propsWithTs)
-    } yield {
-        fetchSnapshotEdge(edge).map { case (queryParam, edgeOpt, kvOpt) =>
-          edgeOpt.toSeq.map(e => EdgeWithScore(e, 1.0))
-        }
-      }
-
-    Future.sequence(futures).map { edgeWithScoreLs =>
-      val s2EdgeWithScoreLs = edgeWithScoreLs.flatMap { ls =>
-        ls.map { edgeWithScore =>
-          S2EdgeWithScore(edgeWithScore.edge, edgeWithScore.score)
-        }
-      }
-      StepResult(results = s2EdgeWithScoreLs, grouped = Nil, degreeEdges = Nil)
-    }
-  }
-
-
-
-  @tailrec
-  final def randomInt(sampleNumber: Int, range: Int, set: Set[Int] = Set.empty[Int]): Set[Int] = {
-    if (range < sampleNumber || set.size == sampleNumber) set
-    else randomInt(sampleNumber, range, set + Random.nextInt(range))
-  }
-
-  protected def sample(queryRequest: QueryRequest, edges: Seq[EdgeWithScore], n: Int): Seq[EdgeWithScore] = {
-    if (edges.size <= n){
-      edges
-    }else{
-      val plainEdges = if (queryRequest.queryParam.offset == 0) {
-        edges.tail
-      } else edges
-
-      val randoms = randomInt(n, plainEdges.size)
-      var samples = List.empty[EdgeWithScore]
-      var idx = 0
-      plainEdges.foreach { e =>
-        if (randoms.contains(idx)) samples = e :: samples
-        idx += 1
-      }
-      samples.toSeq
-    }
-
-  }
-
-  protected def normalize(edgeWithScores: Seq[EdgeWithScore]): Seq[EdgeWithScore] = {
-    val sum = edgeWithScores.foldLeft(0.0) { case (acc, cur) => acc + cur.score }
-    edgeWithScores.map { edgeWithScore =>
-      edgeWithScore.copy(score = edgeWithScore.score / sum)
-    }
-  }
   /** end of query */
 
   /** Mutation Builder */
@@ -1317,51 +1014,78 @@ abstract class Storage[R](val graph: Graph,
 
   /** EdgeMutate */
   def indexedEdgeMutations(edgeMutate: EdgeMutate): Seq[SKeyValue] = {
+    // skip sampling for delete operation
     val deleteMutations = edgeMutate.edgesToDelete.flatMap { indexEdge =>
-      indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete))
+      indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Delete, durability = indexEdge.label.durability))
     }
+
     val insertMutations = edgeMutate.edgesToInsert.flatMap { indexEdge =>
-      indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put))
+      if (indexEdge.isOutEdge) indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
+      else {
+        // For InEdge
+        indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
+      }
     }
 
     deleteMutations ++ insertMutations
   }
 
   def snapshotEdgeMutations(edgeMutate: EdgeMutate): Seq[SKeyValue] =
-    edgeMutate.newSnapshotEdge.map(e => snapshotEdgeSerializer(e).toKeyValues).getOrElse(Nil)
+    edgeMutate.newSnapshotEdge.map(e => snapshotEdgeSerializer(e).toKeyValues.map(_.copy(durability = e.label.durability))).getOrElse(Nil)
+
+  def incrementsInOut(edgeMutate: EdgeMutate): (Seq[SKeyValue], Seq[SKeyValue]) = {
+
+    def filterOutDegree(e: IndexEdge): Boolean = true
 
-  def increments(edgeMutate: EdgeMutate): Seq[SKeyValue] =
     (edgeMutate.edgesToDelete.isEmpty, edgeMutate.edgesToInsert.isEmpty) match {
       case (true, true) =>
 
         /** when there is no need to update. shouldUpdate == false */
-        List.empty
+        (Nil, Nil)
       case (true, false) =>
 
         /** no edges to delete but there is new edges to insert so increase degree by 1 */
-        edgeMutate.edgesToInsert.flatMap { e => buildIncrementsAsync(e) }
+        val (inEdges, outEdges) = edgeMutate.edgesToInsert.partition(_.isInEdge)
+
+        val in = inEdges.filter(filterOutDegree).flatMap(buildIncrementsAsync(_))
+        val out = outEdges.filter(filterOutDegree).flatMap(buildIncrementsAsync(_))
+
+        in -> out
       case (false, true) =>
 
         /** no edges to insert but there is old edges to delete so decrease degree by 1 */
-        edgeMutate.edgesToDelete.flatMap { e => buildIncrementsAsync(e, -1L) }
+        val (inEdges, outEdges) = edgeMutate.edgesToDelete.partition(_.isInEdge)
+
+        val in = inEdges.filter(filterOutDegree).flatMap(buildIncrementsAsync(_, -1))
+        val out = outEdges.filter(filterOutDegree).flatMap(buildIncrementsAsync(_, -1))
+
+        in -> out
       case (false, false) =>
 
         /** update on existing edges so no change on degree */
-        List.empty
+        (Nil, Nil)
     }
+  }
+
+  def increments(edgeMutate: EdgeMutate): Seq[SKeyValue] = {
+    val (in, out) = incrementsInOut(edgeMutate)
+    in ++ out
+  }
 
   /** IndexEdge */
   def buildIncrementsAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] = {
-    val newProps = indexedEdge.props ++ Map(LabelMeta.degreeSeq -> InnerValLikeWithTs.withLong(amount, indexedEdge.ts, indexedEdge.schemaVer))
+    val newProps = indexedEdge.props ++ Map(LabelMeta.degree -> InnerValLikeWithTs.withLong(amount, indexedEdge.ts, indexedEdge.schemaVer))
     val _indexedEdge = indexedEdge.copy(props = newProps)
-    indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment))
+    indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment, durability = _indexedEdge.label.durability))
   }
 
   def buildIncrementsCountAsync(indexedEdge: IndexEdge, amount: Long = 1L): Seq[SKeyValue] = {
-    val newProps = indexedEdge.props ++ Map(LabelMeta.countSeq -> InnerValLikeWithTs.withLong(amount, indexedEdge.ts, indexedEdge.schemaVer))
+    val newProps = indexedEdge.props ++ Map(LabelMeta.count -> InnerValLikeWithTs.withLong(amount, indexedEdge.ts, indexedEdge.schemaVer))
     val _indexedEdge = indexedEdge.copy(props = newProps)
-    indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment))
+    indexEdgeSerializer(_indexedEdge).toKeyValues.map(_.copy(operation = SKeyValue.Increment, durability = _indexedEdge.label.durability))
   }
+
+  //TODO: ServiceColumn do not have durability property yet.
   def buildDeleteBelongsToId(vertex: Vertex): Seq[SKeyValue] = {
     val kvs = vertexSerializer(vertex).toKeyValues
     val kv = kvs.head
@@ -1383,10 +1107,23 @@ abstract class Storage[R](val graph: Graph,
     }
   }
 
+  def buildDegreePuts(edge: Edge, degreeVal: Long): Seq[SKeyValue] = {
+    val kvs = edge.edgesWithIndexValid.flatMap { _indexEdge =>
+      val newProps = Map(LabelMeta.degree -> InnerValLikeWithTs.withLong(degreeVal, _indexEdge.ts, _indexEdge.schemaVer))
+      val indexEdge = _indexEdge.copy(props = newProps)
+
+      indexEdgeSerializer(indexEdge).toKeyValues.map(_.copy(operation = SKeyValue.Put, durability = indexEdge.label.durability))
+    }
+
+    kvs
+  }
+
   def buildPutsAll(vertex: Vertex): 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 info: Map[String, String] = Map("className" -> this.getClass.getSimpleName)
 }


[3/7] incubator-s2graph git commit: [S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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
index 69926fa..d2a7de7 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageDeserializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageDeserializable.scala
@@ -21,6 +21,7 @@ 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
 
@@ -36,16 +37,17 @@ object StorageDeserializable {
   def bytesToKeyValues(bytes: Array[Byte],
                        offset: Int,
                        length: Int,
-                       version: String): (Array[(Byte, InnerValLike)], Int) = {
+                       schemaVer: String,
+                       label: Label): (Array[(LabelMeta, InnerValLike)], Int) = {
     var pos = offset
     val len = bytes(pos)
     pos += 1
-    val kvs = new Array[(Byte, InnerValLike)](len)
+    val kvs = new Array[(LabelMeta, InnerValLike)](len)
     var i = 0
     while (i < len) {
-      val k = bytes(pos)
+      val k = label.labelMetaMap(bytes(pos))
       pos += 1
-      val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, version)
+      val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, schemaVer)
       pos += numOfBytesUsed
       kvs(i) = (k -> v)
       i += 1
@@ -57,16 +59,17 @@ object StorageDeserializable {
 
   def bytesToKeyValuesWithTs(bytes: Array[Byte],
                              offset: Int,
-                             version: String): (Array[(Byte, InnerValLikeWithTs)], Int) = {
+                             schemaVer: String,
+                             label: Label): (Array[(LabelMeta, InnerValLikeWithTs)], Int) = {
     var pos = offset
     val len = bytes(pos)
     pos += 1
-    val kvs = new Array[(Byte, InnerValLikeWithTs)](len)
+    val kvs = new Array[(LabelMeta, InnerValLikeWithTs)](len)
     var i = 0
     while (i < len) {
-      val k = bytes(pos)
+      val k = label.labelMetaMap(bytes(pos))
       pos += 1
-      val (v, numOfBytesUsed) = InnerValLikeWithTs.fromBytes(bytes, pos, 0, version)
+      val (v, numOfBytesUsed) = InnerValLikeWithTs.fromBytes(bytes, pos, 0, schemaVer)
       pos += numOfBytesUsed
       kvs(i) = (k -> v)
       i += 1
@@ -78,15 +81,15 @@ object StorageDeserializable {
 
   def bytesToProps(bytes: Array[Byte],
                    offset: Int,
-                   version: String): (Array[(Byte, InnerValLike)], Int) = {
+                   schemaVer: String): (Array[(LabelMeta, InnerValLike)], Int) = {
     var pos = offset
     val len = bytes(pos)
     pos += 1
-    val kvs = new Array[(Byte, InnerValLike)](len)
+    val kvs = new Array[(LabelMeta, InnerValLike)](len)
     var i = 0
     while (i < len) {
-      val k = HBaseType.EMPTY_SEQ_BYTE
-      val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, version)
+      val k = LabelMeta.empty
+      val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, schemaVer)
       pos += numOfBytesUsed
       kvs(i) = (k -> v)
       i += 1
@@ -98,17 +101,19 @@ object StorageDeserializable {
   }
 
   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](queryParam: QueryParam, kvs: Seq[T], version: String, cacheElementOpt: Option[E]): Option[E] = {
+  def fromKeyValues[T: CanSKeyValue](checkLabel: Option[Label], kvs: Seq[T], version: String, cacheElementOpt: Option[E]): Option[E] = {
     try {
-      Option(fromKeyValuesInner(queryParam, kvs, version, cacheElementOpt))
+      Option(fromKeyValuesInner(checkLabel, kvs, version, cacheElementOpt))
     } catch {
       case e: Exception =>
         logger.error(s"${this.getClass.getName} fromKeyValues failed.", e)
         None
     }
   }
-  def fromKeyValuesInner[T: CanSKeyValue](queryParam: QueryParam, kvs: Seq[T], version: String, cacheElementOpt: Option[E]): E
+  def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label], kvs: Seq[T], version: String, cacheElementOpt: Option[E]): E
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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
index b7326f5..c1efe7b 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerializable.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/StorageSerializable.scala
@@ -20,31 +20,33 @@
 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[(Byte, InnerValLike)]): Array[Byte] = {
+  def propsToBytes(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, v.bytes)
+    for ((_, v) <- props) bytes = Bytes.add(bytes, v.bytes)
     bytes
   }
 
-  def propsToKeyValues(props: Seq[(Byte, InnerValLike)]): Array[Byte] = {
+  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), v.bytes)
+    for ((k, v) <- props) bytes = Bytes.add(bytes, Array.fill(1)(k.seq), v.bytes)
     bytes
   }
 
-  def propsToKeyValuesWithTs(props: Seq[(Byte, InnerValLikeWithTs)]): Array[Byte] = {
+  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), v.bytes)
+    for ((k, v) <- props) bytes = Bytes.add(bytes, Array.fill(1)(k.seq), v.bytes)
     bytes
   }
 
@@ -53,13 +55,17 @@ object StorageSerializable {
     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
 
-  val table: Array[Byte]
-  val ts: Long
+  def table: Array[Byte]
+  def ts: Long
 
   def toRowKey: Array[Byte]
   def toQualifier: Array[Byte]
@@ -70,7 +76,7 @@ trait StorageSerializable[E] {
     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/66bdf1bc/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 b52ba53..e63dfea 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
@@ -19,11 +19,13 @@
 
 package org.apache.s2graph.core.storage.hbase
 
+
+
 import java.util
 import java.util.Base64
 
-import com.stumbleupon.async.Deferred
-import com.typesafe.config.{Config, ConfigFactory}
+import com.stumbleupon.async.{Callback, Deferred}
+import com.typesafe.config.Config
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hbase.client.{ConnectionFactory, Durability}
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm
@@ -35,15 +37,17 @@ import org.apache.hadoop.security.UserGroupInformation
 import org.apache.s2graph.core._
 import org.apache.s2graph.core.mysqls.LabelMeta
 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.utils._
 import org.hbase.async.FilterList.Operator.MUST_PASS_ALL
 import org.hbase.async._
 
 import scala.collection.JavaConversions._
-import scala.collection.{Map, Seq}
-import scala.concurrent.duration.Duration
+import scala.collection.mutable.ArrayBuffer
 import scala.concurrent._
+import scala.concurrent.duration.Duration
+import scala.util.Try
 import scala.util.hashing.MurmurHash3
 
 
@@ -79,12 +83,15 @@ object AsynchbaseStorage {
     logger.info(s"Asynchbase: ${client.getConfig.dumpConfiguration()}")
     client
   }
+
+  case class ScanWithRange(scan: Scanner, offset: Int, limit: Int)
+  type AsyncRPC = Either[GetRequest, ScanWithRange]
 }
 
 
 class AsynchbaseStorage(override val graph: Graph,
                         override val config: Config)(implicit ec: ExecutionContext)
-  extends Storage[Deferred[StepInnerResult]](graph, config) {
+  extends Storage[AsyncRPC, Deferred[StepResult]](graph, config) {
 
   import Extensions.DeferOps
 
@@ -92,26 +99,41 @@ class AsynchbaseStorage(override val graph: Graph,
    * Asynchbase client setup.
    * note that we need two client, one for bulk(withWait=false) and another for withWait=true
    */
-  val configWithFlush = config.withFallback(ConfigFactory.parseMap(Map("hbase.rpcs.buffered_flush_interval" -> "0")))
-  val client = AsynchbaseStorage.makeClient(config)
-
-  private val clientWithFlush = AsynchbaseStorage.makeClient(config, "hbase.rpcs.buffered_flush_interval" -> "0")
-  private val clients = Seq(client, clientWithFlush)
   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 emptyStepResult = new util.ArrayList[StepResult]()
+
   private def client(withWait: Boolean): HBaseClient = if (withWait) clientWithFlush else client
 
+  import CanDefer._
+
   /** Future Cache to squash request */
-  private val futureCache = new DeferCache[StepInnerResult, Deferred, Deferred](config, StepInnerResult.Empty, "FutureCache", useMetric = true)
+  lazy private val futureCache = new DeferCache[StepResult, Deferred, Deferred](config, StepResult.Empty, "AsyncHbaseFutureCache", useMetric = true)
 
   /** Simple Vertex Cache */
-  private val vertexCache = new DeferCache[Seq[SKeyValue], Promise, Future](config, Seq.empty[SKeyValue])
+  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.zookeeper.quorum")) Option(config.getString("hbase.zookeeper.quorum"))
+    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.
@@ -120,37 +142,75 @@ class AsynchbaseStorage(override val graph: Graph,
     if (kvs.isEmpty) Future.successful(true)
     else {
       val _client = client(withWait)
-      val futures = kvs.map { kv =>
-        val _defer = kv.operation match {
-          case SKeyValue.Put => _client.put(new PutRequest(kv.table, kv.row, kv.cf, kv.qualifier, kv.value, kv.timestamp))
-          case SKeyValue.Delete =>
-            if (kv.qualifier == null) _client.delete(new DeleteRequest(kv.table, kv.row, kv.cf, kv.timestamp))
-            else _client.delete(new DeleteRequest(kv.table, kv.row, kv.cf, kv.qualifier, kv.timestamp))
-          case SKeyValue.Increment =>
-            _client.atomicIncrement(new AtomicIncrementRequest(kv.table, kv.row, kv.cf, kv.qualifier, Bytes.toLong(kv.value)))
-        }
-        val future = _defer.withCallback { ret => true }.recoverWith { ex =>
+      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
-        }.toFuture
-
+        }
         if (withWait) future else Future.successful(true)
       }
 
-      Future.sequence(futures).map(_.forall(identity))
+      /** 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)
     }
   }
 
-
-  override def fetchSnapshotEdgeKeyValues(hbaseRpc: AnyRef): Future[Seq[SKeyValue]] = {
-    val defer = fetchKeyValuesInner(hbaseRpc)
-    defer.toFuture.map { kvsArr =>
+  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)
-      } toSeq
+      }
     }
   }
 
+  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.
@@ -162,7 +222,7 @@ class AsynchbaseStorage(override val graph: Graph,
   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).withCallback(ret => ret.booleanValue()).toFuture
+    client(withWait = true).compareAndSet(put, expected).map(true.booleanValue())(ret => ret.booleanValue()).toFuture(true)
   }
 
 
@@ -182,24 +242,24 @@ class AsynchbaseStorage(override val graph: Graph,
    * @param queryRequest
    * @return Scanner or GetRequest with proper setup with StartKey, EndKey, RangeFilter.
    */
-  override def buildRequest(queryRequest: QueryRequest): AnyRef = {
+  override def buildRequest(queryRequest: QueryRequest, edge: Edge): AsyncRPC = {
     import Serializable._
     val queryParam = queryRequest.queryParam
     val label = queryParam.label
-    val edge = toRequestEdge(queryRequest)
 
     val serializer = if (queryParam.tgtVertexInnerIdOpt.isDefined) {
       val snapshotEdge = edge.toSnapshotEdge
       snapshotEdgeSerializer(snapshotEdge)
     } else {
-      val indexEdge = IndexEdge(edge.srcVertex, edge.tgtVertex, edge.labelWithDir,
+      val indexEdge = IndexEdge(edge.srcVertex, edge.tgtVertex, edge.label, edge.dir,
         edge.op, edge.version, queryParam.labelOrderSeq, edge.propsWithTs)
       indexEdgeSerializer(indexEdge)
     }
 
-    val (rowKey, qualifier) = (serializer.toRowKey, serializer.toQualifier)
+    val rowKey = serializer.toRowKey
+    val (minTs, maxTs) = queryParam.durationOpt.getOrElse((0L, Long.MaxValue))
 
-    val (minTs, maxTs) = queryParam.duration.getOrElse((0L, Long.MaxValue))
+    val (intervalMaxBytes, intervalMinBytes) = queryParam.buildInterval(Option(edge))
 
     label.schemaVersion match {
       case HBaseType.VERSION4 if queryParam.tgtVertexInnerIdOpt.isEmpty =>
@@ -215,28 +275,26 @@ class AsynchbaseStorage(override val graph: Graph,
         val srcIdBytes = VertexId.toSourceVertexId(indexEdge.srcVertex.id).bytes
         val labelWithDirBytes = indexEdge.labelWithDir.bytes
         val labelIndexSeqWithIsInvertedBytes = StorageSerializable.labelOrderSeqWithIsInverted(indexEdge.labelIndexSeq, isInverted = false)
-        //        val labelIndexSeqWithIsInvertedStopBytes =  StorageSerializable.labelOrderSeqWithIsInverted(indexEdge.labelIndexSeq, isInverted = true)
-        val baseKey = Bytes.add(srcIdBytes, labelWithDirBytes, Bytes.add(labelIndexSeqWithIsInvertedBytes, Array.fill(1)(edge.op)))
+        val baseKey = Bytes.add(srcIdBytes, labelWithDirBytes, labelIndexSeqWithIsInvertedBytes)
+
         val (startKey, stopKey) =
-          if (queryParam.columnRangeFilter != null) {
+          if (queryParam.intervalOpt.isDefined) {
             // interval is set.
             val _startKey = queryParam.cursorOpt match {
-              case Some(cursor) => Bytes.add(Base64.getDecoder.decode(cursor), Array.fill(1)(0))
-              case None => Bytes.add(baseKey, queryParam.columnRangeFilterMinBytes)
+              case Some(cursor) => Base64.getDecoder.decode(cursor)
+              case None => Bytes.add(baseKey, intervalMaxBytes)
             }
-            (_startKey, Bytes.add(baseKey, queryParam.columnRangeFilterMaxBytes))
+            (_startKey , Bytes.add(baseKey, intervalMinBytes))
           } else {
-            /*
-             * note: since propsToBytes encode size of property map at first byte, we are sure about max value here
-             */
+            /**
+              * 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) => Bytes.add(Base64.getDecoder.decode(cursor), Array.fill(1)(0))
+              case Some(cursor) => Base64.getDecoder.decode(cursor)
               case None => baseKey
             }
             (_startKey, Bytes.add(baseKey, Array.fill(1)(-1)))
           }
-//                logger.debug(s"[StartKey]: ${startKey.toList}")
-//                logger.debug(s"[StopKey]: ${stopKey.toList}")
 
         scanner.setStartKey(startKey)
         scanner.setStopKey(stopKey)
@@ -244,15 +302,23 @@ class AsynchbaseStorage(override val graph: Graph,
         if (queryParam.limit == Int.MinValue) logger.debug(s"MinValue: $queryParam")
 
         scanner.setMaxVersions(1)
-        scanner.setMaxNumRows(queryParam.offset + queryParam.limit)
+        // 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.rpcTimeoutInMillis)
+        scanner.setRpcTimeout(queryParam.rpcTimeout)
+
         // SET option for this rpc properly.
-        scanner
+        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, qualifier)
+          new GetRequest(label.hbaseTableName.getBytes, rowKey, edgeCf, serializer.toQualifier)
         } else {
           new GetRequest(label.hbaseTableName.getBytes, rowKey, edgeCf)
         }
@@ -261,12 +327,14 @@ class AsynchbaseStorage(override val graph: Graph,
         get.setFailfast(true)
         get.setMinTimestamp(minTs)
         get.setMaxTimestamp(maxTs)
-        get.setTimeout(queryParam.rpcTimeoutInMillis)
+        get.setTimeout(queryParam.rpcTimeout)
 
         val pagination = new ColumnPaginationFilter(queryParam.limit, queryParam.offset)
-        get.setFilter(new FilterList(pagination +: Option(queryParam.columnRangeFilter).toSeq, MUST_PASS_ALL))
-
-        get
+        val columnRangeFilterOpt = queryParam.intervalOpt.map { interval =>
+          new ColumnRangeFilter(intervalMaxBytes, true, intervalMinBytes, true)
+        }
+        get.setFilter(new FilterList(pagination +: columnRangeFilterOpt.toSeq, MUST_PASS_ALL))
+        Left(get)
     }
   }
 
@@ -274,81 +342,81 @@ class AsynchbaseStorage(override val graph: Graph,
    * we are using future cache to squash requests into same key on storage.
    *
    * @param queryRequest
-   * @param prevStepScore
    * @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,
-                     prevStepScore: Double,
                      isInnerCall: Boolean,
-                     parentEdges: Seq[EdgeWithScore]): Deferred[StepInnerResult] = {
+                     parentEdges: Seq[EdgeWithScore]): Deferred[StepResult] = {
 
-    def fetchInner(hbaseRpc: AnyRef): Deferred[StepInnerResult] = {
-      val queryParam = queryRequest.queryParam
+    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
+      }
 
-      fetchKeyValuesInner(hbaseRpc).withCallback { kvs =>
-        val (startOffset, length) = queryParam.label.schemaVersion match {
-          case HBaseType.VERSION4 => (queryParam.offset, queryParam.limit)
+      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)
         }
 
-        val edgeWithScores = toEdges(kvs, queryParam, prevStepScore, isInnerCall, parentEdges, startOffset, length)
-        if (edgeWithScores.isEmpty) StepInnerResult.Empty
-        else {
-          val head = edgeWithScores.head
-          val (degreeEdges, indexEdges) =
-            if (head.edge.isDegree) (Seq(head), edgeWithScores.tail)
-            else (Nil, edgeWithScores)
-
-          val normalized =
-            if (queryRequest.queryParam.shouldNormalize) normalize(indexEdges)
-            else indexEdges
-
-          val sampled = if (queryRequest.queryParam.sample >= 0) {
-            sample(queryRequest, normalized, queryRequest.queryParam.sample)
-          } else normalized
-
-          StepInnerResult(edgesWithScoreLs = sampled, degreeEdges)
-        }
-      } recoverWith { ex =>
-        logger.error(s"fetchInner failed. fallback return. $hbaseRpc}", ex)
-        StepInnerResult.Failure
+        toEdges(kvs, queryRequest, prevStepScore, isInnerCall, parentEdges, startOffset, len)
       }
     }
 
     val queryParam = queryRequest.queryParam
     val cacheTTL = queryParam.cacheTTLInMillis
-    val request = buildRequest(queryRequest)
+    /** with version 4, request's type is (Scanner, (Int, Int)). otherwise GetRequest. */
+
+    val edge = toRequestEdge(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.cacheKeyBytes, toCacheKeyBytes(request))
+      val cacheKeyBytes = Bytes.add(queryRequest.query.queryOption.cacheKeyBytes, requestCacheKey)
+
+//      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)
+      }
 
-  override def fetches(queryRequestWithScoreLs: scala.Seq[(QueryRequest, Double)],
-                       prevStepEdges: Predef.Map[VertexId, scala.Seq[EdgeWithScore]]): Future[scala.Seq[StepInnerResult]] = {
-    val defers: Seq[Deferred[StepInnerResult]] = for {
-      (queryRequest, prevStepScore) <- queryRequestWithScoreLs
-      parentEdges <- prevStepEdges.get(queryRequest.vertex.id)
-    } yield fetch(queryRequest, prevStepScore, isInnerCall = false, parentEdges)
-
-    val grouped: Deferred[util.ArrayList[StepInnerResult]] = Deferred.group(defers)
-    grouped withCallback {
-      queryResults: util.ArrayList[StepInnerResult] =>
-        queryResults.toIndexedSeq
-    } toFuture
+    val grouped: Deferred[util.ArrayList[StepResult]] = Deferred.groupInOrder(defers)
+    grouped.map(emptyStepResult) { queryResults: util.ArrayList[StepResult] =>
+      queryResults.toSeq
+    }.toFuture(emptyStepResult)
   }
 
 
-  def fetchVertexKeyValues(request: AnyRef): Future[Seq[SKeyValue]] = fetchSnapshotEdgeKeyValues(request)
+  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)
+
   /**
    * 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.
@@ -357,35 +425,44 @@ class AsynchbaseStorage(override val graph: Graph,
    * @param withWait
    * @return
    */
-  override def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long)]] = {
+  override def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]] = {
+
     val _client = client(withWait)
-    val defers: Seq[Deferred[(Boolean, Long)]] = for {
+    val defers: Seq[Deferred[(Boolean, Long, Long)]] = for {
       edge <- edges
     } yield {
-        val edgeWithIndex = edge.edgesWithIndex.head
-        val countWithTs = edge.propsWithTs(LabelMeta.countSeq)
-        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 defer = _client.bufferAtomicIncrement(request) withCallback { resultCount: java.lang.Long =>
-          (true, resultCount.longValue())
-        } recoverWith { ex =>
-          logger.error(s"mutation failed. $request", ex)
-          (false, -1L)
+        val futures: List[Deferred[(Boolean, Long, Long)]] = for {
+          relEdge <- edge.relatedEdges
+          edgeWithIndex <- relEdge.edgesWithIndexValid
+        } yield {
+          val countWithTs = edge.propsWithTs(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))
         }
-        if (withWait) defer
-        else Deferred.fromResult((true, -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)]] = Deferred.groupInOrder(defers)
-    grouped.toFuture.map(_.toSeq)
+    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, timeout)
+    Await.result(client.flush().toFuture(new AnyRef), timeout)
   }
 
 
@@ -394,40 +471,37 @@ class AsynchbaseStorage(override val graph: Graph,
                            cfs: List[String],
                            regionMultiplier: Int,
                            ttl: Option[Int],
-                           compressionAlgorithm: String): Unit = {
+                           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")
       val admin = getAdmin(zkAddr)
-      val regionCount = admin.getClusterStatus.getServersSize * regionMultiplier
+      val regionCount = totalRegionCount.getOrElse(admin.getClusterStatus.getServersSize * regionMultiplier)
       try {
         if (!admin.tableExists(TableName.valueOf(tableName))) {
-          try {
-            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)
-              if (ttl.isDefined) columnDesc.setTimeToLive(ttl.get)
-              desc.addFamily(columnDesc)
-            }
-
-            if (regionCount <= 1) admin.createTable(desc)
-            else admin.createTable(desc, getStartKey(regionCount), getEndKey(regionCount), regionCount)
-          } catch {
-            case e: Throwable =>
-              logger.error(s"$zkAddr, $tableName failed with $e", e)
-              throw e
+          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)
+            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.")
         }
@@ -445,12 +519,12 @@ class AsynchbaseStorage(override val graph: Graph,
 
   /** Asynchbase implementation override default getVertices to use future Cache */
   override def getVertices(vertices: Seq[Vertex]): Future[Seq[Vertex]] = {
-    def fromResult(queryParam: QueryParam,
-                   kvs: Seq[SKeyValue],
+    def fromResult(kvs: Seq[SKeyValue],
                    version: String): Option[Vertex] = {
 
       if (kvs.isEmpty) None
-      else vertexDeserializer.fromKeyValues(queryParam, kvs, version, None)
+      else vertexDeserializer.fromKeyValues(None, kvs, version, None)
+//        .map(S2Vertex(graph, _))
     }
 
     val futures = vertices.map { vertex =>
@@ -461,55 +535,85 @@ class AsynchbaseStorage(override val graph: Graph,
       get.maxVersions(1)
 
       val cacheKey = MurmurHash3.stringHash(get.toString)
-      vertexCache.getOrElseUpdate(cacheKey, cacheTTL = 10000)(fetchVertexKeyValues(get)).map { kvs =>
-        fromResult(QueryParam.Empty, kvs, vertex.serviceColumn.schemaVersion)
+      vertexCache.getOrElseUpdate(cacheKey, cacheTTL = 10000)(fetchVertexKeyValues(Left(get))).map { kvs =>
+        fromResult(kvs, vertex.serviceColumn.schemaVersion)
       }
     }
 
     Future.sequence(futures).map { result => result.toList.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: AnyRef): Deferred[util.ArrayList[KeyValue]] = {
+  private def fetchKeyValuesInner(rpc: AsyncRPC): Deferred[util.ArrayList[KeyValue]] = {
     rpc match {
-      case getRequest: GetRequest => client.get(getRequest)
-      case scanner: Scanner =>
-        scanner.nextRows().withCallback { 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()
-          ls
-        }.recoverWith { ex =>
-          logger.error(s"fetchKeyValuesInner failed.", ex)
-          scanner.close()
-          emptyKeyValues
-        }
+      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: AnyRef): Array[Byte] = {
+  private def toCacheKeyBytes(hbaseRpc: AsyncRPC): Array[Byte] = {
+    /** with version 4, request's type is (Scanner, (Int, Int)). otherwise GetRequest. */
     hbaseRpc match {
-      case getRequest: GetRequest => getRequest.key()
-      case scanner: Scanner => scanner.getCurrentKey()
+      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")
-        Array.empty[Byte]
+        throw new RuntimeException(s"toCacheKeyBytes: $hbaseRpc")
     }
   }
 
@@ -520,8 +624,6 @@ class AsynchbaseStorage(override val graph: Graph,
     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")

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 e2b7c2f..2428173 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
@@ -20,141 +20,125 @@
 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.mysqls.{Label, LabelMeta}
 import org.apache.s2graph.core.storage.StorageDeserializable._
-import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable, SKeyValue, StorageDeserializable}
+import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable, StorageDeserializable}
 import org.apache.s2graph.core.types._
-import org.apache.s2graph.core.{GraphUtil, IndexEdge, QueryParam, Vertex}
+import org.apache.s2graph.core.utils.logger
+import org.apache.s2graph.core.{GraphUtil, IndexEdge, Vertex}
 import scala.collection.immutable
 
+object IndexEdgeDeserializable{
+  def getNewInstance() = new IndexEdgeDeserializable()
+}
 class IndexEdgeDeserializable(bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[IndexEdge] {
    import StorageDeserializable._
 
-   type QualifierRaw = (Array[(Byte, InnerValLike)], VertexId, Byte, Boolean, Int)
-   type ValueRaw = (Array[(Byte, InnerValLike)], Int)
+   type QualifierRaw = (Array[(LabelMeta, InnerValLike)], VertexId, Byte, Boolean, Int)
+   type ValueRaw = (Array[(LabelMeta, InnerValLike)], Int)
 
-   private def parseDegreeQualifier(kv: SKeyValue, version: String): QualifierRaw = {
-     //    val degree = Bytes.toLong(kv.value)
-     val degree = bytesToLongFunc(kv.value, 0)
-     val idxPropsRaw = Array(LabelMeta.degreeSeq -> InnerVal.withLong(degree, version))
-     val tgtVertexIdRaw = VertexId(HBaseType.DEFAULT_COL_ID, InnerVal.withStr("0", version))
-     (idxPropsRaw, tgtVertexIdRaw, GraphUtil.operations("insert"), false, 0)
-   }
+   override def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
+                                                    _kvs: Seq[T],
+                                                    schemaVer: String,
+                                                    cacheElementOpt: Option[IndexEdge]): IndexEdge = {
+
+     assert(_kvs.size == 1)
+
+     //     val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
+     val kv = implicitly[CanSKeyValue[T]].toSKeyValue(_kvs.head)
+//     logger.debug(s"[DES]: ${kv.toLogString}}")
 
-   private def parseQualifier(kv: SKeyValue, version: String): QualifierRaw = {
-     var qualifierLen = 0
+     val version = kv.timestamp
+     //    logger.debug(s"[Des]: ${kv.row.toList}, ${kv.qualifier.toList}, ${kv.value.toList}")
      var pos = 0
-     val (idxPropsRaw, idxPropsLen, tgtVertexIdRaw, tgtVertexIdLen) = {
-       val (props, endAt) = bytesToProps(kv.qualifier, pos, version)
+     val (srcVertexId, srcIdLen) = SourceVertexId.fromBytes(kv.row, pos, kv.row.length, schemaVer)
+     pos += srcIdLen
+     val labelWithDir = LabelWithDirection(Bytes.toInt(kv.row, pos, 4))
+     pos += 4
+     val (labelIdxSeq, isInverted) = bytesToLabelIndexSeqWithIsInverted(kv.row, pos)
+     pos += 1
+
+     val label = checkLabel.getOrElse(Label.findById(labelWithDir.labelId))
+//     val op = kv.row(pos)
+//     pos += 1
+
+     if (pos == kv.row.length) {
+       // degree
+       //      val degreeVal = Bytes.toLong(kv.value)
+       val degreeVal = bytesToLongFunc(kv.value, 0)
+       val ts = kv.timestamp
+       val tsInnerValLikeWithTs = InnerValLikeWithTs.withLong(ts, ts, schemaVer)
+       val props = Map(LabelMeta.timestamp -> tsInnerValLikeWithTs,
+         LabelMeta.degree -> InnerValLikeWithTs.withLong(degreeVal, ts, schemaVer))
+       val tgtVertexId = VertexId(HBaseType.DEFAULT_COL_ID, InnerVal.withStr("0", schemaVer))
+       IndexEdge(Vertex(srcVertexId, ts), Vertex(tgtVertexId, ts), label, labelWithDir.dir, GraphUtil.defaultOpByte, ts, labelIdxSeq, props,  tsInnerValOpt = Option(tsInnerValLikeWithTs.innerVal))
+     } else {
+       // not degree edge
+       val (idxPropsRaw, endAt) = bytesToProps(kv.row, pos, schemaVer)
        pos = endAt
-       qualifierLen += endAt
-       val (tgtVertexId, tgtVertexIdLen) = if (endAt == kv.qualifier.length) {
+
+
+       val (tgtVertexIdRaw, tgtVertexIdLen) = if (endAt == kv.row.length - 1) {
          (HBaseType.defaultTgtVertexId, 0)
        } else {
-         TargetVertexId.fromBytes(kv.qualifier, endAt, kv.qualifier.length, version)
+         TargetVertexId.fromBytes(kv.row, endAt, kv.row.length - 1, schemaVer)
        }
-       qualifierLen += tgtVertexIdLen
-       (props, endAt, tgtVertexId, tgtVertexIdLen)
-     }
-     val (op, opLen) =
-       if (kv.qualifier.length == qualifierLen) (GraphUtil.defaultOpByte, 0)
-       else (kv.qualifier(qualifierLen), 1)
+       val op = kv.row(kv.row.length-1)
+
+       val allProps = immutable.Map.newBuilder[LabelMeta, InnerValLikeWithTs]
+       val index = label.indicesMap.getOrElse(labelIdxSeq, throw new RuntimeException(s"invalid index seq: ${label.id.get}, ${labelIdxSeq}"))
+
+       /** process indexProps */
+       val size = idxPropsRaw.length
+       (0 until size).foreach { ith =>
+         val meta = index.sortKeyTypesArray(ith)
+         val (k, v) = idxPropsRaw(ith)
+         if (k == LabelMeta.degree) allProps += LabelMeta.degree -> InnerValLikeWithTs(v, version)
+         else allProps += meta -> InnerValLikeWithTs(v, version)
+       }
+//       for {
+//         (meta, (k, v)) <- index.sortKeyTypes.zip(idxPropsRaw)
+//       } {
+//         if (k == LabelMeta.degree) allProps += LabelMeta.degree -> InnerValLikeWithTs(v, version)
+//         else {
+//           allProps += meta -> InnerValLikeWithTs(v, version)
+//         }
+//       }
+
+       /** process props */
+       if (op == GraphUtil.operations("incrementCount")) {
+         //        val countVal = Bytes.toLong(kv.value)
+         val countVal = bytesToLongFunc(kv.value, 0)
+         allProps += (LabelMeta.count -> InnerValLikeWithTs.withLong(countVal, version, schemaVer))
+       } else {
+         val (props, endAt) = bytesToKeyValues(kv.value, 0, kv.value.length, schemaVer, label)
+         props.foreach { case (k, v) =>
+           allProps += (k -> InnerValLikeWithTs(v, version))
+         }
+       }
+       val _mergedProps = allProps.result()
+       val (mergedProps, tsInnerValLikeWithTs) = _mergedProps.get(LabelMeta.timestamp) match {
+         case None =>
+           val tsInnerVal = InnerValLikeWithTs.withLong(version, version, schemaVer)
+           val mergedProps = _mergedProps + (LabelMeta.timestamp -> InnerValLikeWithTs.withLong(version, version, schemaVer))
+           (mergedProps, tsInnerVal)
+         case Some(tsInnerVal) =>
+           (_mergedProps, tsInnerVal)
+       }
+//       val mergedProps =
+//         if (_mergedProps.contains(LabelMeta.timestamp)) _mergedProps
+//         else _mergedProps + (LabelMeta.timestamp -> InnerValLikeWithTs.withLong(version, version, schemaVer))
 
-     qualifierLen += opLen
+       /** process tgtVertexId */
+       val tgtVertexId =
+         mergedProps.get(LabelMeta.to) match {
+           case None => tgtVertexIdRaw
+           case Some(vId) => TargetVertexId(HBaseType.DEFAULT_COL_ID, vId.innerVal)
+         }
 
-     (idxPropsRaw, tgtVertexIdRaw, op, tgtVertexIdLen != 0, qualifierLen)
-   }
 
-   private def parseValue(kv: SKeyValue, version: String): ValueRaw = {
-     val (props, endAt) = bytesToKeyValues(kv.value, 0, kv.value.length, version)
-     (props, endAt)
-   }
+       IndexEdge(Vertex(srcVertexId, version), Vertex(tgtVertexId, version), label, labelWithDir.dir, op, version, labelIdxSeq, mergedProps, tsInnerValOpt = Option(tsInnerValLikeWithTs.innerVal))
 
-   private def parseDegreeValue(kv: SKeyValue, version: String): ValueRaw = {
-     (Array.empty[(Byte, InnerValLike)], 0)
+     }
    }
-
-  override def fromKeyValuesInner[T: CanSKeyValue](queryParam: QueryParam,
-                                                   _kvs: Seq[T],
-                                                   schemaVer: String,
-                                                   cacheElementOpt: Option[IndexEdge]): IndexEdge = {
-
-    assert(_kvs.size == 1)
-
-    val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
-
-    val kv = kvs.head
-    val version = kv.timestamp
-    //    logger.debug(s"[Des]: ${kv.row.toList}, ${kv.qualifier.toList}, ${kv.value.toList}")
-    var pos = 0
-    val (srcVertexId, srcIdLen) = SourceVertexId.fromBytes(kv.row, pos, kv.row.length, schemaVer)
-    pos += srcIdLen
-    val labelWithDir = LabelWithDirection(Bytes.toInt(kv.row, pos, 4))
-    pos += 4
-    val (labelIdxSeq, isInverted) = bytesToLabelIndexSeqWithIsInverted(kv.row, pos)
-    pos += 1
-
-    val op = kv.row(pos)
-    pos += 1
-
-    if (pos == kv.row.length) {
-      // degree
-      //      val degreeVal = Bytes.toLong(kv.value)
-      val degreeVal = bytesToLongFunc(kv.value, 0)
-      val ts = kv.timestamp
-      val props = Map(LabelMeta.timeStampSeq -> InnerValLikeWithTs.withLong(ts, ts, schemaVer),
-        LabelMeta.degreeSeq -> InnerValLikeWithTs.withLong(degreeVal, ts, schemaVer))
-      val tgtVertexId = VertexId(HBaseType.DEFAULT_COL_ID, InnerVal.withStr("0", schemaVer))
-      IndexEdge(Vertex(srcVertexId, ts), Vertex(tgtVertexId, ts), labelWithDir, op, ts, labelIdxSeq, props)
-    } else {
-      // not degree edge
-
-
-      val (idxPropsRaw, endAt) = bytesToProps(kv.row, pos, schemaVer)
-      pos = endAt
-      val (tgtVertexIdRaw, tgtVertexIdLen) = if (endAt == kv.row.length) {
-        (HBaseType.defaultTgtVertexId, 0)
-      } else {
-        TargetVertexId.fromBytes(kv.row, endAt, kv.row.length, schemaVer)
-      }
-
-      val allProps = immutable.Map.newBuilder[Byte, InnerValLikeWithTs]
-      val index = queryParam.label.indicesMap.getOrElse(labelIdxSeq, throw new RuntimeException(s"invalid index seq: ${queryParam.label.id.get}, ${labelIdxSeq}"))
-
-      /* process indexProps */
-      for {
-        (seq, (k, v)) <- index.metaSeqs.zip(idxPropsRaw)
-      } {
-        if (k == LabelMeta.degreeSeq) allProps += k -> InnerValLikeWithTs(v, version)
-        else allProps += seq -> InnerValLikeWithTs(v, version)
-      }
-
-      /* process props */
-      if (op == GraphUtil.operations("incrementCount")) {
-        //        val countVal = Bytes.toLong(kv.value)
-        val countVal = bytesToLongFunc(kv.value, 0)
-        allProps += (LabelMeta.countSeq -> InnerValLikeWithTs.withLong(countVal, version, schemaVer))
-      } else {
-        val (props, endAt) = bytesToKeyValues(kv.value, 0, kv.value.length, schemaVer)
-        props.foreach { case (k, v) =>
-          allProps += (k -> InnerValLikeWithTs(v, version))
-        }
-      }
-      val _mergedProps = allProps.result()
-      val mergedProps =
-        if (_mergedProps.contains(LabelMeta.timeStampSeq)) _mergedProps
-        else _mergedProps + (LabelMeta.timeStampSeq -> InnerValLikeWithTs.withLong(version, version, schemaVer))
-
-      /* process tgtVertexId */
-      val tgtVertexId =
-        mergedProps.get(LabelMeta.toSeq) match {
-          case None => tgtVertexIdRaw
-          case Some(vId) => TargetVertexId(HBaseType.DEFAULT_COL_ID, vId.innerVal)
-        }
-
-
-      IndexEdge(Vertex(srcVertexId, version), Vertex(tgtVertexId, version), labelWithDir, op, version, labelIdxSeq, mergedProps)
-
-    }
-  }
-}
+ }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 a76bd1f..cd242dc 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,16 +21,16 @@ 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.{SKeyValue, Serializable, StorageSerializable}
+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._
 
-
-class IndexEdgeSerializable(indexEdge: IndexEdge) extends Serializable[IndexEdge] {
+class IndexEdgeSerializable(indexEdge: IndexEdge, longToBytes: Long => Array[Byte] = longToBytes) extends Serializable[IndexEdge] {
    import StorageSerializable._
 
-   override val ts = indexEdge.version
-   override val table = indexEdge.label.hbaseTableName.getBytes()
+   override def ts = indexEdge.version
+   override def table = indexEdge.label.hbaseTableName.getBytes()
 
    def idxPropsMap = indexEdge.orders.toMap
    def idxPropsBytes = propsToBytes(indexEdge.orders)
@@ -43,25 +43,25 @@ class IndexEdgeSerializable(indexEdge: IndexEdge) extends Serializable[IndexEdge
      val row = Bytes.add(srcIdBytes, labelWithDirBytes, labelIndexSeqWithIsInvertedBytes)
      //    logger.error(s"${row.toList}\n${srcIdBytes.toList}\n${labelWithDirBytes.toList}\n${labelIndexSeqWithIsInvertedBytes.toList}")
 
-     val qualifier =
-       if (indexEdge.degreeEdge) Array.empty[Byte]
-       else
-         idxPropsMap.get(LabelMeta.toSeq) match {
+     if (indexEdge.degreeEdge) row
+     else {
+       val qualifier = idxPropsMap.get(LabelMeta.to) match {
            case None => Bytes.add(idxPropsBytes, VertexId.toTargetVertexId(indexEdge.tgtVertex.id).bytes)
            case Some(vId) => idxPropsBytes
          }
 
-     /* TODO search usage of op byte. if there is no, then remove opByte */
-     Bytes.add(row, Array.fill(1)(GraphUtil.defaultOpByte), qualifier)
+       val opByte = if (indexEdge.op == GraphUtil.operations("incrementCount")) indexEdge.op else GraphUtil.defaultOpByte
+       Bytes.add(row, qualifier, Array.fill(1)(opByte))
+     }
    }
 
    override def toQualifier: Array[Byte] = Array.empty[Byte]
 
    override def toValue: Array[Byte] =
      if (indexEdge.degreeEdge)
-       Bytes.toBytes(indexEdge.props(LabelMeta.degreeSeq).innerVal.toString().toLong)
+       longToBytes(indexEdge.props(LabelMeta.degree).innerVal.toString().toLong)
      else if (indexEdge.op == GraphUtil.operations("incrementCount"))
-       Bytes.toBytes(indexEdge.props(LabelMeta.countSeq).innerVal.toString().toLong)
+       longToBytes(indexEdge.props(LabelMeta.count).innerVal.toString().toLong)
      else propsToKeyValues(indexEdge.metas.toSeq)
 
  }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 eb3d765..534667b 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
@@ -19,122 +19,126 @@
 
 package org.apache.s2graph.core.storage.serde.indexedge.wide
 
-import org.apache.s2graph.core.mysqls.LabelMeta
+import org.apache.s2graph.core.mysqls.{Label, LabelMeta}
 import org.apache.s2graph.core.storage.StorageDeserializable._
 import org.apache.s2graph.core.storage._
 import org.apache.s2graph.core.types._
-import org.apache.s2graph.core.{GraphUtil, IndexEdge, QueryParam, Vertex}
-import scala.collection.immutable
-
-import scala.collection.immutable
-
+import org.apache.s2graph.core.{GraphUtil, IndexEdge, Vertex}
 import scala.collection.immutable
 
 class IndexEdgeDeserializable(bytesToLongFunc: (Array[Byte], Int) => Long = bytesToLong) extends Deserializable[IndexEdge] {
-
-
-  import StorageDeserializable._
-
-  type QualifierRaw = (Array[(Byte, InnerValLike)], VertexId, Byte, Boolean, Int)
-  type ValueRaw = (Array[(Byte, InnerValLike)], Int)
-
-  private def parseDegreeQualifier(kv: SKeyValue, version: String): QualifierRaw = {
-    //    val degree = Bytes.toLong(kv.value)
-    val degree = bytesToLongFunc(kv.value, 0)
-    val idxPropsRaw = Array(LabelMeta.degreeSeq -> InnerVal.withLong(degree, version))
-    val tgtVertexIdRaw = VertexId(HBaseType.DEFAULT_COL_ID, InnerVal.withStr("0", version))
-    (idxPropsRaw, tgtVertexIdRaw, GraphUtil.operations("insert"), false, 0)
-  }
-
-  private def parseQualifier(kv: SKeyValue, version: String): QualifierRaw = {
-    var qualifierLen = 0
-    var pos = 0
-    val (idxPropsRaw, idxPropsLen, tgtVertexIdRaw, tgtVertexIdLen) = {
-      val (props, endAt) = bytesToProps(kv.qualifier, pos, version)
-      pos = endAt
-      qualifierLen += endAt
-      val (tgtVertexId, tgtVertexIdLen) = if (endAt == kv.qualifier.length) {
-        (HBaseType.defaultTgtVertexId, 0)
-      } else {
-        TargetVertexId.fromBytes(kv.qualifier, endAt, kv.qualifier.length, version)
-      }
-      qualifierLen += tgtVertexIdLen
-      (props, endAt, tgtVertexId, tgtVertexIdLen)
-    }
-    val (op, opLen) =
-      if (kv.qualifier.length == qualifierLen) (GraphUtil.defaultOpByte, 0)
-      else (kv.qualifier(qualifierLen), 1)
-
-    qualifierLen += opLen
-
-    (idxPropsRaw, tgtVertexIdRaw, op, tgtVertexIdLen != 0, qualifierLen)
-  }
-
-  private def parseValue(kv: SKeyValue, version: String): ValueRaw = {
-    val (props, endAt) = bytesToKeyValues(kv.value, 0, kv.value.length, version)
-    (props, endAt)
-  }
-
-  private def parseDegreeValue(kv: SKeyValue, version: String): ValueRaw = {
-    (Array.empty[(Byte, InnerValLike)], 0)
-  }
-
-  override def fromKeyValuesInner[T: CanSKeyValue](queryParam: QueryParam,
-                                                   _kvs: Seq[T],
-                                                   schemaVer: String,
-                                                   cacheElementOpt: Option[IndexEdge]): IndexEdge = {
-    assert(_kvs.size == 1)
-
-    val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
-
-    val kv = kvs.head
-    val version = kv.timestamp
-
-    val (srcVertexId, labelWithDir, labelIdxSeq, _, _) = cacheElementOpt.map { e =>
-      (e.srcVertex.id, e.labelWithDir, e.labelIndexSeq, false, 0)
-    }.getOrElse(parseRow(kv, schemaVer))
-
-    val (idxPropsRaw, tgtVertexIdRaw, op, tgtVertexIdInQualifier, _) =
-      if (kv.qualifier.isEmpty) parseDegreeQualifier(kv, schemaVer)
-      else parseQualifier(kv, schemaVer)
-
-    val allProps = immutable.Map.newBuilder[Byte, InnerValLikeWithTs]
-    val index = queryParam.label.indicesMap.getOrElse(labelIdxSeq, throw new RuntimeException(s"invalid index seq: ${queryParam.label.id.get}, ${labelIdxSeq}"))
-
-    /* process indexProps */
-    for {
-      (seq, (k, v)) <- index.metaSeqs.zip(idxPropsRaw)
-    } {
-      if (k == LabelMeta.degreeSeq) allProps += k -> InnerValLikeWithTs(v, version)
-      else allProps += seq -> InnerValLikeWithTs(v, version)
-    }
-
-    /* process props */
-    if (op == GraphUtil.operations("incrementCount")) {
-      //      val countVal = Bytes.toLong(kv.value)
-      val countVal = bytesToLongFunc(kv.value, 0)
-      allProps += (LabelMeta.countSeq -> InnerValLikeWithTs.withLong(countVal, version, schemaVer))
-    } else if (kv.qualifier.isEmpty) {
-      val countVal = bytesToLongFunc(kv.value, 0)
-      allProps += (LabelMeta.degreeSeq -> InnerValLikeWithTs.withLong(countVal, version, schemaVer))
-    } else {
-      val (props, _) = bytesToKeyValues(kv.value, 0, kv.value.length, schemaVer)
-      props.foreach { case (k, v) => allProps += (k -> InnerValLikeWithTs(v, version)) }
-    }
-
-    val _mergedProps = allProps.result()
-    val mergedProps =
-      if (_mergedProps.contains(LabelMeta.timeStampSeq)) _mergedProps
-      else _mergedProps + (LabelMeta.timeStampSeq -> InnerValLikeWithTs.withLong(version, version, schemaVer))
-
-    /* process tgtVertexId */
-    val tgtVertexId =
-      mergedProps.get(LabelMeta.toSeq) match {
-        case None => tgtVertexIdRaw
-        case Some(vId) => TargetVertexId(HBaseType.DEFAULT_COL_ID, vId.innerVal)
-      }
-
-    IndexEdge(Vertex(srcVertexId, version), Vertex(tgtVertexId, version), labelWithDir, op, version, labelIdxSeq, mergedProps)
-
-  }
-}
+   import StorageDeserializable._
+
+   type QualifierRaw = (Array[(LabelMeta, InnerValLike)], VertexId, Byte, Boolean, Int)
+   type ValueRaw = (Array[(LabelMeta, InnerValLike)], Int)
+
+   private def parseDegreeQualifier(kv: SKeyValue, schemaVer: String): QualifierRaw = {
+     //    val degree = Bytes.toLong(kv.value)
+     val degree = bytesToLongFunc(kv.value, 0)
+     val idxPropsRaw = Array(LabelMeta.degree -> InnerVal.withLong(degree, schemaVer))
+     val tgtVertexIdRaw = VertexId(HBaseType.DEFAULT_COL_ID, InnerVal.withStr("0", schemaVer))
+     (idxPropsRaw, tgtVertexIdRaw, GraphUtil.operations("insert"), false, 0)
+   }
+
+   private def parseQualifier(kv: SKeyValue, schemaVer: String): QualifierRaw = {
+     var qualifierLen = 0
+     var pos = 0
+     val (idxPropsRaw, idxPropsLen, tgtVertexIdRaw, tgtVertexIdLen) = {
+       val (props, endAt) = bytesToProps(kv.qualifier, pos, schemaVer)
+       pos = endAt
+       qualifierLen += endAt
+       val (tgtVertexId, tgtVertexIdLen) = if (endAt == kv.qualifier.length) {
+         (HBaseType.defaultTgtVertexId, 0)
+       } else {
+         TargetVertexId.fromBytes(kv.qualifier, endAt, kv.qualifier.length, schemaVer)
+       }
+       qualifierLen += tgtVertexIdLen
+       (props, endAt, tgtVertexId, tgtVertexIdLen)
+     }
+     val (op, opLen) =
+       if (kv.qualifier.length == qualifierLen) (GraphUtil.defaultOpByte, 0)
+       else (kv.qualifier(qualifierLen), 1)
+
+     qualifierLen += opLen
+
+     (idxPropsRaw, tgtVertexIdRaw, op, tgtVertexIdLen != 0, qualifierLen)
+   }
+
+   override def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
+                                                    _kvs: Seq[T],
+                                                    schemaVer: String,
+                                                    cacheElementOpt: Option[IndexEdge]): IndexEdge = {
+     assert(_kvs.size == 1)
+
+//     val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
+
+     val kv = implicitly[CanSKeyValue[T]].toSKeyValue(_kvs.head)
+     val version = kv.timestamp
+
+     val (srcVertexId, labelWithDir, labelIdxSeq, _, _) = cacheElementOpt.map { e =>
+       (e.srcVertex.id, e.labelWithDir, e.labelIndexSeq, false, 0)
+     }.getOrElse(parseRow(kv, schemaVer))
+
+     val label = checkLabel.getOrElse(Label.findById(labelWithDir.labelId))
+
+     val (idxPropsRaw, tgtVertexIdRaw, op, tgtVertexIdInQualifier, _) =
+       if (kv.qualifier.isEmpty) parseDegreeQualifier(kv, schemaVer)
+       else parseQualifier(kv, schemaVer)
+
+     val allProps = immutable.Map.newBuilder[LabelMeta, InnerValLikeWithTs]
+     val index = label.indicesMap.getOrElse(labelIdxSeq, throw new RuntimeException(s"invalid index seq: ${label.id.get}, ${labelIdxSeq}"))
+
+     /** process indexProps */
+     val size = idxPropsRaw.length
+     (0 until size).foreach { ith =>
+       val meta = index.sortKeyTypesArray(ith)
+       val (k, v) = idxPropsRaw(ith)
+       if (k == LabelMeta.degree) allProps += LabelMeta.degree -> InnerValLikeWithTs(v, version)
+       else allProps += meta -> InnerValLikeWithTs(v, version)
+     }
+//     for {
+//       (seq, (k, v)) <- index.sortKeyTypes.zip(idxPropsRaw)
+//     } {
+//       if (k == LabelMeta.degree) allProps += LabelMeta.degree -> InnerValLikeWithTs(v, version)
+//       else allProps += seq -> InnerValLikeWithTs(v, version)
+//     }
+
+     /** process props */
+     if (op == GraphUtil.operations("incrementCount")) {
+       //      val countVal = Bytes.toLong(kv.value)
+       val countVal = bytesToLongFunc(kv.value, 0)
+       allProps += (LabelMeta.count -> InnerValLikeWithTs.withLong(countVal, version, schemaVer))
+     } else if (kv.qualifier.isEmpty) {
+       val countVal = bytesToLongFunc(kv.value, 0)
+       allProps += (LabelMeta.degree -> InnerValLikeWithTs.withLong(countVal, version, schemaVer))
+     } else {
+       val (props, _) = bytesToKeyValues(kv.value, 0, kv.value.length, schemaVer, label)
+       props.foreach { case (k, v) =>
+         allProps += (k -> InnerValLikeWithTs(v, version))
+       }
+     }
+
+     val _mergedProps = allProps.result()
+     val (mergedProps, tsInnerValLikeWithTs) = _mergedProps.get(LabelMeta.timestamp) match {
+       case None =>
+         val tsInnerVal = InnerValLikeWithTs.withLong(version, version, schemaVer)
+         val mergedProps = _mergedProps + (LabelMeta.timestamp -> InnerValLikeWithTs.withLong(version, version, schemaVer))
+         (mergedProps, tsInnerVal)
+       case Some(tsInnerVal) =>
+         (_mergedProps, tsInnerVal)
+     }
+//     val mergedProps =
+//       if (_mergedProps.contains(LabelMeta.timestamp)) _mergedProps
+//            else _mergedProps + (LabelMeta.timestamp -> InnerValLikeWithTs.withLong(version, version, schemaVer))
+
+     /** process tgtVertexId */
+     val tgtVertexId =
+       mergedProps.get(LabelMeta.to) match {
+         case None => tgtVertexIdRaw
+         case Some(vId) => TargetVertexId(HBaseType.DEFAULT_COL_ID, vId.innerVal)
+       }
+
+     IndexEdge(Vertex(srcVertexId, version), Vertex(tgtVertexId, version), label, labelWithDir.dir, op, version, labelIdxSeq, mergedProps, tsInnerValOpt = Option(tsInnerValLikeWithTs.innerVal))
+
+   }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 c700e53..211b159 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
@@ -24,12 +24,13 @@ 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._
 
-class IndexEdgeSerializable(indexEdge: IndexEdge) extends Serializable[IndexEdge] {
+class IndexEdgeSerializable(indexEdge: IndexEdge, longToBytes: Long => Array[Byte] = longToBytes) extends Serializable[IndexEdge] {
    import StorageSerializable._
 
-   override val ts = indexEdge.version
-   override val table = indexEdge.label.hbaseTableName.getBytes()
+   override def ts = indexEdge.version
+   override def table = indexEdge.label.hbaseTableName.getBytes()
 
    def idxPropsMap = indexEdge.orders.toMap
    def idxPropsBytes = propsToBytes(indexEdge.orders)
@@ -49,7 +50,7 @@ class IndexEdgeSerializable(indexEdge: IndexEdge) extends Serializable[IndexEdge
        if (indexEdge.op == GraphUtil.operations("incrementCount")) {
          Bytes.add(idxPropsBytes, tgtIdBytes, Array.fill(1)(indexEdge.op))
        } else {
-         idxPropsMap.get(LabelMeta.toSeq) match {
+         idxPropsMap.get(LabelMeta.to) match {
            case None => Bytes.add(idxPropsBytes, tgtIdBytes)
            case Some(vId) => idxPropsBytes
          }
@@ -59,9 +60,9 @@ class IndexEdgeSerializable(indexEdge: IndexEdge) extends Serializable[IndexEdge
 
   override def toValue: Array[Byte] =
     if (indexEdge.degreeEdge)
-      Bytes.toBytes(indexEdge.props(LabelMeta.degreeSeq).innerVal.toString().toLong)
+      longToBytes(indexEdge.props(LabelMeta.degree).innerVal.toString().toLong)
     else if (indexEdge.op == GraphUtil.operations("incrementCount"))
-      Bytes.toBytes(indexEdge.props(LabelMeta.countSeq).innerVal.toString().toLong)
+      longToBytes(indexEdge.props(LabelMeta.count).innerVal.toString().toLong)
     else propsToKeyValues(indexEdge.metas.toSeq)
 
  }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 368e3f3..91b8db1 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,11 @@
 package org.apache.s2graph.core.storage.serde.snapshotedge.tall
 
 import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.mysqls.{LabelIndex, LabelMeta}
+import org.apache.s2graph.core.mysqls.{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.types.{HBaseType, LabelWithDirection, SourceAndTargetVertexIdPair, SourceVertexId}
-import org.apache.s2graph.core.{Edge, QueryParam, SnapshotEdge, Vertex}
+import org.apache.s2graph.core.{Edge, SnapshotEdge, Vertex}
 
 class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
 
@@ -34,7 +34,7 @@ class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
     (statusCode.toByte, op.toByte)
   }
 
-  override def fromKeyValuesInner[T: CanSKeyValue](queryParam: QueryParam,
+  override def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
                                                    _kvs: Seq[T],
                                                    version: String,
                                                    cacheElementOpt: Option[SnapshotEdge]): SnapshotEdge = {
@@ -42,9 +42,10 @@ class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
     assert(kvs.size == 1)
 
     val kv = kvs.head
-    val schemaVer = queryParam.label.schemaVersion
+    val label = checkLabel.get
+    val schemaVer = label.schemaVersion
     val cellVersion = kv.timestamp
-    /* rowKey */
+    /** rowKey */
     def parseRowV3(kv: SKeyValue, version: String) = {
       var pos = 0
       val (srcIdAndTgtId, srcIdAndTgtIdLen) = SourceAndTargetVertexIdPair.fromBytes(kv.row, pos, kv.row.length, version)
@@ -64,13 +65,14 @@ class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
     val srcVertexId = SourceVertexId(HBaseType.DEFAULT_COL_ID, srcInnerId)
     val tgtVertexId = SourceVertexId(HBaseType.DEFAULT_COL_ID, tgtInnerId)
 
-    val (props, op, ts, statusCode, _pendingEdgeOpt) = {
+    val (props, op, ts, statusCode, _pendingEdgeOpt, tsInnerVal) = {
       var pos = 0
       val (statusCode, op) = statusCodeWithOp(kv.value(pos))
       pos += 1
-      val (props, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer)
+      val (props, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer, label)
       val kvsMap = props.toMap
-      val ts = kvsMap(LabelMeta.timeStampSeq).innerVal.toString.toLong
+      val tsInnerVal = kvsMap(LabelMeta.timestamp).innerVal
+      val ts = tsInnerVal.toString.toLong
 
       pos = endAt
       val _pendingEdgeOpt =
@@ -80,24 +82,24 @@ class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
           pos += 1
           //          val versionNum = Bytes.toLong(kv.value, pos, 8)
           //          pos += 8
-          val (pendingEdgeProps, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer)
+          val (pendingEdgeProps, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer, label)
           pos = endAt
           val lockTs = Option(Bytes.toLong(kv.value, pos, 8))
 
           val pendingEdge =
             Edge(Vertex(srcVertexId, cellVersion),
               Vertex(tgtVertexId, cellVersion),
-              labelWithDir, pendingEdgeOp,
+              label, labelWithDir.dir, pendingEdgeOp,
               cellVersion, pendingEdgeProps.toMap,
-              statusCode = pendingEdgeStatusCode, lockTs = lockTs)
+              statusCode = pendingEdgeStatusCode, lockTs = lockTs, tsInnerValOpt = Option(tsInnerVal))
           Option(pendingEdge)
         }
 
-      (kvsMap, op, ts, statusCode, _pendingEdgeOpt)
+      (kvsMap, op, ts, statusCode, _pendingEdgeOpt, tsInnerVal)
     }
 
     SnapshotEdge(Vertex(srcVertexId, ts), Vertex(tgtVertexId, ts),
-      labelWithDir, op, cellVersion, props, statusCode = statusCode,
-      pendingEdgeOpt = _pendingEdgeOpt, lockTs = None)
+      label, labelWithDir.dir, op, cellVersion, props, statusCode = statusCode,
+      pendingEdgeOpt = _pendingEdgeOpt, lockTs = None, tsInnerValOpt = Option(tsInnerVal))
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 4f7c17b..fc84469 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
@@ -29,8 +29,8 @@ import org.apache.s2graph.core.types.SourceAndTargetVertexIdPair
 class SnapshotEdgeSerializable(snapshotEdge: SnapshotEdge) extends Serializable[SnapshotEdge] {
   import StorageSerializable._
 
-  override val ts = snapshotEdge.version
-  override val table = snapshotEdge.label.hbaseTableName.getBytes()
+  override def ts = snapshotEdge.version
+  override def table = snapshotEdge.label.hbaseTableName.getBytes()
 
   def statusCodeWithOp(statusCode: Byte, op: Byte): Array[Byte] = {
     val byte = (((statusCode << 4) | op).toByte)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 8aca2cf..8d95e77 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,11 @@
 package org.apache.s2graph.core.storage.serde.snapshotedge.wide
 
 import org.apache.hadoop.hbase.util.Bytes
-import org.apache.s2graph.core.mysqls.{LabelIndex, LabelMeta}
+import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
 import org.apache.s2graph.core.storage.StorageDeserializable._
 import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable, StorageDeserializable}
 import org.apache.s2graph.core.types.TargetVertexId
-import org.apache.s2graph.core.{Edge, QueryParam, SnapshotEdge, Vertex}
+import org.apache.s2graph.core.{Edge, SnapshotEdge, Vertex}
 
 class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
 
@@ -34,7 +34,7 @@ class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
     (statusCode.toByte, op.toByte)
   }
 
-  override def fromKeyValuesInner[T: CanSKeyValue](queryParam: QueryParam,
+  override def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
                                                    _kvs: Seq[T],
                                                    version: String,
                                                    cacheElementOpt: Option[SnapshotEdge]): SnapshotEdge = {
@@ -42,21 +42,23 @@ class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
     assert(kvs.size == 1)
 
     val kv = kvs.head
-    val schemaVer = queryParam.label.schemaVersion
+    val label = checkLabel.get
+    val schemaVer = label.schemaVersion
     val cellVersion = kv.timestamp
 
     val (srcVertexId, labelWithDir, _, _, _) = cacheElementOpt.map { e =>
       (e.srcVertex.id, e.labelWithDir, LabelIndex.DefaultSeq, true, 0)
     }.getOrElse(parseRow(kv, schemaVer))
 
-    val (tgtVertexId, props, op, ts, statusCode, _pendingEdgeOpt) = {
+    val (tgtVertexId, props, op, ts, statusCode, _pendingEdgeOpt, tsInnerVal) = {
       val (tgtVertexId, _) = TargetVertexId.fromBytes(kv.qualifier, 0, kv.qualifier.length, schemaVer)
       var pos = 0
       val (statusCode, op) = statusCodeWithOp(kv.value(pos))
       pos += 1
-      val (props, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer)
+      val (props, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer, label)
       val kvsMap = props.toMap
-      val ts = kvsMap(LabelMeta.timeStampSeq).innerVal.toString.toLong
+      val tsInnerVal = kvsMap(LabelMeta.timestamp).innerVal
+      val ts = tsInnerVal.toString.toLong
 
       pos = endAt
       val _pendingEdgeOpt =
@@ -66,24 +68,24 @@ class SnapshotEdgeDeserializable extends Deserializable[SnapshotEdge] {
           pos += 1
           //          val versionNum = Bytes.toLong(kv.value, pos, 8)
           //          pos += 8
-          val (pendingEdgeProps, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer)
+          val (pendingEdgeProps, endAt) = bytesToKeyValuesWithTs(kv.value, pos, schemaVer, label)
           pos = endAt
           val lockTs = Option(Bytes.toLong(kv.value, pos, 8))
 
           val pendingEdge =
             Edge(Vertex(srcVertexId, cellVersion),
               Vertex(tgtVertexId, cellVersion),
-              labelWithDir, pendingEdgeOp,
+              label, labelWithDir.dir, pendingEdgeOp,
               cellVersion, pendingEdgeProps.toMap,
-              statusCode = pendingEdgeStatusCode, lockTs = lockTs)
+              statusCode = pendingEdgeStatusCode, lockTs = lockTs, tsInnerValOpt = Option(tsInnerVal))
           Option(pendingEdge)
         }
 
-      (tgtVertexId, kvsMap, op, ts, statusCode, _pendingEdgeOpt)
+      (tgtVertexId, kvsMap, op, ts, statusCode, _pendingEdgeOpt, tsInnerVal)
     }
 
     SnapshotEdge(Vertex(srcVertexId, ts), Vertex(tgtVertexId, ts),
-      labelWithDir, op, cellVersion, props, statusCode = statusCode,
-      pendingEdgeOpt = _pendingEdgeOpt, lockTs = None)
+      label, labelWithDir.dir, op, cellVersion, props, statusCode = statusCode,
+      pendingEdgeOpt = _pendingEdgeOpt, lockTs = None, tsInnerValOpt = Option(tsInnerVal))
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 757ef1b..4ceb4a8 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
@@ -34,8 +34,8 @@ import org.apache.s2graph.core.types.VertexId
 class SnapshotEdgeSerializable(snapshotEdge: SnapshotEdge) extends Serializable[SnapshotEdge] {
   import StorageSerializable._
 
-  override val ts = snapshotEdge.version
-  override val table = snapshotEdge.label.hbaseTableName.getBytes()
+  override def ts = snapshotEdge.version
+  override def table = snapshotEdge.label.hbaseTableName.getBytes()
 
   def statusCodeWithOp(statusCode: Byte, op: Byte): Array[Byte] = {
     val byte = (((statusCode << 4) | op).toByte)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 737c2a8..3ec17ab 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
@@ -6,9 +6,9 @@
  * 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
@@ -19,18 +19,19 @@
 
 package org.apache.s2graph.core.storage.serde.vertex
 
-import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core.mysqls.Label
+import org.apache.s2graph.core.storage.StorageDeserializable._
 import org.apache.s2graph.core.storage.{CanSKeyValue, Deserializable}
 import org.apache.s2graph.core.types.{InnerVal, InnerValLike, VertexId}
 import org.apache.s2graph.core.{QueryParam, Vertex}
 
 import scala.collection.mutable.ListBuffer
 
-class VertexDeserializable extends Deserializable[Vertex] {
-  def fromKeyValuesInner[T: CanSKeyValue](queryParam: QueryParam,
-                                     _kvs: Seq[T],
-                                     version: String,
-                                     cacheElementOpt: Option[Vertex]): Vertex = {
+class VertexDeserializable(bytesToInt: (Array[Byte], Int) => Int = bytesToInt) extends Deserializable[Vertex] {
+  def fromKeyValuesInner[T: CanSKeyValue](checkLabel: Option[Label],
+                                          _kvs: Seq[T],
+                                          version: String,
+                                          cacheElementOpt: Option[Vertex]): Vertex = {
 
     val kvs = _kvs.map { kv => implicitly[CanSKeyValue[T]].toSKeyValue(kv) }
 
@@ -46,7 +47,7 @@ class VertexDeserializable extends Deserializable[Vertex] {
     } {
       val propKey =
         if (kv.qualifier.length == 1) kv.qualifier.head.toInt
-        else Bytes.toInt(kv.qualifier)
+        else bytesToInt(kv.qualifier, 0)
 
       val ts = kv.timestamp
       if (ts > maxTs) maxTs = ts
@@ -63,4 +64,3 @@ class VertexDeserializable extends Deserializable[Vertex] {
     Vertex(vertexId, maxTs, propsMap.toMap, belongLabelIds = belongLabelIds)
   }
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 6bb162c..77bbb87 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
@@ -6,9 +6,9 @@
  * 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
@@ -19,11 +19,12 @@
 
 package org.apache.s2graph.core.storage.serde.vertex
 
-import org.apache.hadoop.hbase.util.Bytes
 import org.apache.s2graph.core.Vertex
+import org.apache.s2graph.core.storage.StorageSerializable._
 import org.apache.s2graph.core.storage.{SKeyValue, Serializable}
+import org.apache.s2graph.core.utils.logger
 
-case class VertexSerializable(vertex: Vertex) extends Serializable[Vertex] {
+case class VertexSerializable(vertex: Vertex, intToBytes: Int => Array[Byte] = intToBytes) extends Serializable[Vertex] {
 
   override val table = vertex.hbaseTableName.getBytes
   override val ts = vertex.ts
@@ -37,10 +38,10 @@ case class VertexSerializable(vertex: Vertex) extends Serializable[Vertex] {
   /** vertex override toKeyValues since vertex expect to produce multiple sKeyValues */
   override def toKeyValues: Seq[SKeyValue] = {
     val row = toRowKey
-    val base = for ((k, v) <- vertex.props ++ vertex.defaultProps) yield Bytes.toBytes(k) -> v.bytes
-    val belongsTo = vertex.belongLabelIds.map { labelId => Bytes.toBytes(Vertex.toPropKey(labelId)) -> Array.empty[Byte] }
+    val base = for ((k, v) <- vertex.props ++ vertex.defaultProps) yield intToBytes(k) -> v.bytes
+    val belongsTo = vertex.belongLabelIds.map { labelId => intToBytes(Vertex.toPropKey(labelId)) -> Array.empty[Byte] }
     (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/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/types/HBaseType.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/types/HBaseType.scala b/s2core/src/main/scala/org/apache/s2graph/core/types/HBaseType.scala
index c03319d..b885bc6 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/types/HBaseType.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/types/HBaseType.scala
@@ -20,6 +20,7 @@
 package org.apache.s2graph.core.types
 
 import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core.mysqls.LabelMeta
 
 object HBaseType {
   val VERSION4 = "v4"
@@ -28,7 +29,7 @@ object HBaseType {
   val VERSION1 = "v1"
 //  val DEFAULT_VERSION = VERSION2
   val DEFAULT_VERSION = VERSION3
-  val EMPTY_SEQ_BYTE = Byte.MaxValue
+//  val EMPTY_SEQ_BYTE = Byte.MaxValue
   val DEFAULT_COL_ID = 0
   val bitsForDir = 2
   val maxBytes = Bytes.toBytes(Int.MaxValue)
@@ -100,7 +101,7 @@ object HBaseDeserializable {
     val kvs = new Array[(Byte, InnerValLike)](len)
     var i = 0
     while (i < len) {
-      val k = EMPTY_SEQ_BYTE
+      val k = LabelMeta.emptySeq
       val (v, numOfBytesUsed) = InnerVal.fromBytes(bytes, pos, 0, version)
       pos += numOfBytesUsed
       kvs(i) = (k -> v)



[7/7] incubator-s2graph git commit: [S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

Posted by st...@apache.org.
[S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

JIRA:
  [S2GRAPH-122] https://issues.apache.org/jira/browse/S2GRAPH-122

Pull Request:
  Closes #97

Authors
  DO YUNG YOON: steamshon@apache.org


Project: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/commit/66bdf1bc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/tree/66bdf1bc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s2graph/diff/66bdf1bc

Branch: refs/heads/master
Commit: 66bdf1bc09b1a976a8d9d0f07b6af572a6074aae
Parents: 8dbb9a3
Author: DO YUNG YOON <st...@apache.org>
Authored: Wed Nov 16 17:32:02 2016 +0100
Committer: DO YUNG YOON <st...@apache.org>
Committed: Wed Nov 16 17:32:02 2016 +0100

----------------------------------------------------------------------
 CHANGES                                         |    2 +
 dev_support/docker-compose.yml                  |    5 -
 dev_support/graph_mysql/schema.sql              |  248 ++--
 .../loader/subscriber/GraphSubscriber.scala     |  145 ---
 .../loader/subscriber/TransferToHFile.scala     |   28 +-
 .../scala/org/apache/s2graph/core/Edge.scala    |  266 ++--
 .../scala/org/apache/s2graph/core/Graph.scala   | 1164 ++++++++++++++----
 .../apache/s2graph/core/GraphExceptions.scala   |   24 +-
 .../org/apache/s2graph/core/GraphUtil.scala     |    8 +-
 .../org/apache/s2graph/core/Management.scala    |   97 +-
 .../org/apache/s2graph/core/PostProcess.scala   |  195 ++-
 .../org/apache/s2graph/core/QueryParam.scala    |  575 +++------
 .../org/apache/s2graph/core/QueryResult.scala   |  328 ++---
 .../scala/org/apache/s2graph/core/Vertex.scala  |   59 -
 .../org/apache/s2graph/core/mysqls/Bucket.scala |   26 +-
 .../apache/s2graph/core/mysqls/Experiment.scala |   13 +-
 .../org/apache/s2graph/core/mysqls/Label.scala  |  203 ++-
 .../apache/s2graph/core/mysqls/LabelIndex.scala |   13 +-
 .../apache/s2graph/core/mysqls/LabelMeta.scala  |   64 +-
 .../org/apache/s2graph/core/mysqls/Model.scala  |   37 +-
 .../apache/s2graph/core/mysqls/Service.scala    |   15 +-
 .../s2graph/core/parsers/WhereParser.scala      |  103 +-
 .../s2graph/core/rest/RequestParser.scala       |  334 ++---
 .../apache/s2graph/core/rest/RestHandler.scala  |  118 +-
 .../apache/s2graph/core/storage/SKeyValue.scala |   12 +-
 .../apache/s2graph/core/storage/Storage.scala   |  771 ++++--------
 .../core/storage/StorageDeserializable.scala    |   35 +-
 .../core/storage/StorageSerializable.scala      |   24 +-
 .../core/storage/hbase/AsynchbaseStorage.scala  |  454 ++++---
 .../tall/IndexEdgeDeserializable.scala          |  222 ++--
 .../indexedge/tall/IndexEdgeSerializable.scala  |   26 +-
 .../wide/IndexEdgeDeserializable.scala          |  230 ++--
 .../indexedge/wide/IndexEdgeSerializable.scala  |   13 +-
 .../tall/SnapshotEdgeDeserializable.scala       |   30 +-
 .../tall/SnapshotEdgeSerializable.scala         |    4 +-
 .../wide/SnapshotEdgeDeserializable.scala       |   28 +-
 .../wide/SnapshotEdgeSerializable.scala         |    4 +-
 .../serde/vertex/VertexDeserializable.scala     |   20 +-
 .../serde/vertex/VertexSerializable.scala       |   15 +-
 .../apache/s2graph/core/types/HBaseType.scala   |    5 +-
 .../s2graph/core/types/InnerValLike.scala       |   15 +-
 .../apache/s2graph/core/utils/DeferCache.scala  |    5 +-
 .../apache/s2graph/core/utils/Extentions.scala  |   65 +-
 .../s2graph/core/utils/SafeUpdateCache.scala    |    9 +-
 .../org/apache/s2graph/core/EdgeTest.scala      |   48 +-
 .../s2graph/core/Integrate/CrudTest.scala       |    6 +-
 .../core/Integrate/IntegrateCommon.scala        |   20 +-
 .../s2graph/core/Integrate/QueryTest.scala      |  530 ++++----
 .../core/Integrate/WeakLabelDeleteTest.scala    |    6 +-
 .../apache/s2graph/core/ManagementTest.scala    |  117 +-
 .../apache/s2graph/core/QueryParamTest.scala    |  316 ++---
 .../s2graph/core/TestCommonWithModels.scala     |    8 +
 .../apache/s2graph/core/models/ModelTest.scala  |   90 --
 .../s2graph/core/mysqls/ExperimentSpec.scala    |   83 --
 .../s2graph/core/parsers/WhereParserTest.scala  |  168 +--
 .../core/storage/hbase/IndexEdgeTest.scala      |   31 +-
 .../org/apache/s2graph/rest/netty/Server.scala  |   63 +-
 .../rest/play/controllers/AdminController.scala |    2 +-
 .../rest/play/controllers/EdgeController.scala  |    6 +-
 59 files changed, 4071 insertions(+), 3480 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 73b3597..01d1cc8 100644
--- a/CHANGES
+++ b/CHANGES
@@ -95,6 +95,8 @@ Release 0.1.0 - unreleased
     S2GRAPH-127: Refactor ExceptionHander Object into Class (Committed by DOYUNG YOON).
 
     S2GRAPH-121: Create `Result` class to hold traverse result edges (Committed by DOYUNG YOON).
+ 
+    S2GRAPH-122: Change data types of Edge/IndexEdge/SnapshotEdge (Committed by DOYUNG YOON).
 
   BUG FIXES
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/dev_support/docker-compose.yml
----------------------------------------------------------------------
diff --git a/dev_support/docker-compose.yml b/dev_support/docker-compose.yml
index 5531fde..fb95a39 100644
--- a/dev_support/docker-compose.yml
+++ b/dev_support/docker-compose.yml
@@ -13,11 +13,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-graph:
-    image: s2rest_play:latest
-    container_name: graph
-    net: container:graph_hbase
-
 graph_mysql:
     build: graph_mysql
     container_name: graph_mysql

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/dev_support/graph_mysql/schema.sql
----------------------------------------------------------------------
diff --git a/dev_support/graph_mysql/schema.sql b/dev_support/graph_mysql/schema.sql
index 862062b..df283a8 100644
--- a/dev_support/graph_mysql/schema.sql
+++ b/dev_support/graph_mysql/schema.sql
@@ -35,17 +35,17 @@ SET FOREIGN_KEY_CHECKS = 0;
 -- ----------------------------
 DROP TABLE IF EXISTS `services`;
 CREATE TABLE `services` (
-  `id` integer NOT NULL AUTO_INCREMENT,
-  `service_name` varchar(64) NOT NULL,
-  `access_token` varchar(64) NOT NULL,
-  `cluster` varchar(255) NOT NULL,
-  `hbase_table_name` varchar(255) NOT NULL,
-  `pre_split_size` integer NOT NULL default 0,
-  `hbase_table_ttl` integer,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_service_name` (`service_name`),
-  INDEX `idx_access_token` (`access_token`),
-  INDEX `idx_cluster` (cluster(75))
+	`id` integer NOT NULL AUTO_INCREMENT,
+	`service_name` varchar(64) NOT NULL,
+	`access_token` varchar(64) NOT NULL,
+	`cluster` varchar(255) NOT NULL,
+	`hbase_table_name` varchar(255) NOT NULL,
+	`pre_split_size` integer NOT NULL default 0,
+	`hbase_table_ttl` integer,
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_service_name` (`service_name`),
+	INDEX `idx_access_token` (`access_token`),
+	INDEX `idx_cluster` (cluster(75))
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 
@@ -54,13 +54,13 @@ CREATE TABLE `services` (
 -- ----------------------------
 DROP TABLE IF EXISTS `service_columns`;
 CREATE TABLE `service_columns` (
-  `id` integer NOT NULL AUTO_INCREMENT,
-  `service_id` integer NOT NULL,
-  `column_name` varchar(64) NOT NULL,
-  `column_type` varchar(8) NOT NULL,
-  `schema_version` varchar(8) NOT NULL default 'v2',
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_service_id_column_name` (`service_id`, `column_name`)
+	`id` integer NOT NULL AUTO_INCREMENT,
+	`service_id` integer NOT NULL,
+	`column_name` varchar(64) NOT NULL,
+	`column_type` varchar(8) NOT NULL,
+	`schema_version` varchar(8) NOT NULL default 'v2',
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_service_id_column_name` (`service_id`, `column_name`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 ALTER TABLE service_columns add FOREIGN KEY(service_id) REFERENCES services(id) ON DELETE CASCADE;
@@ -71,14 +71,14 @@ ALTER TABLE service_columns add FOREIGN KEY(service_id) REFERENCES services(id)
 -- ----------------------------
 DROP TABLE IF EXISTS `column_metas`;
 CREATE TABLE `column_metas` (
-  `id` integer NOT NULL AUTO_INCREMENT,
-  `column_id` integer NOT NULL,
-  `name` varchar(64) NOT NULL,
-  `seq` tinyint	NOT NULL,
-  `data_type` varchar(8) NOT NULL DEFAULT 'string',
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_column_id_name` (`column_id`, `name`),
-  INDEX `idx_column_id_seq` (`column_id`, `seq`)
+	`id` integer NOT NULL AUTO_INCREMENT,
+	`column_id` integer NOT NULL,
+	`name` varchar(64) NOT NULL,
+	`seq` tinyint	NOT NULL,
+	`data_type` varchar(8) NOT NULL DEFAULT 'string',
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_column_id_name` (`column_id`, `name`),
+	INDEX `idx_column_id_seq` (`column_id`, `seq`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 ALTER TABLE column_metas ADD FOREIGN KEY(column_id) REFERENCES service_columns(id) ON DELETE CASCADE;
@@ -89,32 +89,33 @@ ALTER TABLE column_metas ADD FOREIGN KEY(column_id) REFERENCES service_columns(i
 
 DROP TABLE IF EXISTS `labels`;
 CREATE TABLE `labels` (
-  `id` integer NOT NULL AUTO_INCREMENT,
-  `label` varchar(64) NOT NULL,
-  `src_service_id` integer NOT NULL,
-  `src_column_name` varchar(64) NOT NULL,
-  `src_column_type` varchar(8) NOT NULL,
-  `tgt_service_id` integer NOT NULL,
-  `tgt_column_name` varchar(64) NOT NULL,
-  `tgt_column_type` varchar(8) NOT NULL,
-  `is_directed` tinyint	NOT NULL DEFAULT 1,
-  `service_name` varchar(64),
-  `service_id` integer NOT NULL,
-  `consistency_level` varchar(8) NOT NULL DEFAULT 'weak',
-  `hbase_table_name` varchar(255) NOT NULL DEFAULT 's2graph',
-  `hbase_table_ttl` integer,
-  `schema_version` varchar(8) NOT NULL default 'v2',
-  `is_async` tinyint(4) NOT NULL default '0',
-  `compressionAlgorithm` varchar(64) NOT NULL DEFAULT 'lz4',
-  `options` text,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_label` (`label`),
-  INDEX `idx_src_column_name` (`src_column_name`),
-  INDEX	`idx_tgt_column_name` (`tgt_column_name`),
-  INDEX `idx_src_service_id` (`src_service_id`),
-  INDEX `idx_tgt_service_id` (`tgt_service_id`),
-  INDEX `idx_service_name` (`service_name`), 
-  INDEX `idx_service_id` (`service_id`)
+	`id` integer NOT NULL AUTO_INCREMENT,
+	`label` varchar(128) NOT NULL,
+	`src_service_id` integer NOT NULL,
+	`src_column_name` varchar(64) NOT NULL,
+	`src_column_type` varchar(8) NOT NULL,
+	`tgt_service_id` integer NOT NULL,
+	`tgt_column_name` varchar(64) NOT NULL,
+	`tgt_column_type` varchar(8) NOT NULL,
+	`is_directed` tinyint	NOT NULL DEFAULT 1,
+	`service_name` varchar(64),
+	`service_id` integer NOT NULL,
+	`consistency_level` varchar(8) NOT NULL DEFAULT 'weak',
+	`hbase_table_name` varchar(255) NOT NULL DEFAULT 's2graph',
+	`hbase_table_ttl` integer,
+	`schema_version` varchar(8) NOT NULL default 'v2',
+	`is_async` tinyint(4) NOT NULL default '0',
+	`compressionAlgorithm` varchar(64) NOT NULL DEFAULT 'lz4',
+	`options` text,
+	`deleted_at` datetime DEFAULT NULL,
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_label` (`label`),
+	INDEX `idx_src_column_name` (`src_column_name`),
+	INDEX	`idx_tgt_column_name` (`tgt_column_name`),
+	INDEX `idx_src_service_id` (`src_service_id`),
+	INDEX `idx_tgt_service_id` (`tgt_service_id`),
+	INDEX `idx_service_name` (`service_name`),
+	INDEX `idx_service_id` (`service_id`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 ALTER TABLE labels add FOREIGN KEY(service_id) REFERENCES services(id);
@@ -126,16 +127,16 @@ ALTER TABLE labels add FOREIGN KEY(service_id) REFERENCES services(id);
 -- ----------------------------
 DROP TABLE IF EXISTS `label_metas`;
 CREATE TABLE `label_metas` (
-  `id` integer NOT NULL AUTO_INCREMENT,
-  `label_id` integer NOT NULL,
-  `name` varchar(64) NOT NULL,
-  `seq` tinyint	NOT NULL,
-  `default_value` varchar(64) NOT NULL,
-  `data_type` varchar(8) NOT NULL DEFAULT 'long',
-  `used_in_index` tinyint	NOT NULL DEFAULT 0,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_label_id_name` (`label_id`, `name`),
-  INDEX `idx_label_id_seq` (`label_id`, `seq`)
+	`id` integer NOT NULL AUTO_INCREMENT,
+	`label_id` integer NOT NULL,
+	`name` varchar(64) NOT NULL,
+	`seq` tinyint	NOT NULL,
+	`default_value` varchar(64) NOT NULL,
+	`data_type` varchar(8) NOT NULL DEFAULT 'long',
+	`used_in_index` tinyint	NOT NULL DEFAULT 0,
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_label_id_name` (`label_id`, `name`),
+	INDEX `idx_label_id_seq` (`label_id`, `seq`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 ALTER TABLE label_metas ADD FOREIGN KEY(label_id) REFERENCES labels(id) ON DELETE CASCADE;
@@ -146,15 +147,18 @@ ALTER TABLE label_metas ADD FOREIGN KEY(label_id) REFERENCES labels(id) ON DELET
 -- ----------------------------
 DROP TABLE IF EXISTS `label_indices`;
 CREATE TABLE `label_indices` (
-  `id` int(11) NOT NULL AUTO_INCREMENT,
-  `label_id` int(11) NOT NULL,
-  `name` varchar(64) NOT NULL DEFAULT '_PK',
-  `seq` tinyint(4) NOT NULL,
-  `meta_seqs` varchar(64) NOT NULL,
-  `formulars` varchar(255) DEFAULT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_label_id_seq` (`label_id`,`meta_seqs`),
-  UNIQUE KEY `ux_label_id_name` (`label_id`,`name`)
+	`id` int(11) NOT NULL AUTO_INCREMENT,
+	`label_id` int(11) NOT NULL,
+	`name` varchar(64) NOT NULL DEFAULT '_PK',
+	`seq` tinyint(4) NOT NULL,
+	`meta_seqs` varchar(64) NOT NULL,
+	`formulars` varchar(255) DEFAULT NULL,
+	`dir` int DEFAULT NULL,
+	`options` text,
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_label_id_seq` (`label_id`,`meta_seqs`),
+	UNIQUE KEY `ux_label_id_name` (`label_id`,`name`),
+	UNIQUE KEY `ux_label_id_meta_seqs_dir` (`label_id`,`meta_seqs`,`dir`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 ALTER TABLE label_indices ADD FOREIGN KEY(label_id) REFERENCES labels(id) ON DELETE CASCADE;
@@ -165,15 +169,15 @@ ALTER TABLE label_indices ADD FOREIGN KEY(label_id) REFERENCES labels(id) ON DEL
 -- ----------------------------
 DROP TABLE IF EXISTS `experiments`;
 CREATE TABLE `experiments` (
-  `id` integer NOT NULL AUTO_INCREMENT,
-  `service_id` integer NOT NULL,
-  `service_name` varchar(128) NOT NULL,
-  `name` varchar(64) NOT NULL,
-  `description` varchar(255) NOT NULL,
-  `experiment_type` varchar(8) NOT NULL DEFAULT 'u',
-  `total_modular` int NOT NULL DEFAULT 100,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_service_id_name` (`service_id`, `name`)
+	`id` integer NOT NULL AUTO_INCREMENT,
+	`service_id` integer NOT NULL,
+	`service_name` varchar(128) NOT NULL,
+	`name` varchar(64) NOT NULL,
+	`description` varchar(255) NOT NULL,
+	`experiment_type` varchar(8) NOT NULL DEFAULT 'u',
+	`total_modular` int NOT NULL DEFAULT 100,
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_service_id_name` (`service_id`, `name`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 -- ALTER TABLE experiments ADD FOREIGN KEY(service_id) REFERENCES service(id) ON DELETE CASCADE;
@@ -182,23 +186,24 @@ CREATE TABLE `experiments` (
 -- ----------------------------
 --  Table structure for `buckets`
 -- ----------------------------
+DROP TABLE IF EXISTS `buckets`;
 CREATE TABLE `buckets` (
-  `id` integer NOT NULL AUTO_INCREMENT,
-  `experiment_id` integer NOT NULL,
-  `modular` varchar(64) NOT NULL,
-  `http_verb` varchar(8) NOT NULL,
-  `api_path` text NOT NULL,
-  `uuid_key` varchar(128),
-  `uuid_placeholder` varchar(64),
-  `request_body` text NOT NULL,
-  `timeout` int NOT NULL DEFAULT 1000,
-  `impression_id` varchar(64) NOT NULL,
-  `is_graph_query` tinyint NOT NULL DEFAULT 1,
-  `is_empty` tinyint NOT NULL DEFAULT 0,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `ux_impression_id` (`impression_id`),
-  INDEX `idx_experiment_id` (`experiment_id`),
-  INDEX `idx_impression_id` (`impression_id`)
+	`id` integer NOT NULL AUTO_INCREMENT,
+	`experiment_id` integer NOT NULL,
+	`modular` varchar(64) NOT NULL,
+	`http_verb` varchar(8) NOT NULL,
+	`api_path` text NOT NULL,
+	`uuid_key` varchar(128),
+	`uuid_placeholder` varchar(64),
+	`request_body` text NOT NULL,
+	`timeout` int NOT NULL DEFAULT 1000,
+	`impression_id` varchar(64) NOT NULL,
+	`is_graph_query` tinyint NOT NULL DEFAULT 1,
+	`is_empty` tinyint NOT NULL DEFAULT 0,
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `ux_impression_id` (`impression_id`),
+	INDEX `idx_experiment_id` (`experiment_id`),
+	INDEX `idx_impression_id` (`impression_id`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;
 
 SET FOREIGN_KEY_CHECKS = 1;
@@ -209,29 +214,30 @@ SET FOREIGN_KEY_CHECKS = 1;
 -- ----------------------------
 DROP TABLE IF EXISTS `counter`;
 CREATE TABLE `counter` (
-  `id` int(11) unsigned NOT NULL AUTO_INCREMENT,
-  `use_flag` tinyint(1) NOT NULL DEFAULT '0',
-  `version` smallint(1) NOT NULL DEFAULT '1',
-  `service` varchar(64) NOT NULL DEFAULT '',
-  `action` varchar(64) NOT NULL DEFAULT '',
-  `item_type` int(11) NOT NULL DEFAULT '0',
-  `auto_comb` tinyint(1) NOT NULL DEFAULT '1',
-  `dimension` varchar(1024) NOT NULL,
-  `use_profile` tinyint(1) NOT NULL DEFAULT '0',
-  `bucket_imp_id` varchar(64) DEFAULT NULL,
-  `use_exact` tinyint(1) NOT NULL DEFAULT '1',
-  `use_rank` tinyint(1) NOT NULL DEFAULT '1',
-  `ttl` int(11) NOT NULL DEFAULT '172800',
-  `daily_ttl` int(11) DEFAULT NULL,
-  `hbase_table` varchar(1024) DEFAULT NULL,
-  `interval_unit` varchar(1024) DEFAULT NULL,
-  `rate_action_id` int(11) unsigned DEFAULT NULL,
-  `rate_base_id` int(11) unsigned DEFAULT NULL,
-  `rate_threshold` int(11) DEFAULT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `svc` (`service`,`action`),
-  KEY `rate_action_id` (`rate_action_id`),
-  KEY `rate_base_id` (`rate_base_id`),
-  CONSTRAINT `rate_action_id` FOREIGN KEY (`rate_action_id`) REFERENCES `counter` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION,
-  CONSTRAINT `rate_base_id` FOREIGN KEY (`rate_base_id`) REFERENCES `counter` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION
+	`id` int(11) unsigned NOT NULL AUTO_INCREMENT,
+	`use_flag` tinyint(1) NOT NULL DEFAULT '0',
+	`version` smallint(1) NOT NULL DEFAULT '1',
+	`service` varchar(64) NOT NULL DEFAULT '',
+	`action` varchar(64) NOT NULL DEFAULT '',
+	`item_type` int(11) NOT NULL DEFAULT '0',
+	`auto_comb` tinyint(1) NOT NULL DEFAULT '1',
+	`dimension` varchar(1024) NOT NULL,
+	`use_profile` tinyint(1) NOT NULL DEFAULT '0',
+	`bucket_imp_id` varchar(64) DEFAULT NULL,
+	`use_exact` tinyint(1) NOT NULL DEFAULT '1',
+	`use_rank` tinyint(1) NOT NULL DEFAULT '1',
+	`ttl` int(11) NOT NULL DEFAULT '172800',
+	`daily_ttl` int(11) DEFAULT NULL,
+	`hbase_table` varchar(1024) DEFAULT NULL,
+	`interval_unit` varchar(1024) DEFAULT NULL,
+	`rate_action_id` int(11) unsigned DEFAULT NULL,
+	`rate_base_id` int(11) unsigned DEFAULT NULL,
+	`rate_threshold` int(11) DEFAULT NULL,
+	`top_k` int(11) DEFAULT NULL,
+	PRIMARY KEY (`id`),
+	UNIQUE KEY `svc` (`service`,`action`),
+	KEY `rate_action_id` (`rate_action_id`),
+	KEY `rate_base_id` (`rate_base_id`),
+	CONSTRAINT `rate_action_id` FOREIGN KEY (`rate_action_id`) REFERENCES `counter` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION,
+	CONSTRAINT `rate_base_id` FOREIGN KEY (`rate_base_id`) REFERENCES `counter` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4;

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
----------------------------------------------------------------------
diff --git a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
index 9e9fe4c..05aed34 100644
--- a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
+++ b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/GraphSubscriber.scala
@@ -207,148 +207,3 @@ object GraphSubscriberHelper extends WithKafka {
   }
 }
 
-//object GraphSubscriber extends SparkApp with WithKafka {
-//  val sleepPeriod = 5000
-//  val usages =
-//    s"""
-//       |/**
-//       |* this job read edge format(TSV) from HDFS file system then bulk load edges into s2graph. assumes that newLabelName is already created by API.
-//       |* params:
-//       |*  0. hdfsPath: where is your data in hdfs. require full path with hdfs:// predix
-//       |*  1. dbUrl: jdbc database connection string to specify database for meta.
-//       |*  2. labelMapping: oldLabel:newLabel delimited by ,
-//       |*  3. zkQuorum: target hbase zkQuorum where this job will publish data to.
-//       |*  4. hTableName: target hbase physical table name where this job will publish data to.
-//       |*  5. batchSize: how many edges will be batched for Put request to target hbase.
-//       |*  6. kafkaBrokerList: using kafka as fallback queue. when something goes wrong during batch, data needs to be replay will be stored in kafka.
-//       |*  7. kafkaTopic: fallback queue topic.
-//       |*  8. edgeAutoCreate: true if need to create reversed edge automatically.
-//       |*
-//       |* after this job finished, s2graph will have data with sequence corresponding newLabelName.
-//       |* change this newLabelName to ogirinalName if you want to online replace of label.
-//       |*
-//       |*/
-//     """.stripMargin
-//
-//  override def run() = {
-//    /**
-//     * Main function
-//     */
-//    println(args.toList)
-////    if (args.length != 10) {
-////      System.err.println(usages)
-////      System.exit(1)
-////    }
-//    val hdfsPath = args(0)
-//    val dbUrl = args(1)
-//    val labelMapping = GraphSubscriberHelper.toLabelMapping(args(2))
-//
-//    val zkQuorum = args(3)
-//    val hTableName = args(4)
-//    val batchSize = args(5).toInt
-//    val kafkaBrokerList = args(6)
-//    val kafkaTopic = args(7)
-//    val edgeAutoCreate = args(8).toBoolean
-//    val vertexDegreePathOpt = if (args.length >= 10) GraphSubscriberHelper.toOption(args(9)) else None
-//
-//    val conf = sparkConf(s"$hdfsPath: GraphSubscriber")
-//    val sc = new SparkContext(conf)
-//    val mapAcc = sc.accumulable(HashMap.empty[String, Long], "counter")(HashMapParam[String, Long](_ + _))
-//
-//
-//    if (!GraphSubscriberHelper.isValidQuorum(zkQuorum)) throw new RuntimeException(s"$zkQuorum is not valid.")
-//
-//    /** this job expect only one hTableName. all labels in this job will be stored in same physical hbase table */
-//    try {
-//
-//      import GraphSubscriberHelper._
-//      // set local driver setting.
-//      val phase = System.getProperty("phase")
-//      GraphSubscriberHelper.apply(phase, dbUrl, zkQuorum, kafkaBrokerList)
-//
-//      /** copy when oldLabel exist and newLabel done exist. otherwise ignore. */
-//
-//      if (labelMapping.isEmpty) {
-//        // pass
-//      } else {
-//        for {
-//          (oldLabelName, newLabelName) <- labelMapping
-//        } {
-//          Management.copyLabel(oldLabelName, newLabelName, toOption(hTableName))
-//        }
-//      }
-//
-//      vertexDegreePathOpt.foreach { vertexDegreePath =>
-//        val vertexDegrees = sc.textFile(vertexDegreePath).filter(line => line.split("\t").length == 4).map { line =>
-//          val tokens = line.split("\t")
-//          (tokens(0), tokens(1), tokens(2), tokens(3).toInt)
-//        }
-//        vertexDegrees.foreachPartition { partition =>
-//
-//          // init Graph
-//          val phase = System.getProperty("phase")
-//          GraphSubscriberHelper.apply(phase, dbUrl, zkQuorum, kafkaBrokerList)
-//
-//          partition.grouped(batchSize).foreach { msgs =>
-//            try {
-//              val start = System.currentTimeMillis()
-//              val counts = GraphSubscriberHelper.storeDegreeBulk(zkQuorum, hTableName)(msgs, labelMapping)(Some(mapAcc))
-//              for ((k, v) <- counts) {
-//                mapAcc +=(k, v)
-//              }
-//              val duration = System.currentTimeMillis() - start
-//              println(s"[Success]: store, $mapAcc, $duration, $zkQuorum, $hTableName")
-//            } catch {
-//              case e: Throwable =>
-//                println(s"[Failed]: store $e")
-//
-//                msgs.foreach { msg =>
-//                  GraphSubscriberHelper.report(msg.toString(), Some(e.getMessage()), topic = kafkaTopic)
-//                }
-//            }
-//          }
-//        }
-//      }
-//
-//
-//      val msgs = sc.textFile(hdfsPath)
-//      msgs.foreachPartition(partition => {
-//        // set executor setting.
-//        val phase = System.getProperty("phase")
-//        GraphSubscriberHelper.apply(phase, dbUrl, zkQuorum, kafkaBrokerList)
-//
-//        partition.grouped(batchSize).foreach { msgs =>
-//          try {
-//            val start = System.currentTimeMillis()
-//            //            val counts =
-//            //              GraphSubscriberHelper.store(msgs, GraphSubscriberHelper.toOption(newLabelName))(Some(mapAcc))
-//            val counts =
-//              GraphSubscriberHelper.storeBulk(zkQuorum, hTableName)(msgs, labelMapping, edgeAutoCreate)(Some(mapAcc))
-//
-//            for ((k, v) <- counts) {
-//              mapAcc +=(k, v)
-//            }
-//            val duration = System.currentTimeMillis() - start
-//            println(s"[Success]: store, $mapAcc, $duration, $zkQuorum, $hTableName")
-//          } catch {
-//            case e: Throwable =>
-//              println(s"[Failed]: store $e")
-//
-//              msgs.foreach { msg =>
-//                GraphSubscriberHelper.report(msg, Some(e.getMessage()), topic = kafkaTopic)
-//              }
-//          }
-//        }
-//      })
-//
-//      logInfo(s"counter: $mapAcc")
-//      println(s"Stats: ${mapAcc}")
-//
-//    } catch {
-//      case e: Throwable =>
-//        println(s"job failed with exception: $e")
-//        throw e
-//    }
-//
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
----------------------------------------------------------------------
diff --git a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
index f347ba9..9ebff03 100644
--- a/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
+++ b/loader/src/main/scala/org/apache/s2graph/loader/subscriber/TransferToHFile.scala
@@ -84,11 +84,11 @@ object TransferToHFile extends SparkApp {
     } yield output
   }
   def buildPutRequests(snapshotEdge: SnapshotEdge): List[PutRequest] = {
-    val kvs = GraphSubscriberHelper.g.storage.snapshotEdgeSerializer(snapshotEdge).toKeyValues.toList
+    val kvs = GraphSubscriberHelper.g.getStorage(snapshotEdge.label).snapshotEdgeSerializer(snapshotEdge).toKeyValues.toList
     kvs.map { kv => new PutRequest(kv.table, kv.row, kv.cf, kv.qualifier, kv.value, kv.timestamp) }
   }
   def buildPutRequests(indexEdge: IndexEdge): List[PutRequest] = {
-    val kvs = GraphSubscriberHelper.g.storage.indexEdgeSerializer(indexEdge).toKeyValues.toList
+    val kvs = GraphSubscriberHelper.g.getStorage(indexEdge.label).indexEdgeSerializer(indexEdge).toKeyValues.toList
     kvs.map { kv => new PutRequest(kv.table, kv.row, kv.cf, kv.qualifier, kv.value, kv.timestamp) }
   }
   def buildDegreePutRequests(vertexId: String, labelName: String, direction: String, degreeVal: Long): List[PutRequest] = {
@@ -100,12 +100,11 @@ object TransferToHFile extends SparkApp {
     val vertex = Vertex(SourceVertexId(label.srcColumn.id.get, innerVal))
 
     val ts = System.currentTimeMillis()
-    val propsWithTs = Map(LabelMeta.timeStampSeq -> InnerValLikeWithTs.withLong(ts, ts, label.schemaVersion))
-    val labelWithDir = LabelWithDirection(label.id.get, dir)
-    val edge = Edge(vertex, vertex, labelWithDir, propsWithTs=propsWithTs)
+    val propsWithTs = Map(LabelMeta.timestamp -> InnerValLikeWithTs.withLong(ts, ts, label.schemaVersion))
+    val edge = Edge(vertex, vertex, label, dir, propsWithTs=propsWithTs)
 
     edge.edgesWithIndex.flatMap { indexEdge =>
-      GraphSubscriberHelper.g.storage.indexEdgeSerializer(indexEdge).toKeyValues.map { kv =>
+      GraphSubscriberHelper.g.getStorage(indexEdge.label).indexEdgeSerializer(indexEdge).toKeyValues.map { kv =>
         new PutRequest(kv.table, kv.row, kv.cf, Array.empty[Byte], Bytes.toBytes(degreeVal), kv.timestamp)
       }
     }
@@ -158,7 +157,7 @@ object TransferToHFile extends SparkApp {
 
     val sc = new SparkContext(conf)
 
-    GraphSubscriberHelper.management.createTable(zkQuorum, tableName, List("e", "v"), maxHFilePerResionServer, None, compressionAlgorithm)
+    GraphSubscriberHelper.management.createStorageTable(zkQuorum, tableName, List("e", "v"), maxHFilePerResionServer, None, compressionAlgorithm)
 
     /* set up hbase init */
     val hbaseConf = HBaseConfiguration.create()
@@ -175,20 +174,7 @@ object TransferToHFile extends SparkApp {
       GraphSubscriberHelper.apply(phase, dbUrl, "none", "none")
       toKeyValues(iter.toSeq, labelMapping, autoEdgeCreate)
     }
-    //
-    //    val newRDD = if (!buildDegree) new HFileRDD(kvs)
-    //    else {
-    //      val degreeKVs = buildDegrees(rdd, labelMapping, autoEdgeCreate).reduceByKey { (agg, current) =>
-    //        agg + current
-    //      }.mapPartitions { iter =>
-    //        val phase = System.getProperty("phase")
-    //        GraphSubscriberHelper.apply(phase, dbUrl, "none", "none")
-    //        toKeyValues(iter.toSeq)
-    //      }
-    //      new HFileRDD(kvs ++ degreeKVs)
-    //    }
-    //
-    //    newRDD.toHFile(hbaseConf, zkQuorum, tableName, maxHFilePerResionServer, tmpPath)
+
     val merged = if (!buildDegree) kvs
     else {
       kvs ++ buildDegrees(rdd, labelMapping, autoEdgeCreate).reduceByKey { (agg, current) =>

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala b/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala
index 97730f3..657cfed 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/Edge.scala
@@ -24,38 +24,39 @@ import org.apache.s2graph.core.JSONParser._
 import org.apache.s2graph.core.mysqls.{Label, LabelIndex, LabelMeta}
 import org.apache.s2graph.core.types._
 import org.apache.s2graph.core.utils.logger
-import play.api.libs.json.{JsNumber, Json}
-import scala.collection.JavaConversions._
+import play.api.libs.json.{JsNumber, JsObject, Json}
 import scala.util.hashing.MurmurHash3
 
 
 case class SnapshotEdge(srcVertex: Vertex,
                         tgtVertex: Vertex,
-                        labelWithDir: LabelWithDirection,
+                        label: Label,
+                        direction: Int,
                         op: Byte,
                         version: Long,
-                        props: Map[Byte, InnerValLikeWithTs],
+                        props: Map[LabelMeta, InnerValLikeWithTs],
                         pendingEdgeOpt: Option[Edge],
                         statusCode: Byte = 0,
-                        lockTs: Option[Long]) {
+                        lockTs: Option[Long],
+                        tsInnerValOpt: Option[InnerValLike] = None) {
 
-  if (!props.containsKey(LabelMeta.timeStampSeq)) throw new Exception("Timestamp is required.")
+  lazy val labelWithDir = LabelWithDirection(label.id.get, direction)
+  if (!props.contains(LabelMeta.timestamp)) throw new Exception("Timestamp is required.")
 
-  val label = Label.findById(labelWithDir.labelId)
-  val schemaVer = label.schemaVersion
+//  val label = Label.findById(labelWithDir.labelId)
+  lazy val schemaVer = label.schemaVersion
   lazy val propsWithoutTs = props.mapValues(_.innerVal)
-  val ts = props(LabelMeta.timeStampSeq).innerVal.toString().toLong
+  lazy val ts = props(LabelMeta.timestamp).innerVal.toString().toLong
 
   def toEdge: Edge = {
-    val ts = props.get(LabelMeta.timeStampSeq).map(v => v.ts).getOrElse(version)
-    Edge(srcVertex, tgtVertex, labelWithDir, op,
+    val ts = props.get(LabelMeta.timestamp).map(v => v.ts).getOrElse(version)
+    Edge(srcVertex, tgtVertex, label, direction, op,
       version, props, pendingEdgeOpt = pendingEdgeOpt,
-      statusCode = statusCode, lockTs = lockTs)
+      statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
   }
 
   def propsWithName = (for {
-    (seq, v) <- props
-    meta <- label.metaPropsMap.get(seq)
+    (meta, v) <- props
     jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
   } yield meta.name -> jsValue) ++ Map("version" -> JsNumber(version))
 
@@ -67,17 +68,23 @@ case class SnapshotEdge(srcVertex: Vertex,
 
 case class IndexEdge(srcVertex: Vertex,
                      tgtVertex: Vertex,
-                     labelWithDir: LabelWithDirection,
+                     label: Label,
+                     direction: Int,
                      op: Byte,
                      version: Long,
                      labelIndexSeq: Byte,
-                     props: Map[Byte, InnerValLikeWithTs]) {
-  //  if (!props.containsKey(LabelMeta.timeStampSeq)) throw new Exception("Timestamp is required.")
-  //  assert(props.containsKey(LabelMeta.timeStampSeq))
+                     props: Map[LabelMeta, InnerValLikeWithTs],
+                     tsInnerValOpt: Option[InnerValLike] = None)  {
+//  if (!props.contains(LabelMeta.timeStampSeq)) throw new Exception("Timestamp is required.")
+  //  assert(props.contains(LabelMeta.timeStampSeq))
+  lazy val labelWithDir = LabelWithDirection(label.id.get, direction)
+
+  lazy val isInEdge = labelWithDir.dir == GraphUtil.directions("in")
+  lazy val isOutEdge = !isInEdge
+
+  lazy val ts = props(LabelMeta.timestamp).innerVal.toString.toLong
+  lazy val degreeEdge = props.contains(LabelMeta.degree)
 
-  lazy val ts = props(LabelMeta.timeStampSeq).innerVal.toString.toLong
-  lazy val degreeEdge = props.contains(LabelMeta.degreeSeq)
-  lazy val label = Label.findById(labelWithDir.labelId)
   lazy val schemaVer = label.schemaVersion
   lazy val labelIndex = LabelIndex.findByLabelIdAndSeq(labelWithDir.labelId, labelIndexSeq).get
   lazy val defaultIndexMetas = labelIndex.sortKeyTypes.map { meta =>
@@ -85,33 +92,33 @@ case class IndexEdge(srcVertex: Vertex,
     meta.seq -> innerVal
   }.toMap
 
-  lazy val labelIndexMetaSeqs = labelIndex.metaSeqs
+  lazy val labelIndexMetaSeqs = labelIndex.sortKeyTypes
 
   /** TODO: make sure call of this class fill props as this assumes */
-  lazy val orders = for (k <- labelIndexMetaSeqs) yield {
-    props.get(k) match {
+  lazy val orders = for (meta <- labelIndexMetaSeqs) yield {
+    props.get(meta) match {
       case None =>
 
-        /*
-         * TODO: agly hack
-         * now we double store target vertex.innerId/srcVertex.innerId for easy development. later fix this to only store id once
-         */
-        val v = k match {
-          case LabelMeta.timeStampSeq => InnerVal.withLong(version, schemaVer)
-          case LabelMeta.toSeq => tgtVertex.innerId
-          case LabelMeta.fromSeq => //srcVertex.innerId
-            // for now, it does not make sense to build index on srcVertex.innerId since all edges have same data.
-            throw new RuntimeException("_from on indexProps is not supported")
-          case _ => defaultIndexMetas(k)
+        /**
+          * TODO: agly hack
+          * now we double store target vertex.innerId/srcVertex.innerId for easy development. later fix this to only store id once
+          */
+        val v = meta match {
+          case LabelMeta.timestamp=> InnerVal.withLong(version, schemaVer)
+          case LabelMeta.to => toEdge.tgtVertex.innerId
+          case LabelMeta.from => toEdge.srcVertex.innerId
+          // for now, it does not make sense to build index on srcVertex.innerId since all edges have same data.
+          //            throw new RuntimeException("_from on indexProps is not supported")
+          case _ => toInnerVal(meta.defaultValue, meta.dataType, schemaVer)
         }
 
-        k -> v
-      case Some(v) => k -> v.innerVal
+        meta -> v
+      case Some(v) => meta -> v.innerVal
     }
   }
 
   lazy val ordersKeyMap = orders.map { case (byte, _) => byte }.toSet
-  lazy val metas = for ((k, v) <- props if !ordersKeyMap.contains(k)) yield k -> v.innerVal
+  lazy val metas = for ((meta, v) <- props if !ordersKeyMap.contains(meta)) yield meta -> v.innerVal
 
 //  lazy val propsWithTs = props.map { case (k, v) => k -> InnerValLikeWithTs(v, version) }
 
@@ -121,13 +128,12 @@ case class IndexEdge(srcVertex: Vertex,
   lazy val hasAllPropsForIndex = orders.length == labelIndexMetaSeqs.length
 
   def propsWithName = for {
-    (seq, v) <- props
-    meta <- label.metaPropsMap.get(seq) if seq >= 0
+    (meta, v) <- props if meta.seq >= 0
     jsValue <- innerValToJsValue(v.innerVal, meta.dataType)
   } yield meta.name -> jsValue
 
 
-  def toEdge: Edge = Edge(srcVertex, tgtVertex, labelWithDir, op, version, props)
+  def toEdge: Edge = Edge(srcVertex, tgtVertex, label, direction, op, version, props, tsInnerValOpt = tsInnerValOpt)
 
   // only for debug
   def toLogString() = {
@@ -137,25 +143,37 @@ case class IndexEdge(srcVertex: Vertex,
 
 case class Edge(srcVertex: Vertex,
                 tgtVertex: Vertex,
-                labelWithDir: LabelWithDirection,
+                label: Label,
+                dir: Int,
                 op: Byte = GraphUtil.defaultOpByte,
                 version: Long = System.currentTimeMillis(),
-                propsWithTs: Map[Byte, InnerValLikeWithTs],
+                propsWithTs: Map[LabelMeta, InnerValLikeWithTs],
                 parentEdges: Seq[EdgeWithScore] = Nil,
                 originalEdgeOpt: Option[Edge] = None,
                 pendingEdgeOpt: Option[Edge] = None,
                 statusCode: Byte = 0,
-                lockTs: Option[Long] = None) extends GraphElement {
+                lockTs: Option[Long] = None,
+                tsInnerValOpt: Option[InnerValLike] = None) extends GraphElement {
 
-  if (!props.containsKey(LabelMeta.timeStampSeq)) throw new Exception("Timestamp is required.")
-  //  assert(propsWithTs.containsKey(LabelMeta.timeStampSeq))
-  val schemaVer = label.schemaVersion
-  val ts = propsWithTs(LabelMeta.timeStampSeq).innerVal.toString.toLong
+  lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
+//  if (!props.contains(LabelMeta.timestamp)) throw new Exception("Timestamp is required.")
+  //  assert(propsWithTs.contains(LabelMeta.timeStampSeq))
+  lazy val schemaVer = label.schemaVersion
+  lazy val ts = propsWithTs(LabelMeta.timestamp).innerVal.value match {
+    case b: BigDecimal => b.longValue()
+    case l: Long => l
+    case i: Int => i.toLong
+    case _ => throw new RuntimeException("ts should be in [BigDecimal/Long/Int].")
+  }
+  //FIXME
+  lazy val tsInnerVal = tsInnerValOpt.get.value
+//    propsWithTs(LabelMeta.timestamp).innerVal.value
 
+//  lazy val label = Label.findById(labelWithDir.labelId)
   lazy val srcId = srcVertex.innerIdVal
   lazy val tgtId = tgtVertex.innerIdVal
   lazy val labelName = label.label
-  lazy val direction = GraphUtil.fromDirection(labelWithDir.dir)
+  lazy val direction = GraphUtil.fromDirection(dir)
   lazy val properties = toProps()
 
   def props = propsWithTs.mapValues(_.innerVal)
@@ -165,7 +183,7 @@ case class Edge(srcVertex: Vertex,
     for {
       (labelMeta, defaultVal) <- label.metaPropsDefaultMapInner
     } yield {
-      labelMeta.name -> propsWithTs.getOrElse(labelMeta.seq, defaultVal).innerVal.value
+      labelMeta.name -> propsWithTs.getOrElse(labelMeta, defaultVal).innerVal.value
     }
   }
 
@@ -174,8 +192,9 @@ case class Edge(srcVertex: Vertex,
       val skipReverse = label.extraOptions.get("skipReverse").map(_.as[Boolean]).getOrElse(false)
       if (skipReverse) List(this) else List(this, duplicateEdge)
     } else {
-      val outDir = labelWithDir.copy(dir = GraphUtil.directions("out"))
-      val base = copy(labelWithDir = outDir)
+//      val outDir = labelWithDir.copy(dir = GraphUtil.directions("out"))
+//      val base = copy(labelWithDir = outDir)
+      val base = copy(dir = GraphUtil.directions("out"))
       List(base, base.reverseSrcTgtEdge)
     }
   }
@@ -202,12 +221,11 @@ case class Edge(srcVertex: Vertex,
 
   def duplicateEdge = reverseSrcTgtEdge.reverseDirEdge
 
-  def reverseDirEdge = copy(labelWithDir = labelWithDir.dirToggled)
+//  def reverseDirEdge = copy(labelWithDir = labelWithDir.dirToggled)
+  def reverseDirEdge = copy(dir = GraphUtil.toggleDir(dir))
 
   def reverseSrcTgtEdge = copy(srcVertex = tgtVertex, tgtVertex = srcVertex)
 
-  def label = Label.findById(labelWithDir.labelId)
-
   def labelOrders = LabelIndex.findByLabelIdAll(labelWithDir.labelId)
 
   override def serviceName = label.serviceName
@@ -218,32 +236,32 @@ case class Edge(srcVertex: Vertex,
 
   override def isAsync = label.isAsync
 
-  def isDegree = propsWithTs.contains(LabelMeta.degreeSeq)
+  def isDegree = propsWithTs.contains(LabelMeta.degree)
 
 //  def propsPlusTs = propsWithTs.get(LabelMeta.timeStampSeq) match {
 //    case Some(_) => props
 //    case None => props ++ Map(LabelMeta.timeStampSeq -> InnerVal.withLong(ts, schemaVer))
 //  }
 
-  def propsPlusTsValid = propsWithTs.filter(kv => kv._1 >= 0)
+  def propsPlusTsValid = propsWithTs.filter(kv => LabelMeta.isValidSeq(kv._1.seq))
 
   def edgesWithIndex = for (labelOrder <- labelOrders) yield {
-    IndexEdge(srcVertex, tgtVertex, labelWithDir, op, version, labelOrder.seq, propsWithTs)
+    IndexEdge(srcVertex, tgtVertex, label, dir, op, version, labelOrder.seq, propsWithTs, tsInnerValOpt = tsInnerValOpt)
   }
 
   def edgesWithIndexValid = for (labelOrder <- labelOrders) yield {
-    IndexEdge(srcVertex, tgtVertex, labelWithDir, op, version, labelOrder.seq, propsPlusTsValid)
+    IndexEdge(srcVertex, tgtVertex, label, dir, op, version, labelOrder.seq, propsPlusTsValid, tsInnerValOpt = tsInnerValOpt)
   }
 
   /** force direction as out on invertedEdge */
   def toSnapshotEdge: SnapshotEdge = {
     val (smaller, larger) = (srcForVertex, tgtForVertex)
 
-    val newLabelWithDir = LabelWithDirection(labelWithDir.labelId, GraphUtil.directions("out"))
+//    val newLabelWithDir = LabelWithDirection(labelWithDir.labelId, GraphUtil.directions("out"))
 
-    val ret = SnapshotEdge(smaller, larger, newLabelWithDir, op, version,
-      Map(LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(ts, schemaVer), ts)) ++ propsWithTs,
-      pendingEdgeOpt = pendingEdgeOpt, statusCode = statusCode, lockTs = lockTs)
+    val ret = SnapshotEdge(smaller, larger, label, GraphUtil.directions("out"), op, version,
+      Map(LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(ts, schemaVer), ts)) ++ propsWithTs,
+      pendingEdgeOpt = pendingEdgeOpt, statusCode = statusCode, lockTs = lockTs, tsInnerValOpt = tsInnerValOpt)
     ret
   }
 
@@ -259,16 +277,20 @@ case class Edge(srcVertex: Vertex,
     case _ => false
   }
 
-  def propsWithName = for {
-    (seq, v) <- props
-    meta <- label.metaPropsMap.get(seq) if seq > 0
-    jsValue <- innerValToJsValue(v, meta.dataType)
-  } yield meta.name -> jsValue
+  def defaultPropsWithName = Json.obj("from" -> srcVertex.innerId.toString(), "to" -> tgtVertex.innerId.toString(),
+    "label" -> label.label, "service" -> label.serviceName)
+
+  def propsWithName =
+    for {
+      (meta, v) <- props if meta.seq > 0
+      jsValue <- innerValToJsValue(v, meta.dataType)
+    } yield meta.name -> jsValue
+
 
   def updateTgtVertex(id: InnerValLike) = {
     val newId = TargetVertexId(tgtVertex.id.colId, id)
     val newTgtVertex = Vertex(newId, tgtVertex.ts, tgtVertex.props)
-    Edge(srcVertex, newTgtVertex, labelWithDir, op, version, propsWithTs)
+    Edge(srcVertex, newTgtVertex, label, dir, op, version, propsWithTs, tsInnerValOpt = tsInnerValOpt)
   }
 
   def rank(r: RankParam): Double =
@@ -277,20 +299,15 @@ case class Edge(srcVertex: Vertex,
       var sum: Double = 0
 
       for ((seq, w) <- r.keySeqAndWeights) {
-        seq match {
-          case LabelMeta.countSeq => sum += 1
-          case _ => {
-            propsWithTs.get(seq) match {
-              case None => // do nothing
-              case Some(innerValWithTs) => {
-                val cost = try innerValWithTs.innerVal.toString.toDouble catch {
-                  case e: Exception =>
-                    logger.error("toInnerval failed in rank", e)
-                    1.0
-                }
-                sum += w * cost
-              }
+        propsWithTs.get(seq) match {
+          case None => // do nothing
+          case Some(innerValWithTs) => {
+            val cost = try innerValWithTs.innerVal.toString.toDouble catch {
+              case e: Exception =>
+                logger.error("toInnerval failed in rank", e)
+                1.0
             }
+            sum += w * cost
           }
         }
       }
@@ -298,35 +315,8 @@ case class Edge(srcVertex: Vertex,
     }
 
   def toLogString: String = {
-    val ret =
-      if (propsWithName.nonEmpty)
-        List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, Json.toJson(propsWithName)).map(_.toString)
-      else
-        List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label).map(_.toString)
-
-    ret.mkString("\t")
-  }
-
-  def selectValues(selectColumns: Seq[String],
-                   useToString: Boolean = true,
-                   score: Double = 0.0): Seq[Option[Any]] = {
-    //TODO: Option should be matched in JsonParser anyTo*
-    for {
-      selectColumn <- selectColumns
-    } yield {
-      val valueOpt = selectColumn match {
-        case LabelMeta.from.name | "from" => Option(srcId)
-        case LabelMeta.to.name | "to" => Option(tgtId)
-        case "label" => Option(labelName)
-        case "direction" => Option(direction)
-        case "score" => Option(score)
-        case LabelMeta.timestamp.name | "timestamp" => Option(ts)
-        case _ =>
-          properties.get(selectColumn)
-      }
-      if (useToString) valueOpt.map(_.toString)
-      else valueOpt
-    }
+    val allPropsWithName = defaultPropsWithName ++ Json.toJson(propsWithName).asOpt[JsObject].getOrElse(Json.obj())
+    List(ts, GraphUtil.fromOp(op), "e", srcVertex.innerId, tgtVertex.innerId, label.label, allPropsWithName).mkString("\t")
   }
 }
 
@@ -373,17 +363,17 @@ object Edge {
     val propsWithTs = label.propsToInnerValsWithTs(propsPlusTs, ts)
     val op = GraphUtil.toOp(operation).getOrElse(throw new RuntimeException(s"$operation is not supported."))
 
-    new Edge(srcVertex, tgtVertex, labelWithDir, op = op, version = ts, propsWithTs = propsWithTs)
+    new Edge(srcVertex, tgtVertex, label, dir, op = op, version = ts, propsWithTs = propsWithTs)
   }
 
   /** now version information is required also **/
-  type State = Map[Byte, InnerValLikeWithTs]
+  type State = Map[LabelMeta, InnerValLikeWithTs]
   type PropsPairWithTs = (State, State, Long, String)
   type MergeState = PropsPairWithTs => (State, Boolean)
   type UpdateFunc = (Option[Edge], Edge, MergeState)
 
-  def allPropsDeleted(props: Map[Byte, InnerValLikeWithTs]): Boolean =
-    if (!props.containsKey(LabelMeta.lastDeletedAt)) false
+  def allPropsDeleted(props: Map[LabelMeta, InnerValLikeWithTs]): Boolean =
+    if (!props.contains(LabelMeta.lastDeletedAt)) false
     else {
       val lastDeletedAt = props.get(LabelMeta.lastDeletedAt).get.ts
       val propsWithoutLastDeletedAt = props - LabelMeta.lastDeletedAt
@@ -398,14 +388,14 @@ object Edge {
     val edgesToDelete = requestEdge.relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
     val edgeInverted = Option(requestEdge.toSnapshotEdge)
 
-    (requestEdge, EdgeMutate(edgesToDelete, edgesToInsert = Nil, edgeInverted))
+    (requestEdge, EdgeMutate(edgesToDelete, edgesToInsert = Nil, newSnapshotEdge = edgeInverted))
   }
 
   def buildOperation(invertedEdge: Option[Edge], requestEdges: Seq[Edge]): (Edge, EdgeMutate) = {
     //            logger.debug(s"oldEdge: ${invertedEdge.map(_.toStringRaw)}")
     //            logger.debug(s"requestEdge: ${requestEdge.toStringRaw}")
     val oldPropsWithTs =
-      if (invertedEdge.isEmpty) Map.empty[Byte, InnerValLikeWithTs] else invertedEdge.get.propsWithTs
+      if (invertedEdge.isEmpty) Map.empty[LabelMeta, InnerValLikeWithTs] else invertedEdge.get.propsWithTs
 
     val funcs = requestEdges.map { edge =>
       if (edge.op == GraphUtil.operations("insert")) {
@@ -438,17 +428,18 @@ object Edge {
       for {
         (requestEdge, func) <- requestWithFuncs
       } {
-        val (_newPropsWithTs, _) = func((prevPropsWithTs, requestEdge.propsWithTs, requestEdge.ts, requestEdge.schemaVer))
+        val (_newPropsWithTs, _) = func(prevPropsWithTs, requestEdge.propsWithTs, requestEdge.ts, requestEdge.schemaVer)
         prevPropsWithTs = _newPropsWithTs
         //        logger.debug(s"${requestEdge.toLogString}\n$oldPropsWithTs\n$prevPropsWithTs\n")
       }
       val requestTs = requestEdge.ts
-      /* version should be monotoniously increasing so our RPC mutation should be applied safely */
+      /** version should be monotoniously increasing so our RPC mutation should be applied safely */
       val newVersion = invertedEdge.map(e => e.version + incrementVersion).getOrElse(requestTs)
       val maxTs = prevPropsWithTs.map(_._2.ts).max
       val newTs = if (maxTs > requestTs) maxTs else requestTs
       val propsWithTs = prevPropsWithTs ++
-        Map(LabelMeta.timeStampSeq -> InnerValLikeWithTs(InnerVal.withLong(newTs, requestEdge.label.schemaVersion), newTs))
+        Map(LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(newTs, requestEdge.label.schemaVersion), newTs))
+
       val edgeMutate = buildMutation(invertedEdge, requestEdge, newVersion, oldPropsWithTs, propsWithTs)
 
       //      logger.debug(s"${edgeMutate.toLogString}\n${propsWithTs}")
@@ -460,14 +451,14 @@ object Edge {
   def buildMutation(snapshotEdgeOpt: Option[Edge],
                     requestEdge: Edge,
                     newVersion: Long,
-                    oldPropsWithTs: Map[Byte, InnerValLikeWithTs],
-                    newPropsWithTs: Map[Byte, InnerValLikeWithTs]): EdgeMutate = {
+                    oldPropsWithTs: Map[LabelMeta, InnerValLikeWithTs],
+                    newPropsWithTs: Map[LabelMeta, InnerValLikeWithTs]): EdgeMutate = {
+
     if (oldPropsWithTs == newPropsWithTs) {
       // all requests should be dropped. so empty mutation.
-      //      logger.error(s"Case 1")
       EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = None)
     } else {
-      val withOutDeletedAt = newPropsWithTs.filter(kv => kv._1 != LabelMeta.lastDeletedAt)
+      val withOutDeletedAt = newPropsWithTs.filter(kv => kv._1 != LabelMeta.lastDeletedAtSeq)
       val newOp = snapshotEdgeOpt match {
         case None => requestEdge.op
         case Some(old) =>
@@ -479,26 +470,29 @@ object Edge {
       val newSnapshotEdgeOpt =
         Option(requestEdge.copy(op = newOp, propsWithTs = newPropsWithTs, version = newVersion).toSnapshotEdge)
       // delete request must always update snapshot.
-      if (withOutDeletedAt == oldPropsWithTs && newPropsWithTs.containsKey(LabelMeta.lastDeletedAt)) {
+      if (withOutDeletedAt == oldPropsWithTs && newPropsWithTs.contains(LabelMeta.lastDeletedAt)) {
         // no mutation on indexEdges. only snapshotEdge should be updated to record lastDeletedAt.
-        //        logger.error(s"Case 2")
         EdgeMutate(edgesToDelete = Nil, edgesToInsert = Nil, newSnapshotEdge = newSnapshotEdgeOpt)
       } else {
-        //        logger.error(s"Case 3")
         val edgesToDelete = snapshotEdgeOpt match {
           case Some(snapshotEdge) if snapshotEdge.op != GraphUtil.operations("delete") =>
-            snapshotEdge.copy(op = GraphUtil.defaultOpByte).
-              relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
+            snapshotEdge.copy(op = GraphUtil.defaultOpByte)
+              .relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
           case _ => Nil
         }
 
         val edgesToInsert =
           if (newPropsWithTs.isEmpty || allPropsDeleted(newPropsWithTs)) Nil
           else
-            requestEdge.copy(version = newVersion, propsWithTs = newPropsWithTs, op = GraphUtil.defaultOpByte).
-              relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
-
-        EdgeMutate(edgesToDelete = edgesToDelete, edgesToInsert = edgesToInsert, newSnapshotEdge = newSnapshotEdgeOpt)
+            requestEdge.copy(
+              version = newVersion,
+              propsWithTs = newPropsWithTs,
+              op = GraphUtil.defaultOpByte
+            ).relatedEdges.flatMap { relEdge => relEdge.edgesWithIndexValid }
+
+        EdgeMutate(edgesToDelete = edgesToDelete,
+          edgesToInsert = edgesToInsert,
+          newSnapshotEdge = newSnapshotEdgeOpt)
       }
     }
   }
@@ -520,7 +514,7 @@ object Edge {
 
         case None =>
           assert(oldValWithTs.ts >= lastDeletedAt)
-          if (oldValWithTs.ts >= requestTs || k < 0) Some(k -> oldValWithTs)
+          if (oldValWithTs.ts >= requestTs || k.seq < 0) Some(k -> oldValWithTs)
           else {
             shouldReplace = true
             None
@@ -575,7 +569,7 @@ object Edge {
     val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
       propsWithTs.get(k) match {
         case Some(newValWithTs) =>
-          if (k == LabelMeta.timeStampSeq) {
+          if (k == LabelMeta.timestamp) {
             val v = if (oldValWithTs.ts >= newValWithTs.ts) oldValWithTs
             else {
               shouldReplace = true
@@ -625,7 +619,7 @@ object Edge {
       }
     }
     val existInOld = for ((k, oldValWithTs) <- oldPropsWithTs) yield {
-      if (k == LabelMeta.timeStampSeq) {
+      if (k == LabelMeta.timestamp) {
         if (oldValWithTs.ts >= requestTs) Some(k -> oldValWithTs)
         else {
           shouldReplace = true


[5/7] incubator-s2graph git commit: [S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 ef40688..b481880 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
@@ -21,48 +21,46 @@ package org.apache.s2graph.core
 
 import com.google.common.hash.Hashing
 import org.apache.hadoop.hbase.util.Bytes
+import org.apache.s2graph.core.DuplicatePolicy.DuplicatePolicy
 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.types.{HBaseSerializable, InnerVal, InnerValLike, LabelWithDirection}
+import org.apache.s2graph.core.rest.TemplateHelper
+import org.apache.s2graph.core.storage.StorageSerializable._
+import org.apache.s2graph.core.types.{InnerVal, InnerValLike, InnerValLikeWithTs, LabelWithDirection}
 import org.hbase.async.ColumnRangeFilter
-import play.api.libs.json.{JsNull, JsNumber, JsValue, Json}
+import play.api.libs.json.{JsString, JsNull, JsValue, Json}
 
-import scala.util.hashing.MurmurHash3
 import scala.util.{Success, Try}
 
 object Query {
   val initialScore = 1.0
   lazy val empty = Query()
-
+  def apply(query: Query): Query = {
+    Query(query.vertices, query.steps, query.queryOption, query.jsonQuery)
+  }
   def toQuery(srcVertices: Seq[Vertex], queryParam: QueryParam) = Query(srcVertices, Vector(Step(List(queryParam))))
 
-  object DuplicatePolicy extends Enumeration {
-    type DuplicatePolicy = Value
-    val First, Sum, CountSum, Raw = Value
-
-    def apply(policy: String): Value = {
-      policy match {
-        case "sum" => Query.DuplicatePolicy.Sum
-        case "countSum" => Query.DuplicatePolicy.CountSum
-        case "raw" => Query.DuplicatePolicy.Raw
-        case _ => DuplicatePolicy.First
-      }
-    }
-  }
 }
 
+case class MinShouldMatchParam(prop: String, count: Int, terms: Set[Any])
+
 object GroupBy {
   val Empty = GroupBy()
 }
 case class GroupBy(keys: Seq[String] = Nil,
-                   limit: Int = Int.MaxValue)
+                   limit: Int = Int.MaxValue,
+                   minShouldMatch: Option[MinShouldMatchParam]= None)
 
 case class MultiQuery(queries: Seq[Query],
                       weights: Seq[Double],
                       queryOption: QueryOption,
                       jsonQuery: JsValue = JsNull)
 
+object QueryOption {
+  val DefaultAscendingVals: Seq[Boolean] = Seq(false, false)
+}
+
 case class QueryOption(removeCycle: Boolean = false,
                        selectColumns: Seq[String] = Seq.empty,
                        groupBy: GroupBy = GroupBy.Empty,
@@ -75,9 +73,27 @@ case class QueryOption(removeCycle: Boolean = false,
                        returnAgg: Boolean = true,
                        scoreThreshold: Double = Double.MinValue,
                        returnDegree: Boolean = true,
-                       impIdOpt: Option[String] = None) {
+                       impIdOpt: Option[String] = None,
+                       shouldPropagateScore: Boolean = true,
+                       ignorePrevStepCache: Boolean = false) {
   val orderByKeys = orderByColumns.map(_._1)
   val ascendingVals = orderByColumns.map(_._2)
+  val selectColumnsMap = selectColumns.map { c => c -> true } .toMap
+  val scoreFieldIdx = orderByKeys.zipWithIndex.find(t => t._1 == "score").map(_._2).getOrElse(-1)
+  val (edgeSelectColumns, propsSelectColumns) = selectColumns.partition(c => LabelMeta.defaultRequiredMetaNames.contains(c))
+  /** */
+  val edgeSelectColumnsFiltered = edgeSelectColumns
+//  val edgeSelectColumnsFiltered = edgeSelectColumns.filterNot(c => groupBy.keys.contains(c))
+  lazy val cacheKeyBytes: Array[Byte] = {
+    val selectBytes = Bytes.toBytes(selectColumns.toString)
+    val groupBytes = Bytes.toBytes(groupBy.keys.toString)
+    val orderByBytes = Bytes.toBytes(orderByColumns.toString)
+    val filterOutBytes = filterOutQuery.map(_.fullCacheBytes).getOrElse(Array.empty[Byte])
+    val returnTreeBytes = Bytes.toBytes(returnTree)
+
+    Seq(selectBytes, groupBytes, orderByBytes, filterOutBytes, returnTreeBytes).foldLeft(Array.empty[Byte])(Bytes.add)
+  }
+
 }
 
 case class Query(vertices: Seq[Vertex] = Seq.empty[Vertex],
@@ -85,57 +101,13 @@ case class Query(vertices: Seq[Vertex] = Seq.empty[Vertex],
                  queryOption: QueryOption = QueryOption(),
                  jsonQuery: JsValue = JsNull) {
 
-  val removeCycle = queryOption.removeCycle
-  val selectColumns = queryOption.selectColumns
-  val groupBy = queryOption.groupBy
-  val orderByColumns = queryOption.orderByColumns
-  val filterOutQuery = queryOption.filterOutQuery
-  val filterOutFields = queryOption.filterOutFields
-  val withScore = queryOption.withScore
-  val returnTree = queryOption.returnTree
-  val limitOpt = queryOption.limitOpt
-  val returnAgg = queryOption.returnAgg
-  val returnDegree = queryOption.returnDegree
-
-  def cacheKeyBytes: Array[Byte] = {
-    val selectBytes = Bytes.toBytes(queryOption.selectColumns.toString)
-    val groupBytes = Bytes.toBytes(queryOption.groupBy.keys.toString)
-    val orderByBytes = Bytes.toBytes(queryOption.orderByColumns.toString)
-    val filterOutBytes = queryOption.filterOutQuery.map(_.cacheKeyBytes).getOrElse(Array.empty[Byte])
-    val returnTreeBytes = Bytes.toBytes(queryOption.returnTree)
-
-    Seq(selectBytes, groupBytes, orderByBytes, filterOutBytes, returnTreeBytes).foldLeft(Array.empty[Byte])(Bytes.add)
-  }
-
-  lazy val selectColumnsSet = queryOption.selectColumns.map { c =>
-    if (c == "_from") "from"
-    else if (c == "_to") "to"
-    else c
-  }.toSet
-
-  /** return logical query id without considering parameter values */
-  def templateId(): JsValue = {
-    Json.toJson(for {
-      step <- steps
-      queryParam <- step.queryParams.sortBy(_.labelWithDir.labelId)
-    } yield {
-        Json.obj("label" -> queryParam.label.label, "direction" -> GraphUtil.fromDirection(queryParam.labelWithDir.dir))
-      })
-  }
-
-  def impressionId(): JsNumber = {
-    val hash = MurmurHash3.stringHash(templateId().toString())
-    JsNumber(hash)
-  }
-
-  def cursorStrings(): Seq[Seq[String]] = {
-    //Don`t know how to replace all cursor keys in json
-    steps.map { step =>
-      step.queryParams.map { queryParam =>
-        queryParam.cursorOpt.getOrElse("")
-      }
-    }
+  lazy val fullCacheBytes = {
+    val srcBytes = vertices.map(_.innerId.bytes).foldLeft(Array.empty[Byte])(Bytes.add)
+    val stepBytes = steps.map(_.cacheKeyBytes).foldLeft(Array.empty[Byte])(Bytes.add)
+    val queryOptionBytes = queryOption.cacheKeyBytes
+    Bytes.add(srcBytes, stepBytes, queryOptionBytes)
   }
+  lazy val fullCacheKey: Long = Hashing.murmur3_128().hashBytes(fullCacheBytes).asLong()
 }
 
 object EdgeTransformer {
@@ -148,7 +120,7 @@ object EdgeTransformer {
  * TODO: step wise outputFields should be used with nextStepLimit, nextStepThreshold.
  * @param jsValue
  */
-case class EdgeTransformer(queryParam: QueryParam, jsValue: JsValue) {
+case class EdgeTransformer(jsValue: JsValue) {
   val Delimiter = "\\$"
   val targets = jsValue.asOpt[List[Vector[String]]].toList
   val fieldsLs = for {
@@ -159,7 +131,8 @@ case class EdgeTransformer(queryParam: QueryParam, jsValue: JsValue) {
 
   def toHashKeyBytes: Array[Byte] = if (isDefault) Array.empty[Byte] else Bytes.toBytes(jsValue.toString)
 
-  def replace(fmt: String,
+  def replace(queryParam: QueryParam,
+              fmt: String,
               values: Seq[InnerValLike],
               nextStepOpt: Option[Step]): Seq[InnerValLike] = {
 
@@ -189,19 +162,19 @@ case class EdgeTransformer(queryParam: QueryParam, jsValue: JsValue) {
     }
   }
 
-  def toInnerValOpt(edge: Edge, fieldName: String): Option[InnerValLike] = {
+  def toInnerValOpt(queryParam: QueryParam, edge: Edge, fieldName: String): Option[InnerValLike] = {
     fieldName match {
       case LabelMeta.to.name => Option(edge.tgtVertex.innerId)
       case LabelMeta.from.name => Option(edge.srcVertex.innerId)
       case _ =>
         for {
           labelMeta <- queryParam.label.metaPropsInvMap.get(fieldName)
-          value <- edge.propsWithTs.get(labelMeta.seq)
+          value <- edge.propsWithTs.get(labelMeta)
         } yield value.innerVal
     }
   }
 
-  def transform(edge: Edge, nextStepOpt: Option[Step]): Seq[Edge] = {
+  def transform(queryParam: QueryParam, edge: Edge, nextStepOpt: Option[Step]): Seq[Edge] = {
     if (isDefault) Seq(edge)
     else {
       val edges = for {
@@ -209,10 +182,10 @@ case class EdgeTransformer(queryParam: QueryParam, jsValue: JsValue) {
         innerVal <- {
           if (fields.size == 1) {
             val fieldName = fields.head
-            toInnerValOpt(edge, fieldName).toSeq
+            toInnerValOpt(queryParam, edge, fieldName).toSeq
           } else {
             val fmt +: fieldNames = fields
-            replace(fmt, fieldNames.flatMap(fieldName => toInnerValOpt(edge, fieldName)), nextStepOpt)
+            replace(queryParam, fmt, fieldNames.flatMap(fieldName => toInnerValOpt(queryParam, edge, fieldName)), nextStepOpt)
           }
         }
       } yield edge.updateTgtVertex(innerVal).copy(originalEdgeOpt = Option(edge))
@@ -227,17 +200,18 @@ object Step {
   val Delimiter = "|"
 }
 
-case class Step(queryParams: List[QueryParam],
+case class Step(queryParams: Seq[QueryParam],
                 labelWeights: Map[Int, Double] = Map.empty,
-                //                scoreThreshold: Double = 0.0,
                 nextStepScoreThreshold: Double = 0.0,
                 nextStepLimit: Int = -1,
-                cacheTTL: Long = -1) {
+                cacheTTL: Long = -1,
+                groupBy: GroupBy = GroupBy.Empty) {
 
-  lazy val excludes = queryParams.filter(_.exclude)
-  lazy val includes = queryParams.filterNot(_.exclude)
-  lazy val excludeIds = excludes.map(x => x.labelWithDir.labelId -> true).toMap
+//  lazy val excludes = queryParams.filter(_.exclude)
+//  lazy val includes = queryParams.filterNot(_.exclude)
+//  lazy val excludeIds = excludes.map(x => x.labelWithDir.labelId -> true).toMap
 
+  lazy val cacheKeyBytes = queryParams.map(_.toCacheKeyRaw(Array.empty[Byte])).foldLeft(Array.empty[Byte])(Bytes.add)
   def toCacheKey(lss: Seq[Long]): Long = Hashing.murmur3_128().hashBytes(toCacheKeyRaw(lss)).asLong()
 //    MurmurHash3.bytesHash(toCacheKeyRaw(lss))
 
@@ -265,359 +239,202 @@ case class VertexParam(vertices: Seq[Vertex]) {
 
 }
 
-//object RankParam {
-//  def apply(labelId: Int, keyAndWeights: Seq[(Byte, Double)]) = {
-//    new RankParam(labelId, keyAndWeights)
-//  }
-//}
+object RankParam {
+  val Default = RankParam()
+}
 
-case class RankParam(labelId: Int, var keySeqAndWeights: Seq[(Byte, Double)] = Seq.empty[(Byte, Double)]) {
+case class RankParam(keySeqAndWeights: Seq[(LabelMeta, Double)] = Seq((LabelMeta.count, 1.0))) {
   // empty => Count
   lazy val rankKeysWeightsMap = keySeqAndWeights.toMap
 
-  def defaultKey() = {
-    this.keySeqAndWeights = List((LabelMeta.countSeq, 1.0))
-    this
-  }
-
   def toHashKeyBytes(): Array[Byte] = {
     var bytes = Array.empty[Byte]
-    keySeqAndWeights.map { case (key, weight) =>
-      bytes = Bytes.add(bytes, Array.fill(1)(key), Bytes.toBytes(weight))
+    keySeqAndWeights.map { case (labelMeta, weight) =>
+      bytes = Bytes.add(bytes, Array.fill(1)(labelMeta.seq), Bytes.toBytes(weight))
     }
     bytes
   }
 }
-case class S2Request(labelName: String,
-                     direction: String = "out",
-                     ts: Long = System.currentTimeMillis(),
-                     options: Map[String, Any] = Map.empty) {
-  val label = Label.findByName(labelName).getOrElse(throw new LabelNotExistException(labelName))
-  val dir = GraphUtil.toDir(direction).getOrElse(throw new RuntimeException(s"$direction is not supported."))
-  val labelWithDir = LabelWithDirection(label.id.get, dir)
-  //TODO: need to merge options into queryParam.
-  val queryParam = QueryParam(labelWithDir, ts)
-
-}
 object QueryParam {
-  lazy val Empty = QueryParam(LabelWithDirection(0, 0))
+  lazy val Empty = QueryParam(labelName = "")
   lazy val DefaultThreshold = Double.MinValue
   val Delimiter = ","
   val maxMetaByte = (-1).toByte
   val fillArray = Array.fill(100)(maxMetaByte)
-}
 
-case class QueryParam(labelWithDir: LabelWithDirection, timestamp: Long = System.currentTimeMillis()) {
-
-  import HBaseSerializable._
-  import Query.DuplicatePolicy
-  import Query.DuplicatePolicy._
-
-  lazy val label = Label.findById(labelWithDir.labelId)
-  val DefaultKey = LabelIndex.DefaultSeq
-  val fullKey = DefaultKey
-
-  var labelOrderSeq = fullKey
-
-  var sample = -1
-  var limit = 10
-  var offset = 0
-  var rank = new RankParam(labelWithDir.labelId, List(LabelMeta.countSeq -> 1))
-
-  var duration: Option[(Long, Long)] = None
-  var isInverted: Boolean = false
-
-  var columnRangeFilter: ColumnRangeFilter = null
-
-  var hasFilters: Map[Byte, InnerValLike] = Map.empty[Byte, InnerValLike]
-  var where: Try[Where] = Success(WhereParser.success)
-  var whereRawOpt: Option[String] = None
-  var duplicatePolicy = DuplicatePolicy.First
-  var rpcTimeoutInMillis = 1000
-  var maxAttempt = 2
-  var includeDegree = false
-  var tgtVertexInnerIdOpt: Option[InnerValLike] = None
-  var cacheTTLInMillis: Long = -1L
-  var threshold = QueryParam.DefaultThreshold
-  var timeDecay: Option[TimeDecay] = None
-  var transformer: EdgeTransformer = EdgeTransformer(this, EdgeTransformer.DefaultJson)
-  var scorePropagateOp: String = "multiply"
-  var scorePropagateShrinkage: Long = 500
-  var exclude = false
-  var include = false
-  var shouldNormalize= false
-  var cursorOpt: Option[String] = None
-
-  var columnRangeFilterMinBytes = Array.empty[Byte]
-  var columnRangeFilterMaxBytes = Array.empty[Byte]
-
-  lazy val srcColumnWithDir = label.srcColumnWithDir(labelWithDir.dir)
-  lazy val tgtColumnWithDir = label.tgtColumnWithDir(labelWithDir.dir)
-
-  def toBytes(idxSeq: Byte, offset: Int, limit: Int, isInverted: Boolean): Array[Byte] = {
-    val front = Array[Byte](idxSeq, if (isInverted) 1.toByte else 0.toByte)
-    Bytes.add(front, Bytes.toBytes((offset.toLong << 32 | limit)))
+  def apply(labelWithDirection: LabelWithDirection): QueryParam = {
+    val label = Label.findById(labelWithDirection.labelId)
+    val direction = GraphUtil.fromDirection(labelWithDirection.dir)
+    QueryParam(labelName = label.label, direction = direction)
   }
-
-  /**
-   * consider only I/O specific parameters.
-   * properties that is used on Graph.filterEdges should not be considered.
-   * @param bytes
-   * @return
-   */
-  def toCacheKey(bytes: Array[Byte]): Long = {
-    val hashBytes = toCacheKeyRaw(bytes)
-    Hashing.murmur3_128().hashBytes(hashBytes).asLong()
-//    MurmurHash3.bytesHash(hashBytes)
-  }
-
-  def toCacheKeyRaw(bytes: Array[Byte]): Array[Byte] = {
-    val transformBytes = transformer.toHashKeyBytes
+}
+case class QueryParam(labelName: String,
+                        direction: String = "out",
+                        offset: Int = 0,
+                        limit: Int = 100,
+                        sample: Int = -1,
+                        maxAttempt: Int = 2,
+                        rpcTimeout: Int = 1000,
+                        cacheTTLInMillis: Long = -1L,
+                        indexName: String = LabelIndex.DefaultName,
+                        where: Try[Where] = Success(WhereParser.success),
+                        timestamp: Long = System.currentTimeMillis(),
+                        threshold: Double = Double.MinValue,
+                        rank: RankParam = RankParam.Default,
+                        intervalOpt: Option[((Seq[(String, JsValue)]), Seq[(String, JsValue)])] = None,
+                        durationOpt: Option[(Long, Long)] = None,
+                        exclude: Boolean = false,
+                        include: Boolean = false,
+                        has: Map[String, Any] = Map.empty,
+                        duplicatePolicy: DuplicatePolicy = DuplicatePolicy.First,
+                        includeDegree: Boolean = false,
+                        scorePropagateShrinkage: Long = 500L,
+                        scorePropagateOp: String = "multiply",
+                        shouldNormalize: Boolean = false,
+                        whereRawOpt: Option[String] = None,
+                        cursorOpt: Option[String] = None,
+                        tgtVertexIdOpt: Option[Any] = None,
+                        edgeTransformer: EdgeTransformer = EdgeTransformer(EdgeTransformer.DefaultJson),
+                        timeDecay: Option[TimeDecay] = None) {
+  import JSONParser._
+
+  //TODO: implement this.
+  lazy val whereHasParent = true
+
+  lazy val label = Label.findByName(labelName).getOrElse(throw LabelNotExistException(labelName))
+  lazy val dir = GraphUtil.toDir(direction).getOrElse(throw new RuntimeException(s"not supported direction: $direction"))
+
+  lazy val labelWithDir = LabelWithDirection(label.id.get, dir)
+  lazy val labelOrderSeq =
+    if (indexName == LabelIndex.DefaultName) LabelIndex.DefaultSeq
+    else label.indexNameMap.getOrElse(indexName, throw new RuntimeException(s"$indexName indexName is not found.")).seq
+
+  lazy val tgtVertexInnerIdOpt = tgtVertexIdOpt.map { id =>
+    val tmp = label.tgtColumnWithDir(dir)
+    toInnerVal(id, tmp.columnType, tmp.schemaVersion)
+  }
+
+  def buildInterval(edgeOpt: Option[Edge]) = intervalOpt match {
+    case None => Array.empty[Byte] -> Array.empty[Byte]
+    case Some(interval) =>
+      val (froms, tos) = interval
+
+      val len = label.indicesMap(labelOrderSeq).sortKeyTypes.size.toByte
+      val (maxBytes, minBytes) = paddingInterval(len, froms, tos, edgeOpt)
+
+      maxBytes -> minBytes
+  }
+
+  lazy val isSnapshotEdge = tgtVertexInnerIdOpt.isDefined
+
+  /** since degree info is located on first always */
+  lazy val (innerOffset, innerLimit) = if (intervalOpt.isEmpty) {
+    if (offset == 0) (offset, if (limit == Int.MaxValue) limit else limit + 1)
+    else (offset + 1, limit)
+  } else (offset, limit)
+
+  lazy val optionalCacheKey: Array[Byte] = {
+    val transformBytes = edgeTransformer.toHashKeyBytes
     //TODO: change this to binrary format.
     val whereBytes = Bytes.toBytes(whereRawOpt.getOrElse(""))
-    val durationBytes = duration.map { case (min, max) =>
+    val durationBytes = durationOpt.map { case (min, max) =>
       val minTs = min / cacheTTLInMillis
       val maxTs = max / cacheTTLInMillis
       Bytes.add(Bytes.toBytes(minTs), Bytes.toBytes(maxTs))
     } getOrElse Array.empty[Byte]
-//    Bytes.toBytes(duration.toString)
+
     val conditionBytes = Bytes.add(transformBytes, whereBytes, durationBytes)
-    Bytes.add(Bytes.add(bytes, labelWithDir.bytes, toBytes(labelOrderSeq, offset, limit, isInverted)), rank.toHashKeyBytes(),
-      Bytes.add(columnRangeFilterMinBytes, columnRangeFilterMaxBytes, conditionBytes))
-  }
 
-  def isInverted(isInverted: Boolean): QueryParam = {
-    this.isInverted = isInverted
-    this
+    // Interval cache bytes is moved to fetch method
+    Bytes.add(Bytes.add(toBytes(offset, limit), rank.toHashKeyBytes()), conditionBytes)
   }
 
-  def labelOrderSeq(labelOrderSeq: Byte): QueryParam = {
-    this.labelOrderSeq = labelOrderSeq
-    this
+  def toBytes(offset: Int, limit: Int): Array[Byte] = {
+    Bytes.add(Bytes.toBytes(offset), Bytes.toBytes(limit))
   }
 
-  def sample(n: Int): QueryParam = {
-    this.sample = n
-    this
+  def toCacheKey(bytes: Array[Byte]): Long = {
+    val hashBytes = toCacheKeyRaw(bytes)
+    Hashing.murmur3_128().hashBytes(hashBytes).asLong()
   }
 
-  def limit(offset: Int, limit: Int): QueryParam = {
-    /** since degree info is located on first always */
-    this.limit = limit
-    this.offset = offset
-
-    if (this.columnRangeFilter == null) {
-      if (offset == 0) this.limit = limit + 1
-      else this.offset = offset + 1
-    }
-    //    this.columnPaginationFilter = new ColumnPaginationFilter(this.limit, this.offset)
-    this
+  def toCacheKeyRaw(bytes: Array[Byte]): Array[Byte] = {
+    Bytes.add(bytes, optionalCacheKey)
   }
 
-  def interval(fromTo: Option[(Seq[(Byte, InnerValLike)], Seq[(Byte, InnerValLike)])]): QueryParam = {
-    fromTo match {
-      case Some((from, to)) => interval(from, to)
-      case _ => this
-    }
-  }
+  private def convertToInner(kvs: Seq[(String, JsValue)], edgeOpt: Option[Edge]): Seq[(LabelMeta, InnerValLike)] = {
+    kvs.map { case (propKey, propValJs) =>
+      propValJs match {
+        case JsString(in) if edgeOpt.isDefined && in.contains("_parent.") =>
+          val parentLen = in.split("_parent.").length - 1
+          val edge = (0 until parentLen).foldLeft(edgeOpt.get) { case (acc, _) => acc.parentEdges.head.edge }
 
-  def paddingInterval(len: Byte, from: Seq[(Byte, InnerValLike)], to: Seq[(Byte, InnerValLike)]) = {
-    val fromVal = Bytes.add(propsToBytes(from), QueryParam.fillArray)
-    val toVal = propsToBytes(to)
+          val timePivot = edge.ts
+          val replaced = TemplateHelper.replaceVariable(timePivot, in).trim
 
-    toVal(0) = len
-    fromVal(0) = len
+          val (_propKey, _padding) = replaced.span(ch => !ch.isDigit && ch != '-' && ch != '+' && ch != ' ')
+          val propKey = _propKey.split("_parent.").last
+          val padding = Try(_padding.trim.toLong).getOrElse(0L)
 
-    val minMax = (toVal, fromVal) // inverted
-    minMax
-  }
+          val labelMeta = edge.label.metaPropsInvMap.getOrElse(propKey, throw new RuntimeException(s"$propKey not found in ${edge} labelMetas."))
 
-  def interval(from: Seq[(Byte, InnerValLike)], to: Seq[(Byte, InnerValLike)]): QueryParam = {
-    val len = label.indicesMap(labelOrderSeq).sortKeyTypes.size.toByte
-    val (minBytes, maxBytes) = paddingInterval(len, from, to)
+          val propVal =
+            if (InnerVal.isNumericType(labelMeta.dataType)) {
+              InnerVal.withLong(edge.props(labelMeta).value.asInstanceOf[BigDecimal].longValue() + padding, label.schemaVersion)
+            } else {
+              edge.props(labelMeta)
+            }
 
-    this.columnRangeFilterMaxBytes = maxBytes
-    this.columnRangeFilterMinBytes = minBytes
-    this.columnRangeFilter = new ColumnRangeFilter(minBytes, true, maxBytes, true)
-    this
-  }
+          labelMeta -> propVal
+        case _ =>
+          val labelMeta = label.metaPropsInvMap.getOrElse(propKey, throw new RuntimeException(s"$propKey not found in labelMetas."))
+          val propVal = jsValueToInnerVal(propValJs, labelMeta.dataType, label.schemaVersion)
 
-  def duration(minMaxTs: Option[(Long, Long)]): QueryParam = {
-    minMaxTs match {
-      case Some((minTs, maxTs)) => duration(minTs, maxTs)
-      case _ => this
+          labelMeta -> propVal.get
+      }
     }
   }
 
-  def duration(minTs: Long, maxTs: Long): QueryParam = {
-    this.duration = Some((minTs, maxTs))
-    this
-  }
-
-  def rank(r: RankParam): QueryParam = {
-    this.rank = r
-    this
-  }
-
-  def exclude(filterOut: Boolean): QueryParam = {
-    this.exclude = filterOut
-    this
-  }
-
-  def include(filterIn: Boolean): QueryParam = {
-    this.include = filterIn
-    this
-  }
-
-  def has(hasFilters: Map[Byte, InnerValLike]): QueryParam = {
-    this.hasFilters = hasFilters
-    this
-  }
-
-  def where(whereTry: Try[Where]): QueryParam = {
-    this.where = whereTry
-    this
-  }
-
-  def duplicatePolicy(policy: Option[DuplicatePolicy]): QueryParam = {
-    this.duplicatePolicy = policy.getOrElse(DuplicatePolicy.First)
-    this
-  }
+  def paddingInterval(len: Byte, froms: Seq[(String, JsValue)], tos: Seq[(String, JsValue)], edgeOpt: Option[Edge] = None) = {
+    val fromInnerVal = convertToInner(froms, edgeOpt)
+    val toInnerVal = convertToInner(tos, edgeOpt)
 
-  def rpcTimeout(millis: Int): QueryParam = {
-    this.rpcTimeoutInMillis = millis
-    this
-  }
-
-  def maxAttempt(attempt: Int): QueryParam = {
-    this.maxAttempt = attempt
-    this
-  }
-
-  def includeDegree(includeDegree: Boolean): QueryParam = {
-    this.includeDegree = includeDegree
-    this
-  }
-
-  def scorePropagateShrinkage(scorePropagateShrinkage: Long): QueryParam = {
-    this.scorePropagateShrinkage = scorePropagateShrinkage
-    this
-  }
+    val fromVal = Bytes.add(propsToBytes(fromInnerVal), QueryParam.fillArray)
+    val toVal = propsToBytes(toInnerVal)
 
-  def tgtVertexInnerIdOpt(other: Option[InnerValLike]): QueryParam = {
-    this.tgtVertexInnerIdOpt = other
-    this
-  }
+    toVal(0) = len
+    fromVal(0) = len
 
-  def cacheTTLInMillis(other: Long): QueryParam = {
-    this.cacheTTLInMillis = other
-    this
+    val minMax = (toVal, fromVal) // inverted
+    minMax
   }
 
-  def timeDecay(other: Option[TimeDecay]): QueryParam = {
-    this.timeDecay = other
-    this
+  def toLabelMetas(names: Seq[String]): Set[LabelMeta] = {
+    val m = for {
+      name <- names
+      labelMeta <- label.metaPropsInvMap.get(name)
+    } yield labelMeta
+    m.toSet
   }
+}
 
-  def threshold(other: Double): QueryParam = {
-    this.threshold = other
-    this
-  }
+object DuplicatePolicy extends Enumeration {
+  type DuplicatePolicy = Value
+  val First, Sum, CountSum, Raw = Value
 
-  def transformer(other: Option[JsValue]): QueryParam = {
-    other match {
-      case Some(js) => this.transformer = EdgeTransformer(this, js)
-      case None =>
+  def apply(policy: String): Value = {
+    policy match {
+      case "sum" => DuplicatePolicy.Sum
+      case "countSum" => DuplicatePolicy.CountSum
+      case "raw" => DuplicatePolicy.Raw
+      case _ => DuplicatePolicy.First
     }
-    this
-  }
-
-  def scorePropagateOp(scorePropagateOp: String): QueryParam = {
-    this.scorePropagateOp = scorePropagateOp
-    this
-  }
-
-  def shouldNormalize(shouldNormalize: Boolean): QueryParam = {
-    this.shouldNormalize = shouldNormalize
-    this
   }
-
-  def whereRawOpt(sqlOpt: Option[String]): QueryParam = {
-    this.whereRawOpt = sqlOpt
-    this
-  }
-
-  def cursorOpt(cursorOpt: Option[String]): QueryParam = {
-    this.cursorOpt = cursorOpt
-    this
-  }
-
-  def isSnapshotEdge = tgtVertexInnerIdOpt.isDefined
-
-  override def toString = {
-    List(label.label, labelOrderSeq, offset, limit, rank,
-      duration, isInverted, exclude, include, hasFilters).mkString("\t")
-    //      duration, isInverted, exclude, include, hasFilters, outputFields).mkString("\t")
-  }
-
-  //
-  //  def buildGetRequest(srcVertex: Vertex) = {
-  //    val (srcColumn, tgtColumn) = label.srcTgtColumn(labelWithDir.dir)
-  //    val (srcInnerId, tgtInnerId) = tgtVertexInnerIdOpt match {
-  //      case Some(tgtVertexInnerId) => // _to is given.
-  //        /** we use toInvertedEdgeHashLike so dont need to swap src, tgt */
-  //        val src = InnerVal.convertVersion(srcVertex.innerId, srcColumn.columnType, label.schemaVersion)
-  //        val tgt = InnerVal.convertVersion(tgtVertexInnerId, tgtColumn.columnType, label.schemaVersion)
-  //        (src, tgt)
-  //      case None =>
-  //        val src = InnerVal.convertVersion(srcVertex.innerId, srcColumn.columnType, label.schemaVersion)
-  //        (src, src)
-  //    }
-  //
-  //    val (srcVId, tgtVId) = (SourceVertexId(srcColumn.id.get, srcInnerId), TargetVertexId(tgtColumn.id.get, tgtInnerId))
-  //    val (srcV, tgtV) = (Vertex(srcVId), Vertex(tgtVId))
-  //    val edge = Edge(srcV, tgtV, labelWithDir)
-  //
-  //    val get = if (tgtVertexInnerIdOpt.isDefined) {
-  //      val snapshotEdge = edge.toInvertedEdgeHashLike
-  //      val kv = snapshotEdge.kvs.head
-  //      new GetRequest(label.hbaseTableName.getBytes, kv.row, edgeCf, kv.qualifier)
-  //    } else {
-  //      val indexedEdgeOpt = edge.edgesWithIndex.find(e => e.labelIndexSeq == labelOrderSeq)
-  //      assert(indexedEdgeOpt.isDefined)
-  //      val indexedEdge = indexedEdgeOpt.get
-  //      val kv = indexedEdge.kvs.head
-  //      val table = label.hbaseTableName.getBytes
-  //        //kv.table //
-  //      val rowKey = kv.row // indexedEdge.rowKey.bytes
-  //      val cf = edgeCf
-  //      new GetRequest(table, rowKey, cf)
-  //    }
-  //
-  //    val (minTs, maxTs) = duration.getOrElse((0L, Long.MaxValue))
-  //
-  //    get.maxVersions(1)
-  //    get.setFailfast(true)
-  //    get.setMaxResultsPerColumnFamily(limit)
-  //    get.setRowOffsetPerColumnFamily(offset)
-  //    get.setMinTimestamp(minTs)
-  //    get.setMaxTimestamp(maxTs)
-  //    get.setTimeout(rpcTimeoutInMillis)
-  //    if (columnRangeFilter != null) get.setFilter(columnRangeFilter)
-  //    //    get.setMaxAttempt(maxAttempt.toByte)
-  //    //    get.setRpcTimeout(rpcTimeoutInMillis)
-  //
-  //    //    if (columnRangeFilter != null) get.filter(columnRangeFilter)
-  //    //    logger.debug(s"Get: $get, $offset, $limit")
-  //
-  //    get
-  //  }
 }
-
 case class TimeDecay(initial: Double = 1.0,
                      lambda: Double = 0.1,
                      timeUnit: Double = 60 * 60 * 24,
-                     labelMetaSeq: Byte = LabelMeta.timeStampSeq) {
+                     labelMeta: LabelMeta = LabelMeta.timestamp) {
   def decay(diff: Double): Double = {
     //FIXME
     val ret = initial * Math.pow(1.0 - lambda, diff / timeUnit)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala b/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
index 5b2622f..dd7e45d 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryResult.scala
@@ -19,71 +19,106 @@
 
 package org.apache.s2graph.core
 
-import org.apache.s2graph.core.mysqls.LabelMeta
-import org.apache.s2graph.core.types.{InnerVal, InnerValLikeWithTs}
+import org.apache.s2graph.core.mysqls.{Label, LabelMeta}
+import org.apache.s2graph.core.types.{InnerVal, InnerValLikeWithTs, VertexId}
 import org.apache.s2graph.core.utils.logger
 
-import scala.collection.mutable.ListBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.collection.{Seq, mutable}
 
 object QueryResult {
-  def fromVertices(query: Query): StepInnerResult = {
+  def fromVertices(graph: Graph,
+                   query: Query): StepResult = {
     if (query.steps.isEmpty || query.steps.head.queryParams.isEmpty) {
-      StepInnerResult.Empty
+      StepResult.Empty
     } else {
       val queryParam = query.steps.head.queryParams.head
       val label = queryParam.label
       val currentTs = System.currentTimeMillis()
-      val propsWithTs = Map(LabelMeta.timeStampSeq ->
+      val propsWithTs = Map(LabelMeta.timestamp ->
         InnerValLikeWithTs(InnerVal.withLong(currentTs, label.schemaVersion), currentTs))
-      val edgeWithScoreLs = for {
+      val edgeWithScores = for {
         vertex <- query.vertices
       } yield {
-          val edge = Edge(vertex, vertex, queryParam.labelWithDir, propsWithTs = propsWithTs)
-          val edgeWithScore = EdgeWithScore(edge, Graph.DefaultScore)
+          val edge = Edge(vertex, vertex, label, queryParam.labelWithDir.dir, propsWithTs = propsWithTs)
+          val edgeWithScore = EdgeWithScore(edge, Graph.DefaultScore, queryParam.label)
           edgeWithScore
         }
-      StepInnerResult(edgesWithScoreLs = edgeWithScoreLs, Nil, false)
+      StepResult(edgeWithScores = edgeWithScores, grouped = Nil, degreeEdges = Nil, false)
     }
   }
 }
-/** inner traverse */
-object StepInnerResult {
-  val Failure = StepInnerResult(Nil, Nil, true)
-  val Empty = StepInnerResult(Nil, Nil, false)
-}
-case class StepInnerResult(edgesWithScoreLs: Seq[EdgeWithScore],
-                           degreeEdges: Seq[EdgeWithScore],
-                           isFailure: Boolean = false) {
-  val isEmpty = edgesWithScoreLs.isEmpty
-}
 
 case class QueryRequest(query: Query,
                         stepIdx: Int,
                         vertex: Vertex,
-                        queryParam: QueryParam)
-
+                        queryParam: QueryParam,
+                        prevStepScore: Double = 1.0,
+                        labelWeight: Double = 1.0) {
+  val nextStepOpt =
+    if (stepIdx < query.steps.size - 1) Option(query.steps(stepIdx + 1))
+    else None
+}
 
-case class QueryResult(edgeWithScoreLs: Seq[EdgeWithScore] = Nil,
-                       tailCursor: Array[Byte] = Array.empty,
-                       timestamp: Long = System.currentTimeMillis(),
-                       isFailure: Boolean = false)
+trait WithScore[T] {
+  def score(me: T): Double
+  def withNewScore(me: T, newScore: Double): T
+}
 
-case class EdgeWithScore(edge: Edge, score: Double)
+object WithScore {
+  implicit val impEdgeWithScore = new WithScore[EdgeWithScore] {
+    override def score(me: EdgeWithScore): Double = me.score
 
+    override def withNewScore(me: EdgeWithScore, newScore: Double): EdgeWithScore = me.copy(score = newScore)
+  }
+}
 
+case class EdgeWithScore(edge: Edge,
+                         score: Double,
+                         label: Label,
+                         orderByValues: (Any, Any, Any, Any) = StepResult.EmptyOrderByValues,
+                         groupByValues: Seq[Option[Any]] = Nil,
+                         stepGroupByValues: Seq[Option[Any]] = Nil,
+                         filterOutValues: Seq[Option[Any]] = Nil,
+                         accumulatedScores: Map[String, Double] = Map.empty) {
+
+  def toValue(keyName: String): Option[Any] = keyName match {
+    case "from" | "_from" => Option(edge.srcId)
+    case "to" | "_to" => Option(edge.tgtId)
+    case "label" => Option(label.label)
+    case "direction" => Option(edge.dir)
+    case "score" => Option(score)
+    case _ =>
+      label.metaPropsInvMap.get(keyName).flatMap { labelMeta =>
+        edge.propsWithTs.get(labelMeta).orElse(label.metaPropsDefaultMapInner.get(labelMeta)).map(_.innerVal.value)
+      }
+  }
 
+  def toValues(keyNames: Seq[String]): Seq[Option[Any]] = for {
+    keyName <- keyNames
+  } yield toValue(keyName)
 
+}
 
 /** result */
+case class StepResult(edgeWithScores: Seq[EdgeWithScore],
+                      grouped: Seq[(StepResult.GroupByKey, (Double, StepResult.Values))],
+                      degreeEdges: Seq[EdgeWithScore],
+                      isFailure: Boolean = false,
+                      accumulatedCursors: Seq[Seq[Array[Byte]]] = Nil,
+                      cursors: Seq[Array[Byte]] = Nil,
+                      failCount: Int = 0) {
+  //  val isInnerEmpty = innerResults.isEmpty
+  val isEmpty = edgeWithScores.isEmpty
+}
 
 object StepResult {
 
-  type Values = Seq[S2EdgeWithScore]
+  type Values = Seq[EdgeWithScore]
   type GroupByKey = Seq[Option[Any]]
   val EmptyOrderByValues = (None, None, None, None)
   val Empty = StepResult(Nil, Nil, Nil)
-
+  val Failure = StepResult(Nil, Nil, Nil, true, failCount = 1)
 
   def mergeOrdered(left: StepResult.Values,
                    right: StepResult.Values,
@@ -99,177 +134,182 @@ object StepResult {
     }
   }
 
+  def filterOutStepGroupBy(edgesWithScores: Seq[EdgeWithScore],
+                                   groupBy: GroupBy): Seq[EdgeWithScore] =
+    if (groupBy == GroupBy.Empty) edgesWithScores
+    else {
+      groupBy.minShouldMatch match {
+        case None => edgesWithScores
+        case Some(minShouldMatch) =>
+          val MinShouldMatchParam(propKey, count, terms) = minShouldMatch
+
+          val grouped = edgesWithScores.groupBy { edgeWithScore =>
+            edgeWithScore.stepGroupByValues
+          }.filter { case (key, edges) =>
+            val filtered = edges.toStream.filter{ e =>
+              e.toValue(propKey) match {
+                case None => false
+                case Some(v) => terms.contains(v)
+              }
+            }.take(count)
+
+            filtered.lengthCompare(count) >= 0
+          }
+
+          grouped.values.flatten.toSeq
+      }
+    }
+
   def orderBy(queryOption: QueryOption, notOrdered: Values): Values = {
     import OrderingUtil._
-
-    if (queryOption.withScore && queryOption.orderByColumns.nonEmpty) {
-      notOrdered.sortBy(_.orderByValues)(TupleMultiOrdering[Any](queryOption.ascendingVals))
+    if (queryOption.withScore) {
+      val ascendingVals = if (queryOption.ascendingVals.isEmpty) QueryOption.DefaultAscendingVals else queryOption.ascendingVals
+      notOrdered.sortBy(_.orderByValues)(TupleMultiOrdering[Any](ascendingVals))
     } else {
       notOrdered
     }
   }
-  def toOrderByValues(s2Edge: Edge,
-                      score: Double,
-                      orderByKeys: Seq[String]): (Any, Any, Any, Any) = {
-    def toValue(propertyKey: String): Any = {
-      propertyKey match {
-        case "score" => score
-        case "timestamp" | "_timestamp" => s2Edge.ts
-        case _ => s2Edge.properties.get(propertyKey)
-      }
+
+  def updateScoreOnOrderByValues(scoreFieldIndex: Int,
+                                 orderByValues: (Any, Any, Any, Any),
+                                 newScore: Double): (Any, Any, Any, Any) = {
+    scoreFieldIndex match {
+      case 0 => (newScore, orderByValues._2, orderByValues._3, orderByValues._4)
+      case 1 => (orderByValues._1, newScore, orderByValues._3, orderByValues._4)
+      case 2 => (orderByValues._1, orderByValues._2, newScore, orderByValues._4)
+      case 3 => (orderByValues._1, orderByValues._2, orderByValues._3, newScore)
+      case _ => orderByValues
     }
-    if (orderByKeys.isEmpty) (None, None, None, None)
-    else {
-      orderByKeys.length match {
-        case 1 =>
-          (toValue(orderByKeys(0)), None, None, None)
-        case 2 =>
-          (toValue(orderByKeys(0)), toValue(orderByKeys(1)), None, None)
-        case 3 =>
-          (toValue(orderByKeys(0)), toValue(orderByKeys(1)), toValue(orderByKeys(2)), None)
-        case _ =>
-          (toValue(orderByKeys(0)), toValue(orderByKeys(1)), toValue(orderByKeys(2)), toValue(orderByKeys(3)))
-      }
+
+  }
+
+  def toTuple4(values: Seq[Option[Any]]): (Any, Any, Any, Any) = {
+    values.length match {
+      case 1 => (values(0).getOrElse(None), None, None, None)
+      case 2 => (values(0).getOrElse(None), values(1).getOrElse(None), None, None)
+      case 3 => (values(0).getOrElse(None), values(1).getOrElse(None), values(2).getOrElse(None), None)
+      case _ => (values(0).getOrElse(None), values(1).getOrElse(None), values(2).getOrElse(None), values(3).getOrElse(None))
     }
   }
+
   /**
    * merge multiple StepResult into one StepResult.
-   * @param queryOption
+   * @param globalQueryOption
    * @param multiStepResults
+   * @param weights
+   * @param filterOutStepResult
    * @return
    */
-  def merges(queryOption: QueryOption,
+  def merges(globalQueryOption: QueryOption,
              multiStepResults: Seq[StepResult],
-             weights: Seq[Double] = Nil): StepResult = {
+             weights: Seq[Double] = Nil,
+             filterOutStepResult: StepResult): StepResult = {
     val degrees = multiStepResults.flatMap(_.degreeEdges)
-    val ls = new mutable.ListBuffer[S2EdgeWithScore]()
-    val agg= new mutable.HashMap[GroupByKey, ListBuffer[S2EdgeWithScore]]()
+    val ls = new mutable.ListBuffer[EdgeWithScore]()
+    val agg= new mutable.HashMap[GroupByKey, ListBuffer[EdgeWithScore]]()
     val sums = new mutable.HashMap[GroupByKey, Double]()
 
+
+    val filterOutSet = filterOutStepResult.edgeWithScores.foldLeft(Set.empty[Seq[Option[Any]]]) { case (prev, t) =>
+      prev + t.filterOutValues
+    }
+
     for {
       (weight, eachStepResult) <- weights.zip(multiStepResults)
-      (ordered, grouped) = (eachStepResult.results, eachStepResult.grouped)
+      (ordered, grouped) = (eachStepResult.edgeWithScores, eachStepResult.grouped)
     } {
       ordered.foreach { t =>
-        val newScore = t.score * weight
-        ls += t.copy(score = newScore)
+        val filterOutKey = t.filterOutValues
+        if (!filterOutSet.contains(filterOutKey)) {
+          val newScore = t.score * weight
+          val newT = t.copy(score = newScore)
+
+          //          val newOrderByValues = updateScoreOnOrderByValues(globalQueryOption.scoreFieldIdx, t.orderByValues, newScore)
+          val newOrderByValues =
+            if (globalQueryOption.orderByKeys.isEmpty) (newScore, t.edge.tsInnerVal, None, None)
+            else toTuple4(newT.toValues(globalQueryOption.orderByKeys))
+
+          val newGroupByValues = newT.toValues(globalQueryOption.groupBy.keys)
+
+          ls += t.copy(score = newScore, orderByValues = newOrderByValues, groupByValues = newGroupByValues)
+        }
       }
 
       // process each query's stepResult's grouped
       for {
         (groupByKey, (scoreSum, values)) <- grouped
       } {
-        val buffer = agg.getOrElseUpdate(groupByKey, ListBuffer.empty[S2EdgeWithScore])
+        val buffer = agg.getOrElseUpdate(groupByKey, ListBuffer.empty[EdgeWithScore])
         var scoreSum = 0.0
+        var isEmpty = true
         values.foreach { t =>
-          val newScore = t.score * weight
-          buffer += t.copy(score = newScore)
-          scoreSum += newScore
+          val filterOutKey = t.filterOutValues
+          if (!filterOutSet.contains(filterOutKey)) {
+            isEmpty = false
+            val newScore = t.score * weight
+            val newT = t.copy(score = newScore)
+//            val newOrderByValues = updateScoreOnOrderByValues(globalQueryOption.scoreFieldIdx, t.orderByValues, newScore)
+
+            val newOrderByValues =
+              if (globalQueryOption.orderByKeys.isEmpty) (newScore, t.edge.tsInnerVal, None, None)
+              else toTuple4(newT.toValues(globalQueryOption.orderByKeys))
+
+            val newGroupByValues = newT.toValues(globalQueryOption.groupBy.keys)
+
+            buffer += t.copy(score = newScore, orderByValues = newOrderByValues, groupByValues = newGroupByValues)
+            scoreSum += newScore
+          }
         }
-        sums += (groupByKey -> scoreSum)
+        if (!isEmpty) sums += (groupByKey -> scoreSum)
       }
     }
 
     // process global groupBy
-    if (queryOption.groupBy.keys.nonEmpty) {
+    val (ordered, grouped) = if (globalQueryOption.groupBy.keys.nonEmpty) {
       for {
-        s2EdgeWithScore <- ls
-        groupByKey = s2EdgeWithScore.s2Edge.selectValues(queryOption.groupBy.keys)
+        edgeWithScore <- ls
+        groupByKey = edgeWithScore.groupByValues
       } {
-        val buffer = agg.getOrElseUpdate(groupByKey, ListBuffer.empty[S2EdgeWithScore])
-        buffer += s2EdgeWithScore
-        val newScore = sums.getOrElse(groupByKey, 0.0) + s2EdgeWithScore.score
+        val buffer = agg.getOrElseUpdate(groupByKey, ListBuffer.empty[EdgeWithScore])
+        buffer += edgeWithScore
+        val newScore = sums.getOrElse(groupByKey, 0.0) + edgeWithScore.score
         sums += (groupByKey -> newScore)
       }
+      val grouped = for {
+        (groupByKey, scoreSum) <- sums.toSeq.sortBy(_._2 * -1)
+        aggregated = agg(groupByKey) if aggregated.nonEmpty
+        sorted = orderBy(globalQueryOption, aggregated)
+      } yield groupByKey -> (scoreSum, sorted)
+      (Nil, grouped)
+    } else {
+      val ordered = orderBy(globalQueryOption, ls)
+      (ordered, Nil)
     }
 
-
-    val ordered = orderBy(queryOption, ls)
-    val grouped = for {
-      (groupByKey, scoreSum) <- sums.toSeq.sortBy(_._2 * -1)
-      aggregated = agg(groupByKey) if aggregated.nonEmpty
-    } yield groupByKey -> (scoreSum, aggregated)
-
-    StepResult(results = ordered, grouped = grouped, degrees)
+    StepResult(edgeWithScores = ordered, grouped = grouped, degrees, failCount = multiStepResults.map(_.failCount).sum)
   }
 
   //TODO: Optimize this.
   def filterOut(graph: Graph,
                 queryOption: QueryOption,
                 baseStepResult: StepResult,
-                filterOutStepInnerResult: StepInnerResult): StepResult = {
+                filterOutStepResult: StepResult): StepResult = {
 
-    val fields = if (queryOption.filterOutFields.isEmpty) Seq("to") else Seq("to")
-    //    else queryOption.filterOutFields
-    val filterOutSet = filterOutStepInnerResult.edgesWithScoreLs.map { t =>
-      t.edge.selectValues(fields)
-    }.toSet
+    val filterOutSet = filterOutStepResult.edgeWithScores.foldLeft(Set.empty[Seq[Option[Any]]]) { case (prev, t) =>
+      prev + t.filterOutValues
+    }
 
-    val filteredResults = baseStepResult.results.filter { t =>
-      val filterOutKey = t.s2Edge.selectValues(fields)
+    val filteredResults = baseStepResult.edgeWithScores.filter { t =>
+      val filterOutKey = t.filterOutValues
       !filterOutSet.contains(filterOutKey)
     }
 
     val grouped = for {
       (key, (scoreSum, values)) <- baseStepResult.grouped
-      (out, in) = values.partition(v => filterOutSet.contains(v.s2Edge.selectValues(fields)))
+      (out, in) = values.partition(v => filterOutSet.contains(v.filterOutValues))
       newScoreSum = scoreSum - out.foldLeft(0.0) { case (prev, current) => prev + current.score } if in.nonEmpty
     } yield key -> (newScoreSum, in)
 
-
-    StepResult(results = filteredResults, grouped = grouped, baseStepResult.degreeEdges)
-  }
-  def apply(graph: Graph,
-            queryOption: QueryOption,
-            stepInnerResult: StepInnerResult): StepResult = {
-        logger.debug(s"[BeforePostProcess]: ${stepInnerResult.edgesWithScoreLs.size}")
-
-    val results = for {
-      edgeWithScore <- stepInnerResult.edgesWithScoreLs
-    } yield {
-        val edge = edgeWithScore.edge
-        val orderByValues =
-          if (queryOption.orderByColumns.isEmpty) (edgeWithScore.score, None, None, None)
-          else toOrderByValues(edge, edgeWithScore.score, queryOption.orderByKeys)
-
-        S2EdgeWithScore(edge, edgeWithScore.score, orderByValues, edgeWithScore.edge.parentEdges)
-      }
-    /** ordered flatten result */
-    val ordered = orderBy(queryOption, results)
-
-    /** ordered grouped result */
-    val grouped =
-      if (queryOption.groupBy.keys.isEmpty) Nil
-      else {
-        val agg = new mutable.HashMap[GroupByKey, (Double, Values)]()
-        results.groupBy { s2EdgeWithScore =>
-          s2EdgeWithScore.s2Edge.selectValues(queryOption.groupBy.keys, useToString = true)
-        }.map { case (k, ls) =>
-          val (scoreSum, merged) = mergeOrdered(ls, Nil, queryOption)
-          /**
-           * watch out here. by calling toString on Any, we lose type information which will be used
-           * later for toJson.
-           */
-          if (merged.nonEmpty) {
-            val newKey = merged.head.s2Edge.selectValues(queryOption.groupBy.keys, useToString = false)
-            agg += (newKey -> (scoreSum, merged))
-          }
-        }
-        agg.toSeq.sortBy(_._2._1 * -1)
-      }
-
-    val degrees = stepInnerResult.degreeEdges.map(t => S2EdgeWithScore(t.edge, t.score))
-    StepResult(results = ordered, grouped = grouped, degreeEdges = degrees)
+    StepResult(edgeWithScores = filteredResults, grouped = grouped, baseStepResult.degreeEdges, cursors = baseStepResult.cursors, failCount = baseStepResult.failCount + filterOutStepResult.failCount)
   }
 }
-
-case class S2EdgeWithScore(s2Edge: Edge,
-                           score: Double,
-                           orderByValues: (Any, Any, Any, Any) = StepResult.EmptyOrderByValues,
-                           parentEdges: Seq[EdgeWithScore] = Nil)
-
-case class StepResult(results: StepResult.Values,
-                      grouped: Seq[(StepResult.GroupByKey, (Double, StepResult.Values))],
-                      degreeEdges: StepResult.Values) {
-  val isEmpty = results.isEmpty
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala b/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala
index 0377bd8..bbd71ec 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/Vertex.scala
@@ -23,49 +23,6 @@ import org.apache.s2graph.core.JSONParser._
 import org.apache.s2graph.core.mysqls.{ColumnMeta, Service, ServiceColumn}
 import org.apache.s2graph.core.types.{InnerVal, InnerValLike, SourceVertexId, VertexId}
 import play.api.libs.json.Json
-//
-//object S2Vertex {
-//  def apply(graph: Graph, vertex: Vertex): S2Vertex = {
-//    S2Vertex(graph,
-//      vertex.serviceName,
-//      vertex.serviceColumn.columnName,
-//      vertex.innerIdVal,
-//      vertex.serviceColumn.innerValsToProps(vertex.props),
-//      vertex.ts,
-//      GraphUtil.fromOp(vertex.op)
-//    )
-//  }
-//}
-//
-//case class S2Vertex(graph: Graph,
-//                    serviceName: String,
-//                    columnName: String,
-//                    id: Any,
-//                    props: Map[String, Any] = Map.empty,
-//                    ts: Long = System.currentTimeMillis(),
-//                    operation: String = "insert") extends GraphElement {
-//  lazy val vertex = {
-//    val service = Service.findByName(serviceName).getOrElse(throw new RuntimeException(s"$serviceName is not found."))
-//    val column = ServiceColumn.find(service.id.get, columnName).getOrElse(throw new RuntimeException(s"$columnName is not found."))
-//    val op = GraphUtil.toOp(operation).getOrElse(throw new RuntimeException(s"$operation is not supported."))
-//
-//    val srcVertexId = VertexId(column.id.get, toInnerVal(id.toString, column.columnType, column.schemaVersion))
-//    val propsInner = column.propsToInnerVals(props) ++
-//      Map(ColumnMeta.timeStampSeq.toInt -> InnerVal.withLong(ts, column.schemaVersion))
-//
-//    Vertex(srcVertexId, ts, propsInner, op)
-//  }
-//
-//  val uniqueId = (serviceName, columnName, id)
-//
-//  override def isAsync: Boolean = vertex.isAsync
-//
-//  override def toLogString(): String = vertex.toLogString()
-//
-//  override def queueKey: String = vertex.queueKey
-//
-//  override def queuePartitionKey: String = vertex.queuePartitionKey
-//}
 case class Vertex(id: VertexId,
                   ts: Long = System.currentTimeMillis(),
                   props: Map[Int, InnerValLike] = Map.empty[Int, InnerValLike],
@@ -109,22 +66,6 @@ case class Vertex(id: VertexId,
     meta <- ColumnMeta.findByIdAndSeq(id.colId, seq.toByte)
   } yield (meta.name -> v.toString)
 
-  //  /** only used by bulk loader */
-  //  def buildPuts(): List[Put] = {
-  //    //    logger.error(s"put: $this => $rowKey")
-  ////    val put = new Put(rowKey.bytes)
-  ////    for ((q, v) <- qualifiersWithValues) {
-  ////      put.addColumn(vertexCf, q, ts, v)
-  ////    }
-  ////    List(put)
-  //    val kv = kvs.head
-  //    val put = new Put(kv.row)
-  //    kvs.map { kv =>
-  //      put.addColumn(kv.cf, kv.qualifier, kv.timestamp, kv.value)
-  //    }
-  //    List(put)
-  //  }
-
   def toEdgeVertex() = Vertex(SourceVertexId(id.colId, innerId), ts, props, op)
 
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Bucket.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Bucket.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Bucket.scala
index bcd5f0a..43e5db8 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Bucket.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Bucket.scala
@@ -19,12 +19,10 @@
 
 package org.apache.s2graph.core.mysqls
 
-/**
- * Created by shon on 8/5/15.
- */
-
 import scalikejdbc._
 
+import scala.util.Try
+
 object Bucket extends Model[Bucket] {
 
   val rangeDelimiter = "~"
@@ -53,7 +51,7 @@ object Bucket extends Model[Bucket] {
 
   def toRange(str: String): Option[(Int, Int)] = {
     val range = str.split(rangeDelimiter)
-    if (range.length == 2) Option((range.head.toInt, range.last.toInt))
+    if (range.length == 2) Option(range.head.toInt, range.last.toInt)
     else None
   }
 
@@ -69,6 +67,24 @@ object Bucket extends Model[Bucket] {
       sql.single().apply()
     }
   }
+
+  def insert(experiment: Experiment, modular: String, httpVerb: String, apiPath: String,
+             requestBody: String, timeout: Int, impressionId: String,
+             isGraphQuery: Boolean, isEmpty: Boolean)
+            (implicit session: DBSession = AutoSession): Try[Bucket] = {
+    Try {
+      sql"""
+            INSERT INTO buckets(experiment_id, modular, http_verb, api_path, request_body, timeout, impression_id,
+             is_graph_query, is_empty)
+            VALUES (${experiment.id.get}, $modular, $httpVerb, $apiPath, $requestBody, $timeout, $impressionId,
+             $isGraphQuery, $isEmpty)
+        """
+        .updateAndReturnGeneratedKey().apply()
+    }.map { newId =>
+      Bucket(Some(newId.toInt), experiment.id.get, modular, httpVerb, apiPath, requestBody, timeout, impressionId,
+        isGraphQuery, isEmpty)
+    }
+  }
 }
 
 case class Bucket(id: Option[Int],

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Experiment.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Experiment.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Experiment.scala
index cd825f4..0b16449 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Experiment.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Experiment.scala
@@ -22,7 +22,7 @@ package org.apache.s2graph.core.mysqls
 import org.apache.s2graph.core.GraphUtil
 import scalikejdbc._
 
-import scala.util.Random
+import scala.util.{Try, Random}
 
 object Experiment extends Model[Experiment] {
   val ImpressionKey = "S2-Impression-Id"
@@ -60,6 +60,17 @@ object Experiment extends Model[Experiment] {
         .map { rs => Experiment(rs) }.single.apply
     )
   }
+
+  def insert(service: Service, name: String, description: String, experimentType: String = "t", totalModular: Int = 100)
+            (implicit session: DBSession = AutoSession): Try[Experiment] = {
+    Try {
+      sql"""INSERT INTO experiments(service_id, service_name, `name`, description, experiment_type, total_modular)
+         VALUES(${service.id.get}, ${service.serviceName}, $name, $description, $experimentType, $totalModular)"""
+        .updateAndReturnGeneratedKey().apply()
+    }.map { newId =>
+      Experiment(Some(newId.toInt), service.id.get, name, description, experimentType, totalModular)
+    }
+  }
 }
 
 case class Experiment(id: Option[Int],

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Label.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Label.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Label.scala
index f7318f6..fdef677 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Label.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Label.scala
@@ -21,13 +21,14 @@ package org.apache.s2graph.core.mysqls
 
 import java.util.Calendar
 
+import com.typesafe.config.Config
 import org.apache.s2graph.core.GraphExceptions.ModelNotFoundException
 import org.apache.s2graph.core.GraphUtil
 import org.apache.s2graph.core.JSONParser._
 import org.apache.s2graph.core.Management.JsonModel.{Index, Prop}
-import org.apache.s2graph.core.types.{InnerValLike, InnerValLikeWithTs}
+import org.apache.s2graph.core.types.{InnerVal, InnerValLike, InnerValLikeWithTs}
 import org.apache.s2graph.core.utils.logger
-import play.api.libs.json.{JsObject, JsValue, Json}
+import play.api.libs.json.{JsArray, JsObject, JsValue, Json}
 import scalikejdbc._
 
 object Label extends Model[Label] {
@@ -57,7 +58,8 @@ object Label extends Model[Label] {
       sql"""
         select *
         from labels
-        where label = ${labelName}""".map { rs => Label(rs) }.single.apply()
+        where label = ${labelName}
+        and deleted_at is null """.map { rs => Label(rs) }.single.apply()
 
     if (useCache) withCache(cacheKey)(labelOpt)
     else labelOpt
@@ -101,7 +103,8 @@ object Label extends Model[Label] {
       sql"""
         select 	*
         from 	labels
-        where 	id = ${id}"""
+        where 	id = ${id}
+        and deleted_at is null"""
         .map { rs => Label(rs) }.single.apply())
   }
 
@@ -111,7 +114,8 @@ object Label extends Model[Label] {
       sql"""
         select 	*
         from 	labels
-        where 	id = ${id}"""
+        where 	id = ${id}
+        and deleted_at is null"""
         .map { rs => Label(rs) }.single.apply()).get
   }
 
@@ -124,6 +128,7 @@ object Label extends Model[Label] {
           from	labels
           where	tgt_column_name = ${col.columnName}
           and service_id = ${col.serviceId}
+          and deleted_at is null
         """.map { rs => Label(rs) }.list().apply())
   }
 
@@ -136,20 +141,21 @@ object Label extends Model[Label] {
           from	labels
           where	src_column_name = ${col.columnName}
           and service_id = ${col.serviceId}
+          and deleted_at is null
         """.map { rs => Label(rs) }.list().apply())
   }
 
   def findBySrcServiceId(serviceId: Int)(implicit session: DBSession = AutoSession): List[Label] = {
     val cacheKey = "srcServiceId=" + serviceId
     withCaches(cacheKey)(
-      sql"""select * from labels where src_service_id = ${serviceId}""".map { rs => Label(rs) }.list().apply
+      sql"""select * from labels where src_service_id = ${serviceId} and deleted_at is null""".map { rs => Label(rs) }.list().apply
     )
   }
 
   def findByTgtServiceId(serviceId: Int)(implicit session: DBSession = AutoSession): List[Label] = {
     val cacheKey = "tgtServiceId=" + serviceId
     withCaches(cacheKey)(
-      sql"""select * from labels where tgt_service_id = ${serviceId}""".map { rs => Label(rs) }.list().apply
+      sql"""select * from labels where tgt_service_id = ${serviceId} and deleted_at is null""".map { rs => Label(rs) }.list().apply
     )
   }
 
@@ -183,14 +189,14 @@ object Label extends Model[Label] {
         val tgtServiceId = tgtService.id.get
         val serviceId = service.id.get
 
-        /* insert serviceColumn */
+        /** insert serviceColumn */
         val srcCol = ServiceColumn.findOrInsert(srcServiceId, srcColumnName, Some(srcColumnType), schemaVersion)
         val tgtCol = ServiceColumn.findOrInsert(tgtServiceId, tgtColumnName, Some(tgtColumnType), schemaVersion)
 
         if (srcCol.columnType != srcColumnType) throw new RuntimeException(s"source service column type not matched ${srcCol.columnType} != ${srcColumnType}")
         if (tgtCol.columnType != tgtColumnType) throw new RuntimeException(s"target service column type not matched ${tgtCol.columnType} != ${tgtColumnType}")
 
-        /* create label */
+        /** create label */
         Label.findByName(labelName, useCache = false).getOrElse {
 
           val createdId = insert(labelName, srcServiceId, srcColumnName, srcColumnType,
@@ -224,7 +230,7 @@ object Label extends Model[Label] {
   }
 
   def findAll()(implicit session: DBSession = AutoSession) = {
-    val ls = sql"""select * from labels""".map { rs => Label(rs) }.list().apply()
+    val ls = sql"""select * from labels where deleted_at is null""".map { rs => Label(rs) }.list().apply()
     putsToCache(ls.map { x =>
       val cacheKey = s"id=${x.id.get}"
       (cacheKey -> x)
@@ -233,6 +239,7 @@ object Label extends Model[Label] {
       val cacheKey = s"label=${x.label}"
       (cacheKey -> x)
     })
+    ls
   }
 
   def updateName(oldName: String, newName: String)(implicit session: DBSession = AutoSession) = {
@@ -264,6 +271,21 @@ object Label extends Model[Label] {
     }
     cnt
   }
+
+  def markDeleted(label: Label)(implicit session: DBSession = AutoSession) = {
+
+    logger.info(s"mark deleted label: $label")
+    val oldName = label.label
+    val now = Calendar.getInstance().getTime
+    val newName = s"deleted_${now.getTime}_"+ label.label
+    val cnt = sql"""update labels set label = ${newName}, deleted_at = ${now} where id = ${label.id.get}""".update.apply()
+    val cacheKeys = List(s"id=${label.id}", s"label=${oldName}")
+    cacheKeys.foreach { key =>
+      expireCache(key)
+      expireCaches(key)
+    }
+    cnt
+  }
 }
 
 case class Label(id: Option[Int], label: String,
@@ -274,10 +296,9 @@ case class Label(id: Option[Int], label: String,
                  schemaVersion: String, isAsync: Boolean = false,
                  compressionAlgorithm: String,
                  options: Option[String]) {
+  def metas(useCache: Boolean = true) = LabelMeta.findAllByLabelId(id.get, useCache = useCache)
 
-  def metas = LabelMeta.findAllByLabelId(id.get)
-
-  def metaSeqsToNames = metas.map(x => (x.seq, x.name)) toMap
+  def indices(useCache: Boolean = true) = LabelIndex.findByLabelIdAll(id.get, useCache = useCache)
 
   //  lazy val firstHBaseTableName = hbaseTableName.split(",").headOption.getOrElse(Config.HBASE_TABLE_NAME)
   lazy val srcService = Service.findById(srcServiceId)
@@ -356,83 +377,131 @@ case class Label(id: Option[Int], label: String,
 //    }
 //  }"""))
 
-  lazy val extraOptions: Map[String, JsValue] = options match {
-    case None => Map.empty
-    case Some(v) =>
-      try {
-        Json.parse(v).asOpt[JsObject].map { obj => obj.fields.toMap }.getOrElse(Map.empty)
-      } catch {
-        case e: Exception =>
-          logger.error(s"An error occurs while parsing the extra label option: ${label}", e)
-          Map.empty
-      }
+  lazy val tokens: Set[String] = extraOptions.get("tokens").fold(Set.empty[String]) {
+    case JsArray(tokens) => tokens.map(_.as[String]).toSet
+    case _ =>
+      logger.error("Invalid token JSON")
+      Set.empty[String]
+  }
+
+  lazy val extraOptions = Model.extraOptions(options)
+
+  lazy val durability = extraOptions.get("durability").map(_.as[Boolean]).getOrElse(true)
+
+  lazy val storageConfigOpt: Option[Config] = toStorageConfig
+
+  def toStorageConfig: Option[Config] = {
+    Model.toStorageConfig(extraOptions)
   }
 
+
   def srcColumnWithDir(dir: Int) = {
-    if (dir == GraphUtil.directions("out")) srcColumn else tgtColumn
+    // GraphUtil.directions("out"
+    if (dir == 0) srcColumn else tgtColumn
   }
 
   def tgtColumnWithDir(dir: Int) = {
-    if (dir == GraphUtil.directions("out")) tgtColumn else srcColumn
+    // GraphUtil.directions("out"
+    if (dir == 0) tgtColumn else srcColumn
   }
 
-  def srcTgtColumn(dir: Int) =
-    if (isDirected) {
-      (srcColumnWithDir(dir), tgtColumnWithDir(dir))
-    } else {
-      if (dir == GraphUtil.directions("in")) {
-        (tgtColumn, srcColumn)
-      } else {
-        (srcColumn, tgtColumn)
-      }
-    }
-
-  def init() = {
-    metas
-    metaSeqsToNames
-    service
-    srcColumn
-    tgtColumn
-    defaultIndex
-    indices
-    metaProps
-  }
+  lazy val tgtSrc = (tgtColumn, srcColumn)
+  lazy val srcTgt = (srcColumn, tgtColumn)
+
+  def srcTgtColumn(dir: Int) = if (dir == 1) tgtSrc else srcTgt
+
+  lazy val EmptyPropsWithTs = Map(LabelMeta.timestamp -> InnerValLikeWithTs(InnerVal.withLong(0, schemaVersion), 0))
+//  def init() = {
+//    metas()
+//    metaSeqsToNames()
+//    service
+//    srcColumn
+//    tgtColumn
+//    defaultIndex
+//    indices
+//    metaProps
+//  }
+
+  //  def srcColumnInnerVal(jsValue: JsValue) = {
+  //    jsValueToInnerVal(jsValue, srcColumnType, version)
+  //  }
+  //  def tgtColumnInnerVal(jsValue: JsValue) = {
+  //    jsValueToInnerVal(jsValue, tgtColumnType, version)
+  //  }
 
   override def toString(): String = {
     val orderByKeys = LabelMeta.findAllByLabelId(id.get)
     super.toString() + orderByKeys.toString()
   }
 
-  lazy val toJson = Json.obj("labelName" -> label,
-    "from" -> srcColumn.toJson, "to" -> tgtColumn.toJson,
-    "isDirected" -> isDirected,
-    "serviceName" -> serviceName,
-    "consistencyLevel" -> consistencyLevel,
-    "schemaVersion" -> schemaVersion,
-    "isAsync" -> isAsync,
-    "compressionAlgorithm" -> compressionAlgorithm,
-    "defaultIndex" -> defaultIndex.map(x => x.toJson),
-    "extraIndex" -> extraIndices.map(exIdx => exIdx.toJson),
-    "metaProps" -> metaProps.filter { labelMeta => LabelMeta.isValidSeqForAdmin(labelMeta.seq) }.map(_.toJson)
-  )
-
+  //  def findLabelIndexSeq(scoring: List[(Byte, Double)]): Byte = {
+  //    if (scoring.isEmpty) LabelIndex.defaultSeq
+  //    else {
+  //      LabelIndex.findByLabelIdAndSeqs(id.get, scoring.map(_._1).sorted).map(_.seq).getOrElse(LabelIndex.defaultSeq)
+  //
+  ////      LabelIndex.findByLabelIdAndSeqs(id.get, scoring.map(_._1).sorted).map(_.seq).getOrElse(LabelIndex.defaultSeq)
+  //    }
+  //  }
+  lazy val toJson = {
+    val allIdxs = LabelIndex.findByLabelIdAll(id.get, useCache = false)
+    val defaultIdxOpt = LabelIndex.findByLabelIdAndSeq(id.get, LabelIndex.DefaultSeq, useCache = false)
+    val extraIdxs = allIdxs.filter(idx => defaultIdxOpt.isDefined && idx.id.get != defaultIdxOpt.get.id.get)
+    val metaProps = LabelMeta.reservedMetas.map { m =>
+      if (m == LabelMeta.to) m.copy(dataType = tgtColumnType)
+      else if (m == LabelMeta.from) m.copy(dataType = srcColumnType)
+      else m
+    } ::: LabelMeta.findAllByLabelId(id.get, useCache = false)
+
+    val defaultIdx = defaultIdxOpt.map(x => x.toJson).getOrElse(Json.obj())
+    val optionsJs = try {
+      val obj = options.map(Json.parse).getOrElse(Json.obj()).as[JsObject]
+      if (!obj.value.contains("tokens")) obj
+      else obj ++ Json.obj("tokens" -> obj.value("tokens").as[Seq[String]].map("*" * _.length))
+
+    } catch { case e: Exception => Json.obj() }
+
+    Json.obj("labelName" -> label,
+      "from" -> srcColumn.toJson, "to" -> tgtColumn.toJson,
+      "isDirected" -> isDirected,
+      "serviceName" -> serviceName,
+      "consistencyLevel" -> consistencyLevel,
+      "schemaVersion" -> schemaVersion,
+      "isAsync" -> isAsync,
+      "compressionAlgorithm" -> compressionAlgorithm,
+      "defaultIndex" -> defaultIdx,
+      "extraIndex" -> extraIdxs.map(exIdx => exIdx.toJson),
+      "metaProps" -> metaProps.filter { labelMeta => LabelMeta.isValidSeqForAdmin(labelMeta.seq) }.map(_.toJson),
+      "options" -> optionsJs
+    )
+  }
 
   def propsToInnerValsWithTs(props: Map[String, Any],
-                             ts: Long = System.currentTimeMillis()): Map[Byte, InnerValLikeWithTs] = {
+                             ts: Long = System.currentTimeMillis()): Map[LabelMeta, InnerValLikeWithTs] = {
     for {
       (k, v) <- props
       labelMeta <- metaPropsInvMap.get(k)
       innerVal = toInnerVal(v.toString, labelMeta.dataType, schemaVersion)
-    } yield labelMeta.seq -> InnerValLikeWithTs(innerVal, ts)
+    } yield labelMeta -> InnerValLikeWithTs(innerVal, ts)
 
   }
 
-  def innerValsWithTsToProps(props: Map[Byte, InnerValLikeWithTs]): Map[String, Any] = {
-    for {
-      (k, v) <- props
-      labelMeta <- metaPropsMap.get(k)
-    } yield {
-      labelMeta.name -> v.innerVal.value
+  def innerValsWithTsToProps(props: Map[LabelMeta, InnerValLikeWithTs],
+                             selectColumns: Map[Byte, Boolean]): Map[String, Any] = {
+    if (selectColumns.isEmpty) {
+      for {
+        (meta, v) <- metaPropsDefaultMapInner ++ props
+      } yield {
+        meta.name -> innerValToAny(v.innerVal, meta.dataType)
+      }
+    } else {
+      for {
+        (k, _) <- selectColumns
+        if k != LabelMeta.toSeq && k != LabelMeta.fromSeq
+        labelMeta <- metaPropsMap.get(k)
+      } yield {
+        val v = props.get(labelMeta).orElse(metaPropsDefaultMapInner.get(labelMeta)).get
+        labelMeta.name -> innerValToAny(v.innerVal, labelMeta.dataType)
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelIndex.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelIndex.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelIndex.scala
index d7736bc..c548868 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelIndex.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelIndex.scala
@@ -23,12 +23,14 @@ package org.apache.s2graph.core.mysqls
  * Created by shon on 6/3/15.
  */
 
-import play.api.libs.json.Json
+import org.apache.s2graph.core.GraphUtil
+import org.apache.s2graph.core.utils.logger
+import play.api.libs.json.{JsObject, JsString, Json}
 import scalikejdbc._
 
 object LabelIndex extends Model[LabelIndex] {
   val DefaultName = "_PK"
-  val DefaultMetaSeqs = Seq(LabelMeta.timeStampSeq)
+  val DefaultMetaSeqs = Seq(LabelMeta.timestampSeq)
   val DefaultSeq = 1.toByte
   val MaxOrderSeq = 7
 
@@ -144,16 +146,11 @@ object LabelIndex extends Model[LabelIndex] {
   }
 }
 
-/**
- * formular
- * ex1): w1, w2, w3
- * ex2): 1.5 * w1^2 + 3.4 * (w1 * w2), w2, w1
- */
-
 case class LabelIndex(id: Option[Int], labelId: Int, name: String, seq: Byte, metaSeqs: Seq[Byte], formulars: String) {
   lazy val label = Label.findById(labelId)
   lazy val metas = label.metaPropsMap
   lazy val sortKeyTypes = metaSeqs.flatMap(metaSeq => label.metaPropsMap.get(metaSeq))
+  lazy val sortKeyTypesArray = sortKeyTypes.toArray
   lazy val propNames = sortKeyTypes.map { labelMeta => labelMeta.name }
   lazy val toJson = Json.obj(
     "name" -> name,

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
index ef5d90a..4a7e931 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/LabelMeta.scala
@@ -32,40 +32,61 @@ object LabelMeta extends Model[LabelMeta] {
 
   /** dummy sequences */
 
-  val fromSeq = -4.toByte
-  val toSeq = -5.toByte
-  val lastOpSeq = -3.toByte
-  val lastDeletedAt = -2.toByte
-  val timeStampSeq = 0.toByte
+  val fromSeq = (-4).toByte
+  val toSeq = (-5).toByte
+  val lastOpSeq = (-3).toByte
+  val lastDeletedAtSeq = (-2).toByte
+  val timestampSeq = (0).toByte
+  val labelSeq = (-6).toByte
+  val directionSeq = -7.toByte
+  val fromHashSeq = -8.toByte
+
   val countSeq = (Byte.MaxValue - 2).toByte
   val degreeSeq = (Byte.MaxValue - 1).toByte
   val maxValue = Byte.MaxValue
-  val emptyValue = Byte.MaxValue
+  val emptySeq = Byte.MaxValue
 
   /** reserved sequences */
   //  val deleted = LabelMeta(id = Some(lastDeletedAt), labelId = lastDeletedAt, name = "lastDeletedAt",
   //    seq = lastDeletedAt, defaultValue = "", dataType = "long")
+  val fromHash = LabelMeta(id = None, labelId = fromHashSeq, name = "_from_hash",
+    seq = fromHashSeq, defaultValue = fromHashSeq.toString, dataType = "long")
   val from = LabelMeta(id = Some(fromSeq), labelId = fromSeq, name = "_from",
-    seq = fromSeq, defaultValue = fromSeq.toString, dataType = "long")
+    seq = fromSeq, defaultValue = fromSeq.toString, dataType = "string")
   val to = LabelMeta(id = Some(toSeq), labelId = toSeq, name = "_to",
-    seq = toSeq, defaultValue = toSeq.toString, dataType = "long")
+    seq = toSeq, defaultValue = toSeq.toString, dataType = "string")
   val timestamp = LabelMeta(id = Some(-1), labelId = -1, name = "_timestamp",
-    seq = timeStampSeq, defaultValue = "0", dataType = "long")
+    seq = timestampSeq, defaultValue = "0", dataType = "long")
   val degree = LabelMeta(id = Some(-1), labelId = -1, name = "_degree",
     seq = degreeSeq, defaultValue = "0", dataType = "long")
   val count = LabelMeta(id = Some(-1), labelId = -1, name = "_count",
     seq = countSeq, defaultValue = "-1", dataType = "long")
+  val lastDeletedAt = LabelMeta(id = Some(-1), labelId = -1, name = "_lastDeletedAt",
+    seq = lastDeletedAtSeq, defaultValue = "-1", dataType = "long")
+  val label = LabelMeta(id = Some(-1), labelId = -1, name = "label",
+    seq = labelSeq, defaultValue = "", dataType = "string")
+  val direction = LabelMeta(id = Some(-1), labelId = -1, name = "direction",
+    seq = directionSeq, defaultValue = "out", dataType = "string")
+  val empty = LabelMeta(id = Some(-1), labelId = -1, name = "_empty",
+    seq = emptySeq, defaultValue = "-1", dataType = "long")
 
   // Each reserved column(_timestamp, timestamp) has same seq number, starts with '_' has high priority
-  val reservedMetas = List(from, to, degree, timestamp, count).flatMap { lm => List(lm, lm.copy(name = lm.name.drop(1))) }.reverse
-  val reservedMetasInner = List(from, to, degree, timestamp, count)
+  val reservedMetas = List(empty, label, direction, lastDeletedAt, from, fromHash, to, degree, timestamp, count).flatMap { lm => List(lm, lm.copy(name = lm.name.drop(1))) }.reverse
+  val reservedMetasInner = List(empty, label, direction, lastDeletedAt, from, fromHash, to, degree, timestamp, count)
+
+  val defaultRequiredMetaNames = Set("from", "_from", "to", "_to", "_from_hash", "label", "direction", "timestamp", "_timestamp")
 
   def apply(rs: WrappedResultSet): LabelMeta = {
     LabelMeta(Some(rs.int("id")), rs.int("label_id"), rs.string("name"), rs.byte("seq"), rs.string("default_value"), rs.string("data_type").toLowerCase)
   }
 
-  def isValidSeq(seq: Byte): Boolean = seq >= 0 && seq <= countSeq
-  def isValidSeqForAdmin(seq: Byte): Boolean = seq > 0 && seq < countSeq
+  /** Note: DegreeSeq should not be included in serializer/deserializer.
+    * only 0 <= seq <= CountSeq(Int.MaxValue - 2), not DegreeSet(Int.MaxValue - 1) should be
+    * included in actual bytes in storage.
+    * */
+  def isValidSeq(seq: Byte): Boolean = seq >= 0 && seq <= countSeq // || seq == fromHashSeq
+
+  def isValidSeqForAdmin(seq: Byte): Boolean = seq > 0 && seq < countSeq // || seq == fromHashSeq
 
   def findById(id: Int)(implicit session: DBSession = AutoSession): LabelMeta = {
     val cacheKey = "id=" + id
@@ -164,6 +185,21 @@ object LabelMeta extends Model[LabelMeta] {
   }
 }
 
-case class LabelMeta(id: Option[Int], labelId: Int, name: String, seq: Byte, defaultValue: String, dataType: String) {
+case class LabelMeta(id: Option[Int],
+                     labelId: Int,
+                     name: String,
+                     seq: Byte,
+                     defaultValue: String,
+                     dataType: String) {
   lazy val toJson = Json.obj("name" -> name, "defaultValue" -> defaultValue, "dataType" -> dataType)
+  override def equals(other: Any): Boolean = {
+    if (!other.isInstanceOf[LabelMeta]) false
+    else {
+      val o = other.asInstanceOf[LabelMeta]
+//      labelId == o.labelId &&
+        seq == o.seq
+    }
+  }
+  override def hashCode(): Int = seq.toInt
+//    (labelId, seq).hashCode()
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Model.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Model.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Model.scala
index bf6dfb7..7a18a49 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Model.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Model.scala
@@ -21,8 +21,10 @@ package org.apache.s2graph.core.mysqls
 
 import java.util.concurrent.Executors
 
-import com.typesafe.config.Config
+import com.typesafe.config.{ConfigFactory, Config}
+import org.apache.s2graph.core.JSONParser
 import org.apache.s2graph.core.utils.{SafeUpdateCache, logger}
+import play.api.libs.json.{Json, JsObject, JsValue}
 import scalikejdbc._
 
 import scala.concurrent.ExecutionContext
@@ -36,6 +38,7 @@ object Model {
   val numOfThread = Runtime.getRuntime.availableProcessors()
   val threadPool = Executors.newFixedThreadPool(numOfThread)
   val ec = ExecutionContext.fromExecutor(threadPool)
+  val useUTF8Encoding = "?useUnicode=true&characterEncoding=utf8"
 
   def apply(config: Config) = {
     maxSize = config.getInt("cache.max.size")
@@ -116,6 +119,34 @@ object Model {
     LabelIndex.findAll()
     ColumnMeta.findAll()
   }
+
+  def extraOptions(options: Option[String]): Map[String, JsValue] = options match {
+    case None => Map.empty
+    case Some(v) =>
+      try {
+        Json.parse(v).asOpt[JsObject].map { obj => obj.fields.toMap }.getOrElse(Map.empty)
+      } catch {
+        case e: Exception =>
+          logger.error(s"An error occurs while parsing the extra label option", e)
+          Map.empty
+      }
+  }
+
+  def toStorageConfig(options: Map[String, JsValue]): Option[Config] = {
+    try {
+      options.get("storage").map { jsValue =>
+        import scala.collection.JavaConverters._
+        val configMap = jsValue.as[JsObject].fieldSet.toMap.map { case (key, value) =>
+          key -> JSONParser.jsValueToAny(value).getOrElse(throw new RuntimeException("!!"))
+        }
+        ConfigFactory.parseMap(configMap.asJava)
+      }
+    } catch {
+      case e: Exception =>
+        logger.error(s"toStorageConfig error. use default storage", e)
+        None
+    }
+  }
 }
 
 trait Model[V] extends SQLSyntaxSupport[V] {
@@ -145,5 +176,9 @@ trait Model[V] extends SQLSyntaxSupport[V] {
   def putsToCaches(kvs: List[(String, List[V])]) = kvs.foreach {
     case (key, values) => listCache.put(key, values)
   }
+
+  def getAllCacheData() : (List[(String, Option[_])], List[(String, List[_])]) = {
+    (optionCache.getAllData(), listCache.getAllData())
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Service.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Service.scala b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Service.scala
index 3e01014..7fdda45 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Service.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/mysqls/Service.scala
@@ -21,6 +21,7 @@ package org.apache.s2graph.core.mysqls
 
 import java.util.UUID
 
+import com.typesafe.config.Config
 import org.apache.s2graph.core.utils.logger
 import play.api.libs.json.Json
 import scalikejdbc._
@@ -94,6 +95,7 @@ object Service extends Model[Service] {
       val cacheKey = s"serviceName=${x.serviceName}"
       (cacheKey -> x)
     })
+    ls
   }
 
   def findAllConn()(implicit session: DBSession = AutoSession): List[String] = {
@@ -101,7 +103,14 @@ object Service extends Model[Service] {
   }
 }
 
-case class Service(id: Option[Int], serviceName: String, accessToken: String, cluster: String, hTableName: String, preSplitSize: Int, hTableTTL: Option[Int]) {
+case class Service(id: Option[Int],
+                   serviceName: String,
+                   accessToken: String,
+                   cluster: String,
+                   hTableName: String,
+                   preSplitSize: Int,
+                   hTableTTL: Option[Int],
+                   options: Option[String] = None) {
   lazy val toJson =
     id match {
       case Some(_id) =>
@@ -110,4 +119,8 @@ case class Service(id: Option[Int], serviceName: String, accessToken: String, cl
       case None =>
         Json.parse("{}")
     }
+
+  lazy val extraOptions = Model.extraOptions(options)
+  lazy val storageConfigOpt: Option[Config] = toStorageConfig
+  def toStorageConfig: Option[Config] = Model.toStorageConfig(extraOptions)
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala b/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
index 1c93667..effb94b 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
@@ -19,11 +19,13 @@
 
 package org.apache.s2graph.core.parsers
 
-import org.apache.s2graph.core.GraphExceptions.WhereParserException
+import org.apache.s2graph.core.GraphExceptions.{LabelNotExistException, WhereParserException}
 import org.apache.s2graph.core.mysqls.{Label, LabelMeta}
 import org.apache.s2graph.core.types.InnerValLike
-import org.apache.s2graph.core.Edge
+import org.apache.s2graph.core.{Edge, GraphUtil}
 import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core.utils.logger
+
 import scala.annotation.tailrec
 import scala.util.Try
 import scala.util.parsing.combinator.JavaTokenParsers
@@ -37,12 +39,11 @@ trait ExtractValue {
     val label = parentEdge.label
     val metaPropInvMap = label.metaPropsInvMap
     val labelMeta = metaPropInvMap.getOrElse(propKey, throw WhereParserException(s"Where clause contains not existing property name: $propKey"))
-    val metaSeq = labelMeta.seq
 
-    metaSeq match {
-      case LabelMeta.from.seq => parentEdge.srcVertex.innerId
-      case LabelMeta.to.seq => parentEdge.tgtVertex.innerId
-      case _ => parentEdge.propsWithTs.get(metaSeq) match {
+    labelMeta match {
+      case LabelMeta.from => parentEdge.srcVertex.innerId
+      case LabelMeta.to => parentEdge.tgtVertex.innerId
+      case _ => parentEdge.propsWithTs.get(labelMeta) match {
         case None => toInnerVal(labelMeta.defaultValue, labelMeta.dataType, label.schemaVersion)
         case Some(edgeVal) => edgeVal.innerVal
       }
@@ -99,7 +100,13 @@ trait Clause extends ExtractValue {
     binOp(propValue, compValue)
   }
 }
-
+object Where {
+  def apply(labelName: String, sql: String): Try[Where] = {
+    val label = Label.findByName(labelName).getOrElse(throw new LabelNotExistException(labelName))
+    val parser = new WhereParser(label)
+    parser.parse(sql)
+  }
+}
 case class Where(clauses: Seq[Clause] = Seq.empty[Clause]) {
   def filter(edge: Edge) =
     if (clauses.isEmpty) true else clauses.map(_.filter(edge)).forall(identity)
@@ -118,18 +125,36 @@ case class Eq(propKey: String, value: String) extends Clause {
 }
 
 case class InWithoutParent(label: Label, propKey: String, values: Set[String]) extends Clause {
-  val innerValLikeLs = values.map { value =>
+  lazy val innerValLikeLsOut = values.map { value =>
     val labelMeta = label.metaPropsInvMap.getOrElse(propKey, throw WhereParserException(s"Where clause contains not existing property name: $propKey"))
     val dataType = propKey match {
       case "_to" | "to" => label.tgtColumn.columnType
       case "_from" | "from" => label.srcColumn.columnType
       case _ => labelMeta.dataType
     }
+
+    toInnerVal(value, dataType, label.schemaVersion)
+  }
+
+  lazy val innerValLikeLsIn = values.map { value =>
+    val labelMeta = label.metaPropsInvMap.getOrElse(propKey, throw WhereParserException(s"Where clause contains not existing property name: $propKey"))
+    val dataType = propKey match {
+      case "_to" | "to" => label.srcColumn.columnType
+      case "_from" | "from" => label.tgtColumn.columnType
+      case _ => labelMeta.dataType
+    }
+
     toInnerVal(value, dataType, label.schemaVersion)
   }
+
   override def filter(edge: Edge): Boolean = {
-    val propVal = propToInnerVal(edge, propKey)
-    innerValLikeLs.contains(propVal)
+    if (edge.dir == GraphUtil.directions("in")) {
+      val propVal = propToInnerVal(edge, propKey)
+      innerValLikeLsIn.contains(propVal)
+    } else {
+      val propVal = propToInnerVal(edge, propKey)
+      innerValLikeLsOut.contains(propVal)
+    }
   }
 }
 
@@ -166,11 +191,15 @@ case class Or(left: Clause, right: Clause) extends Clause {
 
 object WhereParser {
   val success = Where()
+
 }
 
 case class WhereParser(label: Label) extends JavaTokenParsers {
 
-  val anyStr = "[^\\s(),]+".r
+
+  override val stringLiteral = (("'" ~> "(\\\\'|[^'])*".r <~ "'" ) ^^ (_.replace("\\'", "'"))) | anyStr
+
+  val anyStr = "[^\\s(),']+".r
 
   val and = "and|AND".r
 
@@ -190,33 +219,35 @@ case class WhereParser(label: Label) extends JavaTokenParsers {
 
   def identWithDot: Parser[String] = repsep(ident, ".") ^^ { case values => values.mkString(".") }
 
-  def predicate = {
-    identWithDot ~ ("!=" | "=") ~ anyStr ^^ {
-      case f ~ op ~ s =>
-        if (op == "=") Eq(f, s)
-        else Not(Eq(f, s))
-    } | identWithDot ~ (">=" | "<=" | ">" | "<") ~ anyStr ^^ {
-      case f ~ op ~ s => op match {
-        case ">" => Gt(f, s)
-        case ">=" => Or(Gt(f, s), Eq(f, s))
-        case "<" => Lt(f, s)
-        case "<=" => Or(Lt(f, s), Eq(f, s))
-      }
-    } | identWithDot ~ (between ~> anyStr <~ and) ~ anyStr ^^ {
-      case f ~ minV ~ maxV => Between(f, minV, maxV)
-    } | identWithDot ~ (notIn | in) ~ ("(" ~> repsep(anyStr, ",") <~ ")") ^^ {
-      case f ~ op ~ values =>
-        val inClause =
-          if (f.startsWith("_parent")) IN(f, values.toSet)
-          else InWithoutParent(label, f, values.toSet)
-        if (op.toLowerCase == "in") inClause
-        else Not(inClause)
-
-
-      case _ => throw WhereParserException(s"Failed to parse where clause. ")
+  val _eq = identWithDot ~ ("!=" | "=") ~ stringLiteral ^^ {
+    case f ~ op ~ s => if (op == "=") Eq(f, s) else Not(Eq(f, s))
+  }
+
+  val _ltGt = identWithDot ~ (">=" | "<=" | ">" | "<") ~ stringLiteral ^^ {
+    case f ~ op ~ s => op match {
+      case ">" => Gt(f, s)
+      case ">=" => Or(Gt(f, s), Eq(f, s))
+      case "<" => Lt(f, s)
+      case "<=" => Or(Lt(f, s), Eq(f, s))
     }
   }
 
+  val _between = identWithDot ~ (between ~> stringLiteral <~ and) ~ stringLiteral ^^ {
+    case f ~ minV ~ maxV => Between(f, minV, maxV)
+  }
+
+  val _in = identWithDot ~ (notIn | in) ~ ("(" ~> repsep(stringLiteral, ",") <~ ")") ^^ {
+    case f ~ op ~ values =>
+      val inClause =
+        if (f.startsWith("_parent")) IN(f, values.toSet)
+        else InWithoutParent(label, f, values.toSet)
+
+      if (op.toLowerCase == "in") inClause
+      else Not(inClause)
+  }
+
+  def predicate =  _eq | _ltGt | _between | _in
+
   def parse(sql: String): Try[Where] = Try {
     parseAll(where, sql) match {
       case Success(r, q) => r


[6/7] incubator-s2graph git commit: [S2GRAPH-122]: Change data types of Edge/IndexEdge/SnapshotEdge.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala b/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala
index 2cbddea..ca32a14 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/Graph.scala
@@ -19,25 +19,31 @@
 
 package org.apache.s2graph.core
 
-import java.util
-import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.Executors
 
 import com.typesafe.config.{Config, ConfigFactory}
+import org.apache.hadoop.fs.Path
+import org.apache.s2graph.core.GraphExceptions.{FetchAllStepFailException, FetchTimeoutException, LabelNotExistException}
 import org.apache.s2graph.core.JSONParser._
-import org.apache.s2graph.core.mysqls.{Label, Model}
-import org.apache.s2graph.core.parsers.WhereParser
+import org.apache.s2graph.core.mysqls.{Label, LabelMeta, Model, Service}
 import org.apache.s2graph.core.storage.hbase.AsynchbaseStorage
-import org.apache.s2graph.core.types.{InnerVal, LabelWithDirection}
-import org.apache.s2graph.core.utils.logger
+import org.apache.s2graph.core.storage.{SKeyValue, Storage}
+import org.apache.s2graph.core.types._
+import org.apache.s2graph.core.utils.{DeferCache, Extensions, SafeUpdateCache, logger}
 import play.api.libs.json.{JsObject, Json}
 
+import scala.annotation.tailrec
 import scala.collection.JavaConversions._
-import scala.collection._
-import scala.collection.mutable.ListBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.concurrent._
-import scala.util.Try
+import scala.util.{Random, Try}
 
 object Graph {
+
+  type HashKey = (Int, Int, Int, Int, Boolean)
+  type FilterHashKey = (Int, Int)
+
+
   val DefaultScore = 1.0
 
   private val DefaultConfigs: Map[String, AnyRef] = Map(
@@ -65,26 +71,130 @@ object Graph {
     "future.cache.expire.after.write" -> java.lang.Integer.valueOf(10000),
     "future.cache.expire.after.access" -> java.lang.Integer.valueOf(5000),
     "future.cache.metric.interval" -> java.lang.Integer.valueOf(60000),
+    "query.future.cache.max.size" -> java.lang.Integer.valueOf(1000),
+    "query.future.cache.expire.after.write" -> java.lang.Integer.valueOf(10000),
+    "query.future.cache.expire.after.access" -> java.lang.Integer.valueOf(5000),
+    "query.future.cache.metric.interval" -> java.lang.Integer.valueOf(60000),
     "s2graph.storage.backend" -> "hbase",
-    "query.hardlimit" -> java.lang.Integer.valueOf(100000)
+    "query.hardlimit" -> java.lang.Integer.valueOf(100000),
+    "hbase.zookeeper.znode.parent" -> "/hbase",
+    "query.log.sample.rate" -> Double.box(0.05)
   )
 
   var DefaultConfig: Config = ConfigFactory.parseMap(DefaultConfigs)
 
-  /** helpers for filterEdges */
-  type HashKey = (Int, Int, Int, Int, Boolean)
-  type FilterHashKey = (Int, Int)
-  type Result = (ConcurrentHashMap[HashKey, ListBuffer[(Edge, Double)]],
-    ConcurrentHashMap[HashKey, (FilterHashKey, Edge, Double)],
-    ListBuffer[(HashKey, FilterHashKey, Edge, Double)])
+  def toGraphElement(s: String, labelMapping: Map[String, String] = Map.empty): Option[GraphElement] = Try {
+    val parts = GraphUtil.split(s)
+    val logType = parts(2)
+    val element = if (logType == "edge" | logType == "e") {
+      /** current only edge is considered to be bulk loaded */
+      labelMapping.get(parts(5)) match {
+        case None =>
+        case Some(toReplace) =>
+          parts(5) = toReplace
+      }
+      toEdge(parts)
+    } else if (logType == "vertex" | logType == "v") {
+      toVertex(parts)
+    } else {
+      throw new GraphExceptions.JsonParseException("log type is not exist in log.")
+    }
 
-  def toHashKey(queryParam: QueryParam, edge: Edge, isDegree: Boolean): (HashKey, FilterHashKey) = {
-    val src = edge.srcVertex.innerId.hashCode()
-    val tgt = edge.tgtVertex.innerId.hashCode()
-    val hashKey = (src, edge.labelWithDir.labelId, edge.labelWithDir.dir, tgt, isDegree)
-    val filterHashKey = (src, tgt)
+    element
+  } recover {
+    case e: Exception =>
+      logger.error(s"[toElement]: $e", e)
+      None
+  } get
 
-    (hashKey, filterHashKey)
+
+  def toVertex(s: String): Option[Vertex] = {
+    toVertex(GraphUtil.split(s))
+  }
+
+  def toEdge(s: String): Option[Edge] = {
+    toEdge(GraphUtil.split(s))
+  }
+
+  def toEdge(parts: Array[String]): Option[Edge] = Try {
+    val (ts, operation, logType, srcId, tgtId, label) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
+    val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
+    val tempDirection = if (parts.length >= 8) parts(7) else "out"
+    val direction = if (tempDirection != "out" && tempDirection != "in") "out" else tempDirection
+    val edge = Edge.toEdge(srcId, tgtId, label, direction, props, ts.toLong, operation)
+    Option(edge)
+  } recover {
+    case e: Exception =>
+      logger.error(s"[toEdge]: $e", e)
+      throw e
+  } get
+
+  def toVertex(parts: Array[String]): Option[Vertex] = Try {
+    val (ts, operation, logType, srcId, serviceName, colName) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
+    val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
+    val vertex = Vertex.toVertex(serviceName, colName, srcId, props, ts.toLong, operation)
+    Option(vertex)
+  } recover {
+    case e: Throwable =>
+      logger.error(s"[toVertex]: $e", e)
+      throw e
+  } get
+
+  def initStorage(graph: Graph, config: Config)(ec: ExecutionContext): Storage[_, _] = {
+    val storageBackend = config.getString("s2graph.storage.backend")
+    logger.info(s"[InitStorage]: $storageBackend")
+
+    storageBackend match {
+      case "hbase" => new AsynchbaseStorage(graph, config)(ec)
+      case _ => throw new RuntimeException("not supported storage.")
+    }
+  }
+
+  def parseCacheConfig(config: Config, prefix: String): Config = {
+    import scala.collection.JavaConversions._
+
+    val kvs = new java.util.HashMap[String, AnyRef]()
+    for {
+      entry <- config.entrySet()
+      (k, v) = (entry.getKey, entry.getValue) if k.startsWith(prefix)
+    } yield {
+      val newKey = k.replace(prefix, "")
+      kvs.put(newKey, v.unwrapped())
+    }
+    ConfigFactory.parseMap(kvs)
+  }
+
+  /** Global helper functions */
+  @tailrec
+  final def randomInt(sampleNumber: Int, range: Int, set: Set[Int] = Set.empty[Int]): Set[Int] = {
+    if (range < sampleNumber || set.size == sampleNumber) set
+    else randomInt(sampleNumber, range, set + Random.nextInt(range))
+  }
+
+  def sample(queryRequest: QueryRequest, edges: Seq[EdgeWithScore], n: Int): Seq[EdgeWithScore] = {
+    if (edges.size <= n) {
+      edges
+    } else {
+      val plainEdges = if (queryRequest.queryParam.offset == 0) {
+        edges.tail
+      } else edges
+
+      val randoms = randomInt(n, plainEdges.size)
+      var samples = List.empty[EdgeWithScore]
+      var idx = 0
+      plainEdges.foreach { e =>
+        if (randoms.contains(idx)) samples = e :: samples
+        idx += 1
+      }
+      samples
+    }
+  }
+
+  def normalize(edgeWithScores: Seq[EdgeWithScore]): Seq[EdgeWithScore] = {
+    val sum = edgeWithScores.foldLeft(0.0) { case (acc, cur) => acc + cur.score }
+    edgeWithScores.map { edgeWithScore =>
+      edgeWithScore.copy(score = edgeWithScore.score / sum)
+    }
   }
 
   def alreadyVisitedVertices(edgeWithScoreLs: Seq[EdgeWithScore]): Map[(LabelWithDirection, Vertex), Boolean] = {
@@ -100,7 +210,7 @@ object Graph {
   /** common methods for filter out, transform, aggregate queryResult */
   def convertEdges(queryParam: QueryParam, edge: Edge, nextStepOpt: Option[Step]): Seq[Edge] = {
     for {
-      convertedEdge <- queryParam.transformer.transform(edge, nextStepOpt) if !edge.isDegree
+      convertedEdge <- queryParam.edgeTransformer.transform(queryParam, edge, nextStepOpt) if !edge.isDegree
     } yield convertedEdge
   }
 
@@ -110,18 +220,17 @@ object Graph {
       case None => 1.0
       case Some(timeDecay) =>
         val tsVal = try {
-          val labelMeta = edge.label.metaPropsMap(timeDecay.labelMetaSeq)
-          val innerValWithTsOpt = edge.propsWithTs.get(timeDecay.labelMetaSeq)
+          val innerValWithTsOpt = edge.propsWithTs.get(timeDecay.labelMeta)
           innerValWithTsOpt.map { innerValWithTs =>
             val innerVal = innerValWithTs.innerVal
-            labelMeta.dataType match {
+            timeDecay.labelMeta.dataType match {
               case InnerVal.LONG => innerVal.value match {
                 case n: BigDecimal => n.bigDecimal.longValue()
                 case _ => innerVal.toString().toLong
               }
               case _ => innerVal.toString().toLong
             }
-          } getOrElse(edge.ts)
+          } getOrElse (edge.ts)
         } catch {
           case e: Exception =>
             logger.error(s"processTimeDecay error. ${edge.toLogString}", e)
@@ -134,264 +243,891 @@ object Graph {
     tsVal
   }
 
-  def processDuplicates(queryParam: QueryParam,
-                        duplicates: Seq[(FilterHashKey, EdgeWithScore)]): Seq[(FilterHashKey, EdgeWithScore)] = {
+  def processDuplicates[T](queryParam: QueryParam,
+                           duplicates: Seq[(FilterHashKey, T)])(implicit ev: WithScore[T]): Seq[(FilterHashKey, T)] = {
 
     if (queryParam.label.consistencyLevel != "strong") {
       //TODO:
       queryParam.duplicatePolicy match {
-        case Query.DuplicatePolicy.First => Seq(duplicates.head)
-        case Query.DuplicatePolicy.Raw => duplicates
-        case Query.DuplicatePolicy.CountSum =>
+        case DuplicatePolicy.First => Seq(duplicates.head)
+        case DuplicatePolicy.Raw => duplicates
+        case DuplicatePolicy.CountSum =>
           val countSum = duplicates.size
-          Seq(duplicates.head._1 -> duplicates.head._2.copy(score = countSum))
+          val (headFilterHashKey, headEdgeWithScore) = duplicates.head
+          Seq(headFilterHashKey -> ev.withNewScore(headEdgeWithScore, countSum))
         case _ =>
-          val scoreSum = duplicates.foldLeft(0.0) { case (prev, current) => prev + current._2.score }
-          Seq(duplicates.head._1 -> duplicates.head._2.copy(score = scoreSum))
+          val scoreSum = duplicates.foldLeft(0.0) { case (prev, current) => prev + ev.score(current._2) }
+          val (headFilterHashKey, headEdgeWithScore) = duplicates.head
+          Seq(headFilterHashKey -> ev.withNewScore(headEdgeWithScore, scoreSum))
       }
     } else {
       duplicates
     }
   }
+
+  def toHashKey(queryParam: QueryParam, edge: Edge, isDegree: Boolean): (HashKey, FilterHashKey) = {
+    val src = edge.srcVertex.innerId.hashCode()
+    val tgt = edge.tgtVertex.innerId.hashCode()
+    val hashKey = (src, edge.labelWithDir.labelId, edge.labelWithDir.dir, tgt, isDegree)
+    val filterHashKey = (src, tgt)
+
+    (hashKey, filterHashKey)
+  }
+
   def filterEdges(q: Query,
                   stepIdx: Int,
                   queryRequests: Seq[QueryRequest],
-                  queryResultLsFuture: Future[Seq[StepInnerResult]],
+                  queryResultLsFuture: Future[Seq[StepResult]],
                   queryParams: Seq[QueryParam],
-                  alreadyVisited: Map[(LabelWithDirection, Vertex), Boolean] = Map.empty[(LabelWithDirection, Vertex), Boolean])
-                 (implicit ec: scala.concurrent.ExecutionContext): Future[StepInnerResult] = {
+                  alreadyVisited: Map[(LabelWithDirection, Vertex), Boolean] = Map.empty,
+                  buildLastStepInnerResult: Boolean = true,
+                  parentEdges: Map[VertexId, Seq[EdgeWithScore]])
+                 (implicit ec: scala.concurrent.ExecutionContext): Future[StepResult] = {
 
     queryResultLsFuture.map { queryRequestWithResultLs =>
-      if (queryRequestWithResultLs.isEmpty) StepInnerResult.Empty
+      val (cursors, failCount) = {
+        val _cursors = ArrayBuffer.empty[Array[Byte]]
+        var _failCount = 0
+
+        queryRequestWithResultLs.foreach { stepResult =>
+          _cursors.append(stepResult.cursors: _*)
+          _failCount += stepResult.failCount
+        }
+
+        _cursors -> _failCount
+      }
+
+
+      if (queryRequestWithResultLs.isEmpty) StepResult.Empty.copy(failCount = failCount)
       else {
+        val isLastStep = stepIdx == q.steps.size - 1
+        val queryOption = q.queryOption
         val step = q.steps(stepIdx)
 
-        val nextStepOpt = if (stepIdx < q.steps.size - 1) Option(q.steps(stepIdx + 1)) else None
-
-        val excludeLabelWithDirSet = new util.HashSet[(Int, Int)]
-        val includeLabelWithDirSet = new util.HashSet[(Int, Int)]
-        step.queryParams.filter(_.exclude).foreach(l => excludeLabelWithDirSet.add(l.labelWithDir.labelId -> l.labelWithDir.dir))
-        step.queryParams.filter(_.include).foreach(l => includeLabelWithDirSet.add(l.labelWithDir.labelId -> l.labelWithDir.dir))
-
-        val edgesToExclude = new util.HashSet[FilterHashKey]()
-        val edgesToInclude = new util.HashSet[FilterHashKey]()
-
-        val sequentialLs = new ListBuffer[(HashKey, FilterHashKey, EdgeWithScore)]()
-        val agg = new mutable.HashMap[HashKey, ListBuffer[(FilterHashKey, EdgeWithScore)]]()
-        val params = new mutable.HashMap[HashKey, QueryParam]()
-        var numOfDuplicates = 0
-        var numOfTotal = 0
-        queryRequests.zip(queryRequestWithResultLs).foreach { case (queryRequest, stepInnerResult) =>
-          val queryParam = queryRequest.queryParam
-          val labelWeight = step.labelWeights.getOrElse(queryParam.labelWithDir.labelId, 1.0)
-          val includeExcludeKey = queryParam.labelWithDir.labelId -> queryParam.labelWithDir.dir
-          val shouldBeExcluded = excludeLabelWithDirSet.contains(includeExcludeKey)
-          val shouldBeIncluded = includeLabelWithDirSet.contains(includeExcludeKey)
-          val where = queryParam.where.get
-
-          for {
-            edgeWithScore <- stepInnerResult.edgesWithScoreLs
-            if where == WhereParser.success || where.filter(edgeWithScore.edge)
-            (edge, score) = EdgeWithScore.unapply(edgeWithScore).get
-          } {
-            numOfTotal += 1
-            if (queryParam.transformer.isDefault) {
-              val convertedEdge = edge
-
-              val (hashKey, filterHashKey) = toHashKey(queryParam, convertedEdge, isDegree = false)
-
-              /** check if this edge should be exlcuded. */
-              if (shouldBeExcluded) {
-                edgesToExclude.add(filterHashKey)
-              } else {
-                if (shouldBeIncluded) {
-                  edgesToInclude.add(filterHashKey)
-                }
-                val tsVal = processTimeDecay(queryParam, convertedEdge)
-                val newScore = labelWeight * score * tsVal
-                val newEdgeWithScore = EdgeWithScore(convertedEdge, newScore)
-                sequentialLs += ((hashKey, filterHashKey, newEdgeWithScore))
-                agg.get(hashKey) match {
-                  case None =>
-                    val newLs = new ListBuffer[(FilterHashKey, EdgeWithScore)]()
-                    newLs += (filterHashKey -> newEdgeWithScore)
-                    agg += (hashKey -> newLs)
-                  case Some(old) =>
-                    numOfDuplicates += 1
-                    old += (filterHashKey -> newEdgeWithScore)
-                }
-                params += (hashKey -> queryParam)
+        val currentStepResults = queryRequests.view.zip(queryRequestWithResultLs)
+        val shouldBuildInnerResults = !isLastStep || buildLastStepInnerResult
+        val degrees = queryRequestWithResultLs.flatMap(_.degreeEdges)
+
+        if (shouldBuildInnerResults) {
+          val _results = buildResult(q, stepIdx, currentStepResults, parentEdges) { (edgeWithScore, propsSelectColumns) =>
+            edgeWithScore
+          }
+
+          /** process step group by */
+          val results = StepResult.filterOutStepGroupBy(_results, step.groupBy)
+          StepResult(edgeWithScores = results, grouped = Nil, degreeEdges = degrees, cursors = cursors, failCount = failCount)
+
+        } else {
+          val _results = buildResult(q, stepIdx, currentStepResults, parentEdges) { (edgeWithScore, propsSelectColumns) =>
+            val edge = edgeWithScore.edge
+            val score = edgeWithScore.score
+            val label = edgeWithScore.label
+
+            /** Select */
+            val mergedPropsWithTs =
+            if (queryOption.selectColumns.isEmpty) {
+              label.metaPropsDefaultMapInner.map { case (labelMeta, defaultVal) =>
+                labelMeta -> edge.propsWithTs.getOrElse(labelMeta, defaultVal)
               }
             } else {
-              convertEdges(queryParam, edge, nextStepOpt).foreach { convertedEdge =>
-                val (hashKey, filterHashKey) = toHashKey(queryParam, convertedEdge, isDegree = false)
-
-                /** check if this edge should be exlcuded. */
-                if (shouldBeExcluded) {
-                  edgesToExclude.add(filterHashKey)
-                } else {
-                  if (shouldBeIncluded) {
-                    edgesToInclude.add(filterHashKey)
-                  }
-                  val tsVal = processTimeDecay(queryParam, convertedEdge)
-                  val newScore = labelWeight * score * tsVal
-                  val newEdgeWithScore = EdgeWithScore(convertedEdge, newScore)
-                  sequentialLs += ((hashKey, filterHashKey, newEdgeWithScore))
-                  agg.get(hashKey) match {
-                    case None =>
-                      val newLs = new ListBuffer[(FilterHashKey, EdgeWithScore)]()
-                      newLs += (filterHashKey -> newEdgeWithScore)
-                      agg += (hashKey -> newLs)
-                    case Some(old) =>
-                      numOfDuplicates += 1
-                      old += (filterHashKey -> newEdgeWithScore)
-                  }
-                  params += (hashKey -> queryParam)
-                }
+              val initial = Map(LabelMeta.timestamp -> edge.propsWithTs(LabelMeta.timestamp))
+              propsSelectColumns.foldLeft(initial) { case (prev, labelMeta) =>
+                prev + (labelMeta -> edge.propsWithTs.getOrElse(labelMeta, label.metaPropsDefaultMapInner(labelMeta)))
               }
             }
-          }
-        }
 
+            val newEdge = edge.copy(propsWithTs = mergedPropsWithTs)
+            val newEdgeWithScore = edgeWithScore.copy(edge = newEdge)
+            /** OrderBy */
+            val orderByValues =
+             if (queryOption.orderByKeys.isEmpty) (score, edge.tsInnerVal, None, None)
+              else StepResult.toTuple4(newEdgeWithScore.toValues(queryOption.orderByKeys))
+
+            /** StepGroupBy */
+            val stepGroupByValues = newEdgeWithScore.toValues(step.groupBy.keys)
+
+            /** GroupBy */
+            val groupByValues = newEdgeWithScore.toValues(queryOption.groupBy.keys)
+
+            /** FilterOut */
+            val filterOutValues = newEdgeWithScore.toValues(queryOption.filterOutFields)
 
-        val edgeWithScoreLs = new ListBuffer[EdgeWithScore]()
-        if (numOfDuplicates == 0) {
-          // no duplicates at all.
-          for {
-            (hashKey, filterHashKey, edgeWithScore) <- sequentialLs
-            if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
-          } {
-            edgeWithScoreLs += edgeWithScore
+            newEdgeWithScore.copy(orderByValues = orderByValues,
+              stepGroupByValues = stepGroupByValues,
+              groupByValues = groupByValues,
+              filterOutValues = filterOutValues)
           }
-        } else {
-          // need to resolve duplicates.
-          val seen = new mutable.HashSet[HashKey]()
-          for {
-            (hashKey, filterHashKey, edgeWithScore) <- sequentialLs
-            if !seen.contains(hashKey)
-            if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
-          } {
-            val queryParam = params(hashKey)
-            val duplicates = processDuplicates(queryParam, agg(hashKey))
-            duplicates.foreach { case (_, duplicate) =>
-              if (duplicate.score >= queryParam.threshold) {
-                seen += hashKey
-                edgeWithScoreLs += duplicate
+
+          /** process step group by */
+          val results = StepResult.filterOutStepGroupBy(_results, step.groupBy)
+
+          /** process ordered list */
+          val ordered = if (queryOption.groupBy.keys.isEmpty) StepResult.orderBy(queryOption, results) else Nil
+
+          /** process grouped list */
+          val grouped =
+          if (queryOption.groupBy.keys.isEmpty) Nil
+          else {
+            val agg = new scala.collection.mutable.HashMap[StepResult.GroupByKey, (Double, StepResult.Values)]()
+            results.groupBy { edgeWithScore =>
+              //                edgeWithScore.groupByValues.map(_.map(_.toString))
+              edgeWithScore.groupByValues
+            }.foreach { case (k, ls) =>
+              val (scoreSum, merged) = StepResult.mergeOrdered(ls, Nil, queryOption)
+
+              val newScoreSum = scoreSum
+
+              /**
+                * watch out here. by calling toString on Any, we lose type information which will be used
+                * later for toJson.
+                */
+              if (merged.nonEmpty) {
+                val newKey = merged.head.groupByValues
+                agg += (newKey -> (newScoreSum, merged))
               }
             }
+            agg.toSeq.sortBy(_._2._1 * -1)
           }
-        }
 
-        val degrees = queryRequestWithResultLs.flatMap(_.degreeEdges)
-        StepInnerResult(edgesWithScoreLs = edgeWithScoreLs, degreeEdges = degrees)
+          StepResult(edgeWithScores = ordered, grouped = grouped, degreeEdges = degrees, cursors = cursors, failCount = failCount)
+        }
       }
     }
   }
 
-  def toGraphElement(s: String, labelMapping: Map[String, String] = Map.empty): Option[GraphElement] = Try {
-    val parts = GraphUtil.split(s)
-    val logType = parts(2)
-    val element = if (logType == "edge" | logType == "e") {
-      /* current only edge is considered to be bulk loaded */
-      labelMapping.get(parts(5)) match {
-        case None =>
-        case Some(toReplace) =>
-          parts(5) = toReplace
+  private def toEdgeWithScores(queryRequest: QueryRequest,
+                               stepResult: StepResult,
+                               parentEdges: Map[VertexId, Seq[EdgeWithScore]]): Seq[EdgeWithScore] = {
+    val queryOption = queryRequest.query.queryOption
+    val queryParam = queryRequest.queryParam
+    val prevScore = queryRequest.prevStepScore
+    val labelWeight = queryRequest.labelWeight
+    val edgeWithScores = stepResult.edgeWithScores
+
+    val shouldBuildParents = queryOption.returnTree || queryParam.whereHasParent
+    val parents = if (shouldBuildParents) {
+      parentEdges.getOrElse(queryRequest.vertex.id, Nil).map { edgeWithScore =>
+        val edge = edgeWithScore.edge
+        val score = edgeWithScore.score
+        val label = edgeWithScore.label
+
+        /** Select */
+        val mergedPropsWithTs =
+          if (queryOption.selectColumns.isEmpty) {
+            label.metaPropsDefaultMapInner.map { case (labelMeta, defaultVal) =>
+              labelMeta -> edge.propsWithTs.getOrElse(labelMeta, defaultVal)
+            }
+          } else {
+            val initial = Map(LabelMeta.timestamp -> edge.propsWithTs(LabelMeta.timestamp))
+            queryOption.selectColumns.foldLeft(initial) { case (acc, labelMetaName) =>
+              label.metaPropsDefaultMapInnerString.get(labelMetaName) match {
+                case None => acc
+                case Some(defaultValue) =>
+                  val labelMeta = label.metaPropsInvMap(labelMetaName)
+                  acc + (labelMeta -> edge.propsWithTs.getOrElse(labelMeta, defaultValue))
+              }
+            }
+          }
+
+        val newEdge = edge.copy(propsWithTs = mergedPropsWithTs)
+        edgeWithScore.copy(edge = newEdge)
       }
-      toEdge(parts)
-    } else if (logType == "vertex" | logType == "v") {
-      toVertex(parts)
-    } else {
-      throw new GraphExceptions.JsonParseException("log type is not exist in log.")
-    }
+    } else Nil
 
-    element
-  } recover {
-    case e: Exception =>
-      logger.error(s"[toElement]: $e", e)
-      None
-  } get
+    // skip
+    if (queryOption.ignorePrevStepCache) stepResult.edgeWithScores
+    else {
+      val degreeScore = 0.0
 
+      val sampled =
+        if (queryRequest.queryParam.sample >= 0) sample(queryRequest, edgeWithScores, queryRequest.queryParam.sample)
+        else edgeWithScores
 
-  def toVertex(s: String): Option[Vertex] = {
-    toVertex(GraphUtil.split(s))
-  }
+      val withScores = for {
+        edgeWithScore <- sampled
+      } yield {
+        val edge = edgeWithScore.edge
+        val edgeScore = edgeWithScore.score
+        val score = queryParam.scorePropagateOp match {
+          case "plus" => edgeScore + prevScore
+          case "divide" =>
+            if ((prevScore + queryParam.scorePropagateShrinkage) == 0) 0
+            else edgeScore / (prevScore + queryParam.scorePropagateShrinkage)
+          case _ => edgeScore * prevScore
+        }
 
-  def toEdge(s: String): Option[Edge] = {
-    toEdge(GraphUtil.split(s))
+        val tsVal = processTimeDecay(queryParam, edge)
+        val newScore = degreeScore + score
+        //          val newEdge = if (queryOption.returnTree) edge.copy(parentEdges = parents) else edge
+        val newEdge = edge.copy(parentEdges = parents)
+        edgeWithScore.copy(edge = newEdge, score = newScore * labelWeight * tsVal)
+      }
+
+      val normalized =
+        if (queryParam.shouldNormalize) normalize(withScores)
+        else withScores
+
+      normalized
+    }
   }
 
-  def toEdge(parts: Array[String]): Option[Edge] = Try {
-    val (ts, operation, logType, srcId, tgtId, label) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
-    val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
-    val tempDirection = if (parts.length >= 8) parts(7) else "out"
-    val direction = if (tempDirection != "out" && tempDirection != "in") "out" else tempDirection
-    val edge = Edge.toEdge(srcId, tgtId, label, direction, props, ts.toLong, operation)
-    Option(edge)
-  } recover {
-    case e: Exception =>
-      logger.error(s"[toEdge]: $e", e)
-      throw e
-  } get
+  private def buildResult[T](query: Query,
+                             stepIdx: Int,
+                             stepResultLs: Seq[(QueryRequest, StepResult)],
+                             parentEdges: Map[VertexId, Seq[EdgeWithScore]])
+                            (createFunc: (EdgeWithScore, Set[LabelMeta]) => T)
+                            (implicit ev: WithScore[T]): ListBuffer[T] = {
+    import scala.collection._
 
-  def toVertex(parts: Array[String]): Option[Vertex] = Try {
-    val (ts, operation, logType, srcId, serviceName, colName) = (parts(0), parts(1), parts(2), parts(3), parts(4), parts(5))
-    val props = if (parts.length >= 7) fromJsonToProperties(Json.parse(parts(6)).asOpt[JsObject].getOrElse(Json.obj())) else Map.empty[String, Any]
-    val vertex = Vertex.toVertex(serviceName, colName, srcId, props, ts.toLong, operation)
-    Option(vertex)
-  } recover {
-    case e: Throwable =>
-      logger.error(s"[toVertex]: $e", e)
-      throw e
-  } get
+    val results = ListBuffer.empty[T]
+    val sequentialLs: ListBuffer[(HashKey, FilterHashKey, T, QueryParam)] = ListBuffer.empty
+    val duplicates: mutable.HashMap[HashKey, ListBuffer[(FilterHashKey, T)]] = mutable.HashMap.empty
+    val edgesToExclude: mutable.HashSet[FilterHashKey] = mutable.HashSet.empty
+    val edgesToInclude: mutable.HashSet[FilterHashKey] = mutable.HashSet.empty
 
-  def initStorage(graph: Graph, config: Config)(ec: ExecutionContext) = {
-    config.getString("s2graph.storage.backend") match {
-      case "hbase" => new AsynchbaseStorage(graph, config)(ec)
-      case _ => throw new RuntimeException("not supported storage.")
+    var numOfDuplicates = 0
+    val queryOption = query.queryOption
+    val step = query.steps(stepIdx)
+    val excludeLabelWithDirSet = step.queryParams.filter(_.exclude).map(l => l.labelWithDir).toSet
+    val includeLabelWithDirSet = step.queryParams.filter(_.include).map(l => l.labelWithDir).toSet
+
+    stepResultLs.foreach { case (queryRequest, stepInnerResult) =>
+      val queryParam = queryRequest.queryParam
+      val label = queryParam.label
+      val shouldBeExcluded = excludeLabelWithDirSet.contains(queryParam.labelWithDir)
+      val shouldBeIncluded = includeLabelWithDirSet.contains(queryParam.labelWithDir)
+
+      val propsSelectColumns = (for {
+        column <- queryOption.propsSelectColumns
+        labelMeta <- label.metaPropsInvMap.get(column)
+      } yield labelMeta).toSet
+
+      for {
+        edgeWithScore <- toEdgeWithScores(queryRequest, stepInnerResult, parentEdges)
+      } {
+        val edge = edgeWithScore.edge
+        val (hashKey, filterHashKey) = toHashKey(queryParam, edge, isDegree = false)
+        //        params += (hashKey -> queryParam) //
+
+        /** check if this edge should be exlcuded. */
+        if (shouldBeExcluded) {
+          edgesToExclude.add(filterHashKey)
+        } else {
+          if (shouldBeIncluded) {
+            edgesToInclude.add(filterHashKey)
+          }
+          val newEdgeWithScore = createFunc(edgeWithScore, propsSelectColumns)
+
+          sequentialLs += ((hashKey, filterHashKey, newEdgeWithScore, queryParam))
+          duplicates.get(hashKey) match {
+            case None =>
+              val newLs = ListBuffer.empty[(FilterHashKey, T)]
+              newLs += (filterHashKey -> newEdgeWithScore)
+              duplicates += (hashKey -> newLs) //
+            case Some(old) =>
+              numOfDuplicates += 1
+              old += (filterHashKey -> newEdgeWithScore) //
+          }
+        }
+      }
     }
+
+
+    if (numOfDuplicates == 0) {
+      // no duplicates at all.
+      for {
+        (hashKey, filterHashKey, edgeWithScore, _) <- sequentialLs
+        if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
+      } {
+        results += edgeWithScore
+      }
+    } else {
+      // need to resolve duplicates.
+      val seen = new mutable.HashSet[HashKey]()
+      for {
+        (hashKey, filterHashKey, edgeWithScore, queryParam) <- sequentialLs
+        if !edgesToExclude.contains(filterHashKey) || edgesToInclude.contains(filterHashKey)
+        if !seen.contains(hashKey)
+      } {
+        //        val queryParam = params(hashKey)
+        processDuplicates(queryParam, duplicates(hashKey)).foreach { case (_, duplicate) =>
+          if (ev.score(duplicate) >= queryParam.threshold) {
+            seen += hashKey
+            results += duplicate
+          }
+        }
+      }
+    }
+    results
   }
+
 }
 
 class Graph(_config: Config)(implicit val ec: ExecutionContext) {
+
+  import Graph._
+
   val config = _config.withFallback(Graph.DefaultConfig)
 
   Model.apply(config)
   Model.loadCache()
 
-  // TODO: Make storage client by config param
-  val storage = Graph.initStorage(this, config)(ec)
+  val MaxRetryNum = config.getInt("max.retry.number")
+  val MaxBackOff = config.getInt("max.back.off")
+  val BackoffTimeout = config.getInt("back.off.timeout")
+  val DeleteAllFetchCount = config.getInt("delete.all.fetch.count")
+  val DeleteAllFetchSize = config.getInt("delete.all.fetch.size")
+  val FailProb = config.getDouble("hbase.fail.prob")
+  val LockExpireDuration = config.getInt("lock.expire.time")
+  val MaxSize = config.getInt("future.cache.max.size")
+  val ExpireAfterWrite = config.getInt("future.cache.expire.after.write")
+  val ExpireAfterAccess = config.getInt("future.cache.expire.after.access")
 
+  val scheduledEx = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor())
+
+  private def confWithFallback(conf: Config): Config = {
+    conf.withFallback(config)
+  }
+
+  /**
+    * TODO: we need to some way to handle malformed configuration for storage.
+    */
+  val storagePool: scala.collection.mutable.Map[String, Storage[_, _]] = {
+    val labels = Label.findAll()
+    val services = Service.findAll()
+
+    val labelConfigs = labels.flatMap(_.toStorageConfig)
+    val serviceConfigs = services.flatMap(_.toStorageConfig)
+
+    val configs = (labelConfigs ++ serviceConfigs).map { conf =>
+      confWithFallback(conf)
+    }.toSet
+
+    val pools = new java.util.HashMap[Config, Storage[_, _]]()
+    configs.foreach { config =>
+      pools.put(config, Graph.initStorage(this, config)(ec))
+    }
+
+    val m = new java.util.concurrent.ConcurrentHashMap[String, Storage[_, _]]()
+
+    labels.foreach { label =>
+      if (label.storageConfigOpt.isDefined) {
+        m += (s"label:${label.label}" -> pools(label.storageConfigOpt.get))
+      }
+    }
+
+    services.foreach { service =>
+      if (service.storageConfigOpt.isDefined) {
+        m += (s"service:${service.serviceName}" -> pools(service.storageConfigOpt.get))
+      }
+    }
+
+    m
+  }
+
+  val defaultStorage: Storage[_, _] = Graph.initStorage(this, config)(ec)
+
+  /** QueryLevel FutureCache */
+  val queryFutureCache = new DeferCache[StepResult, Promise, Future](parseCacheConfig(config, "query."), empty = StepResult.Empty)
 
   for {
     entry <- config.entrySet() if Graph.DefaultConfigs.contains(entry.getKey)
     (k, v) = (entry.getKey, entry.getValue)
   } logger.info(s"[Initialized]: $k, ${this.config.getAnyRef(k)}")
 
-  /** select */
-  def checkEdges(params: Seq[(Vertex, Vertex, QueryParam)]): Future[StepResult] = storage.checkEdges(params)
+  def getStorage(service: Service): Storage[_, _] = {
+    storagePool.getOrElse(s"service:${service.serviceName}", defaultStorage)
+  }
+
+  def getStorage(label: Label): Storage[_, _] = {
+    storagePool.getOrElse(s"label:${label.label}", defaultStorage)
+  }
+
+  def flushStorage(): Unit = {
+    storagePool.foreach { case (_, storage) =>
+
+      /** flush is blocking */
+      storage.flush()
+    }
+  }
+
+  def fallback = Future.successful(StepResult.Empty)
+
+  def checkEdges(edges: Seq[Edge]): Future[StepResult] = {
+    val futures = for {
+      edge <- edges
+    } yield {
+      fetchSnapshotEdge(edge).map { case (queryParam, edgeOpt, kvOpt) =>
+        edgeOpt.toSeq.map(e => EdgeWithScore(e, 1.0, queryParam.label))
+      }
+    }
+
+    Future.sequence(futures).map { edgeWithScoreLs =>
+      val edgeWithScores = edgeWithScoreLs.flatten
+      StepResult(edgeWithScores = edgeWithScores, grouped = Nil, degreeEdges = Nil)
+    }
+  }
+
+  //  def checkEdges(edges: Seq[Edge]): Future[StepResult] = storage.checkEdges(edges)
+
+  def getEdges(q: Query): Future[StepResult] = {
+    Try {
+      if (q.steps.isEmpty) {
+        // TODO: this should be get vertex query.
+        fallback
+      } else {
+        val filterOutFuture = q.queryOption.filterOutQuery match {
+          case None => fallback
+          case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
+        }
+        for {
+          stepResult <- getEdgesStepInner(q)
+          filterOutInnerResult <- filterOutFuture
+        } yield {
+          if (filterOutInnerResult.isEmpty) stepResult
+          else StepResult.filterOut(this, q.queryOption, stepResult, filterOutInnerResult)
+        }
+      }
+    } recover {
+      case e: Exception =>
+        logger.error(s"getEdgesAsync: $e", e)
+        fallback
+    } get
+  }
+
+  def getEdgesStepInner(q: Query, buildLastStepInnerResult: Boolean = false): Future[StepResult] = {
+    Try {
+      if (q.steps.isEmpty) fallback
+      else {
+
+        val queryOption = q.queryOption
+        def fetch: Future[StepResult] = {
+          val startStepInnerResult = QueryResult.fromVertices(this, q)
+          q.steps.zipWithIndex.foldLeft(Future.successful(startStepInnerResult)) { case (prevStepInnerResultFuture, (step, stepIdx)) =>
+            for {
+              prevStepInnerResult <- prevStepInnerResultFuture
+              currentStepInnerResult <- fetchStep(q, stepIdx, prevStepInnerResult, buildLastStepInnerResult)
+            } yield {
+              currentStepInnerResult.copy(
+                accumulatedCursors = prevStepInnerResult.accumulatedCursors :+ currentStepInnerResult.cursors,
+                failCount = currentStepInnerResult.failCount + prevStepInnerResult.failCount
+              )
+            }
+          }
+        }
+
+        fetch
+      }
+    } recover {
+      case e: Exception =>
+        logger.error(s"getEdgesAsync: $e", e)
+        fallback
+    } get
+  }
+
+  def fetchStep(orgQuery: Query,
+                stepIdx: Int,
+                stepInnerResult: StepResult,
+                buildLastStepInnerResult: Boolean = false): Future[StepResult] = {
+    if (stepInnerResult.isEmpty) Future.successful(StepResult.Empty)
+    else {
+      val edgeWithScoreLs = stepInnerResult.edgeWithScores
+
+      val q = orgQuery
+      val queryOption = orgQuery.queryOption
+      val prevStepOpt = if (stepIdx > 0) Option(q.steps(stepIdx - 1)) else None
+      val prevStepThreshold = prevStepOpt.map(_.nextStepScoreThreshold).getOrElse(QueryParam.DefaultThreshold)
+      val prevStepLimit = prevStepOpt.map(_.nextStepLimit).getOrElse(-1)
+      val step = q.steps(stepIdx)
+
+     val alreadyVisited =
+        if (stepIdx == 0) Map.empty[(LabelWithDirection, Vertex), Boolean]
+        else alreadyVisitedVertices(stepInnerResult.edgeWithScores)
+
+      val initial = (Map.empty[Vertex, Double], Map.empty[Vertex, ArrayBuffer[EdgeWithScore]])
+      val (sums, grouped) = edgeWithScoreLs.foldLeft(initial) { case ((sum, group), edgeWithScore) =>
+        val key = edgeWithScore.edge.tgtVertex
+        val newScore = sum.getOrElse(key, 0.0) + edgeWithScore.score
+        val buffer = group.getOrElse(key, ArrayBuffer.empty[EdgeWithScore])
+        buffer += edgeWithScore
+        (sum + (key -> newScore), group + (key -> buffer))
+      }
+      val groupedByFiltered = sums.filter(t => t._2 >= prevStepThreshold)
+      val prevStepTgtVertexIdEdges = grouped.map(t => t._1.id -> t._2)
+
+      val nextStepSrcVertices = if (prevStepLimit >= 0) {
+        groupedByFiltered.toSeq.sortBy(-1 * _._2).take(prevStepLimit)
+      } else {
+        groupedByFiltered.toSeq
+      }
+
+      val queryRequests = for {
+        (vertex, prevStepScore) <- nextStepSrcVertices
+        queryParam <- step.queryParams
+      } yield {
+        val labelWeight = step.labelWeights.getOrElse(queryParam.labelWithDir.labelId, 1.0)
+        val newPrevStepScore = if (queryOption.shouldPropagateScore) prevStepScore else 1.0
+        QueryRequest(q, stepIdx, vertex, queryParam, newPrevStepScore, labelWeight)
+      }
+
+      val fetchedLs = fetches(queryRequests, prevStepTgtVertexIdEdges)
+
+      filterEdges(orgQuery, stepIdx, queryRequests,
+        fetchedLs, orgQuery.steps(stepIdx).queryParams, alreadyVisited, buildLastStepInnerResult, prevStepTgtVertexIdEdges)(ec)
+    }
+  }
+
+
+  /**
+    * responsible to fire parallel fetch call into storage and create future that will return merged result.
+    *
+    * @param queryRequests
+    * @param prevStepEdges
+    * @return
+    */
+  def fetches(queryRequests: Seq[QueryRequest],
+              prevStepEdges: Map[VertexId, Seq[EdgeWithScore]]): Future[Seq[StepResult]] = {
+
+    val reqWithIdxs = queryRequests.zipWithIndex
+    val requestsPerLabel = reqWithIdxs.groupBy(t => t._1.queryParam.label)
+    val aggFuture = requestsPerLabel.foldLeft(Future.successful(Map.empty[Int, StepResult])) { case (prevFuture, (label, reqWithIdxs)) =>
+      for {
+        prev <- prevFuture
+        cur <- getStorage(label).fetches(reqWithIdxs.map(_._1), prevStepEdges)
+      } yield {
+        prev ++ reqWithIdxs.map(_._2).zip(cur).toMap
+      }
+    }
+    aggFuture.map { agg => agg.toSeq.sortBy(_._1).map(_._2) }
+  }
+
+
+  def getEdgesMultiQuery(mq: MultiQuery): Future[StepResult] = {
+    Try {
+      if (mq.queries.isEmpty) fallback
+      else {
+        val filterOutFuture = mq.queryOption.filterOutQuery match {
+          case None => fallback
+          case Some(filterOutQuery) => getEdgesStepInner(filterOutQuery)
+        }
+
+        val multiQueryFutures = Future.sequence(mq.queries.map { query => getEdges(query) })
+        for {
+          multiQueryResults <- multiQueryFutures
+          filterOutInnerResult <- filterOutFuture
+        } yield {
+          StepResult.merges(mq.queryOption, multiQueryResults, mq.weights, filterOutInnerResult)
+        }
+      }
+    } recover {
+      case e: Exception =>
+        logger.error(s"getEdgesAsync: $e", e)
+        fallback
+    } get
+  }
+
+
+  def fetchSnapshotEdge(edge: Edge): Future[(QueryParam, Option[Edge], Option[SKeyValue])] = {
+    /** TODO: Fix this. currently fetchSnapshotEdge should not use future cache
+      * so use empty cacheKey.
+      * */
+    val queryParam = QueryParam(labelName = edge.label.label,
+      direction = GraphUtil.fromDirection(edge.labelWithDir.dir),
+      tgtVertexIdOpt = Option(edge.tgtVertex.innerIdVal),
+      cacheTTLInMillis = -1)
+    val q = Query.toQuery(Seq(edge.srcVertex), queryParam)
+    val queryRequest = QueryRequest(q, 0, edge.srcVertex, queryParam)
+
+    val storage = getStorage(edge.label)
+    storage.fetchSnapshotEdgeKeyValues(queryRequest).map { kvs =>
+      val (edgeOpt, kvOpt) =
+        if (kvs.isEmpty) (None, None)
+        else {
+          val snapshotEdgeOpt = storage.toSnapshotEdge(kvs.head, queryRequest, isInnerCall = true, parentEdges = Nil)
+          val _kvOpt = kvs.headOption
+          (snapshotEdgeOpt, _kvOpt)
+        }
+      (queryParam, edgeOpt, kvOpt)
+    } recoverWith { case ex: Throwable =>
+      logger.error(s"fetchQueryParam failed. fallback return.", ex)
+      throw FetchTimeoutException(s"${edge.toLogString}")
+    }
+  }
+
+  def getVertices(vertices: Seq[Vertex]): Future[Seq[Vertex]] = {
+    val verticesWithIdx = vertices.zipWithIndex
+    val futures = verticesWithIdx.groupBy { case (v, idx) => v.service }.map { case (service, vertexGroup) =>
+      getStorage(service).getVertices(vertexGroup.map(_._1)).map(_.zip(vertexGroup.map(_._2)))
+    }
+
+    Future.sequence(futures).map { ls =>
+      ls.flatten.toSeq.sortBy(_._2).map(_._1)
+    }
+  }
+
+  /** mutate */
+  def deleteAllAdjacentEdges(srcVertices: Seq[Vertex],
+                             labels: Seq[Label],
+                             dir: Int,
+                             ts: Long): Future[Boolean] = {
+
+    val requestTs = ts
+    val vertices = srcVertices
+    /** create query per label */
+    val queries = for {
+      label <- labels
+    } yield {
+      val queryParam = QueryParam(labelName = label.label, direction = GraphUtil.fromDirection(dir),
+        offset = 0, limit = DeleteAllFetchSize, duplicatePolicy = DuplicatePolicy.Raw)
+      val step = Step(List(queryParam))
+      Query(vertices, Vector(step))
+    }
+
+    //    Extensions.retryOnSuccessWithBackoff(MaxRetryNum, Random.nextInt(MaxBackOff) + 1) {
+    val retryFuture = Extensions.retryOnSuccess(DeleteAllFetchCount) {
+      fetchAndDeleteAll(queries, requestTs)
+    } { case (allDeleted, deleteSuccess) =>
+      allDeleted && deleteSuccess
+    }.map { case (allDeleted, deleteSuccess) => allDeleted && deleteSuccess }
+
+    retryFuture onFailure {
+      case ex =>
+        logger.error(s"[Error]: deleteAllAdjacentEdges failed.")
+    }
 
-  def getEdges(q: Query): Future[StepResult] = storage.getEdges(q)
+    retryFuture
+  }
 
-  def getEdgesMultiQuery(mq: MultiQuery): Future[StepResult] = storage.getEdgesMultiQuery(mq)
+  def fetchAndDeleteAll(queries: Seq[Query], requestTs: Long): Future[(Boolean, Boolean)] = {
+    val future = for {
+      stepInnerResultLs <- Future.sequence(queries.map(getEdgesStepInner(_, true)))
+      (allDeleted, ret) <- deleteAllFetchedEdgesLs(stepInnerResultLs, requestTs)
+    } yield {
+      //        logger.debug(s"fetchAndDeleteAll: ${allDeleted}, ${ret}")
+      (allDeleted, ret)
+    }
 
-  def getVertices(vertices: Seq[Vertex]): Future[Seq[Vertex]] = storage.getVertices(vertices)
+    Extensions.retryOnFailure(MaxRetryNum) {
+      future
+    } {
+      logger.error(s"fetch and deleteAll failed.")
+      (true, false)
+    }
 
-  /** write */
-  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]] =
-    storage.mutateElements(elements, withWait)
+  def deleteAllFetchedEdgesLs(stepInnerResultLs: Seq[StepResult],
+                              requestTs: Long): Future[(Boolean, Boolean)] = {
+    stepInnerResultLs.foreach { stepInnerResult =>
+      if (stepInnerResult.isFailure) throw new RuntimeException("fetched result is fallback.")
+    }
+    val futures = for {
+      stepInnerResult <- stepInnerResultLs
+      deleteStepInnerResult = buildEdgesToDelete(stepInnerResult, requestTs)
+      if deleteStepInnerResult.edgeWithScores.nonEmpty
+    } yield {
+      val head = deleteStepInnerResult.edgeWithScores.head
+      val label = head.edge.label
+      val ret = label.schemaVersion match {
+        case HBaseType.VERSION3 | HBaseType.VERSION4 =>
+          if (label.consistencyLevel == "strong") {
+            /**
+              * read: snapshotEdge on queryResult = O(N)
+              * write: N x (relatedEdges x indices(indexedEdge) + 1(snapshotEdge))
+              */
+            mutateEdges(deleteStepInnerResult.edgeWithScores.map(_.edge), withWait = true).map(_.forall(identity))
+          } else {
+            getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
+          }
+        case _ =>
 
-  def mutateEdges(edges: Seq[Edge], withWait: Boolean = false): Future[Seq[Boolean]] = storage.mutateEdges(edges, withWait)
+          /**
+            * read: x
+            * write: N x ((1(snapshotEdge) + 2(1 for incr, 1 for delete) x indices)
+            */
+          getStorage(label).deleteAllFetchedEdgesAsyncOld(deleteStepInnerResult, requestTs, MaxRetryNum)
+      }
+      ret
+    }
 
-  def mutateVertices(vertices: Seq[Vertex], withWait: Boolean = false): Future[Seq[Boolean]] = storage.mutateVertices(vertices, withWait)
+    if (futures.isEmpty) {
+      // all deleted.
+      Future.successful(true -> true)
+    } else {
+      Future.sequence(futures).map { rets => false -> rets.forall(identity) }
+    }
+  }
 
-  def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long)]] = storage.incrementCounts(edges, withWait)
+  def buildEdgesToDelete(stepInnerResult: StepResult, requestTs: Long): StepResult = {
+    val filtered = stepInnerResult.edgeWithScores.filter { edgeWithScore =>
+      (edgeWithScore.edge.ts < requestTs) && !edgeWithScore.edge.isDegree
+    }
+    if (filtered.isEmpty) StepResult.Empty
+    else {
+      val head = filtered.head
+      val label = head.edge.label
+      val edgeWithScoreLs = filtered.map { edgeWithScore =>
+        val (newOp, newVersion, newPropsWithTs) = label.consistencyLevel match {
+          case "strong" =>
+            val _newPropsWithTs = edgeWithScore.edge.propsWithTs ++
+                Map(LabelMeta.timestamp -> InnerValLikeWithTs.withLong(requestTs, requestTs, label.schemaVersion))
+            (GraphUtil.operations("delete"), requestTs, _newPropsWithTs)
+          case _ =>
+            val oldEdge = edgeWithScore.edge
+            (oldEdge.op, oldEdge.version, oldEdge.propsWithTs)
+        }
+
+        val copiedEdge =
+          edgeWithScore.edge.copy(op = newOp, version = newVersion, propsWithTs = newPropsWithTs)
+
+        val edgeToDelete = edgeWithScore.copy(edge = copiedEdge)
+        //      logger.debug(s"delete edge from deleteAll: ${edgeToDelete.edge.toLogString}")
+        edgeToDelete
+      }
+      //Degree edge?
+      StepResult(edgeWithScores = edgeWithScoreLs, grouped = Nil, degreeEdges = Nil, false)
+    }
+  }
+
+  //  def deleteAllAdjacentEdges(srcVertices: List[Vertex], labels: Seq[Label], dir: Int, ts: Long): Future[Boolean] =
+  //    storage.deleteAllAdjacentEdges(srcVertices, labels, dir, ts)
+
+  def mutateElements(elements: Seq[GraphElement],
+                     withWait: Boolean = false): Future[Seq[Boolean]] = {
+
+    val edgeBuffer = ArrayBuffer[(Edge, Int)]()
+    val vertexBuffer = ArrayBuffer[(Vertex, Int)]()
+
+    elements.zipWithIndex.foreach {
+      case (e: Edge, idx: Int) => edgeBuffer.append((e, idx))
+      case (v: Vertex, idx: Int) => vertexBuffer.append((v, idx))
+      case any@_ => logger.error(s"Unknown type: ${any}")
+    }
+
+    val edgeFutureWithIdx = mutateEdges(edgeBuffer.map(_._1), withWait).map { result =>
+      edgeBuffer.map(_._2).zip(result)
+    }
+
+    val vertexFutureWithIdx = mutateVertices(vertexBuffer.map(_._1), withWait).map { result =>
+      vertexBuffer.map(_._2).zip(result)
+    }
+
+    val graphFuture = for {
+      edgesMutated <- edgeFutureWithIdx
+      verticesMutated <- vertexFutureWithIdx
+    } yield (edgesMutated ++ verticesMutated).sortBy(_._1).map(_._2)
+
+    graphFuture
+
+  }
+
+  //  def mutateEdges(edges: Seq[Edge], withWait: Boolean = false): Future[Seq[Boolean]] = storage.mutateEdges(edges, withWait)
+
+  def mutateEdges(edges: Seq[Edge], withWait: Boolean = false): Future[Seq[Boolean]] = {
+    val edgeWithIdxs = edges.zipWithIndex
+
+    val (strongEdges, weakEdges) =
+      edgeWithIdxs.partition { case (edge, idx) =>
+        val e = edge
+        e.label.consistencyLevel == "strong" && e.op != GraphUtil.operations("insertBulk")
+      }
+
+    val weakEdgesFutures = weakEdges.groupBy { case (edge, idx) => edge.label.hbaseZkAddr }.map { case (zkQuorum, edgeWithIdxs) =>
+      val futures = edgeWithIdxs.groupBy(_._1.label).map { case (label, edgeGroup) =>
+        val storage = getStorage(label)
+        val edges = edgeGroup.map(_._1)
+        val idxs = edgeGroup.map(_._2)
+
+        /** multiple edges with weak consistency level will be processed as batch */
+        val mutations = edges.flatMap { edge =>
+          val (_, edgeUpdate) =
+            if (edge.op == GraphUtil.operations("delete")) Edge.buildDeleteBulk(None, edge)
+            else Edge.buildOperation(None, Seq(edge))
+
+          storage.buildVertexPutsAsync(edge) ++ storage.indexedEdgeMutations(edgeUpdate) ++ storage.snapshotEdgeMutations(edgeUpdate) ++ storage.increments(edgeUpdate)
+        }
+
+        storage.writeToStorage(zkQuorum, mutations, withWait).map { ret =>
+          idxs.map(idx => idx -> ret)
+        }
+      }
+      Future.sequence(futures)
+    }
+    val (strongDeleteAll, strongEdgesAll) = strongEdges.partition { case (edge, idx) => edge.op == GraphUtil.operations("deleteAll") }
+
+    val deleteAllFutures = strongDeleteAll.map { case (edge, idx) =>
+      deleteAllAdjacentEdges(Seq(edge.srcVertex), Seq(edge.label), edge.labelWithDir.dir, edge.ts).map(idx -> _)
+    }
+
+    val strongEdgesFutures = strongEdgesAll.groupBy { case (edge, idx) => edge.label }.map { case (label, edgeGroup) =>
+      val edges = edgeGroup.map(_._1)
+      val idxs = edgeGroup.map(_._2)
+      val storage = getStorage(label)
+      storage.mutateStrongEdges(edges, withWait = true).map { rets =>
+        idxs.zip(rets)
+      }
+    }
+
+    for {
+      weak <- Future.sequence(weakEdgesFutures)
+      deleteAll <- Future.sequence(deleteAllFutures)
+      strong <- Future.sequence(strongEdgesFutures)
+    } yield {
+      (deleteAll ++ weak.flatten.flatten ++ strong.flatten).sortBy(_._1).map(_._2)
+    }
+  }
+
+  def mutateVertices(vertices: Seq[Vertex], withWait: Boolean = false): Future[Seq[Boolean]] = {
+    val verticesWithIdx = vertices.zipWithIndex
+    val futures = verticesWithIdx.groupBy { case (v, idx) => v.service }.map { case (service, vertexGroup) =>
+      getStorage(service).mutateVertices(vertexGroup.map(_._1), withWait).map(_.zip(vertexGroup.map(_._2)))
+    }
+    Future.sequence(futures).map { ls => ls.flatten.toSeq.sortBy(_._2).map(_._1) }
+  }
+
+  def incrementCounts(edges: Seq[Edge], withWait: Boolean): Future[Seq[(Boolean, Long, Long)]] = {
+    val edgesWithIdx = edges.zipWithIndex
+    val futures = edgesWithIdx.groupBy { case (e, idx) => e.label }.map { case (label, edgeGroup) =>
+      getStorage(label).incrementCounts(edgeGroup.map(_._1), withWait).map(_.zip(edgeGroup.map(_._2)))
+    }
+    Future.sequence(futures).map { ls => ls.flatten.toSeq.sortBy(_._2).map(_._1) }
+  }
+
+  def updateDegree(edge: Edge, degreeVal: Long = 0): Future[Boolean] = {
+    val label = edge.label
+
+    val storage = getStorage(label)
+    val kvs = storage.buildDegreePuts(edge, degreeVal)
+
+    storage.writeToStorage(edge.label.service.cluster, kvs, withWait = true)
+  }
 
   def shutdown(): Unit = {
-    storage.flush()
+    flushStorage()
     Model.shutdown()
   }
+
+  def addEdge(srcId: Any,
+              tgtId: Any,
+              labelName: String,
+              direction: String = "out",
+              props: Map[String, Any] = Map.empty,
+              ts: Long = System.currentTimeMillis(),
+              operation: String = "insert",
+              withWait: Boolean = true): Future[Boolean] = {
+
+    val innerEdges = Seq(Edge.toEdge(srcId, tgtId, labelName, direction, props.toMap, ts, operation))
+    mutateEdges(innerEdges, withWait).map(_.headOption.getOrElse(false))
+  }
+
+  def addVertex(serviceName: String,
+                columnName: String,
+                id: Any,
+                props: Map[String, Any] = Map.empty,
+                ts: Long = System.currentTimeMillis(),
+                operation: String = "insert",
+                withWait: Boolean = true): Future[Boolean] = {
+    val innerVertices = Seq(Vertex.toVertex(serviceName, columnName, id, props.toMap, ts, operation))
+    mutateVertices(innerVertices, withWait).map(_.headOption.getOrElse(false))
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/GraphExceptions.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/GraphExceptions.scala b/s2core/src/main/scala/org/apache/s2graph/core/GraphExceptions.scala
index 2f090cf..2cc1063 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/GraphExceptions.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/GraphExceptions.scala
@@ -20,6 +20,24 @@
 package org.apache.s2graph.core
 
 object GraphExceptions {
+  var fillStckTrace = true
+  class BaseException(msg : String) extends Exception(msg){
+    override def fillInStackTrace : Exception = {
+      if(fillStckTrace) super.fillInStackTrace()
+      this
+    }
+  }
+  class NoStackException(msg : String) extends Exception(msg){
+    override def fillInStackTrace : Exception = {
+      this
+    }
+  }
+
+  class NoStackCauseException(msg : String, ex: Throwable ) extends Exception(msg, ex){
+    override def fillInStackTrace : Exception = {
+      this
+    }
+  }
 
   case class JsonParseException(msg: String) extends Exception(msg)
 
@@ -43,7 +61,11 @@ object GraphExceptions {
 
   case class InvalidHTableException(msg: String) extends Exception(msg)
 
-  case class FetchTimeoutException(msg: String) extends Exception(msg)
+  case class FetchTimeoutException(msg: String) extends NoStackException(msg)
 
   case class DropRequestException(msg: String) extends Exception(msg)
+
+  case class FetchAllStepFailException(msg: String) extends Exception(msg)
+
+  case class AccessDeniedException(amsg: String) extends Exception(amsg)
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/GraphUtil.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/GraphUtil.scala b/s2core/src/main/scala/org/apache/s2graph/core/GraphUtil.scala
index ebfee7a..939b596 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/GraphUtil.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/GraphUtil.scala
@@ -50,8 +50,8 @@ object GraphUtil {
     val d = direction.trim().toLowerCase match {
       case "directed" | "d" => Some(0)
       case "undirected" | "u" => Some(2)
-      case "out" => Some(0)
-      case "in" => Some(1)
+      case "out" | "o" => Some(0)
+      case "in" | "i" => Some(1)
       case _ => None
     }
     d.map(x => x.toByte)
@@ -61,8 +61,8 @@ object GraphUtil {
     direction.trim().toLowerCase match {
       case "directed" | "d" => 0
       case "undirected" | "u" => 2
-      case "out" => 0
-      case "in" => 1
+      case "out" | "o" => 0
+      case "in" | "i" => 1
       case _ => 2
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/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 9ef7c14..89acc63 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/Management.scala
@@ -25,6 +25,7 @@ import org.apache.s2graph.core.mysqls._
 import org.apache.s2graph.core.types.HBaseType._
 import org.apache.s2graph.core.types._
 import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core.utils.logger
 import play.api.libs.json.Reads._
 import play.api.libs.json._
 
@@ -104,8 +105,8 @@ object Management {
     }
   }
 
-  def findLabel(labelName: String): Option[Label] = {
-    Label.findByName(labelName, useCache = false)
+  def findLabel(labelName: String, useCache: Boolean = false): Option[Label] = {
+    Label.findByName(labelName, useCache = useCache)
   }
 
   def deleteLabel(labelName: String) = {
@@ -117,6 +118,16 @@ object Management {
     }
   }
 
+  def markDeletedLabel(labelName: String) = {
+    Model withTx { implicit session =>
+      Label.findByName(labelName, useCache = false).foreach { label =>
+        // rename & delete_at column filled with current time
+        Label.markDeleted(label)
+      }
+      labelName
+    }
+  }
+
   def addIndex(labelStr: String, indices: Seq[Index]): Try[Label] = {
     Model withTx { implicit session =>
       val label = Label.findByName(labelStr).getOrElse(throw LabelNotExistException(s"$labelStr not found"))
@@ -205,12 +216,12 @@ object Management {
 
   }
 
-  def toProps(label: Label, js: Seq[(String, JsValue)]): Seq[(Byte, InnerValLike)] = {
+  def toProps(label: Label, js: Seq[(String, JsValue)]): Seq[(LabelMeta, InnerValLike)] = {
     val props = for {
       (k, v) <- js
       meta <- label.metaPropsInvMap.get(k)
       innerVal <- jsValueToInnerVal(v, meta.dataType, label.schemaVersion)
-    } yield (meta.seq, innerVal)
+    } yield (meta, innerVal)
 
     props
   }
@@ -248,15 +259,18 @@ object Management {
 
 class Management(graph: Graph) {
   import Management._
-  val storage = graph.storage
 
-  def createTable(zkAddr: String,
+  def createStorageTable(zkAddr: String,
                   tableName: String,
                   cfs: List[String],
                   regionMultiplier: Int,
                   ttl: Option[Int],
-                  compressionAlgorithm: String = DefaultCompressionAlgorithm): Unit =
-    storage.createTable(zkAddr, tableName, cfs, regionMultiplier, ttl, compressionAlgorithm)
+                  compressionAlgorithm: String = DefaultCompressionAlgorithm,
+                  replicationScopeOpt: Option[Int] = None,
+                  totalRegionCount: Option[Int] = None): Unit = {
+    graph.defaultStorage.createTable(zkAddr, tableName, cfs, regionMultiplier, ttl, compressionAlgorithm, replicationScopeOpt, totalRegionCount)
+  }
+
 
   /** HBase specific code */
   def createService(serviceName: String,
@@ -265,9 +279,9 @@ class Management(graph: Graph) {
                     compressionAlgorithm: String = DefaultCompressionAlgorithm): Try[Service] = {
 
     Model withTx { implicit session =>
-      val service = Service.findOrInsert(serviceName, cluster, hTableName, preSplitSize, hTableTTL, compressionAlgorithm)
-      /* create hbase table for service */
-      storage.createTable(cluster, hTableName, List("e", "v"), preSplitSize, hTableTTL, compressionAlgorithm)
+      val service = Service.findOrInsert(serviceName, cluster, hTableName, preSplitSize, hTableTTL.orElse(Some(Integer.MAX_VALUE)), compressionAlgorithm)
+      /** create hbase table for service */
+      graph.getStorage(service).createTable(service.cluster, service.hTableName, List("e", "v"), service.preSplitSize, service.hTableTTL, compressionAlgorithm)
       service
     }
   }
@@ -292,35 +306,26 @@ class Management(graph: Graph) {
                   compressionAlgorithm: String = "gz",
                   options: Option[String]): Try[Label] = {
 
-    val labelOpt = Label.findByName(label, useCache = false)
+    if (label.length > LABEL_NAME_MAX_LENGTH ) throw new LabelNameTooLongException(s"Label name ${label} too long.( max length : ${LABEL_NAME_MAX_LENGTH}} )")
+    if (hTableName.isEmpty && hTableTTL.isDefined) throw new RuntimeException("if want to specify ttl, give hbaseTableName also")
 
+    val labelOpt = Label.findByName(label, useCache = false)
     Model withTx { implicit session =>
-      labelOpt match {
-        case Some(l) =>
-          throw new GraphExceptions.LabelAlreadyExistException(s"Label name ${l.label} already exist.")
-        case None =>
-          /** create all models */
-          if (label.length > LABEL_NAME_MAX_LENGTH ) throw new LabelNameTooLongException(s"Label name ${label} too long.( max length : 40 )")
-          val newLabel = Label.insertAll(label,
-            srcServiceName, srcColumnName, srcColumnType,
-            tgtServiceName, tgtColumnName, tgtColumnType,
-            isDirected, serviceName, indices, props, consistencyLevel,
-            hTableName, hTableTTL, schemaVersion, isAsync, compressionAlgorithm, options)
-
-          /* create hbase table */
-          val service = newLabel.service
-          (hTableName, hTableTTL) match {
-            case (None, None) => // do nothing
-            case (None, Some(hbaseTableTTL)) => throw new RuntimeException("if want to specify ttl, give hbaseTableName also")
-            case (Some(hbaseTableName), None) =>
-              // create own hbase table with default ttl on service level.
-              storage.createTable(service.cluster, hbaseTableName, List("e", "v"), service.preSplitSize, service.hTableTTL, compressionAlgorithm)
-            case (Some(hbaseTableName), Some(hbaseTableTTL)) =>
-              // create own hbase table with own ttl.
-              storage.createTable(service.cluster, hbaseTableName, List("e", "v"), service.preSplitSize, hTableTTL, compressionAlgorithm)
-          }
-          newLabel
-      }
+      if (labelOpt.isDefined) throw new LabelAlreadyExistException(s"Label name ${label} already exist.")
+
+      /** create all models */
+      val newLabel = Label.insertAll(label,
+        srcServiceName, srcColumnName, srcColumnType,
+        tgtServiceName, tgtColumnName, tgtColumnType,
+        isDirected, serviceName, indices, props, consistencyLevel,
+        hTableName, hTableTTL, schemaVersion, isAsync, compressionAlgorithm, options)
+
+      /** 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)
+
+      newLabel
     }
   }
 
@@ -331,12 +336,11 @@ class Management(graph: Graph) {
    * copy label when if oldLabel exist and newLabel do not exist.
    * copy label: only used by bulk load job. not sure if we need to parameterize hbase cluster.
    */
-  def copyLabel(oldLabelName: String, newLabelName: String, hTableName: Option[String]) = {
+  def copyLabel(oldLabelName: String, newLabelName: String, hTableName: Option[String]): Try[Label] = {
     val old = Label.findByName(oldLabelName, useCache = false).getOrElse(throw new LabelNotExistException(s"Old label $oldLabelName not exists."))
-    if (Label.findByName(newLabelName, useCache = false).isDefined) throw new LabelAlreadyExistException(s"New label $newLabelName already exists.")
 
-    val allProps = old.metas.map { labelMeta => Prop(labelMeta.name, labelMeta.defaultValue, labelMeta.dataType) }
-    val allIndices = old.indices.map { index => Index(index.name, index.propNames) }
+    val allProps = old.metas(useCache = false).map { labelMeta => Prop(labelMeta.name, labelMeta.defaultValue, labelMeta.dataType) }
+    val allIndices = old.indices(useCache = false).map { index => Index(index.name, index.propNames) }
 
     createLabel(newLabelName, old.srcService.serviceName, old.srcColumnName, old.srcColumnType,
       old.tgtService.serviceName, old.tgtColumnName, old.tgtColumnType,
@@ -344,4 +348,13 @@ class Management(graph: Graph) {
       allIndices, allProps,
       old.consistencyLevel, hTableName, old.hTableTTL, old.schemaVersion, old.isAsync, old.compressionAlgorithm, old.options)
   }
+
+  def getCurrentStorageInfo(labelName: String): Try[Map[String, String]] = for {
+    label <- Try(Label.findByName(labelName, useCache = false).get)
+  } yield {
+    val storage = graph.getStorage(label)
+    storage.info
+  }
+
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/66bdf1bc/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala b/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
index 7cc2420..6c8563c 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/PostProcess.scala
@@ -19,12 +19,18 @@
 
 package org.apache.s2graph.core
 
-import org.apache.s2graph.core.GraphExceptions.BadQueryException
+import java.util.Base64
+
+import com.google.protobuf.ByteString
+import org.apache.s2graph.core.GraphExceptions.{BadQueryException, LabelNotExistException}
 import org.apache.s2graph.core.mysqls._
-import org.apache.s2graph.core.types.{InnerVal, InnerValLike}
+import org.apache.s2graph.core.types.{InnerVal, InnerValLike, InnerValLikeWithTs}
 import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core.rest.RequestParser
+import org.apache.s2graph.core.utils.logger
 import play.api.libs.json.{Json, _}
 
+import scala.collection.JavaConversions._
 import scala.collection.immutable
 import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 
@@ -40,74 +46,100 @@ object PostProcess {
    * Result Entity score field name
    */
   val emptyDegrees = Seq.empty[JsValue]
-  val timeoutResults = Json.obj("size" -> 0, "degrees" -> Json.arr(), "results" -> Json.arr(), "isTimeout" -> true)
-  val emptyResults = Json.obj("size" -> 0, "degrees" -> Json.arr(), "results" -> Json.arr(), "isEmpty" -> true)
+  val emptyResults = Json.obj("size" -> 0, "degrees" -> Json.arr(), "results" -> Json.arr(), "isEmpty" -> true, "rpcFail" -> 0)
+
   def badRequestResults(ex: => Exception) = ex match {
     case ex: BadQueryException => Json.obj("message" -> ex.msg)
     case _ => Json.obj("message" -> ex.getMessage)
   }
 
-  val SCORE_FIELD_NAME = "scoreSum"
-  val reservedColumns = Set("cacheRemain", "from", "to", "label", "direction", "_timestamp", "timestamp", "score", "props")
-
-
   def s2EdgeParent(graph: Graph,
+                   queryOption: QueryOption,
                    parentEdges: Seq[EdgeWithScore]): JsValue = {
     if (parentEdges.isEmpty) JsNull
     else {
       val ancestors = for {
         current <- parentEdges
-        parents = s2EdgeParent(graph, current.edge.parentEdges) if parents != JsNull
+        parents = s2EdgeParent(graph, queryOption, current.edge.parentEdges) if parents != JsNull
       } yield {
           val s2Edge = current.edge.originalEdgeOpt.getOrElse(current.edge)
-          s2EdgeToJsValue(s2Edge, current.score, false, parents = parents)
+          s2EdgeToJsValue(queryOption, current.copy(edge = s2Edge), false, parents = parents, checkSelectColumns = true)
         }
       Json.toJson(ancestors)
     }
   }
 
-  def s2EdgeToJsValue(s2Edge: Edge,
-                      score: Double,
+  def s2EdgeToJsValue(queryOption: QueryOption,
+                      edgeWithScore: EdgeWithScore,
                       isDegree: Boolean = false,
-                      parents: JsValue = JsNull): JsValue = {
+                      parents: JsValue = JsNull,
+                      checkSelectColumns: Boolean = false): JsValue = {
+    //    val builder = immutable.Map.newBuilder[String, JsValue]
+    val builder = ArrayBuffer.empty[(String, JsValue)]
+    val s2Edge = edgeWithScore.edge
+    val score = edgeWithScore.score
+    val label = edgeWithScore.label
     if (isDegree) {
-      Json.obj(
-        "from" -> anyValToJsValue(s2Edge.srcId),
-        "label" -> s2Edge.labelName,
-        LabelMeta.degree.name -> anyValToJsValue(s2Edge.propsWithTs(LabelMeta.degreeSeq).innerVal.value)
-      )
+      builder += ("from" -> anyValToJsValue(s2Edge.srcId).get)
+      builder += ("label" -> anyValToJsValue(label.label).get)
+      builder += ("direction" -> anyValToJsValue(s2Edge.direction).get)
+      builder += (LabelMeta.degree.name -> anyValToJsValue(s2Edge.propsWithTs(LabelMeta.degree).innerVal.value).get)
+      JsObject(builder)
     } else {
-      Json.obj("from" -> anyValToJsValue(s2Edge.srcId),
-        "to" -> anyValToJsValue(s2Edge.tgtId),
-        "label" -> s2Edge.labelName,
-        "score" -> score,
-        "props" -> JSONParser.propertiesToJson(s2Edge.properties),
-        "direction" -> s2Edge.direction,
-        "timestamp" -> anyValToJsValue(s2Edge.ts),
-        "parents" -> parents
-      )
-    }
-  }
+      if (queryOption.withScore) builder += ("score" -> anyValToJsValue(score).get)
 
-  def withImpressionId(queryOption: QueryOption,
-                        size: Int,
-                        degrees: Seq[JsValue],
-                        results: Seq[JsValue]): JsValue = {
-    queryOption.impIdOpt match {
-      case None => Json.obj(
-        "size" -> size,
-        "degrees" -> degrees,
-        "results" -> results
-      )
-      case Some(impId) =>
-        Json.obj(
-          "size" -> size,
-          "degrees" -> degrees,
-          "results" -> results,
-          Experiment.ImpressionKey -> impId
-        )
+      if (queryOption.selectColumns.isEmpty) {
+        builder += ("from" -> anyValToJsValue(s2Edge.srcId).get)
+        builder += ("to" -> anyValToJsValue(s2Edge.tgtId).get)
+        builder += ("label" -> anyValToJsValue(label.label).get)
+
+        val innerProps = ArrayBuffer.empty[(String, JsValue)]
+        for {
+          (labelMeta, v) <- edgeWithScore.edge.propsWithTs
+          jsValue <- anyValToJsValue(v.innerVal.value)
+        } {
+          innerProps += (labelMeta.name -> jsValue)
+        }
+
+
+        builder += ("props" -> JsObject(innerProps))
+        builder += ("direction" -> anyValToJsValue(s2Edge.direction).get)
+        builder += ("timestamp" -> anyValToJsValue(s2Edge.tsInnerVal).get)
+        builder += ("_timestamp" -> anyValToJsValue(s2Edge.tsInnerVal).get) // backward compatibility
+        if (parents != JsNull) builder += ("parents" -> parents)
+        //          Json.toJson(builder.result())
+        JsObject(builder)
+      } else {
+        queryOption.selectColumnsMap.foreach { case (columnName, _) =>
+          columnName match {
+            case "from" => builder += ("from" -> anyValToJsValue(s2Edge.srcId).get)
+            case "_from" => builder += ("_from" -> anyValToJsValue(s2Edge.srcId).get)
+            case "to" => builder += ("to" -> anyValToJsValue(s2Edge.tgtId).get)
+            case "_to" => builder += ("_to" -> anyValToJsValue(s2Edge.tgtId).get)
+            case "label" => builder += ("label" -> anyValToJsValue(label.label).get)
+            case "direction" => builder += ("direction" -> anyValToJsValue(s2Edge.direction).get)
+            case "timestamp" => builder += ("timestamp" -> anyValToJsValue(s2Edge.tsInnerVal).get)
+            case "_timestamp" => builder += ("_timestamp" -> anyValToJsValue(s2Edge.tsInnerVal).get)
+            case _ => // should not happen
+
+          }
+        }
+        val innerProps = ArrayBuffer.empty[(String, JsValue)]
+        for {
+          (labelMeta, v) <- edgeWithScore.edge.propsWithTs
+          if !checkSelectColumns || queryOption.selectColumnsMap.contains(labelMeta.name)
+          jsValue <- anyValToJsValue(v.innerVal.value)
+        } {
+          innerProps += (labelMeta.name -> jsValue)
+        }
+
+        builder += ("props" -> JsObject(innerProps))
+        if (parents != JsNull) builder += ("parents" -> parents)
+        JsObject(builder)
+      }
     }
   }
+
   def s2VertexToJson(s2Vertex: Vertex): Option[JsValue] = {
     val props = for {
       (k, v) <- s2Vertex.properties
@@ -140,6 +172,7 @@ object PostProcess {
 
     val kvs = new ArrayBuffer[(String, JsValue)]()
 
+
     kvs.append("size" -> JsNumber(size))
     kvs.append("degrees" -> JsArray(degrees))
     kvs.append("results" -> JsArray(results))
@@ -149,28 +182,71 @@ object PostProcess {
     JsObject(kvs)
   }
 
-  def toJson(graph: Graph,
-             queryOption: QueryOption,
-             stepResult: StepResult): JsValue = {
+  def buildJsonWith(js: JsValue)(implicit fn: (String, JsValue) => JsValue): JsValue = js match {
+    case JsObject(obj) => JsObject(obj.map { case (k, v) => k -> buildJsonWith(fn(k, v)) })
+    case JsArray(arr) => JsArray(arr.map(buildJsonWith(_)))
+    case _ => js
+  }
+
+  def toJson(orgQuery: Option[JsValue])(graph: Graph,
+                                        queryOption: QueryOption,
+                                        stepResult: StepResult): JsValue = {
+
+    // [[cursor, cursor], [cursor]]
+    lazy val cursors: Seq[Seq[String]] = stepResult.accumulatedCursors.map { stepCursors =>
+      stepCursors.map { cursor => new String(Base64.getEncoder.encode(cursor)) }
+    }
+
+    lazy val cursorJson: JsValue = Json.toJson(cursors)
+
+    // build nextQuery with (original query + cursors)
+    lazy val nextQuery: Option[JsValue] = {
+      if (cursors.exists { stepCursors => stepCursors.exists(_ != "") }) {
+        val cursorIter = cursors.iterator
 
+        orgQuery.map { query =>
+          buildJsonWith(query) { (key, js) =>
+            if (key == "step") {
+              val currentCursor = cursorIter.next
+              val res = js.as[Seq[JsObject]].toStream.zip(currentCursor).filterNot(_._2 == "").map { case (obj, cursor) =>
+                val label = (obj \ "label").as[String]
+                if (Label.findByName(label).get.schemaVersion == "v4") obj + ("cursor" -> JsString(cursor))
+                else {
+                  val limit = (obj \ "limit").asOpt[Int].getOrElse(RequestParser.defaultLimit)
+                  val offset = (obj \ "offset").asOpt[Int].getOrElse(0)
+                  obj + ("offset" -> JsNumber(offset + limit))
+                }
+              }
 
+              JsArray(res)
+            } else js
+          }
+        }
+      } else Option(JsNull)
+    }
+
+    val limitOpt = queryOption.limitOpt
+    val selectColumns = queryOption.selectColumnsMap
     val degrees =
-      if (queryOption.returnDegree) stepResult.degreeEdges.map(t => s2EdgeToJsValue(t.s2Edge, t.score, true))
+      if (queryOption.returnDegree) stepResult.degreeEdges.map(t => s2EdgeToJsValue(queryOption, t, true, JsNull))
       else emptyDegrees
 
     if (queryOption.groupBy.keys.isEmpty) {
       // no group by specified on query.
+      val results = if (limitOpt.isDefined) stepResult.edgeWithScores.take(limitOpt.get) else stepResult.edgeWithScores
+      val ls = results.map { t =>
+        val parents = if (queryOption.returnTree) s2EdgeParent(graph, queryOption, t.edge.parentEdges) else JsNull
 
-      val ls = stepResult.results.map { t =>
-        val parents = if (queryOption.returnTree) s2EdgeParent(graph, t.parentEdges) else JsNull
-        s2EdgeToJsValue(t.s2Edge, t.score, false, parents)
+        s2EdgeToJsValue(queryOption, t, false, parents)
       }
-      withImpressionId(queryOption, ls.size, degrees, ls)
+
+      withOptionalFields(queryOption, ls.size, degrees, ls, stepResult.failCount, cursorJson, nextQuery)
     } else {
 
+      val grouped = if (limitOpt.isDefined) stepResult.grouped.take(limitOpt.get) else stepResult.grouped
       val results =
         for {
-          (groupByValues, (scoreSum, edges)) <- stepResult.grouped
+          (groupByValues, (scoreSum, edges)) <- grouped
         } yield {
           val groupByKeyValues = queryOption.groupBy.keys.zip(groupByValues).map { case (k, valueOpt) =>
             k -> valueOpt.flatMap(anyValToJsValue).getOrElse(JsNull)
@@ -185,8 +261,8 @@ object PostProcess {
             )
           } else {
             val agg = edges.map { t =>
-              val parents = if (queryOption.returnTree) s2EdgeParent(graph, t.parentEdges) else JsNull
-              s2EdgeToJsValue(t.s2Edge, t.score, false, parents)
+              val parents = if (queryOption.returnTree) s2EdgeParent(graph, queryOption, t.edge.parentEdges) else JsNull
+              s2EdgeToJsValue(queryOption, t, false, parents)
             }
             val aggJson = Json.toJson(agg)
             Json.obj(
@@ -196,7 +272,8 @@ object PostProcess {
             )
           }
         }
-      withImpressionId(queryOption, results.size, degrees, results)
+
+      withOptionalFields(queryOption, results.size, degrees, results, stepResult.failCount, cursorJson, nextQuery)
     }
   }
-}
+}
\ No newline at end of file