switch (entry.getValue().getAction().getType()) { case KILL_QUERY: Future<Boolean> killFuture = wm.applyKillSessionAsync(wmTezSession, entry.getValue().getViolationMsg()); killFutures.put(wmTezSession, killFuture); break; case MOVE_TO_POOL: String destPoolName = entry.getValue().getAction().getPoolName(); Future<Boolean> moveFuture = wm.applyMoveSessionAsync(wmTezSession, destPoolName); moveFutures.put(wmTezSession, moveFuture); break;
public static TezSessionState getSession(TezSessionState session, HiveConf conf, MappingInput input, boolean isUnmanagedLlapMode, WmContext wmContext) throws Exception { Set<String> desiredCounters = new HashSet<>(); // 1. If WM is not present just go to unmanaged. WorkloadManager wm = WorkloadManager.getInstance(); if (wm == null) { LOG.debug("Using unmanaged session - WM is not initialized"); return getUnmanagedSession(session, conf, desiredCounters, isUnmanagedLlapMode, wmContext); } // 2. We will ask WM for a preliminary mapping. This allows us to escape to the unmanaged path // quickly in the common case. It's still possible that resource plan will be updated and // our preliminary mapping won't work out. We'll handle that below. if (!wm.isManaged(input)) { LOG.info("Using unmanaged session - no mapping for " + input); return getUnmanagedSession(session, conf, desiredCounters, isUnmanagedLlapMode, wmContext); } // 3. Finally, try WM. try { // Note: this may just block to wait for a session based on parallelism. LOG.info("Getting a WM session for " + input); WmTezSession result = wm.getSession(session, input, conf, wmContext); result.setWmContext(wmContext); wm.updateTriggers(result); return result; } catch (WorkloadManager.NoPoolMappingException ex) { LOG.info("NoPoolMappingException thrown. Getting an un-managed session"); return getUnmanagedSession(session, conf, desiredCounters, isUnmanagedLlapMode, wmContext); } }
this.yarnQueue = yarnQueue; this.conf = conf; this.totalQueryParallelism = determineQueryParallelism(plan); this.allocationManager = qam; this.allocationManager.setClusterChangedCallback(() -> notifyOfClusterStateChange()); conf, ConfVars.HIVE_SERVER2_TEZ_WM_AM_REGISTRY_TIMEOUT, TimeUnit.MILLISECONDS); tezAmPool = new TezSessionPool<>(conf, totalQueryParallelism, true, oldSession -> createSession(oldSession == null ? null : oldSession.getConf())); restrictedConfig = new RestrictedConfigChecker(conf); wmThread = new Thread(() -> runWmThread(), "Workload management master"); wmThread.setDaemon(true); wmThread.start(); updateResourcePlanAsync(plan).get(); // Wait for the initial resource plan to be applied.
private void initAndStartWorkloadManager(final WMFullResourcePlan resourcePlan) { if (!StringUtils.isEmpty(wmQueue)) { // Initialize workload management. LOG.info("Initializing workload management"); try { wm = WorkloadManager.create(wmQueue, getHiveConf(), resourcePlan); wm.start(); LOG.info("Workload manager initialized."); } catch (Exception e) { throw new ServiceException("Unable to instantiate and start Workload Manager", e); } } else { LOG.info("Workload management is not enabled."); } }
plan.getPlan().setDefaultPoolPath("A"); final WorkloadManager wm = new WorkloadManagerForTest("test", conf, qam, plan); wm.start(); TezSessionPool<WmTezSession> tezAmPool = wm.getTezAmPool(); WmTezSession sessionA1 = (WmTezSession) wm.getSession( null, mappingInput("A", null), conf, null); final AtomicReference<WmTezSession> sessionA2 = new AtomicReference<>(); wm.updateResourcePlanAsync(null).get(); TezSessionState r = wm.getSession(null, mappingInput("A", null), conf, null); fail("Expected an error but got " + r); } catch (WorkloadManager.NoPoolMappingException ex) { wm.updateResourcePlanAsync(plan).get(); sessionA1 = (WmTezSession) wm.getSession(null, mappingInput("A", null), conf, null); assertEquals("A", sessionA1.getPoolName()); sessionA1.returnToSessionManager();
plan.setMappings(Lists.newArrayList(mapping("A", "A"), mapping("B", "B"), mapping("C", "C"))); final WorkloadManager wm = new WorkloadManagerForTest("test", conf, qam, plan); wm.start(); WmTezSession sessionA1 = (WmTezSession) wm.getSession(null, mappingInput("A"), conf); Map<String, SessionTriggerProvider> allSessionProviders = wm.getAllSessionTriggerProviders(); assertEquals(1, allSessionProviders.get("A").getSessions().size()); assertEquals(0, allSessionProviders.get("B").getSessions().size()); Future<Boolean> future = wm.applyMoveSessionAsync(sessionA1, "B.y"); assertNotNull(future.get()); assertTrue(future.get()); wm.addTestEvent().get(); allSessionProviders = wm.getAllSessionTriggerProviders(); assertEquals(0, allSessionProviders.get("A").getSessions().size()); assertEquals(0, allSessionProviders.get("B").getSessions().size()); future = wm.applyMoveSessionAsync(sessionA1, "C"); assertNotNull(future.get()); assertTrue(future.get()); wm.addTestEvent().get(); allSessionProviders = wm.getAllSessionTriggerProviders(); assertEquals(0, allSessionProviders.get("A").getSessions().size()); assertEquals(0, allSessionProviders.get("B").getSessions().size()); future = wm.applyMoveSessionAsync(sessionA1, "B.x"); assertNotNull(future.get()); assertTrue(future.get());
@Test(timeout = 10000) public void testReuse() throws Exception { HiveConf conf = createConf(); MockQam qam = new MockQam(); WorkloadManager wm = new WorkloadManagerForTest("test", conf, 1, qam); wm.start(); TezSessionState nonPool = mock(TezSessionState.class); when(nonPool.getConf()).thenReturn(conf); doNothing().when(nonPool).close(anyBoolean()); TezSessionState session = wm.getSession(nonPool, mappingInput("user"), conf); verify(nonPool).close(anyBoolean()); assertNotSame(nonPool, session); session.returnToSessionManager(); TezSessionPoolSession diffPool = mock(TezSessionPoolSession.class); when(diffPool.getConf()).thenReturn(conf); doNothing().when(diffPool).returnToSessionManager(); session = wm.getSession(diffPool, mappingInput("user"), conf); verify(diffPool).returnToSessionManager(); assertNotSame(diffPool, session); TezSessionState session2 = wm.getSession(session, mappingInput("user"), conf); assertSame(session, session2); }
plan.setMappings(Lists.newArrayList(mapping("U", "A"))); final WorkloadManager wm = new WorkloadManagerForTest("test", conf, qam, plan); wm.start(); WmTezSession sessionA1 = (WmTezSession) wm.getSession(null, mappingInput("U"), conf); assertEquals("A", sessionA1.getPoolName()); assertEquals(0.5f, sessionA1.getClusterFraction(), EPSILON); pool("A", 1, 0.6f), pool("B", 1, 0.4f))); plan.setMappings(Lists.newArrayList(mapping("U", "B"))); wm.updateResourcePlanAsync(plan); WmTezSession sessionB1 = (WmTezSession) wm.getSession(null, mappingInput("U"), conf); assertEquals("B", sessionB1.getPoolName()); assertEquals(0.4f, sessionB1.getClusterFraction(), EPSILON);
@Test(timeout=10000) public void testClusterChange() throws Exception { final HiveConf conf = createConf(); MockQam qam = new MockQam(); WMFullResourcePlan plan = new WMFullResourcePlan(plan(), Lists.newArrayList(pool("A", 2, 1f))); plan.getPlan().setDefaultPoolPath("A"); final WorkloadManager wm = new WorkloadManagerForTest("test", conf, qam, plan); wm.start(); WmTezSession session1 = (WmTezSession) wm.getSession(null, mappingInput("A"), conf), session2 = (WmTezSession) wm.getSession(null, mappingInput("A"), conf); assertEquals(0.5, session1.getClusterFraction(), EPSILON); assertEquals(0.5, session2.getClusterFraction(), EPSILON); qam.assertWasCalledAndReset(); // If cluster info changes, qam should be called with the same fractions. wm.notifyOfClusterStateChange(); assertEquals(0.5, session1.getClusterFraction(), EPSILON); assertEquals(0.5, session2.getClusterFraction(), EPSILON); qam.assertWasCalledAndReset(); session1.returnToSessionManager(); session2.returnToSessionManager(); }
@Test(timeout = 10000) public void testReopen() throws Exception { // We should always get a different object, and cluster fraction should be propagated. HiveConf conf = createConf(); MockQam qam = new MockQam(); WorkloadManager wm = new WorkloadManagerForTest("test", conf, 1, qam); wm.start(); WmTezSession session = (WmTezSession) wm.getSession( null, mappingInput("user"), conf); assertEquals(1.0, session.getClusterFraction(), EPSILON); qam.assertWasCalledAndReset(); WmTezSession session2 = (WmTezSession) session.reopen(); assertNotSame(session, session2); wm.addTestEvent().get(); assertEquals(session2.toString(), 1.0, session2.getClusterFraction(), EPSILON); assertFalse(session.hasClusterFraction()); qam.assertWasCalledAndReset(); }
mapping("p2", "r1.p2"), mapping("r1", "r1"), mapping("r2", "r2"))); WorkloadManager wm = new WorkloadManagerForTest("test", conf, qam, plan); wm.start(); assertEquals(5, wm.getNumSessions()); WmTezSession session05of06 = (WmTezSession) wm.getSession( null, mappingInput("p1"), conf); assertEquals(0.3, session05of06.getClusterFraction(), EPSILON); WmTezSession session03of06 = (WmTezSession) wm.getSession( null, mappingInput("p2"), conf); assertEquals(0.18, session03of06.getClusterFraction(), EPSILON); WmTezSession session03of06_2 = (WmTezSession) wm.getSession( null, mappingInput("p2"), conf); assertEquals(0.09, session03of06.getClusterFraction(), EPSILON); assertEquals(0.09, session03of06_2.getClusterFraction(), EPSILON); WmTezSession session02of06 = (WmTezSession) wm.getSession( null,mappingInput("r1"), conf); assertEquals(0.12, session02of06.getClusterFraction(), EPSILON); WmTezSession session04 = (WmTezSession) wm.getSession( null, mappingInput("r2"), conf); assertEquals(0.4, session04.getClusterFraction(), EPSILON);
@VisibleForTesting public WmTezSession getSession( TezSessionState session, MappingInput input, HiveConf conf) throws Exception { return getSession(session, input, conf, null); }
@Override public void notifyOfClusterStateChange() { super.notifyOfClusterStateChange(); try { ensureWm(); } catch (InterruptedException | ExecutionException e) { throw new RuntimeException(e); } }
ListenableFuture<Boolean> future = wm.updateResourcePlanAsync(appliedRp); boolean isOk = false; try {
public void handleUpdateError(int endpointVersion) { wmParent.addUpdateError(this, endpointVersion); }
/** Called once, when HS2 initializes. */ public static WorkloadManager create(String yarnQueue, HiveConf conf, WMFullResourcePlan plan) throws ExecutionException, InterruptedException { assert INSTANCE == null; // We could derive the expected number of AMs to pass in. // Note: we pass a null token here; the tokens to talk to plugin endpoints will only be // known once the AMs register, and they are different for every AM (unlike LLAP token). LlapPluginEndpointClientImpl amComm = new LlapPluginEndpointClientImpl(conf, null, -1); QueryAllocationManager qam = new GuaranteedTasksAllocator(conf, amComm); return (INSTANCE = new WorkloadManager(amComm, yarnQueue, conf, qam, plan)); }
try { kq.killQuery(queryId, reason, toKill.getConf()); addKillQueryResult(toKill, true); killCtx.killSessionFuture.set(true); wmEvent.endEvent(toKill); addKillQueryResult(toKill, false); });
plan.setMappings(Lists.newArrayList(mapping("A", "A"))); final WorkloadManager wm = new WorkloadManagerForTest("test", conf, qam, plan); wm.start(); TezSessionPool<WmTezSession> pool = wm.getTezAmPool(); WmTezSession oob = pool.getSession(); pool("A", 4, 1.0f))); plan.setMappings(Lists.newArrayList(mapping("A", "A"))); wm.updateResourcePlanAsync(plan); WmTezSession oob2 = pool.getSession(), oob3 = pool.getSession(), wm.updateResourcePlanAsync(plan); wm.addTestEvent().get(); assertEquals(0, pool.getCurrentSize()); sessionA1.get().returnToSessionManager(); wm.updateResourcePlanAsync(plan); oob2 = pool.getSession(); oob3 = pool.getSession(); plan = new WMFullResourcePlan(plan(), Lists.newArrayList(pool("A", 1, 1.0f))); plan.setMappings(Lists.newArrayList(mapping("A", "A"))); wm.updateResourcePlanAsync(plan); plan = new WMFullResourcePlan(plan(), Lists.newArrayList(pool("A", 2, 1.0f))); plan.setMappings(Lists.newArrayList(mapping("A", "A"))); wm.updateResourcePlanAsync(plan); wm.addTestEvent().get();
plan.setMappings(Lists.newArrayList(mapping("A", "A"), mapping("B", "B"))); final WorkloadManager wm = new WorkloadManagerForTest("test", conf, qam, plan); wm.start(); WmTezSession sessionA1 = (WmTezSession) wm.getSession(null, mappingInput("A"), conf); Map<String, SessionTriggerProvider> allSessionProviders = wm.getAllSessionTriggerProviders(); assertEquals(1, allSessionProviders.get("A").getSessions().size()); assertEquals(0, allSessionProviders.get("B").getSessions().size()); Future<Boolean> future = wm.applyMoveSessionAsync(sessionA1, "B"); assertNotNull(future.get()); assertTrue(future.get()); wm.addTestEvent().get(); allSessionProviders = wm.getAllSessionTriggerProviders(); assertEquals(0, allSessionProviders.get("A").getSessions().size()); assertEquals(1, allSessionProviders.get("B").getSessions().size()); WmTezSession sessionA2 = (WmTezSession) wm.getSession(null, mappingInput("A"), conf); allSessionProviders = wm.getAllSessionTriggerProviders(); assertEquals(1, allSessionProviders.get("A").getSessions().size()); assertEquals(1, allSessionProviders.get("B").getSessions().size()); future = wm.applyMoveSessionAsync(sessionA2, "B"); assertNotNull(future.get()); assertTrue(future.get()); wm.addTestEvent().get(); allSessionProviders = wm.getAllSessionTriggerProviders();
@Test(timeout = 10000) public void testQueueName() throws Exception { HiveConf conf = createConf(); MockQam qam = new MockQam(); WorkloadManager wm = new WorkloadManagerForTest("test", conf, 1, qam); wm.start(); // The queue should be ignored. conf.set(TezConfiguration.TEZ_QUEUE_NAME, "test2"); TezSessionState session = wm.getSession(null, mappingInput("user"), conf); assertEquals("test", session.getQueueName()); assertEquals("test", conf.get(TezConfiguration.TEZ_QUEUE_NAME)); session.setQueueName("test2"); session = wm.getSession(session, mappingInput("user"), conf); assertEquals("test", session.getQueueName()); }