@@ -42,6 +42,7 @@ import org.apache.spark.sql.connect.planner.PythonStreamingQueryListener
42
42
import org .apache .spark .sql .connect .planner .StreamingForeachBatchHelper
43
43
import org .apache .spark .sql .connect .service .ExecuteKey
44
44
import org .apache .spark .sql .connect .service .SessionHolder .{ERROR_CACHE_SIZE , ERROR_CACHE_TIMEOUT_SEC }
45
+ import org .apache .spark .sql .execution .QueryExecution
45
46
import org .apache .spark .sql .streaming .StreamingQueryListener
46
47
import org .apache .spark .util .{SystemClock , Utils }
47
48
@@ -450,14 +451,14 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
450
451
*/
451
452
private [connect] def usePlanCache (rel : proto.Relation , cachePlan : Boolean )(
452
453
transform : proto.Relation => LogicalPlan ): LogicalPlan = {
453
- val planCacheEnabled = Option (session)
454
- .forall(_.sessionState.conf.getConf(Connect .CONNECT_SESSION_PLAN_CACHE_ENABLED , true ))
455
454
// We only cache plans that have a plan ID.
456
- val hasPlanId = rel.hasCommon && rel.getCommon.hasPlanId
455
+ val planCacheEnabled = rel.hasCommon && rel.getCommon.hasPlanId &&
456
+ Option (session)
457
+ .forall(_.sessionState.conf.getConf(Connect .CONNECT_SESSION_PLAN_CACHE_ENABLED , true ))
457
458
458
459
def getPlanCache (rel : proto.Relation ): Option [LogicalPlan ] =
459
460
planCache match {
460
- case Some (cache) if planCacheEnabled && hasPlanId =>
461
+ case Some (cache) if planCacheEnabled =>
461
462
Option (cache.getIfPresent(rel)) match {
462
463
case Some (plan) =>
463
464
logDebug(s " Using cached plan for relation ' $rel': $plan" )
@@ -466,20 +467,36 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
466
467
}
467
468
case _ => None
468
469
}
469
- def putPlanCache (rel : proto.Relation , plan : LogicalPlan ): Unit =
470
+ def putPlanCache (rel : proto.Relation , plan : LogicalPlan ): LogicalPlan = {
470
471
planCache match {
471
- case Some (cache) if planCacheEnabled && hasPlanId =>
472
- cache.put(rel, plan)
473
- case _ =>
472
+ case Some (cache) if planCacheEnabled =>
473
+ val analyzedPlan = if (plan.analyzed) {
474
+ plan
475
+ } else {
476
+ val qe = new QueryExecution (session, plan)
477
+ if (qe.isLazyAnalysis) {
478
+ // The plan is intended to be lazily analyzed.
479
+ plan
480
+ } else {
481
+ // Make sure that the plan is fully analyzed before being cached.
482
+ qe.assertAnalyzed()
483
+ qe.analyzed
484
+ }
485
+ }
486
+ cache.put(rel, analyzedPlan)
487
+ analyzedPlan
488
+ case _ => plan
474
489
}
490
+ }
475
491
476
492
getPlanCache(rel)
477
493
.getOrElse({
478
494
val plan = transform(rel)
479
495
if (cachePlan) {
480
496
putPlanCache(rel, plan)
497
+ } else {
498
+ plan
481
499
}
482
- plan
483
500
})
484
501
}
485
502
0 commit comments