Skip to content

Commit

Permalink
Support CompositeQueryExpression and tests for Composite index planning
Browse files Browse the repository at this point in the history
  • Loading branch information
craigtaverner committed Mar 9, 2017
1 parent ca5c9da commit ffb7b56
Show file tree
Hide file tree
Showing 8 changed files with 131 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,14 @@ case class RangeQueryExpression[T](expression: T) extends QueryExpression[T] {
override def map[R](f: (T) => R) = RangeQueryExpression(f(expression))
}

case class CompositeQueryExpression[T](expression: T) extends QueryExpression[T] {
def map[R](f: (T) => R) = CompositeQueryExpression(f(expression))
}

case class CompositeRangeQueryExpression[T](expression: T) extends QueryExpression[T] {
override def map[R](f: (T) => R) = CompositeRangeQueryExpression(f(expression))
}

case class SchemaIndex(variable: String, label: String, properties: Seq[String], kind: SchemaIndexKind, query: Option[QueryExpression[Expression]])
extends StartItem(variable, query.map(q => Arguments.LegacyExpression(q.expression)).toIndexedSeq :+ Arguments.Index(label, properties))
with ReadOnlyStartItem with Hint with NodeStartItemVariables
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,12 @@ object indexQuery extends GraphElementPropertyFunctions {
labelName: String,
propertyNames: Seq[String]): Iterator[Node] = queryExpression match {

// Index exact value seek on single value
case SingleQueryExpression(inner) =>
val value = inner(m)(state)
lookupNodes(value, index).toIterator

// Index exact value seek on multiple values, by combining the results of multiple index seeks
case ManyQueryExpression(inner) =>
inner(m)(state) match {
case IsList(coll) => coll.toSet.toIndexedSeq.flatMap {
Expand All @@ -50,6 +52,19 @@ object indexQuery extends GraphElementPropertyFunctions {
case _ => throw new CypherTypeException(s"Expected the value for looking up :$labelName(${propertyNames.mkString(",")}) to be a collection but it was not.")
}

// Index exact value seek on multiple values, making use of a composite index over all values
case CompositeQueryExpression(inner) =>
inner(m)(state) match {
case IsList(coll) if coll.size == propertyNames.size => index(coll.flatMap{
case null => throw new CypherTypeException(s"Cannot handle null value in composite index search: ${propertyNames.zip(coll).map(p => s"${p._1}=${p._2}").mkString(", ")}")
case value => Some(makeValueNeoSafe(value))
}).toIterator
case null => Iterator.empty
case IsList(coll) => throw new CypherTypeException(s"Expected ${propertyNames.size} values for looking up :$labelName(${propertyNames.mkString(",")}) but there were only ${coll.size}: ${coll.mkString(",")}")
case _ => throw new CypherTypeException(s"Expected the value for looking up :$labelName(${propertyNames.mkString(",")}) to be a collection but it was not.")
}

// Index range seek over range of values
case RangeQueryExpression(rangeWrapper) =>
val range = rangeWrapper match {
case s: PrefixSeekRangeExpression =>
Expand All @@ -59,6 +74,10 @@ object indexQuery extends GraphElementPropertyFunctions {
innerRange.mapBounds(expression => makeValueNeoSafe(expression(m)(state)))
}
index(range).toIterator

// Composite index range seek over ranges of multiple values
case RangeQueryExpression(rangeWrapper) =>
throw new CypherTypeException("Composite index searches for ranges not yet supported")
}

private def lookupNodes(value: Any, index: Any => GenTraversableOnce[Node]) = value match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
*/
package org.neo4j.cypher.internal.compiler.v3_2.planner.logical.steps

import org.neo4j.cypher.internal.compiler.v3_2.commands.{ManyQueryExpression, QueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.commands.{CompositeQueryExpression, QueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.planner.logical.LeafPlansForVariable.maybeLeafPlans
import org.neo4j.cypher.internal.compiler.v3_2.planner.logical._
import org.neo4j.cypher.internal.compiler.v3_2.planner.logical.plans._
Expand Down Expand Up @@ -164,7 +164,7 @@ abstract class AbstractIndexSeekLeafPlanner extends LeafPlanner with LeafPlanFro
plannables.head.queryExpression
else {
val pos = plannables.head.queryExpression.expression.position
ManyQueryExpression(ListLiteral(plannables.map(_.queryExpression.expression))(pos))
CompositeQueryExpression(ListLiteral(plannables.map(_.queryExpression.expression))(pos))
}
val entryConstructor: (Seq[Expression]) => LogicalPlan =
constructPlan(idName, LabelToken(labelName, labelId), propertyKeyTokens,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.mockito.Matchers.any
import org.mockito.Mockito._
import org.neo4j.cypher.internal.compiler.v3_2._
import org.neo4j.cypher.internal.compiler.v3_2.commands.expressions.{ListLiteral, Literal, Variable}
import org.neo4j.cypher.internal.compiler.v3_2.commands.{ManyQueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.commands.{CompositeQueryExpression, ManyQueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.spi.QueryContext
import org.neo4j.cypher.internal.frontend.v3_2.ast._
import org.neo4j.cypher.internal.frontend.v3_2.test_helpers.{CypherFunSuite, WindowsStringSafe}
Expand Down Expand Up @@ -183,6 +183,28 @@ class NodeIndexSeekPipeTest extends CypherFunSuite with AstConstructionTestSuppo
result.map(_("n")).toList should equal(List(node, node))
}

test("should handle index lookups for composite index lookups over multiple values") {
// given
val queryState = QueryStateHelper.emptyWith(// WHERE n.prop = 'hello' AND n.prop2 = 'world']
query = indexFor(
Seq("hello", "world") -> Iterator(node),
"hello" -> Iterator(node, node2)
)
)

// when
val pipe = NodeIndexSeekPipe("n", label,
propertyKey :+ PropertyKeyToken(PropertyKeyName("prop2") _, PropertyKeyId(11)),
CompositeQueryExpression(ListLiteral(
Literal("hello"),
Literal("world")
)))()
val result = pipe.createResults(queryState)

// then
result.map(_("n")).toList should equal(List(node))
}

test("should give a helpful error message") {
// given
val queryState = QueryStateHelper.empty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ package org.neo4j.cypher.internal.compiler.v3_2.planner.logical

import org.neo4j.cypher.internal.compiler.v3_2._
import org.neo4j.cypher.internal.compiler.v3_2.ast.{InequalitySeekRangeWrapper, PrefixSeekRangeWrapper}
import org.neo4j.cypher.internal.compiler.v3_2.commands.{ManyQueryExpression, RangeQueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.commands.{CompositeQueryExpression, ManyQueryExpression, RangeQueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.planner.BeLikeMatcher._
import org.neo4j.cypher.internal.compiler.v3_2.planner.LogicalPlanningTestSupport2
import org.neo4j.cypher.internal.compiler.v3_2.planner.logical.Metrics.QueryGraphSolverInput
Expand Down Expand Up @@ -219,6 +219,7 @@ class LeafPlanningIntegrationTest extends CypherFunSuite with LogicalPlanningTes
)(solved)
)
}

test("should plan index seek by string range for textual inequality predicate") {
(new given {
indexOn("Person", "name")
Expand Down Expand Up @@ -432,6 +433,67 @@ class LeafPlanningIntegrationTest extends CypherFunSuite with LogicalPlanningTes
}
}

//
// Composite indexes
//

test("should plan composite index seek when there is an index on two properties and both are in equality predicates") {
implicit val plan = new given {
indexOn("Awesome", Seq("prop","prop2"))
} getLogicalPlanFor "MATCH (n:Awesome) WHERE n.prop = 42 AND n.prop2 = 'foo' RETURN n"

plan._2 should equal(
NodeIndexSeek(
"n",
LabelToken("Awesome", LabelId(0)),
Seq(
PropertyKeyToken(PropertyKeyName("prop") _, PropertyKeyId(0)),
PropertyKeyToken(PropertyKeyName("prop2") _, PropertyKeyId(1))),
CompositeQueryExpression(ListLiteral(Seq(SignedDecimalIntegerLiteral("42") _, StringLiteral("foo") _)) _),
Set.empty)(solved)
)
}

test("should plan composite index seek when there is an index on two properties and both are in equality predicates regardless of predicate order") {
implicit val plan = new given {
indexOn("Awesome", Seq("prop","prop2"))
} getLogicalPlanFor "MATCH (n:Awesome) WHERE n.prop2 = 'foo' AND n.prop = 42 RETURN n"

plan._2 should equal(
NodeIndexSeek(
"n",
LabelToken("Awesome", LabelId(0)),
Seq(
PropertyKeyToken(PropertyKeyName("prop") _, PropertyKeyId(0)),
PropertyKeyToken(PropertyKeyName("prop2") _, PropertyKeyId(1))),
CompositeQueryExpression(ListLiteral(Seq(SignedDecimalIntegerLiteral("42") _, StringLiteral("foo") _)) _),
Set.empty)(solved)
)
}

test("should plan composite index seek and filter when there is an index on two properties and both are in equality predicates together with other predicates") {
implicit val plan = new given {
indexOn("Awesome", Seq("prop","prop2"))
} getLogicalPlanFor "MATCH (n:Awesome) WHERE n.prop2 = 'foo' AND exists(n.name) AND n.prop = 42 RETURN n"

plan._2 should equal(
Selection(Seq(FunctionInvocation(FunctionName("exists") _, Property(varFor("n"), PropertyKeyName("name") _) _) _),
NodeIndexSeek(
"n",
LabelToken("Awesome", LabelId(0)),
Seq(
PropertyKeyToken(PropertyKeyName("prop") _, PropertyKeyId(0)),
PropertyKeyToken(PropertyKeyName("prop2") _, PropertyKeyId(1))),
CompositeQueryExpression(ListLiteral(Seq(SignedDecimalIntegerLiteral("42") _, StringLiteral("foo") _)) _),
Set.empty)(solved)
)(solved)
)
}

//
// index hints
//

test("should plan hinted label scans") {

implicit val plan = new given {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
package org.neo4j.cypher.internal.compiler.v3_2.planner.logical.plans

import org.neo4j.cypher.internal.frontend.v3_2.ast._
import org.neo4j.cypher.internal.compiler.v3_2.commands.{ManyQueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.commands.{CompositeQueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.planner.BeLikeMatcher._
import org.neo4j.cypher.internal.compiler.v3_2.planner._
import org.neo4j.cypher.internal.compiler.v3_2.planner.logical.steps.{indexSeekLeafPlanner, uniqueIndexSeekLeafPlanner}
Expand Down Expand Up @@ -108,7 +108,7 @@ class IndexSeekLeafPlannerTest extends CypherFunSuite with LogicalPlanningTestSu
resultPlans should beLike {
case Seq(NodeIndexSeek(`idName`, LabelToken("Awesome", _),
Seq(PropertyKeyToken("prop", _), PropertyKeyToken("prop2", _)),
ManyQueryExpression(ListLiteral(Seq(`lit42`, `lit6`))), _)) => ()
CompositeQueryExpression(ListLiteral(Seq(`lit42`, `lit6`))), _)) => ()
}
}
}
Expand Down Expand Up @@ -143,7 +143,7 @@ class IndexSeekLeafPlannerTest extends CypherFunSuite with LogicalPlanningTestSu
resultPlans should beLike {
case Seq(NodeIndexSeek(`idName`, LabelToken("Awesome", _),
Seq(PropertyKeyToken("prop", _), PropertyKeyToken("prop2", _)),
ManyQueryExpression(ListLiteral(Seq(`lit42`, `lit6`))), _)) => ()
CompositeQueryExpression(ListLiteral(Seq(`lit42`, `lit6`))), _)) => ()
}
}
}
Expand Down Expand Up @@ -178,7 +178,7 @@ class IndexSeekLeafPlannerTest extends CypherFunSuite with LogicalPlanningTestSu
resultPlans should beLike {
case Seq(NodeIndexSeek(`idName`, LabelToken("Awesome", _),
props@Seq(_*),
ManyQueryExpression(ListLiteral(vals@Seq(_*))), _))
CompositeQueryExpression(ListLiteral(vals@Seq(_*))), _))
if assertPropsAndValuesMatch(propertyNames, values, props, vals) => ()
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.neo4j.cypher.{ExecutionEngineFunSuite, NewPlannerTestSupport}
/**
* These tests are testing the actual index implementation, thus they should all check the actual result.
* If you only want to verify that plans using indexes are actually planned, please use
* [[org.neo4j.cypher.internal.compiler.v3_1.planner.logical.LeafPlanningIntegrationTest]]
* [[org.neo4j.cypher.internal.compiler.v3_2.planner.logical.LeafPlanningIntegrationTest]]
*/
class NodeIndexScanAcceptanceTest extends ExecutionEngineFunSuite with NewPlannerTestSupport{

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.neo4j.cypher.internal.compiled_runtime.v3_2.codegen.ir.aggregation.Di
import org.neo4j.cypher.internal.compiled_runtime.v3_2.codegen.ir.expressions.ExpressionConverter._
import org.neo4j.cypher.internal.compiled_runtime.v3_2.codegen.ir.expressions._
import org.neo4j.cypher.internal.compiled_runtime.v3_2.codegen.spi.SortItem
import org.neo4j.cypher.internal.compiler.v3_2.commands.{ManyQueryExpression, QueryExpression, RangeQueryExpression, SingleQueryExpression}
import org.neo4j.cypher.internal.compiler.v3_2.commands._
import org.neo4j.cypher.internal.compiler.v3_2.helpers.{One, ZeroOneOrMany}
import org.neo4j.cypher.internal.compiler.v3_2.planner.logical.plans._
import org.neo4j.cypher.internal.compiler.v3_2.planner.logical.{SortDescription, plans}
Expand Down Expand Up @@ -167,10 +167,12 @@ object LogicalPlanConverter {
val (methodHandle, actions :: tl) = context.popParent().consume(context, this)
val opName = context.registerOperator(logicalPlan)
val indexSeekInstruction = valueExpr match {

//single expression, do a index lookup for that value
case SingleQueryExpression(e) =>
val expression = createExpression(e)(context)
indexSeekFun(opName, context.namer.newVarName(), expression, nodeVar, actions)

//collection, create set and for each element of the set do an index lookup
case ManyQueryExpression(e: ast.ListLiteral) =>
val expression = ToSet(createExpression(e)(context))
Expand All @@ -180,6 +182,7 @@ object LogicalPlanConverter {
ForEachExpression(expressionVar, expression,
indexSeekFun(opName, context.namer.newVarName(), LoadVariable(expressionVar), nodeVar,
actions))

//Unknown, try to cast to collection and then same as above
case ManyQueryExpression(e) =>
val expression = ToSet(CastToCollection(createExpression(e)(context)))
Expand All @@ -189,9 +192,16 @@ object LogicalPlanConverter {
indexSeekFun(opName, context.namer.newVarName(), LoadVariable(expressionVar), nodeVar,
actions))

//collection used in composite index search, pass entire collection to index seek
case CompositeQueryExpression(e: ast.ListLiteral) =>
throw new CantCompileQueryException(s"To be done")

case e: RangeQueryExpression[_] =>
throw new CantCompileQueryException(s"To be done")

case e: CompositeRangeQueryExpression[_] =>
throw new CantCompileQueryException(s"To be done")

case e => throw new InternalException(s"$e is not a valid QueryExpression")
}

Expand Down

0 comments on commit ffb7b56

Please sign in to comment.