Skip to content

Commit

Permalink
Unify top-level ExecutionPlan and CacheableExecutableQuery
Browse files Browse the repository at this point in the history
  • Loading branch information
alexaverbuch authored and fickludd committed Jun 26, 2018
1 parent be139f1 commit ceedbed
Show file tree
Hide file tree
Showing 11 changed files with 82 additions and 72 deletions.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -43,5 +43,5 @@ trait Compiler {
tracer: CompilationPhaseTracer,
preParsingNotifications: Set[org.neo4j.graphdb.Notification],
transactionalContext: TransactionalContext
): CacheableExecutableQuery
): ExecutableQuery
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,36 @@ import org.neo4j.kernel.api.query.PlannerInfo
import org.neo4j.kernel.impl.query.TransactionalContext
import org.neo4j.values.virtual.MapValue

trait ExecutionPlan {
/**
* A fully compiled query in executable form.
*/
trait ExecutableQuery {

/**
* Execute this executable query.
*
* @param transactionalContext the transaction in which to execute
* @param executionMode the execution mode
* @param params the parameters
* @return the query result
*/
def run(transactionalContext: TransactionalContext, executionMode: CypherExecutionMode, params: MapValue): Result

/**
* The reusability state of this executable query.
*/
def reusabilityState(lastCommittedTxId: () => Long, ctx: TransactionalContext): ReusabilityState

// This is to force eager calculation
val plannerInfo: PlannerInfo

/**
* Names of all parameters for this query, explicit and auto-parametrized.
*/
val paramNames: Seq[String]

/**
* The names and values of the auto-parametrized parameters for this query.
*/
val extractedParams: MapValue
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,9 +56,9 @@ class ExecutionEngine(val queryService: GraphDatabaseQueryService,

private val preParser = new PreParser(config.version, config.planner, config.runtime, config.queryCacheSize)
private val lastCommittedTxIdProvider = LastCommittedTxIdProvider(queryService)
private def planReusabilitiy(cachedExecutableQuery: CacheableExecutableQuery,
private def planReusabilitiy(executableQuery: ExecutableQuery,
transactionalContext: TransactionalContext): ReusabilityState =
cachedExecutableQuery.plan.reusabilityState(lastCommittedTxIdProvider, transactionalContext)
executableQuery.reusabilityState(lastCommittedTxIdProvider, transactionalContext)

// Log on stale query discard from query cache
private val log = logProvider.getLog( getClass )
Expand All @@ -69,11 +69,11 @@ class ExecutionEngine(val queryService: GraphDatabaseQueryService,
})

private val planStalenessCaller =
new PlanStalenessCaller[CacheableExecutableQuery](clock,
config.statsDivergenceCalculator,
lastCommittedTxIdProvider,
planReusabilitiy)
private val queryCache: QueryCache[String, CacheableExecutableQuery] =
new PlanStalenessCaller[ExecutableQuery](clock,
config.statsDivergenceCalculator,
lastCommittedTxIdProvider,
planReusabilitiy)
private val queryCache: QueryCache[String, ExecutableQuery] =
new QueryCache(config.queryCacheSize, planStalenessCaller, cacheTracer)

private val masterCompiler: MasterCompiler =
Expand All @@ -91,13 +91,13 @@ class ExecutionEngine(val queryService: GraphDatabaseQueryService,

try {
val preParsedQuery = preParser.preParseQuery(query, profile)
val cachedExecutableQuery = getOrCompile(context, preParsedQuery, queryTracer)
val executableQuery = getOrCompile(context, preParsedQuery, queryTracer)
if (preParsedQuery.executionMode.name != "explain") {
checkParameters(cachedExecutableQuery.paramNames, params, cachedExecutableQuery.extractedParams)
checkParameters(executableQuery.paramNames, params, executableQuery.extractedParams)
}
val combinedParams = params.updatedWith(cachedExecutableQuery.extractedParams)
context.executingQuery().planningCompleted(cachedExecutableQuery.plan.plannerInfo)
cachedExecutableQuery.plan.run(context, preParsedQuery.executionMode, combinedParams)
val combinedParams = params.updatedWith(executableQuery.extractedParams)
context.executingQuery().planningCompleted(executableQuery.plannerInfo)
executableQuery.run(context, preParsedQuery.executionMode, combinedParams)

} catch {
case t: Throwable =>
Expand All @@ -117,7 +117,7 @@ class ExecutionEngine(val queryService: GraphDatabaseQueryService,
private def getOrCompile(context: TransactionalContext,
preParsedQuery: PreParsedQuery,
tracer: QueryCompilationEvent
): CacheableExecutableQuery = {
): ExecutableQuery = {
val cacheKey = preParsedQuery.statementWithVersionAndPlanner

// create transaction and query context
Expand All @@ -137,7 +137,7 @@ class ExecutionEngine(val queryService: GraphDatabaseQueryService,
case _: CacheHit[_] |
_: CacheDisabled[_] =>
val executableQuery = cacheLookup.executableQuery
if (schemaHelper.lockLabels(schemaToken, executableQuery.plan, preParsedQuery.version, tc)) {
if (schemaHelper.lockLabels(schemaToken, executableQuery, preParsedQuery.version, tc)) {
tc.cleanForReuse()
return executableQuery
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ class MasterCompiler(graph: GraphDatabaseQueryService,
def compile(preParsedQuery: PreParsedQuery,
tracer: CompilationPhaseTracer,
transactionalContext: TransactionalContext
): CacheableExecutableQuery = {
): ExecutableQuery = {

var notifications = Set.newBuilder[org.neo4j.graphdb.Notification]
val supportedRuntimes3_1 = Seq(CypherRuntimeOption.interpreted, CypherRuntimeOption.default)
Expand All @@ -115,7 +115,7 @@ class MasterCompiler(graph: GraphDatabaseQueryService,
* @param preParsedQuery the query to compile
* @return the compiled query
*/
def innerCompile(preParsedQuery: PreParsedQuery): CacheableExecutableQuery = {
def innerCompile(preParsedQuery: PreParsedQuery): ExecutableQuery = {

if ((preParsedQuery.version == CypherVersion.v3_3 || preParsedQuery.version == CypherVersion.v3_5) && preParsedQuery.planner == CypherPlannerOption.rule) {
notifications += rulePlannerUnavailableFallbackNotification(preParsedQuery.offset)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,5 +144,5 @@ class QueryCache[QUERY_KEY <: AnyRef, EXECUTABLE_QUERY <: AnyRef](val maximumSiz
}

object QueryCache {
val NOT_PRESENT: CacheableExecutableQuery = null
val NOT_PRESENT: ExecutableQuery = null
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ class SchemaHelper(val queryCache: QueryCache[_,_]) {
}

def lockLabels(schemaTokenBefore: SchemaToken,
executionPlan: ExecutionPlan,
executionPlan: ExecutableQuery,
version: CypherVersion,
tc: TransactionalContext): Boolean = {
val labelIds: Seq[Long] = extractPlanLabels(executionPlan, version, tc)
Expand All @@ -59,7 +59,7 @@ class SchemaHelper(val queryCache: QueryCache[_,_]) {
true
}

private def extractPlanLabels(plan: ExecutionPlan, version: CypherVersion, tc: TransactionalContext): Seq[Long] = {
private def extractPlanLabels(plan: ExecutableQuery, version: CypherVersion, tc: TransactionalContext): Seq[Long] = {
import scala.collection.JavaConverters._

def planLabels = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ case class CypherCurrentCompiler[CONTEXT <: RuntimeContext](planner: CypherPlann
override def compile(preParsedQuery: PreParsedQuery,
tracer: CompilationPhaseTracer,
preParsingNotifications: Set[Notification],
transactionalContext: TransactionalContext): CacheableExecutableQuery = {
transactionalContext: TransactionalContext): ExecutableQuery = {

val logicalPlanResult =
planner.parseAndPlan(preParsedQuery, tracer, preParsingNotifications, transactionalContext)
Expand All @@ -82,14 +82,19 @@ case class CypherCurrentCompiler[CONTEXT <: RuntimeContext](planner: CypherPlann

val executionPlan3_5 = runtime.compileToExecutable(logicalPlanResult.logicalPlanState, runtimeContext)

val executionPlan = new ExecutionPlanWrapper(executionPlan3_5, preParsingNotifications, logicalPlanResult.reusability)
CacheableExecutableQuery(executionPlan, logicalPlanResult.paramNames, logicalPlanResult.extractedParams)
new CypherExecutableQuery(
executionPlan3_5,
preParsingNotifications,
logicalPlanResult.reusability,
logicalPlanResult.paramNames,
logicalPlanResult.extractedParams)
}

protected class ExecutionPlanWrapper(inner: ExecutionPlan_v3_5,
preParsingNotifications: Set[org.neo4j.graphdb.Notification],
reusabilityState: ReusabilityState)
extends ExecutionPlan {
protected class CypherExecutableQuery(inner: ExecutionPlan_v3_5,
preParsingNotifications: Set[org.neo4j.graphdb.Notification],
reusabilityState: ReusabilityState,
override val paramNames: Seq[String],
override val extractedParams: MapValue) extends ExecutableQuery {

private val searchMonitor = kernelMonitors.newMonitor(classOf[IndexSearchMonitor])

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,8 +67,11 @@ trait Cypher23Compiler extends CachingPlanner[PreparedQuery] with Compiler {

implicit val executionMonitor: QueryExecutionMonitor = kernelMonitors.newMonitor(classOf[QueryExecutionMonitor])

class ExecutionPlanWrapper(inner: ExecutionPlan_v2_3, preParsingNotifications: Set[org.neo4j.graphdb.Notification], offSet: frontend.v2_3.InputPosition)
extends ExecutionPlan {
class Cypher23ExecutableQuery(inner: ExecutionPlan_v2_3,
preParsingNotifications: Set[org.neo4j.graphdb.Notification],
offSet: frontend.v2_3.InputPosition,
override val paramNames: Seq[String],
override val extractedParams: MapValue) extends ExecutableQuery {

private def queryContext(transactionalContext: TransactionalContextWrapper): QueryContext =
new ExceptionTranslatingQueryContext(new TransactionBoundQueryContext(transactionalContext))
Expand Down Expand Up @@ -121,7 +124,7 @@ trait Cypher23Compiler extends CachingPlanner[PreparedQuery] with Compiler {
tracer: CompilationPhaseTracer,
preParsingNotifications: Set[org.neo4j.graphdb.Notification],
transactionalContext: TransactionalContext
): CacheableExecutableQuery = {
): ExecutableQuery = {

exceptionHandler.runSafely {
val notificationLogger = new RecordingNotificationLogger
Expand All @@ -137,8 +140,12 @@ trait Cypher23Compiler extends CachingPlanner[PreparedQuery] with Compiler {

// Log notifications/warnings from planning
executionPlan2_3.notifications(planContext).foreach(notificationLogger += _)
val executionPlan = new ExecutionPlanWrapper(executionPlan2_3, preParsingNotifications, position2_3)
CacheableExecutableQuery(executionPlan, Seq.empty[String], ValueConversion.asValues(extractedParameters))
new Cypher23ExecutableQuery(
executionPlan2_3,
preParsingNotifications,
position2_3,
Seq.empty[String],
ValueConversion.asValues(extractedParameters))
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,8 +66,11 @@ trait Cypher31Compiler extends CachingPlanner[PreparedQuerySyntax] with Compiler

implicit val executionMonitor: QueryExecutionMonitor = kernelMonitors.newMonitor(classOf[QueryExecutionMonitor])

class ExecutionPlanWrapper(inner: ExecutionPlan_v3_1, preParsingNotifications: Set[org.neo4j.graphdb.Notification], offSet: InputPosition3_1)
extends ExecutionPlan {
class Cypher31ExecutableQuery(inner: ExecutionPlan_v3_1,
preParsingNotifications: Set[org.neo4j.graphdb.Notification],
offSet: InputPosition3_1,
override val paramNames: Seq[String],
override val extractedParams: MapValue) extends ExecutableQuery {

private val searchMonitor = kernelMonitors.newMonitor(classOf[IndexSearchMonitor])

Expand Down Expand Up @@ -123,7 +126,7 @@ trait Cypher31Compiler extends CachingPlanner[PreparedQuerySyntax] with Compiler
tracer: v3_5.phases.CompilationPhaseTracer,
preParsingNotifications: Set[org.neo4j.graphdb.Notification],
transactionalContext: TransactionalContext
): CacheableExecutableQuery = {
): ExecutableQuery = {

exceptionHandler.runSafely {
val notificationLogger = new RecordingNotificationLogger
Expand All @@ -141,8 +144,12 @@ trait Cypher31Compiler extends CachingPlanner[PreparedQuerySyntax] with Compiler
// Log notifications/warnings from planning
executionPlan3_1.notifications(planContext).foreach(notificationLogger += _)

val executionPlan = new ExecutionPlanWrapper(executionPlan3_1, preParsingNotifications, position3_1)
CacheableExecutableQuery(executionPlan, Seq.empty[String], ValueConversion.asValues(extractedParameters))
new Cypher31ExecutableQuery(
executionPlan3_1,
preParsingNotifications,
position3_1,
Seq.empty[String],
ValueConversion.asValues(extractedParameters))
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -228,5 +228,5 @@ class PlanStalenessCallerTest extends CypherFunSuite {
}

private def label(i: Int): LabelId = LabelId(i)
private def not_used(plan: ExecutionPlan, tc: TransactionalContext) = FineToReuse
private def not_used(plan: ExecutableQuery, tc: TransactionalContext) = FineToReuse
}

0 comments on commit ceedbed

Please sign in to comment.