diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java index a8360bd..85dc66c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java @@ -156,6 +156,7 @@ // The initial plan initalization future, to wait for the plan to apply during setup. private ListenableFuture initRpFuture; private LlapPluginEndpointClientImpl amComm; + private WMFullResourcePlan initialRP; private static final FutureCallback FATAL_ERROR_CALLBACK = new FutureCallback() { @Override @@ -190,7 +191,7 @@ public static WorkloadManager create(String yarnQueue, HiveConf conf, WMFullReso this.yarnQueue = yarnQueue; this.conf = conf; this.totalQueryParallelism = determineQueryParallelism(plan); - this.initRpFuture = this.updateResourcePlanAsync(plan); + this.initialRP = plan; this.amComm = amComm; if (this.amComm != null) { this.amComm.init(conf); @@ -234,6 +235,9 @@ public void start() throws Exception { } allocationManager.start(); wmThread.start(); + + // apply initial RP after starting WM thread for proper initialization of user pool mappings + initRpFuture = this.updateResourcePlanAsync(initialRP); initRpFuture.get(); // Wait for the initial resource plan to be applied. final long triggerValidationIntervalMs = HiveConf.getTimeVar(conf,