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 2018/06/21 01:20:33 UTC

[1/7] incubator-s2graph git commit: Refactor WhereParser to accept GraphElement, not Edge only.

Repository: incubator-s2graph
Updated Branches:
  refs/heads/master 2083d136e -> 558264490


Refactor WhereParser to accept GraphElement, not Edge only.


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

Branch: refs/heads/master
Commit: 3b692fbead9accd6f9d0d9764800e15e945c90bb
Parents: 32eb344
Author: DO YUNG YOON <st...@apache.org>
Authored: Fri Jun 15 14:59:13 2018 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Fri Jun 15 14:59:13 2018 +0900

----------------------------------------------------------------------
 .../s2graph/core/S2EdgePropertyHelper.scala     |   2 +-
 .../s2graph/core/parsers/WhereParser.scala      | 102 +++++++++++--------
 .../s2graph/core/parsers/WhereParserTest.scala  |  22 +++-
 3 files changed, 81 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/3b692fbe/s2core/src/main/scala/org/apache/s2graph/core/S2EdgePropertyHelper.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2EdgePropertyHelper.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2EdgePropertyHelper.scala
index 8c609a0..03d9d4c 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2EdgePropertyHelper.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2EdgePropertyHelper.scala
@@ -55,7 +55,7 @@ object S2EdgePropertyHelper {
       case "label" => Option(InnerValLikeWithTs(InnerVal.withStr(e.innerLabel.label, e.innerLabel.schemaVersion), e.ts))
       case "direction" => Option(InnerValLikeWithTs(InnerVal.withStr(e.getDirection(), e.innerLabel.schemaVersion), e.ts))
       case _ =>
-        e.innerLabel.metaPropsInvMap.get(key).map(labelMeta => e.propertyValueInner(labelMeta))
+        e.innerLabel.metaPropsInvMap.get(key).map(labelMeta => propertyValueInner(e, labelMeta))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/3b692fbe/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 75e9657..f0d42b2 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,10 @@
 
 package org.apache.s2graph.core.parsers
 
-import org.apache.s2graph.core.GraphExceptions.{LabelNotExistException, WhereParserException}
-import org.apache.s2graph.core.schema.{Label, LabelMeta}
-import org.apache.s2graph.core.types.InnerValLike
-import org.apache.s2graph.core.{S2EdgeLike}
+import org.apache.s2graph.core.GraphExceptions.WhereParserException
 import org.apache.s2graph.core.JSONParser._
+import org.apache.s2graph.core._
+import org.apache.s2graph.core.types.InnerValLike
 
 import scala.annotation.tailrec
 import scala.util.Try
@@ -32,21 +31,40 @@ import scala.util.parsing.combinator.JavaTokenParsers
 trait ExtractValue {
   val parent = "_parent."
 
-  def propToInnerVal(edge: S2EdgeLike, key: String) = {
-    val (propKey, parentEdge) = findParentEdge(edge, key)
+  private def throwEx(key: String) = {
+    throw WhereParserException(s"Where clause contains not existing property name: $key")
+  }
 
-    val label = parentEdge.innerLabel
-    val metaPropInvMap = label.metaPropsInvMap
-    val labelMeta = metaPropInvMap.getOrElse(propKey, throw WhereParserException(s"Where clause contains not existing property name: $propKey"))
+  def propToInnerVal(element: GraphElement, key: String): InnerValLike = {
+    element match {
+      case e: S2EdgeLike => edgePropToInnerVal(e, key)
+      case v: S2VertexLike => vertexPropToInnerVal(v, key)
+      case _ => throw new IllegalArgumentException("only S2EdgeLike/S2VertexLike supported.")
+    }
+  }
 
-    labelMeta match {
-      case LabelMeta.from => parentEdge.srcVertex.innerId
-      case LabelMeta.to => parentEdge.tgtVertex.innerId
-      case _ => parentEdge.propertyValueInner(labelMeta).innerVal
+  def valueToCompare(element: GraphElement, key: String, value: String): InnerValLike = {
+    element match {
+      case e: S2EdgeLike => edgeValueToCompare(e, key, value)
+      case v: S2VertexLike => vertexValueToCompare(v, key, value)
+      case _ => throw new IllegalArgumentException("only S2EdgeLike/S2VertexLike supported.")
     }
   }
 
-  def valueToCompare(edge: S2EdgeLike, key: String, value: String) = {
+  private def edgePropToInnerVal(edge: S2EdgeLike, key: String): InnerValLike = {
+    val (propKey, parentEdge) = findParentEdge(edge, key)
+
+    val innerValLikeWithTs =
+      S2EdgePropertyHelper.propertyValue(parentEdge, propKey).getOrElse(throwEx(propKey))
+
+    innerValLikeWithTs.innerVal
+  }
+
+  private def vertexPropToInnerVal(vertex: S2VertexLike, key: String): InnerValLike = {
+    S2VertexPropertyHelper.propertyValue(vertex, key).getOrElse(throwEx(key))
+  }
+
+  private def edgeValueToCompare(edge: S2EdgeLike, key: String, value: String): InnerValLike = {
     val label = edge.innerLabel
     if (value.startsWith(parent) || label.metaPropsInvMap.contains(value)) propToInnerVal(edge, value)
     else {
@@ -63,6 +81,12 @@ trait ExtractValue {
     }
   }
 
+  private def vertexValueToCompare(vertex: S2VertexLike, key: String, value: String): InnerValLike = {
+    val columnMeta = vertex.serviceColumn.metasInvMap.getOrElse(key, throw WhereParserException(s"Where clause contains not existing property name: $key"))
+
+    toInnerVal(value, columnMeta.dataType, vertex.serviceColumn.schemaVersion)
+  }
+
   @tailrec
   private def findParent(edge: S2EdgeLike, depth: Int): S2EdgeLike =
     if (depth > 0) findParent(edge.getParentEdges().head.edge, depth - 1)
@@ -87,11 +111,11 @@ trait Clause extends ExtractValue {
 
   def or(otherField: Clause): Clause = Or(this, otherField)
 
-  def filter(edge: S2EdgeLike): Boolean
+  def filter(element: GraphElement): Boolean
 
-  def binaryOp(binOp: (InnerValLike, InnerValLike) => Boolean)(propKey: String, value: String)(edge: S2EdgeLike): Boolean = {
-    val propValue = propToInnerVal(edge, propKey)
-    val compValue = valueToCompare(edge, propKey, value)
+  def binaryOp(binOp: (InnerValLike, InnerValLike) => Boolean)(propKey: String, value: String)(element: GraphElement): Boolean = {
+    val propValue = propToInnerVal(element, propKey)
+    val compValue = valueToCompare(element, propKey, value)
 
     binOp(propValue, compValue)
   }
@@ -106,29 +130,27 @@ object Where {
 }
 
 case class Where(clauses: Seq[Clause] = Seq.empty[Clause]) {
-  def filter(edge: S2EdgeLike) =
-    if (clauses.isEmpty) true else clauses.map(_.filter(edge)).forall(identity)
+  def filter(element: GraphElement) =
+    if (clauses.isEmpty) true else clauses.map(_.filter(element)).forall(identity)
 }
 
 case class Gt(propKey: String, value: String) extends Clause {
-  override def filter(edge: S2EdgeLike): Boolean = binaryOp(_ > _)(propKey, value)(edge)
+  override def filter(element: GraphElement): Boolean = binaryOp(_ > _)(propKey, value)(element)
 }
 
 case class Lt(propKey: String, value: String) extends Clause {
-  override def filter(edge: S2EdgeLike): Boolean = binaryOp(_ < _)(propKey, value)(edge)
+  override def filter(element: GraphElement): Boolean = binaryOp(_ < _)(propKey, value)(element)
 }
 
 case class Eq(propKey: String, value: String) extends Clause {
-  override def filter(edge: S2EdgeLike): Boolean = binaryOp(_ == _)(propKey, value)(edge)
+  override def filter(element: GraphElement): Boolean = binaryOp(_ == _)(propKey, value)(element)
 }
 
 case class InWithoutParent(propKey: String, values: Set[String]) extends Clause {
-  override def filter(edge: S2EdgeLike): Boolean = {
-    val label = edge.innerLabel
-
-    val propVal = propToInnerVal(edge, propKey)
+  override def filter(element: GraphElement): Boolean = {
+    val propVal = propToInnerVal(element, propKey)
     val innerVaLs = values.map { value =>
-      toInnerVal(value, label.metaPropsInvMap(propKey).dataType, label.schemaVersion)
+      valueToCompare(element, propKey, value)
     }
 
     innerVaLs(propVal)
@@ -136,41 +158,41 @@ case class InWithoutParent(propKey: String, values: Set[String]) extends Clause
 }
 
 case class Contains(propKey: String, value: String) extends Clause {
-  override def filter(edge: S2EdgeLike): Boolean = {
-    val propVal = propToInnerVal(edge, propKey)
+  override def filter(element: GraphElement): Boolean = {
+    val propVal = propToInnerVal(element, propKey)
     propVal.value.toString.contains(value)
   }
 }
 
 case class IN(propKey: String, values: Set[String]) extends Clause {
-  override def filter(edge: S2EdgeLike): Boolean = {
-    val propVal = propToInnerVal(edge, propKey)
+  override def filter(element: GraphElement): Boolean = {
+    val propVal = propToInnerVal(element, propKey)
     values.exists { value =>
-      valueToCompare(edge, propKey, value) == propVal
+      valueToCompare(element, propKey, value) == propVal
     }
   }
 }
 
 case class Between(propKey: String, minValue: String, maxValue: String) extends Clause {
-  override def filter(edge: S2EdgeLike): Boolean = {
-    val propVal = propToInnerVal(edge, propKey)
-    val minVal = valueToCompare(edge, propKey, minValue)
-    val maxVal = valueToCompare(edge, propKey, maxValue)
+  override def filter(element: GraphElement): Boolean = {
+    val propVal = propToInnerVal(element, propKey)
+    val minVal = valueToCompare(element, propKey, minValue)
+    val maxVal = valueToCompare(element, propKey, maxValue)
 
     minVal <= propVal && propVal <= maxVal
   }
 }
 
 case class Not(self: Clause) extends Clause {
-  override def filter(edge: S2EdgeLike) = !self.filter(edge)
+  override def filter(element: GraphElement) = !self.filter(element)
 }
 
 case class And(left: Clause, right: Clause) extends Clause {
-  override def filter(edge: S2EdgeLike) = left.filter(edge) && right.filter(edge)
+  override def filter(element: GraphElement) = left.filter(element) && right.filter(element)
 }
 
 case class Or(left: Clause, right: Clause) extends Clause {
-  override def filter(edge: S2EdgeLike) = left.filter(edge) || right.filter(edge)
+  override def filter(element: GraphElement) = left.filter(element) || right.filter(element)
 }
 
 object WhereParser {

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/3b692fbe/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 342d9c6..0859d9c 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
@@ -20,7 +20,7 @@
 package org.apache.s2graph.core.parsers
 
 import org.apache.s2graph.core._
-import org.apache.s2graph.core.schema.{ServiceColumn, Label, LabelMeta}
+import org.apache.s2graph.core.schema._
 import org.apache.s2graph.core.rest.TemplateHelper
 import org.apache.s2graph.core.types._
 import org.apache.s2graph.core.utils.logger
@@ -37,11 +37,11 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
   val ts = System.currentTimeMillis()
   val dummyTs = LabelMeta.timestamp -> InnerValLikeWithTs.withLong(ts, ts, label.schemaVersion)
 
-  def validate(label: Label)(edge: S2EdgeLike)(sql: String)(expected: Boolean) = {
+  def validate(label: Label)(element: GraphElement)(sql: String)(expected: Boolean) = {
     def debug(whereOpt: Try[Where]) = {
       println("==================")
       println(s"$whereOpt")
-      println(s"$edge")
+      println(s"$element")
       println("==================")
     }
 
@@ -50,7 +50,7 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
       debug(whereOpt)
       whereOpt.get // touch exception
     } else {
-      val ret = whereOpt.get.filter(edge)
+      val ret = whereOpt.get.filter(element)
       if (ret != expected) {
         debug(whereOpt)
       }
@@ -294,4 +294,18 @@ class WhereParserTest extends FunSuite with Matchers with TestCommonWithModels {
     println(whereOpt)
   }
 
+  test("test vertex with WhereParser.") {
+    val service = Service.findByName(serviceName).getOrElse(throw new IllegalStateException(s"$serviceName is not found."))
+    val column = ServiceColumn.find(service.id.get, columnName).getOrElse(throw new IllegalStateException(s"$columnName is not found."))
+
+    val vertexId = builder.newVertexId(service, column, 1)
+    val states = Map(ColumnMeta.lastModifiedAtColumn -> InnerVal.withLong(10, column.schemaVersion))
+    val vertex = builder.newVertex(vertexId)
+    S2Vertex.fillPropsWithTs(vertex, states)
+
+    val f = validate(label)(vertex) _
+    f(s"lastModifiedAt < 1")(false)
+    f(s"lastModifiedAt > 1")(true)
+    f(s"lastModifiedAt = 10")(true)
+  }
 }


[4/7] incubator-s2graph git commit: add filter param on serviceColumn field

Posted by st...@apache.org.
add filter param on serviceColumn field


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

Branch: refs/heads/master
Commit: 6a294060793563c3dc222c6a77817b9f4d18a137
Parents: 8e86536
Author: daewon <da...@apache.org>
Authored: Fri Jun 15 16:44:10 2018 +0900
Committer: daewon <da...@apache.org>
Committed: Fri Jun 15 16:44:10 2018 +0900

----------------------------------------------------------------------
 .../src/main/scala/org/apache/s2graph/graphql/types/S2Type.scala  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/6a294060/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/S2Type.scala
----------------------------------------------------------------------
diff --git a/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/S2Type.scala b/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/S2Type.scala
index 49e6b82..1324126 100644
--- a/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/S2Type.scala
+++ b/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/S2Type.scala
@@ -139,7 +139,8 @@ object S2Type {
           Argument("ids", OptionInputType(ListInputType(toScalarType(column.columnType)))),
           Argument("search", OptionInputType(StringType)),
           Argument("offset", OptionInputType(IntType), defaultValue = 0),
-          Argument("limit", OptionInputType(IntType), defaultValue = 100)
+          Argument("limit", OptionInputType(IntType), defaultValue = 100),
+          Argument("filter", OptionInputType(StringType), "desc here")
         ),
         description = Option("desc here"),
         resolve = c => {


[5/7] incubator-s2graph git commit: Merge pull request #17 from daewon/S2GRAPH-223

Posted by st...@apache.org.
Merge pull request #17 from daewon/S2GRAPH-223

Added Filter arguement on ServiceColumn field.

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

Branch: refs/heads/master
Commit: e9110083e4e628f5b4bdb8a7c6c178beab18707f
Parents: 8e86536 6a29406
Author: Doyung Yoon <st...@apache.org>
Authored: Fri Jun 15 16:51:02 2018 +0900
Committer: GitHub <no...@github.com>
Committed: Fri Jun 15 16:51:02 2018 +0900

----------------------------------------------------------------------
 .../src/main/scala/org/apache/s2graph/graphql/types/S2Type.scala  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[7/7] incubator-s2graph git commit: [S2GRAPH-223]: Support WhereParser on Vertex.

Posted by st...@apache.org.
[S2GRAPH-223]: Support WhereParser on Vertex.

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

Pull Request:
  Closes #175

Author
  DO YUNG YOON <st...@apache.org>


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

Branch: refs/heads/master
Commit: 5582644908518c322f0a1809f68473941961e57b
Parents: 2083d13 7d18c65
Author: DO YUNG YOON <st...@apache.org>
Authored: Thu Jun 21 10:19:51 2018 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Thu Jun 21 10:19:51 2018 +0900

----------------------------------------------------------------------
 .../org/apache/s2graph/core/QueryParam.scala    |  24 +++--
 .../s2graph/core/S2EdgePropertyHelper.scala     |   2 +-
 .../scala/org/apache/s2graph/core/S2Graph.scala |  23 +++--
 .../org/apache/s2graph/core/S2GraphLike.scala   |  18 ++--
 .../org/apache/s2graph/core/VertexFetcher.scala |   2 +-
 .../s2graph/core/index/ESIndexProvider.scala    |  10 +-
 .../core/index/LuceneIndexProvider.scala        |   8 +-
 .../s2graph/core/parsers/WhereParser.scala      | 102 +++++++++++--------
 .../s2graph/core/rest/RequestParser.scala       |  18 +++-
 .../apache/s2graph/core/rest/RestHandler.scala  |   7 +-
 .../core/storage/hbase/AsynchbaseStorage.scala  |   2 +-
 .../storage/hbase/AsynchbaseVertexFetcher.scala |  21 ++--
 .../core/storage/rocks/RocksVertexFetcher.scala |   5 +-
 .../core/Integrate/VertexTestHelper.scala       |   8 +-
 .../s2graph/core/index/IndexProviderTest.scala  |   2 +-
 .../s2graph/core/parsers/WhereParserTest.scala  |  22 +++-
 .../s2graph/graphql/types/FieldResolver.scala   |  11 +-
 .../apache/s2graph/graphql/types/S2Type.scala   |   3 +-
 18 files changed, 183 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55826449/s2core/src/main/scala/org/apache/s2graph/core/parsers/WhereParser.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55826449/s2core/src/test/scala/org/apache/s2graph/core/parsers/WhereParserTest.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/55826449/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
----------------------------------------------------------------------


[6/7] incubator-s2graph git commit: add fromVertices on VertexQueryParam.

Posted by st...@apache.org.
add fromVertices on VertexQueryParam.


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

Branch: refs/heads/master
Commit: 7d18c65e7595ab8b55b0fcaef738634d56075e9a
Parents: e911008
Author: DO YUNG YOON <st...@apache.org>
Authored: Thu Jun 21 05:17:40 2018 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Thu Jun 21 08:46:34 2018 +0900

----------------------------------------------------------------------
 .../scala/org/apache/s2graph/core/QueryParam.scala    |  6 +++++-
 .../main/scala/org/apache/s2graph/core/S2Graph.scala  |  4 ++++
 .../scala/org/apache/s2graph/core/S2GraphLike.scala   |  2 ++
 .../s2graph/core/Integrate/VertexTestHelper.scala     | 14 ++++++--------
 4 files changed, 17 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/7d18c65e/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 5fdd3ee..9b0100e 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
@@ -296,8 +296,12 @@ object QueryParam {
 object VertexQueryParam {
   def Empty: VertexQueryParam = VertexQueryParam(Nil)
 
+  def fromVertices(vertices: Seq[S2VertexLike]): VertexQueryParam = {
+    apply(vertices.map(_.id))
+  }
+
   def apply(vertexIds: Seq[VertexId]): VertexQueryParam = {
-    VertexQueryParam(vertexIds)
+    new VertexQueryParam(vertexIds)
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/7d18c65e/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
index f3010f7..95793c8 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
@@ -343,6 +343,10 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends S2Grap
     else matchedVerticesFuture
   }
 
+  override def getVertices(vertices: Seq[S2VertexLike]): Future[Seq[S2VertexLike]] =
+    getVertices(VertexQueryParam.fromVertices(vertices))
+
+
   override def getVertices(queryParam: VertexQueryParam): Future[Seq[S2VertexLike]] = {
     val vertexIdsWithIdx = queryParam.vertexIds.zipWithIndex
     val futures = vertexIdsWithIdx.groupBy { case (vId, idx) => vId.column }.map { case (serviceColumn, vertexGroup) =>

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/7d18c65e/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
index cab866d..629d429 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
@@ -110,6 +110,8 @@ trait S2GraphLike extends Graph {
 
   def shutdown(modelDataDelete: Boolean = false): Unit
 
+  def getVertices(vertices: Seq[S2VertexLike]): Future[Seq[S2VertexLike]]
+
   def getVertices(queryParam: VertexQueryParam): Future[Seq[S2VertexLike]]
 
   def checkEdges(edges: Seq[S2EdgeLike]): Future[StepResult]

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/7d18c65e/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala
index c4136ab..6bb6b2e 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.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
@@ -43,11 +43,10 @@ class VertexTestHelper extends IntegrateCommon {
 
     val vertices = parser.toVertices(payload, "insert", Option(serviceName), Option(columnName))
     val srcVertices = vertices
-    val srcVertexIds = srcVertices.map(_.id)
-    val queryParam = VertexQueryParam(vertexIds = srcVertexIds)
+
     Await.result(graph.mutateVertices(srcVertices, withWait = true), HttpRequestWaitingTime)
 
-    val res = graph.getVertices(queryParam).map { vertices =>
+    val res = graph.getVertices(srcVertices).map { vertices =>
       PostProcess.verticesToJson(vertices)
     }
 
@@ -72,11 +71,10 @@ class VertexTestHelper extends IntegrateCommon {
     val vertices = parser.toVertices(payload, "insert", Option(serviceName),
       Option(stringColumnName))
     val srcVertices = vertices
-    val srcVertexIds = srcVertices.map(_.id)
-    val queryParam = VertexQueryParam(vertexIds = srcVertexIds)
+
     Await.result(graph.mutateVertices(srcVertices, withWait = true), HttpRequestWaitingTime)
 
-    val res = graph.getVertices(queryParam).map { vertices =>
+    val res = graph.getVertices(srcVertices).map { vertices =>
       PostProcess.verticesToJson(vertices)
     }
 


[3/7] incubator-s2graph git commit: - Separate SearchParam from VertexQueryParam.

Posted by st...@apache.org.
- Separate SearchParam from VertexQueryParam.


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

Branch: refs/heads/master
Commit: 8e86536a547dc1fe4c4f3aef29181ac8f2388eed
Parents: f47622f
Author: DO YUNG YOON <st...@apache.org>
Authored: Fri Jun 15 16:16:46 2018 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Fri Jun 15 16:16:46 2018 +0900

----------------------------------------------------------------------
 .../org/apache/s2graph/core/QueryParam.scala      | 11 +++++++----
 .../s2graph/core/index/ESIndexProvider.scala      | 10 +++++-----
 .../s2graph/core/index/LuceneIndexProvider.scala  |  8 ++++----
 .../apache/s2graph/core/rest/RequestParser.scala  | 18 +++++++++++++++---
 .../s2graph/core/Integrate/VertexTestHelper.scala | 10 +++++++---
 .../s2graph/core/index/IndexProviderTest.scala    |  2 +-
 .../s2graph/graphql/types/FieldResolver.scala     | 11 +++++++++--
 7 files changed, 48 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/8e86536a/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 cb1434f..5fdd3ee 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
@@ -294,17 +294,20 @@ object QueryParam {
 }
 
 object VertexQueryParam {
-  def Empty: VertexQueryParam = VertexQueryParam(Nil, 0, 1, None)
+  def Empty: VertexQueryParam = VertexQueryParam(Nil)
 
   def apply(vertexIds: Seq[VertexId]): VertexQueryParam = {
     VertexQueryParam(vertexIds)
   }
 }
 
+object SearchParam {
+  def Empty: SearchParam = SearchParam("", 0, 1)
+}
+case class SearchParam(searchString: String, offset: Int, limit: Int)
+
 case class VertexQueryParam(vertexIds: Seq[VertexId],
-                            offset: Int = 0,
-                            limit: Int = 1,
-                            searchString: Option[String] = None,
+                            searchParamOpt: Option[SearchParam] = None,
                             fetchProp: Boolean = true,
                             where: Try[Where] = Success(WhereParser.success)) {
 }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/8e86536a/s2core/src/main/scala/org/apache/s2graph/core/index/ESIndexProvider.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/index/ESIndexProvider.scala b/s2core/src/main/scala/org/apache/s2graph/core/index/ESIndexProvider.scala
index be1ad6e..c047be2 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/index/ESIndexProvider.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/index/ESIndexProvider.scala
@@ -202,12 +202,12 @@ class ESIndexProvider(config: Config)(implicit ec: ExecutionContext) extends Ind
     val field = vidField
     val empty = new util.ArrayList[VertexId]()
 
-    vertexQueryParam.searchString match {
-      case Some(queryString) =>
+    vertexQueryParam.searchParamOpt match {
+      case Some(searchParam) =>
         fetchInner[VertexId](
-          queryString,
-          vertexQueryParam.offset,
-          vertexQueryParam.limit,
+          searchParam.searchString,
+          searchParam.offset,
+          searchParam.limit,
           VertexIndexName,
           field,
           Conversions.s2VertexIdReads)(v => VertexId.isValid(v).isDefined)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/8e86536a/s2core/src/main/scala/org/apache/s2graph/core/index/LuceneIndexProvider.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/index/LuceneIndexProvider.scala b/s2core/src/main/scala/org/apache/s2graph/core/index/LuceneIndexProvider.scala
index 00f2e20..efb44f6 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/index/LuceneIndexProvider.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/index/LuceneIndexProvider.scala
@@ -242,16 +242,16 @@ class LuceneIndexProvider(config: Config) extends IndexProvider {
   override def fetchVertexIdsAsync(hasContainers: java.util.List[HasContainer]): Future[util.List[VertexId]] = Future.successful(fetchVertexIds(hasContainers))
 
   override def fetchVertexIdsAsyncRaw(vertexQueryParam: VertexQueryParam): Future[util.List[VertexId]] = {
-    val ret = vertexQueryParam.searchString.fold(util.Arrays.asList[VertexId]()) { queryString =>
+    val ret = vertexQueryParam.searchParamOpt.fold(util.Arrays.asList[VertexId]()) { searchParam =>
       val field = vidField
       try {
         val qp = createQueryParser(field, analyzer)
-        val q = qp.parse(queryString)
+        val q = qp.parse(searchParam.searchString)
 
-        fetchInner[VertexId](q, vertexQueryParam.offset, vertexQueryParam.limit, VertexIndexName, vidField, Conversions.s2VertexIdReads)
+        fetchInner[VertexId](q, searchParam.offset, searchParam.limit, VertexIndexName, vidField, Conversions.s2VertexIdReads)
       } catch {
         case ex: ParseException =>
-          logger.error(s"[IndexProvider]: ${queryString} parse failed.", ex)
+          logger.error(s"[IndexProvider]: ${searchParam.searchString} parse failed.", ex)
           util.Arrays.asList[VertexId]()
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/8e86536a/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 7f8b280..b71da8e 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
@@ -219,12 +219,16 @@ class RequestParser(graph: S2GraphLike) {
     ret.map(_.toMap).getOrElse(Map.empty[String, InnerValLike])
   }
 
-  def extractWhere(label: Label, whereClauseOpt: Option[String]): Try[Where] = {
+  private def toWhereParserKey(column: ServiceColumn, whereString: String): String =
+    s"${column.service.serviceName}_${column.columnName}_${whereString}"
+
+  private def toWhereParserKey(label: Label, whereString: String): String =
+    s"${label.label}_${whereString}"
+
+  private def extractWhereInner(whereParserKey: String, whereClauseOpt: Option[String]): Try[Where] = {
     whereClauseOpt match {
       case None => Success(WhereParser.success)
       case Some(where) =>
-        val whereParserKey = s"${label.label}_${where}"
-
         parserCache.get(whereParserKey, new Callable[Try[Where]] {
           override def call(): Try[Where] = {
             val _where = TemplateHelper.replaceVariable(System.currentTimeMillis(), where)
@@ -239,6 +243,14 @@ class RequestParser(graph: S2GraphLike) {
     }
   }
 
+  def extractWhere(column: ServiceColumn, whereClauseOpt: Option[String]): Try[Where] = {
+    extractWhereInner(toWhereParserKey(column, whereClauseOpt.getOrElse("")), whereClauseOpt)
+  }
+
+  def extractWhere(label: Label, whereClauseOpt: Option[String]): Try[Where] = {
+    extractWhereInner(toWhereParserKey(label, whereClauseOpt.getOrElse("")), whereClauseOpt)
+  }
+
   def extractGroupBy(value: Option[JsValue]): GroupBy = value.map {
     case obj: JsObject =>
       val keys = (obj \ "keys").asOpt[Seq[String]].getOrElse(Nil)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/8e86536a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala
index 62c7e00..c4136ab 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/Integrate/VertexTestHelper.scala
@@ -19,7 +19,7 @@
 
 package org.apache.s2graph.core.Integrate
 
-import org.apache.s2graph.core.PostProcess
+import org.apache.s2graph.core.{PostProcess, VertexQueryParam}
 import play.api.libs.json.{JsValue, Json}
 
 import scala.concurrent.Await
@@ -43,9 +43,11 @@ class VertexTestHelper extends IntegrateCommon {
 
     val vertices = parser.toVertices(payload, "insert", Option(serviceName), Option(columnName))
     val srcVertices = vertices
+    val srcVertexIds = srcVertices.map(_.id)
+    val queryParam = VertexQueryParam(vertexIds = srcVertexIds)
     Await.result(graph.mutateVertices(srcVertices, withWait = true), HttpRequestWaitingTime)
 
-    val res = graph.getVertices(srcVertices).map { vertices =>
+    val res = graph.getVertices(queryParam).map { vertices =>
       PostProcess.verticesToJson(vertices)
     }
 
@@ -70,9 +72,11 @@ class VertexTestHelper extends IntegrateCommon {
     val vertices = parser.toVertices(payload, "insert", Option(serviceName),
       Option(stringColumnName))
     val srcVertices = vertices
+    val srcVertexIds = srcVertices.map(_.id)
+    val queryParam = VertexQueryParam(vertexIds = srcVertexIds)
     Await.result(graph.mutateVertices(srcVertices, withWait = true), HttpRequestWaitingTime)
 
-    val res = graph.getVertices(srcVertices).map { vertices =>
+    val res = graph.getVertices(queryParam).map { vertices =>
       PostProcess.verticesToJson(vertices)
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/8e86536a/s2core/src/test/scala/org/apache/s2graph/core/index/IndexProviderTest.scala
----------------------------------------------------------------------
diff --git a/s2core/src/test/scala/org/apache/s2graph/core/index/IndexProviderTest.scala b/s2core/src/test/scala/org/apache/s2graph/core/index/IndexProviderTest.scala
index c12ad7d..12cec21 100644
--- a/s2core/src/test/scala/org/apache/s2graph/core/index/IndexProviderTest.scala
+++ b/s2core/src/test/scala/org/apache/s2graph/core/index/IndexProviderTest.scala
@@ -80,7 +80,7 @@ class IndexProviderTest extends IntegrateCommon {
 //      val hasContainer = new HasContainer(indexPropsColumnMeta.name, P.eq(Long.box(1)))
       val hasContainer = new HasContainer(serviceColumnField, P.eq(testColumn.columnName))
 
-      val f = graph.searchVertices(VertexQueryParam(0, 100, Option(s"${serviceColumnField}:${testColumn.columnName}")))
+      val f = graph.searchVertices(VertexQueryParam(Nil, Option(SearchParam(s"${serviceColumnField}:${testColumn.columnName}", 0, 100))))
       val a = Await.result(f, Duration("60 sec"))
       println(a)
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/8e86536a/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
----------------------------------------------------------------------
diff --git a/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala b/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
index 6e1a7cc..43b3de6 100644
--- a/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
+++ b/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
@@ -76,6 +76,8 @@ object FieldResolver {
     val ids = c.argOpt[Any]("id").toSeq ++ c.argOpt[List[Any]]("ids").toList.flatten
     val offset = c.arg[Int]("offset")
     val limit = c.arg[Int]("limit")
+    val whereClauseOpt = c.argOpt[String]("filter")
+    val where = c.ctx.parser.extractWhere(column, whereClauseOpt)
 
     val vertices = ids.map(vid => c.ctx.toS2VertexLike(vid, column))
     val searchOpt = c.argOpt[String]("search").map { qs =>
@@ -87,7 +89,9 @@ object FieldResolver {
     val selectedFields = AstHelper.selectedFields(c.astFields)
     val canSkipFetch = selectedFields.forall(f => f == "id" || !columnFields(f))
 
-    val vertexQueryParam = VertexQueryParam(vertices.map(_.id), offset, limit, searchOpt, !canSkipFetch)
+    val searchParamOpt = searchOpt.map(search => SearchParam(search, offset, limit))
+    val vertexQueryParam = VertexQueryParam(vertices.map(_.id),
+      searchParamOpt, fetchProp = !canSkipFetch, where = where)
 
     vertexQueryParam
   }
@@ -102,7 +106,10 @@ object FieldResolver {
     val columnFields = column.metasInvMap.keySet
     val canSkipFetch = selectedFields.forall(f => f == "id" || !columnFields(f))
 
-    val vertexQueryParam = VertexQueryParam(Seq(vertex.id), 0, 1, None, !canSkipFetch)
+    val whereClauseOpt = c.argOpt[String]("filter")
+    val where = c.ctx.parser.extractWhere(column, whereClauseOpt)
+
+    val vertexQueryParam = VertexQueryParam(Seq(vertex.id), None, fetchProp = !canSkipFetch, where = where)
 
     vertexQueryParam
   }


[2/7] incubator-s2graph git commit: - Refactor WhereParser to accept GraphElement. - Refactor S2Graph.getVertices to accept VertexQueryParam.

Posted by st...@apache.org.
- Refactor WhereParser to accept GraphElement.
- Refactor S2Graph.getVertices to accept VertexQueryParam.


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

Branch: refs/heads/master
Commit: f47622fdeaee9fef9d46bf83c6191204dad00043
Parents: 3b692fb
Author: DO YUNG YOON <st...@apache.org>
Authored: Fri Jun 15 15:35:16 2018 +0900
Committer: DO YUNG YOON <st...@apache.org>
Committed: Fri Jun 15 15:35:16 2018 +0900

----------------------------------------------------------------------
 .../org/apache/s2graph/core/QueryParam.scala    | 17 ++++++++++------
 .../scala/org/apache/s2graph/core/S2Graph.scala | 19 +++++++++---------
 .../org/apache/s2graph/core/S2GraphLike.scala   | 20 +++++++++----------
 .../org/apache/s2graph/core/VertexFetcher.scala |  2 +-
 .../apache/s2graph/core/rest/RestHandler.scala  |  7 ++++---
 .../core/storage/hbase/AsynchbaseStorage.scala  |  2 +-
 .../storage/hbase/AsynchbaseVertexFetcher.scala | 21 ++++++++++----------
 .../core/storage/rocks/RocksVertexFetcher.scala |  5 +++--
 .../s2graph/graphql/types/FieldResolver.scala   |  4 ++--
 9 files changed, 52 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/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 21aca12..cb1434f 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/QueryParam.scala
@@ -294,14 +294,19 @@ object QueryParam {
 }
 
 object VertexQueryParam {
-  def Empty: VertexQueryParam = VertexQueryParam(0, 1, None)
+  def Empty: VertexQueryParam = VertexQueryParam(Nil, 0, 1, None)
+
+  def apply(vertexIds: Seq[VertexId]): VertexQueryParam = {
+    VertexQueryParam(vertexIds)
+  }
 }
 
-case class VertexQueryParam(offset: Int,
-                            limit: Int,
-                            searchString: Option[String],
-                            vertexIds: Seq[VertexId] = Nil,
-                            fetchProp: Boolean = true) {
+case class VertexQueryParam(vertexIds: Seq[VertexId],
+                            offset: Int = 0,
+                            limit: Int = 1,
+                            searchString: Option[String] = None,
+                            fetchProp: Boolean = true,
+                            where: Try[Where] = Success(WhereParser.success)) {
 }
 
 case class QueryParam(labelName: String,

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
index d41bc24..f3010f7 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2Graph.scala
@@ -335,18 +335,19 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends S2Grap
     }
 
   def searchVertices(queryParam: VertexQueryParam): Future[Seq[S2VertexLike]] = {
-    val matchedVertices = indexProvider.fetchVertexIdsAsyncRaw(queryParam).map { vids =>
+    val matchedVerticesFuture = indexProvider.fetchVertexIdsAsyncRaw(queryParam).map { vids =>
       (queryParam.vertexIds ++ vids).distinct.map(vid => elementBuilder.newVertex(vid))
     }
 
-    if (queryParam.fetchProp) matchedVertices.flatMap(vs => getVertices(vs))
-    else matchedVertices
+    if (queryParam.fetchProp) matchedVerticesFuture.flatMap(vs => getVertices(queryParam.copy(vertexIds = vs.map(_.id))))
+    else matchedVerticesFuture
   }
 
-  override def getVertices(vertices: Seq[S2VertexLike]): Future[Seq[S2VertexLike]] = {
-    val verticesWithIdx = vertices.zipWithIndex
-    val futures = verticesWithIdx.groupBy { case (v, idx) => v.serviceColumn }.map { case (serviceColumn, vertexGroup) =>
-      getVertexFetcher(serviceColumn).fetchVertices(vertices).map(_.zip(vertexGroup.map(_._2)))
+  override def getVertices(queryParam: VertexQueryParam): Future[Seq[S2VertexLike]] = {
+    val vertexIdsWithIdx = queryParam.vertexIds.zipWithIndex
+    val futures = vertexIdsWithIdx.groupBy { case (vId, idx) => vId.column }.map { case (serviceColumn, vertexGroup) =>
+      val (vertexIds, indices) = vertexGroup.unzip
+      getVertexFetcher(serviceColumn).fetchVertices(queryParam.copy(vertexIds = vertexIds)).map(_.zip(indices))
     }
 
     Future.sequence(futures).map { ls =>
@@ -562,8 +563,8 @@ class S2Graph(_config: Config)(implicit val ec: ExecutionContext) extends S2Grap
   }
 
   override def getVertex(vertexId: VertexId): Option[S2VertexLike] = {
-    val v = elementBuilder.newVertex(vertexId)
-    Await.result(getVertices(Seq(v)).map { vertices => vertices.headOption }, WaitTimeout)
+    val queryParam = VertexQueryParam(vertexIds = Seq(vertexId))
+    Await.result(getVertices(queryParam).map { vertices => vertices.headOption }, WaitTimeout)
   }
 
   override def fetchEdges(vertex: S2VertexLike, labelNameWithDirs: Seq[(String, String)]): util.Iterator[Edge] = {

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala b/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
index fb229ec..cab866d 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/S2GraphLike.scala
@@ -110,10 +110,7 @@ trait S2GraphLike extends Graph {
 
   def shutdown(modelDataDelete: Boolean = false): Unit
 
-  def getVertices(vertices: Seq[S2VertexLike]): Future[Seq[S2VertexLike]]
-
-  def getVerticesJava(vertices: util.List[S2VertexLike]): CompletableFuture[util.List[S2VertexLike]] =
-    getVertices(vertices.toSeq).map(_.asJava).toJava.toCompletableFuture
+  def getVertices(queryParam: VertexQueryParam): Future[Seq[S2VertexLike]]
 
   def checkEdges(edges: Seq[S2EdgeLike]): Future[StepResult]
 
@@ -217,22 +214,23 @@ trait S2GraphLike extends Graph {
 
       Await.result(future, WaitTimeout).flatten.iterator
     } else {
-      val vertices = ids.collect {
-        case s2Vertex: S2VertexLike => s2Vertex
-        case vId: VertexId => elementBuilder.newVertex(vId)
-        case vertex: Vertex => elementBuilder.newVertex(vertex.id().asInstanceOf[VertexId])
-        case other@_ => elementBuilder.newVertex(VertexId.fromString(other.toString))
+      val vertexIds = ids.collect {
+        case s2Vertex: S2VertexLike => s2Vertex.id
+        case vId: VertexId => vId
+        case vertex: Vertex => vertex.id().asInstanceOf[VertexId]
+        case other@_ => VertexId.fromString(other.toString)
       }
 
       if (fetchVertices) {
-        val future = getVertices(vertices).map { vs =>
+        val queryParam = VertexQueryParam(vertexIds = vertexIds)
+        val future = getVertices(queryParam).map { vs =>
           val ls = new util.ArrayList[structure.Vertex]()
           ls.addAll(vs)
           ls.iterator()
         }
         Await.result(future, WaitTimeout)
       } else {
-        vertices.iterator
+        vertexIds.map(vId => elementBuilder.newVertex(vId)).iterator
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/s2core/src/main/scala/org/apache/s2graph/core/VertexFetcher.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/VertexFetcher.scala b/s2core/src/main/scala/org/apache/s2graph/core/VertexFetcher.scala
index 4addcab..d6c2e64 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/VertexFetcher.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/VertexFetcher.scala
@@ -27,7 +27,7 @@ import scala.concurrent.{ExecutionContext, Future}
 trait VertexFetcher extends AutoCloseable {
   def init(config: Config)(implicit ec: ExecutionContext): Unit = {}
 
-  def fetchVertices(vertices: Seq[S2VertexLike])(implicit ec: ExecutionContext): Future[Seq[S2VertexLike]]
+  def fetchVertices(vertexQueryParam: VertexQueryParam)(implicit ec: ExecutionContext): Future[Seq[S2VertexLike]]
 
   def fetchVerticesAll()(implicit ec: ExecutionContext): Future[Seq[S2VertexLike]]
 

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/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 c768d81..dfdeca4 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
@@ -212,18 +212,19 @@ class RestHandler(graph: S2GraphLike)(implicit ec: ExecutionContext) {
   private def getVertices(jsValue: JsValue) = {
     val jsonQuery = jsValue
 
-    val vertices = jsonQuery.as[List[JsValue]].flatMap { js =>
+    val vertexIds = jsonQuery.as[List[JsValue]].flatMap { js =>
       val serviceName = (js \ "serviceName").as[String]
       val columnName = (js \ "columnName").as[String]
       for {
         idJson <- (js \ "ids").asOpt[List[JsValue]].getOrElse(List.empty[JsValue])
         id <- jsValueToAny(idJson)
       } yield {
-        graph.toVertex(serviceName, columnName, id)
+        graph.elementBuilder.newVertexId(serviceName)(columnName)(id)
       }
     }
+    val queryParam = VertexQueryParam(vertexIds = vertexIds)
 
-    graph.getVertices(vertices) map { vertices => PostProcess.verticesToJson(vertices) }
+    graph.getVertices(queryParam) map { vertices => PostProcess.verticesToJson(vertices) }
   }
 
   private def buildRequestBody(requestKeyJsonOpt: Option[JsValue], bucket: Bucket, uuid: String): String = {

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/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 89303e6..f670e9c 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
@@ -202,7 +202,7 @@ object AsynchbaseStorage {
     }
   }
 
-  def buildRequest(serDe: StorageSerDe, queryRequest: QueryRequest, vertex: S2VertexLike) = {
+  def buildRequest(serDe: StorageSerDe, vertex: S2VertexLike) = {
     val kvs = serDe.vertexSerializer(vertex).toKeyValues
     val get = new GetRequest(vertex.hbaseTableName.getBytes, kvs.head.row, Serializable.vertexCf)
     //      get.setTimeout(this.singleGetTimeout.toShort)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseVertexFetcher.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseVertexFetcher.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseVertexFetcher.scala
index f16c8e9..4815bf3 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseVertexFetcher.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/hbase/AsynchbaseVertexFetcher.scala
@@ -21,6 +21,7 @@ package org.apache.s2graph.core.storage.hbase
 
 import com.typesafe.config.Config
 import org.apache.s2graph.core._
+import org.apache.s2graph.core.parsers.Where
 import org.apache.s2graph.core.schema.ServiceColumn
 import org.apache.s2graph.core.storage.serde.Serializable
 import org.apache.s2graph.core.storage.{SKeyValue, StorageIO, StorageSerDe}
@@ -40,23 +41,23 @@ class AsynchbaseVertexFetcher(val graph: S2GraphLike,
   import scala.collection.JavaConverters._
 
 
-  private def fetchKeyValues(queryRequest: QueryRequest, vertex: S2VertexLike)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]] = {
-    val rpc = buildRequest(serDe, queryRequest, vertex)
+  private def fetchKeyValues(vertex: S2VertexLike)(implicit ec: ExecutionContext): Future[Seq[SKeyValue]] = {
+    val rpc = buildRequest(serDe, vertex)
     AsynchbaseStorage.fetchKeyValues(client, rpc)
   }
 
-  override def fetchVertices(vertices: Seq[S2VertexLike])(implicit ec: ExecutionContext): Future[Seq[S2VertexLike]] = {
-    def fromResult(kvs: Seq[SKeyValue], version: String): Seq[S2VertexLike] = {
+  override def fetchVertices(vertexQueryParam: VertexQueryParam)(implicit ec: ExecutionContext): Future[Seq[S2VertexLike]] = {
+    def fromResult(kvs: Seq[SKeyValue],
+                   version: String): Seq[S2VertexLike] = {
       if (kvs.isEmpty) Nil
-      else serDe.vertexDeserializer(version).fromKeyValues(kvs, None).toSeq
+      else {
+        serDe.vertexDeserializer(version).fromKeyValues(kvs, None).toSeq.filter(vertexQueryParam.where.get.filter)
+      }
     }
+    val vertices = vertexQueryParam.vertexIds.map(vId => graph.elementBuilder.newVertex(vId))
 
     val futures = vertices.map { vertex =>
-      val queryParam = QueryParam.Empty
-      val q = Query.toQuery(Seq(vertex), Seq(queryParam))
-      val queryRequest = QueryRequest(q, stepIdx = -1, vertex, queryParam)
-
-      fetchKeyValues(queryRequest, vertex).map { kvs =>
+      fetchKeyValues(vertex).map { kvs =>
         fromResult(kvs, vertex.serviceColumn.schemaVersion)
       } recoverWith {
         case ex: Throwable => Future.successful(Nil)

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/s2core/src/main/scala/org/apache/s2graph/core/storage/rocks/RocksVertexFetcher.scala
----------------------------------------------------------------------
diff --git a/s2core/src/main/scala/org/apache/s2graph/core/storage/rocks/RocksVertexFetcher.scala b/s2core/src/main/scala/org/apache/s2graph/core/storage/rocks/RocksVertexFetcher.scala
index 2d3880c..94a0da6 100644
--- a/s2core/src/main/scala/org/apache/s2graph/core/storage/rocks/RocksVertexFetcher.scala
+++ b/s2core/src/main/scala/org/apache/s2graph/core/storage/rocks/RocksVertexFetcher.scala
@@ -43,11 +43,12 @@ class RocksVertexFetcher(val graph: S2GraphLike,
     RocksStorage.fetchKeyValues(vdb, db, rpc)
   }
 
-  override def fetchVertices(vertices: Seq[S2VertexLike])(implicit ec: ExecutionContext): Future[Seq[S2VertexLike]] = {
+  override def fetchVertices(vertexQueryParam: VertexQueryParam)(implicit ec: ExecutionContext): Future[Seq[S2VertexLike]] = {
     def fromResult(kvs: Seq[SKeyValue], version: String): Seq[S2VertexLike] = {
       if (kvs.isEmpty) Nil
-      else serDe.vertexDeserializer(version).fromKeyValues(kvs, None).toSeq
+      else serDe.vertexDeserializer(version).fromKeyValues(kvs, None).toSeq.filter(vertexQueryParam.where.get.filter)
     }
+    val vertices = vertexQueryParam.vertexIds.map(vId => graph.elementBuilder.newVertex(vId))
 
     val futures = vertices.map { vertex =>
       val queryParam = QueryParam.Empty

http://git-wip-us.apache.org/repos/asf/incubator-s2graph/blob/f47622fd/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
----------------------------------------------------------------------
diff --git a/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala b/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
index 478517f..6e1a7cc 100644
--- a/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
+++ b/s2graphql/src/main/scala/org/apache/s2graph/graphql/types/FieldResolver.scala
@@ -87,7 +87,7 @@ object FieldResolver {
     val selectedFields = AstHelper.selectedFields(c.astFields)
     val canSkipFetch = selectedFields.forall(f => f == "id" || !columnFields(f))
 
-    val vertexQueryParam = VertexQueryParam(offset, limit, searchOpt, vertices.map(_.id), !canSkipFetch)
+    val vertexQueryParam = VertexQueryParam(vertices.map(_.id), offset, limit, searchOpt, !canSkipFetch)
 
     vertexQueryParam
   }
@@ -102,7 +102,7 @@ object FieldResolver {
     val columnFields = column.metasInvMap.keySet
     val canSkipFetch = selectedFields.forall(f => f == "id" || !columnFields(f))
 
-    val vertexQueryParam = VertexQueryParam(0, 1, None, Seq(vertex.id), !canSkipFetch)
+    val vertexQueryParam = VertexQueryParam(Seq(vertex.id), 0, 1, None, !canSkipFetch)
 
     vertexQueryParam
   }