Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1212,6 +1212,13 @@ public static void defineFilter(ServletContextHandler ctx, String name,
defineFilter(ctx, filterHolder, fmap);
}

public static void defineFilter(ServletContextHandler ctx, String name,
Filter filter, Map<String,String> parameters, String[] urls) {
FilterHolder filterHolder = getFilterHolder(name, filter, parameters);
FilterMapping fmap = getFilterMapping(name, urls);
defineFilter(ctx, filterHolder, fmap);
}

/**
* Define a filter for a context and set up default url mappings.
*/
Expand All @@ -1229,11 +1236,15 @@ private static FilterMapping getFilterMapping(String name, String[] urls) {
return fmap;
}

private static FilterHolder getFilterHolder(String name, String classname,
private static FilterHolder getFilterHolder(String name, Object filter,
Map<String, String> parameters) {
FilterHolder holder = new FilterHolder();
holder.setName(name);
holder.setClassName(classname);
if (filter instanceof String) {
holder.setClassName((String)filter);
} else {
holder.setFilter((Filter)filter);
}
if (parameters != null) {
holder.setInitParameters(parameters);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,9 @@ void setHostClass(Class<?> cls) {

void setGuiceFilter(GuiceFilter instance) {
guiceFilter = instance;
LOG.info("Registered webapp guice modules");
HttpServer2.defineFilter(httpServer.getWebAppContext(), "guice",
instance, null, new String[] { "/*" });
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -406,10 +406,6 @@ public void setup() {
restCsrfClassName, params,
new String[] {"/*"});
}

HttpServer2.defineFilter(server.getWebAppContext(), "guice",
GuiceFilter.class.getName(), null, new String[] { "/*" });

webapp.setConf(conf);
webapp.setHttpServer(server);
} catch (ClassNotFoundException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;

import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;

import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
Expand Down Expand Up @@ -123,7 +125,9 @@ protected void serviceStart() throws Exception {
.withCSRFProtection(YarnConfiguration.NM_CSRF_PREFIX)
.withXFSProtection(YarnConfiguration.NM_XFS_PREFIX)
.start(this.nmWebApp);
this.port = this.webApp.httpServer().getConnectorAddress(0).getPort();
InetSocketAddress webAddr = this.webApp.httpServer().getConnectorAddress(0);
this.port = webAddr.getPort();
LOG.info("NMWebApp at " + webAddr.getHostName() + ":" + port);
} catch (Exception e) {
String msg = "NMWebapps failed to start.";
LOG.error(msg, e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -592,7 +592,7 @@ private NMLivelinessMonitor createNMLivelinessMonitor() {
protected AMLivelinessMonitor createAMLivelinessMonitor() {
return new AMLivelinessMonitor(this.rmDispatcher);
}

protected RMNodeLabelsManager createNodeLabelManager()
throws InstantiationException, IllegalAccessException {
return new RMNodeLabelsManager();
Expand All @@ -613,7 +613,7 @@ protected AllocationTagsManager createAllocationTagsManager() {
// Use the in memory Placement Constraint Manager.
return new MemoryPlacementConstraintManager();
}

protected DelegationTokenRenewer createDelegationTokenRenewer() {
return new DelegationTokenRenewer();
}
Expand Down Expand Up @@ -771,7 +771,7 @@ protected void serviceInit(Configuration configuration) throws Exception {
AMLivelinessMonitor amFinishingMonitor = createAMLivelinessMonitor();
addService(amFinishingMonitor);
rmContext.setAMFinishingMonitor(amFinishingMonitor);

RMAppLifetimeMonitor rmAppLifetimeMonitor = createRMAppLifetimeMonitor();
addService(rmAppLifetimeMonitor);
rmContext.setRMAppLifetimeMonitor(rmAppLifetimeMonitor);
Expand Down Expand Up @@ -1478,6 +1478,8 @@ protected void startWepApp() {

try {
webApp = builder.start(new RMWebApp(this), uiWebAppContext);
InetSocketAddress webAddr = webApp.httpServer().getConnectorAddress(0);
LOG.info("RMWebApp at " + webAddr.getHostName() + ":" + webAddr.getPort());
} catch (WebAppException e) {
webApp = e.getWebApp();
throw e;
Expand Down Expand Up @@ -1603,7 +1605,7 @@ protected void serviceStart() throws Exception {
transitionToActive();
}
}

protected void doSecureLogin() throws IOException {
InetSocketAddress socAddr = getBindAddress(conf);
SecurityUtil.login(this.conf, YarnConfiguration.RM_KEYTAB,
Expand Down Expand Up @@ -1634,7 +1636,7 @@ protected void serviceStop() throws Exception {
rmContext.setHAServiceState(HAServiceState.STOPPING);
rmStatusInfoBean.unregister();
}

protected ResourceTrackerService createResourceTrackerService() {
return new ResourceTrackerService(this.rmContext, this.nodesListManager,
this.nmLivelinessMonitor,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import org.junit.jupiter.api.Test;

import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URL;

public class TestMiniYarnCluster {

Expand Down Expand Up @@ -153,4 +155,40 @@ void testMultiRMConf() throws IOException {
assertEquals("rm2", conf2.get(YarnConfiguration.RM_HA_ID));
}
}

@Test
void testWebAppPortBinding() throws Exception {
Configuration conf = new YarnConfiguration();
int numNodeManagers = 1;
int numLocalDirs = 1;
int numLogDirs = 1;

try (MiniYARNCluster cluster =
new MiniYARNCluster(TestMiniYarnCluster.class.getSimpleName(),
numNodeManagers, numLocalDirs, numLogDirs, numLogDirs)) {

cluster.init(conf);
cluster.start();

Configuration rmConf = cluster.getResourceManager(0).getConfig();
Configuration nmConf = cluster.getNodeManager(0).getConfig();

URL nmWebappURL =
new URL("http://" + nmConf.get(YarnConfiguration.NM_WEBAPP_ADDRESS) + "/node");
URL rmWebappURL =
new URL("http://" + rmConf.get(YarnConfiguration.RM_WEBAPP_ADDRESS) + "/cluster");

HttpURLConnection nmConn = (HttpURLConnection) nmWebappURL.openConnection();
nmConn.setRequestMethod("GET");
nmConn.connect();
assertEquals(nmConn.getResponseCode(), 200);

// Check the ResourceManager after the NodeManager to make sure it wasn't
// overwritten. Please see YARN-7747.
HttpURLConnection rmConn = (HttpURLConnection) rmWebappURL.openConnection();
rmConn.setRequestMethod("GET");
rmConn.connect();
assertEquals(rmConn.getResponseCode(), 200);
}
}
}