YARN-3333. Rename TimelineAggregator etc. to TimelineCollector. Contributed by Sangjin Lee
This commit is contained in:
parent
9b56364080
commit
2188a07e5b
|
@ -47,6 +47,10 @@
|
|||
<groupId>commons-logging</groupId>
|
||||
<artifactId>commons-logging</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.xml.bind</groupId>
|
||||
<artifactId>jaxb-api</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
|
||||
<dependency>
|
||||
|
|
|
@ -127,7 +127,7 @@ public abstract class AllocateResponse {
|
|||
response.setAMRMToken(amRMToken);
|
||||
return response;
|
||||
}
|
||||
|
||||
|
||||
@Public
|
||||
@Unstable
|
||||
public static AllocateResponse newInstance(int responseId,
|
||||
|
@ -137,13 +137,13 @@ public abstract class AllocateResponse {
|
|||
PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
|
||||
List<Container> increasedContainers,
|
||||
List<Container> decreasedContainers,
|
||||
String aggregatorAddr) {
|
||||
String collectorAddr) {
|
||||
AllocateResponse response =
|
||||
newInstance(responseId, completedContainers, allocatedContainers,
|
||||
updatedNodes, availResources, command, numClusterNodes, preempt,
|
||||
nmTokens, increasedContainers, decreasedContainers);
|
||||
response.setAMRMToken(amRMToken);
|
||||
response.setAggregatorAddr(aggregatorAddr);
|
||||
response.setCollectorAddr(collectorAddr);
|
||||
return response;
|
||||
}
|
||||
|
||||
|
@ -349,16 +349,16 @@ public abstract class AllocateResponse {
|
|||
public abstract void setApplicationPriority(Priority priority);
|
||||
|
||||
/**
|
||||
* The address of aggregator that belong to this app
|
||||
* The address of collector that belong to this app
|
||||
*
|
||||
* @return The address of aggregator that belong to this attempt
|
||||
* @return The address of collector that belong to this attempt
|
||||
*/
|
||||
@Public
|
||||
@Unstable
|
||||
public abstract String getAggregatorAddr();
|
||||
|
||||
public abstract String getCollectorAddr();
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract void setAggregatorAddr(String aggregatorAddr);
|
||||
public abstract void setCollectorAddr(String collectorAddr);
|
||||
|
||||
}
|
||||
|
|
|
@ -47,7 +47,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Get a list of {@link TimelineWriteError} instances
|
||||
*
|
||||
*
|
||||
* @return a list of {@link TimelineWriteError} instances
|
||||
*/
|
||||
@XmlElement(name = "errors")
|
||||
|
@ -57,7 +57,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Add a single {@link TimelineWriteError} instance into the existing list
|
||||
*
|
||||
*
|
||||
* @param error
|
||||
* a single {@link TimelineWriteError} instance
|
||||
*/
|
||||
|
@ -67,7 +67,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Add a list of {@link TimelineWriteError} instances into the existing list
|
||||
*
|
||||
*
|
||||
* @param errors
|
||||
* a list of {@link TimelineWriteError} instances
|
||||
*/
|
||||
|
@ -77,7 +77,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Set the list to the given list of {@link TimelineWriteError} instances
|
||||
*
|
||||
*
|
||||
* @param errors
|
||||
* a list of {@link TimelineWriteError} instances
|
||||
*/
|
||||
|
@ -107,7 +107,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Get the entity Id
|
||||
*
|
||||
*
|
||||
* @return the entity Id
|
||||
*/
|
||||
@XmlElement(name = "entity")
|
||||
|
@ -117,7 +117,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Set the entity Id
|
||||
*
|
||||
*
|
||||
* @param entityId
|
||||
* the entity Id
|
||||
*/
|
||||
|
@ -127,7 +127,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Get the entity type
|
||||
*
|
||||
*
|
||||
* @return the entity type
|
||||
*/
|
||||
@XmlElement(name = "entitytype")
|
||||
|
@ -137,7 +137,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Set the entity type
|
||||
*
|
||||
*
|
||||
* @param entityType
|
||||
* the entity type
|
||||
*/
|
||||
|
@ -147,7 +147,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Get the error code
|
||||
*
|
||||
*
|
||||
* @return an error code
|
||||
*/
|
||||
@XmlElement(name = "errorcode")
|
||||
|
@ -157,7 +157,7 @@ public class TimelineWriteResponse {
|
|||
|
||||
/**
|
||||
* Set the error code to the given error code
|
||||
*
|
||||
*
|
||||
* @param errorCode
|
||||
* an error code
|
||||
*/
|
||||
|
|
|
@ -814,10 +814,10 @@ public class YarnConfiguration extends Configuration {
|
|||
public static final int DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT = 20;
|
||||
|
||||
/** Number of threads container manager uses.*/
|
||||
public static final String NM_AGGREGATOR_SERVICE_THREAD_COUNT =
|
||||
NM_PREFIX + "aggregator-service.thread-count";
|
||||
public static final int DEFAULT_NM_AGGREGATOR_SERVICE_THREAD_COUNT = 5;
|
||||
|
||||
public static final String NM_COLLECTOR_SERVICE_THREAD_COUNT =
|
||||
NM_PREFIX + "collector-service.thread-count";
|
||||
public static final int DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT = 5;
|
||||
|
||||
/** Number of threads used in cleanup.*/
|
||||
public static final String NM_DELETE_THREAD_COUNT =
|
||||
NM_PREFIX + "delete.thread-count";
|
||||
|
@ -845,13 +845,13 @@ public class YarnConfiguration extends Configuration {
|
|||
public static final String DEFAULT_NM_LOCALIZER_ADDRESS = "0.0.0.0:" +
|
||||
DEFAULT_NM_LOCALIZER_PORT;
|
||||
|
||||
/** Address where the aggregator service IPC is.*/
|
||||
public static final String NM_AGGREGATOR_SERVICE_ADDRESS =
|
||||
NM_PREFIX + "aggregator-service.address";
|
||||
public static final int DEFAULT_NM_AGGREGATOR_SERVICE_PORT = 8048;
|
||||
public static final String DEFAULT_NM_AGGREGATOR_SERVICE_ADDRESS =
|
||||
/** Address where the collector service IPC is.*/
|
||||
public static final String NM_COLLECTOR_SERVICE_ADDRESS =
|
||||
NM_PREFIX + "collector-service.address";
|
||||
public static final int DEFAULT_NM_COLLECTOR_SERVICE_PORT = 8048;
|
||||
public static final String DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS =
|
||||
"0.0.0.0:" + DEFAULT_NM_LOCALIZER_PORT;
|
||||
|
||||
|
||||
/** Interval in between cache cleanups.*/
|
||||
public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =
|
||||
NM_PREFIX + "localizer.cache.cleanup.interval-ms";
|
||||
|
|
|
@ -89,7 +89,7 @@ message AllocateResponseProto {
|
|||
repeated ContainerProto decreased_containers = 11;
|
||||
optional hadoop.common.TokenProto am_rm_token = 12;
|
||||
optional PriorityProto application_priority = 13;
|
||||
optional string aggregator_addr = 14;
|
||||
optional string collector_addr = 14;
|
||||
}
|
||||
|
||||
enum SchedulerResourceTypes {
|
||||
|
|
|
@ -70,6 +70,15 @@
|
|||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-timelineservice</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
|
|
|
@ -40,10 +40,10 @@ import java.util.Set;
|
|||
import java.util.Vector;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.GnuParser;
|
||||
|
@ -222,12 +222,12 @@ public class ApplicationMaster {
|
|||
private int appMasterRpcPort = -1;
|
||||
// Tracking url to which app master publishes info for clients to monitor
|
||||
private String appMasterTrackingUrl = "";
|
||||
|
||||
|
||||
private boolean newTimelineService = false;
|
||||
|
||||
|
||||
// For posting entities in new timeline service in a non-blocking way
|
||||
// TODO replace with event loop in TimelineClient.
|
||||
private static ExecutorService threadPool =
|
||||
private static ExecutorService threadPool =
|
||||
Executors.newCachedThreadPool(
|
||||
new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
|
||||
.build());
|
||||
|
@ -331,9 +331,9 @@ public class ApplicationMaster {
|
|||
}
|
||||
appMaster.run();
|
||||
result = appMaster.finish();
|
||||
|
||||
|
||||
threadPool.shutdown();
|
||||
|
||||
|
||||
while (!threadPool.isTerminated()) { // wait for all posting thread to finish
|
||||
try {
|
||||
if (!threadPool.awaitTermination(30, TimeUnit.SECONDS)) {
|
||||
|
@ -427,7 +427,7 @@ public class ApplicationMaster {
|
|||
opts.addOption("container_retry_interval", true,
|
||||
"Interval between each retry, unit is milliseconds");
|
||||
opts.addOption("debug", false, "Dump out debug information");
|
||||
opts.addOption("timeline_service_version", true,
|
||||
opts.addOption("timeline_service_version", true,
|
||||
"Version for timeline service");
|
||||
opts.addOption("help", false, "Print usage");
|
||||
CommandLine cliParser = new GnuParser().parse(opts, args);
|
||||
|
@ -583,7 +583,7 @@ public class ApplicationMaster {
|
|||
if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
|
||||
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
|
||||
if (cliParser.hasOption("timeline_service_version")) {
|
||||
String timelineServiceVersion =
|
||||
String timelineServiceVersion =
|
||||
cliParser.getOptionValue("timeline_service_version", "v1");
|
||||
if (timelineServiceVersion.trim().equalsIgnoreCase("v1")) {
|
||||
newTimelineService = false;
|
||||
|
@ -655,7 +655,7 @@ public class ApplicationMaster {
|
|||
amRMClient = AMRMClientAsync.createAMRMClientAsync(1000, allocListener);
|
||||
amRMClient.init(conf);
|
||||
amRMClient.start();
|
||||
|
||||
|
||||
containerListener = createNMCallbackHandler();
|
||||
nmClientAsync = new NMClientAsyncImpl(containerListener);
|
||||
nmClientAsync.init(conf);
|
||||
|
@ -840,7 +840,7 @@ public class ApplicationMaster {
|
|||
if(timelineClient != null) {
|
||||
timelineClient.stop();
|
||||
}
|
||||
|
||||
|
||||
return success;
|
||||
}
|
||||
|
||||
|
@ -1392,11 +1392,11 @@ public class ApplicationMaster {
|
|||
}
|
||||
|
||||
private static void publishContainerStartEventOnNewTimelineService(
|
||||
final TimelineClient timelineClient, final Container container,
|
||||
final TimelineClient timelineClient, final Container container,
|
||||
final String domainId, final UserGroupInformation ugi) {
|
||||
Runnable publishWrapper = new Runnable() {
|
||||
public void run() {
|
||||
publishContainerStartEventOnNewTimelineServiceBase(timelineClient,
|
||||
publishContainerStartEventOnNewTimelineServiceBase(timelineClient,
|
||||
container, domainId, ugi);
|
||||
}
|
||||
};
|
||||
|
@ -1406,14 +1406,14 @@ public class ApplicationMaster {
|
|||
private static void publishContainerStartEventOnNewTimelineServiceBase(
|
||||
final TimelineClient timelineClient, Container container, String domainId,
|
||||
UserGroupInformation ugi) {
|
||||
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
|
||||
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
|
||||
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
|
||||
entity.setId(container.getId().toString());
|
||||
entity.setType(DSEntity.DS_CONTAINER.toString());
|
||||
//entity.setDomainId(domainId);
|
||||
entity.addInfo("user", ugi.getShortUserName());
|
||||
|
||||
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
|
||||
|
||||
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
|
||||
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
|
||||
event.setTimestamp(System.currentTimeMillis());
|
||||
event.setId(DSEvent.DS_CONTAINER_START.toString());
|
||||
|
@ -1435,29 +1435,29 @@ public class ApplicationMaster {
|
|||
e instanceof UndeclaredThrowableException ? e.getCause() : e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static void publishContainerEndEventOnNewTimelineService(
|
||||
final TimelineClient timelineClient, final ContainerStatus container,
|
||||
final String domainId, final UserGroupInformation ugi) {
|
||||
Runnable publishWrapper = new Runnable() {
|
||||
public void run() {
|
||||
publishContainerEndEventOnNewTimelineServiceBase(timelineClient,
|
||||
publishContainerEndEventOnNewTimelineServiceBase(timelineClient,
|
||||
container, domainId, ugi);
|
||||
}
|
||||
};
|
||||
threadPool.execute(publishWrapper);
|
||||
}
|
||||
|
||||
|
||||
private static void publishContainerEndEventOnNewTimelineServiceBase(
|
||||
final TimelineClient timelineClient, final ContainerStatus container,
|
||||
final String domainId, final UserGroupInformation ugi) {
|
||||
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
|
||||
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
|
||||
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
|
||||
entity.setId(container.getContainerId().toString());
|
||||
entity.setType(DSEntity.DS_CONTAINER.toString());
|
||||
//entity.setDomainId(domainId);
|
||||
entity.addInfo("user", ugi.getShortUserName());
|
||||
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
|
||||
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
|
||||
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
|
||||
event.setTimestamp(System.currentTimeMillis());
|
||||
event.setId(DSEvent.DS_CONTAINER_END.toString());
|
||||
|
@ -1482,28 +1482,28 @@ public class ApplicationMaster {
|
|||
|
||||
private static void publishApplicationAttemptEventOnNewTimelineService(
|
||||
final TimelineClient timelineClient, final String appAttemptId,
|
||||
final DSEvent appEvent, final String domainId,
|
||||
final DSEvent appEvent, final String domainId,
|
||||
final UserGroupInformation ugi) {
|
||||
|
||||
|
||||
Runnable publishWrapper = new Runnable() {
|
||||
public void run() {
|
||||
publishApplicationAttemptEventOnNewTimelineServiceBase(timelineClient,
|
||||
publishApplicationAttemptEventOnNewTimelineServiceBase(timelineClient,
|
||||
appAttemptId, appEvent, domainId, ugi);
|
||||
}
|
||||
};
|
||||
threadPool.execute(publishWrapper);
|
||||
}
|
||||
|
||||
|
||||
private static void publishApplicationAttemptEventOnNewTimelineServiceBase(
|
||||
final TimelineClient timelineClient, String appAttemptId,
|
||||
DSEvent appEvent, String domainId, UserGroupInformation ugi) {
|
||||
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
|
||||
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
|
||||
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
|
||||
entity.setId(appAttemptId);
|
||||
entity.setType(DSEntity.DS_APP_ATTEMPT.toString());
|
||||
//entity.setDomainId(domainId);
|
||||
entity.addInfo("user", ugi.getShortUserName());
|
||||
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
|
||||
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
|
||||
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
|
||||
event.setId(appEvent.toString());
|
||||
event.setTimestamp(System.currentTimeMillis());
|
||||
|
|
|
@ -189,7 +189,7 @@ public class Client {
|
|||
|
||||
// Command line options
|
||||
private Options opts;
|
||||
|
||||
|
||||
private String timelineServiceVersion;
|
||||
|
||||
private static final String shellCommandPath = "shellCommands";
|
||||
|
@ -373,11 +373,11 @@ public class Client {
|
|||
throw new IllegalArgumentException("Invalid virtual cores specified for application master, exiting."
|
||||
+ " Specified virtual cores=" + amVCores);
|
||||
}
|
||||
|
||||
|
||||
if (cliParser.hasOption("timeline_service_version")) {
|
||||
timelineServiceVersion =
|
||||
timelineServiceVersion =
|
||||
cliParser.getOptionValue("timeline_service_version", "v1");
|
||||
if (! (timelineServiceVersion.trim().equalsIgnoreCase("v1") ||
|
||||
if (! (timelineServiceVersion.trim().equalsIgnoreCase("v1") ||
|
||||
timelineServiceVersion.trim().equalsIgnoreCase("v2"))) {
|
||||
throw new IllegalArgumentException(
|
||||
"timeline_service_version is not set properly, should be 'v1' or 'v2'");
|
||||
|
|
|
@ -79,7 +79,7 @@ import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
|
|||
import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeTimelineAggregatorsAuxService;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
|
@ -100,7 +100,7 @@ public class TestDistributedShell {
|
|||
protected YarnConfiguration conf = null;
|
||||
private static final int NUM_NMS = 1;
|
||||
private static final float DEFAULT_TIMELINE_VERSION = 1.0f;
|
||||
private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_aggregator";
|
||||
private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
|
||||
|
||||
protected final static String APPMASTER_JAR =
|
||||
JarFinder.getJar(ApplicationMaster.class);
|
||||
|
@ -161,7 +161,7 @@ public class TestDistributedShell {
|
|||
// enable aux-service based timeline aggregators
|
||||
conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
|
||||
conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME
|
||||
+ ".class", PerNodeTimelineAggregatorsAuxService.class.getName());
|
||||
+ ".class", PerNodeTimelineCollectorsAuxService.class.getName());
|
||||
} else {
|
||||
Assert.fail("Wrong timeline version number: " + timelineVersion);
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ import com.google.common.collect.ImmutableList;
|
|||
public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
|
||||
AbstractService {
|
||||
private static final Log LOG = LogFactory.getLog(AMRMClient.class);
|
||||
|
||||
|
||||
private TimelineClient timelineClient;
|
||||
|
||||
/**
|
||||
|
@ -468,7 +468,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
|
|||
public void registerTimelineClient(TimelineClient timelineClient) {
|
||||
this.timelineClient = timelineClient;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get registered timeline client.
|
||||
* @return
|
||||
|
@ -476,7 +476,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
|
|||
public TimelineClient getRegisteredTimeineClient() {
|
||||
return this.timelineClient;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Wait for <code>check</code> to return true for each 1000 ms.
|
||||
* See also {@link #waitFor(com.google.common.base.Supplier, int)}
|
||||
|
|
|
@ -293,7 +293,7 @@ extends AbstractService {
|
|||
* @return Current number of nodes in the cluster
|
||||
*/
|
||||
public abstract int getClusterNodeCount();
|
||||
|
||||
|
||||
/**
|
||||
* Register TimelineClient to AMRMClient.
|
||||
* @param timelineClient
|
||||
|
@ -301,7 +301,7 @@ extends AbstractService {
|
|||
public void registerTimelineClient(TimelineClient timelineClient) {
|
||||
client.registerTimelineClient(timelineClient);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get registered timeline client.
|
||||
* @return
|
||||
|
|
|
@ -67,8 +67,8 @@ extends AMRMClientAsync<T> {
|
|||
private volatile boolean keepRunning;
|
||||
private volatile float progress;
|
||||
|
||||
private volatile String aggregatorAddr;
|
||||
|
||||
private volatile String collectorAddr;
|
||||
|
||||
private volatile Throwable savedException;
|
||||
|
||||
/**
|
||||
|
@ -354,15 +354,15 @@ extends AMRMClientAsync<T> {
|
|||
if (!allocated.isEmpty()) {
|
||||
handler.onContainersAllocated(allocated);
|
||||
}
|
||||
|
||||
String aggregatorAddress = response.getAggregatorAddr();
|
||||
|
||||
String collectorAddress = response.getCollectorAddr();
|
||||
TimelineClient timelineClient = client.getRegisteredTimeineClient();
|
||||
if (timelineClient != null && aggregatorAddress != null
|
||||
&& !aggregatorAddress.isEmpty()) {
|
||||
if (aggregatorAddr == null ||
|
||||
!aggregatorAddr.equals(aggregatorAddress)) {
|
||||
aggregatorAddr = aggregatorAddress;
|
||||
timelineClient.setTimelineServiceAddress(aggregatorAddress);
|
||||
if (timelineClient != null && collectorAddress != null
|
||||
&& !collectorAddress.isEmpty()) {
|
||||
if (collectorAddr == null ||
|
||||
!collectorAddr.equals(collectorAddress)) {
|
||||
collectorAddr = collectorAddress;
|
||||
timelineClient.setTimelineServiceAddress(collectorAddress);
|
||||
}
|
||||
}
|
||||
progress = handler.getProgress();
|
||||
|
|
|
@ -384,22 +384,22 @@ public class AllocateResponsePBImpl extends AllocateResponse {
|
|||
}
|
||||
this.amrmToken = amRMToken;
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public String getAggregatorAddr() {
|
||||
public String getCollectorAddr() {
|
||||
AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
return p.getAggregatorAddr();
|
||||
return p.getCollectorAddr();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void setAggregatorAddr(String aggregatorAddr) {
|
||||
public void setCollectorAddr(String collectorAddr) {
|
||||
maybeInitBuilder();
|
||||
if (aggregatorAddr == null) {
|
||||
builder.clearAggregatorAddr();
|
||||
if (collectorAddr == null) {
|
||||
builder.clearCollectorAddr();
|
||||
return;
|
||||
}
|
||||
builder.setAggregatorAddr(aggregatorAddr);
|
||||
builder.setCollectorAddr(collectorAddr);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -242,5 +242,5 @@ public abstract class TimelineClient extends AbstractService implements
|
|||
* the timeline service address
|
||||
*/
|
||||
public abstract void setTimelineServiceAddress(String address);
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -119,12 +119,11 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
private TimelineWriter timelineWriter;
|
||||
|
||||
private volatile String timelineServiceAddress;
|
||||
|
||||
|
||||
// Retry parameters for identifying new timeline service
|
||||
// TODO consider to merge with connection retry
|
||||
private int maxServiceRetries;
|
||||
private long serviceRetryInterval;
|
||||
|
||||
private boolean timelineServiceV2 = false;
|
||||
|
||||
@Private
|
||||
|
@ -330,7 +329,7 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
|
||||
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
|
||||
LOG.info("Timeline service address: " + getTimelineServiceAddress());
|
||||
}
|
||||
}
|
||||
super.serviceInit(conf);
|
||||
}
|
||||
|
||||
|
@ -410,16 +409,16 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
YarnException {
|
||||
timelineWriter.putDomain(domain);
|
||||
}
|
||||
|
||||
|
||||
// Used for new timeline service only
|
||||
@Private
|
||||
public void putObjects(String path, MultivaluedMap<String, String> params,
|
||||
public void putObjects(String path, MultivaluedMap<String, String> params,
|
||||
Object obj) throws IOException, YarnException {
|
||||
|
||||
// timelineServiceAddress could haven't be initialized yet
|
||||
|
||||
// timelineServiceAddress could haven't be initialized yet
|
||||
// or stale (only for new timeline service)
|
||||
int retries = pollTimelineServiceAddress(this.maxServiceRetries);
|
||||
|
||||
|
||||
// timelineServiceAddress could be stale, add retry logic here.
|
||||
boolean needRetry = true;
|
||||
while (needRetry) {
|
||||
|
@ -436,13 +435,13 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Check if reaching to maximum of retries.
|
||||
* @param retries
|
||||
* @param e
|
||||
*/
|
||||
private void checkRetryWithSleep(int retries, Exception e) throws
|
||||
private void checkRetryWithSleep(int retries, Exception e) throws
|
||||
YarnException, IOException {
|
||||
if (retries > 0) {
|
||||
try {
|
||||
|
@ -452,8 +451,8 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
}
|
||||
} else {
|
||||
LOG.error(
|
||||
"TimelineClient has reached to max retry times :" +
|
||||
this.maxServiceRetries + " for service address: " +
|
||||
"TimelineClient has reached to max retry times :" +
|
||||
this.maxServiceRetries + " for service address: " +
|
||||
timelineServiceAddress);
|
||||
if (e instanceof YarnException) {
|
||||
throw (YarnException)e;
|
||||
|
@ -499,7 +498,7 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
public void setTimelineServiceAddress(String address) {
|
||||
this.timelineServiceAddress = address;
|
||||
}
|
||||
|
||||
|
||||
private String getTimelineServiceAddress() {
|
||||
return this.timelineServiceAddress;
|
||||
}
|
||||
|
|
|
@ -965,10 +965,10 @@
|
|||
<name>yarn.nodemanager.container-manager.thread-count</name>
|
||||
<value>20</value>
|
||||
</property>
|
||||
|
||||
|
||||
<property>
|
||||
<description>Number of threads aggregator service uses.</description>
|
||||
<name>yarn.nodemanager.aggregator-service.thread-count</name>
|
||||
<description>Number of threads collector service uses.</description>
|
||||
<name>yarn.nodemanager.collector-service.thread-count</name>
|
||||
<value>5</value>
|
||||
</property>
|
||||
|
||||
|
@ -1046,11 +1046,11 @@
|
|||
<name>yarn.nodemanager.localizer.address</name>
|
||||
<value>${yarn.nodemanager.hostname}:8040</value>
|
||||
</property>
|
||||
|
||||
|
||||
|
||||
|
||||
<property>
|
||||
<description>Address where the aggregator service IPC is.</description>
|
||||
<name>yarn.nodemanager.aggregator-service.address</name>
|
||||
<description>Address where the collector service IPC is.</description>
|
||||
<name>yarn.nodemanager.collector-service.address</name>
|
||||
<value>${yarn.nodemanager.hostname}:8048</value>
|
||||
</property>
|
||||
|
||||
|
|
|
@ -135,7 +135,7 @@ public class TestContainerLaunchRPC {
|
|||
|
||||
Assert.fail("timeout exception should have occurred!");
|
||||
}
|
||||
|
||||
|
||||
public static Token newContainerToken(NodeId nodeId, byte[] password,
|
||||
ContainerTokenIdentifier tokenIdentifier) {
|
||||
// RPC layer client expects ip:port as service for tokens
|
||||
|
|
|
@ -150,7 +150,7 @@
|
|||
<include>yarn_server_common_service_protos.proto</include>
|
||||
<include>ResourceTracker.proto</include>
|
||||
<include>SCMUploader.proto</include>
|
||||
<include>aggregatornodemanager_protocol.proto</include>
|
||||
<include>collectornodemanager_protocol.proto</include>
|
||||
</includes>
|
||||
</source>
|
||||
<output>${project.build.directory}/generated-sources/java</output>
|
||||
|
|
|
@ -21,36 +21,37 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||
|
||||
/**
|
||||
* <p>The protocol between an <code>TimelineAggregatorsCollection</code> and a
|
||||
* <code>NodeManager</code> to report a new application aggregator get launched.
|
||||
* <p>The protocol between an <code>TimelineCollectorManager</code> and a
|
||||
* <code>NodeManager</code> to report a new application collector get launched.
|
||||
* </p>
|
||||
*
|
||||
*
|
||||
*/
|
||||
@Private
|
||||
public interface AggregatorNodemanagerProtocol {
|
||||
public interface CollectorNodemanagerProtocol {
|
||||
|
||||
/**
|
||||
*
|
||||
*
|
||||
* <p>
|
||||
* The <code>TimelineAggregatorsCollection</code> provides a list of mapping
|
||||
* between application and aggregator's address in
|
||||
* {@link ReportNewAggregatorsInfoRequest} to a <code>NodeManager</code> to
|
||||
* <em>register</em> aggregator's info, include: applicationId and REST URI to
|
||||
* access aggregator. NodeManager will add them into registered aggregators
|
||||
* The <code>TimelineCollectorManager</code> provides a list of mapping
|
||||
* between application and collector's address in
|
||||
* {@link ReportNewCollectorInfoRequest} to a <code>NodeManager</code> to
|
||||
* <em>register</em> collector's info, include: applicationId and REST URI to
|
||||
* access collector. NodeManager will add them into registered collectors
|
||||
* and register them into <code>ResourceManager</code> afterwards.
|
||||
* </p>
|
||||
*
|
||||
* @param request the request of registering a new aggregator or a list of aggregators
|
||||
* @return
|
||||
*
|
||||
* @param request the request of registering a new collector or a list of
|
||||
* collectors
|
||||
* @return
|
||||
* @throws YarnException
|
||||
* @throws IOException
|
||||
*/
|
||||
ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
|
||||
ReportNewAggregatorsInfoRequest request)
|
||||
ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
||||
ReportNewCollectorInfoRequest request)
|
||||
throws YarnException, IOException;
|
||||
|
||||
|
||||
}
|
|
@ -20,14 +20,14 @@ package org.apache.hadoop.yarn.server.api;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.AggregatorNodemanagerProtocol.AggregatorNodemanagerProtocolService;
|
||||
import org.apache.hadoop.yarn.proto.CollectorNodemanagerProtocol.CollectorNodemanagerProtocolService;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
@ProtocolInfo(
|
||||
protocolName = "org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocolPB",
|
||||
protocolName = "org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB",
|
||||
protocolVersion = 1)
|
||||
public interface AggregatorNodemanagerProtocolPB extends
|
||||
AggregatorNodemanagerProtocolService.BlockingInterface {
|
||||
public interface CollectorNodemanagerProtocolPB extends
|
||||
CollectorNodemanagerProtocolService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -30,18 +30,18 @@ import org.apache.hadoop.security.UserGroupInformation;
|
|||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProto;
|
||||
import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoResponsePBImpl;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class AggregatorNodemanagerProtocolPBClientImpl implements
|
||||
AggregatorNodemanagerProtocol, Closeable {
|
||||
public class CollectorNodemanagerProtocolPBClientImpl implements
|
||||
CollectorNodemanagerProtocol, Closeable {
|
||||
|
||||
// Not a documented config. Only used for tests internally
|
||||
static final String NM_COMMAND_TIMEOUT = YarnConfiguration.YARN_PREFIX
|
||||
|
@ -51,39 +51,39 @@ public class AggregatorNodemanagerProtocolPBClientImpl implements
|
|||
* Maximum of 1 minute timeout for a Node to react to the command
|
||||
*/
|
||||
static final int DEFAULT_COMMAND_TIMEOUT = 60000;
|
||||
|
||||
private AggregatorNodemanagerProtocolPB proxy;
|
||||
|
||||
|
||||
private CollectorNodemanagerProtocolPB proxy;
|
||||
|
||||
@Private
|
||||
public AggregatorNodemanagerProtocolPBClientImpl(long clientVersion,
|
||||
public CollectorNodemanagerProtocolPBClientImpl(long clientVersion,
|
||||
InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, AggregatorNodemanagerProtocolPB.class,
|
||||
RPC.setProtocolEngine(conf, CollectorNodemanagerProtocolPB.class,
|
||||
ProtobufRpcEngine.class);
|
||||
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
||||
|
||||
int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT);
|
||||
proxy =
|
||||
(AggregatorNodemanagerProtocolPB) RPC.getProxy(
|
||||
AggregatorNodemanagerProtocolPB.class,
|
||||
(CollectorNodemanagerProtocolPB) RPC.getProxy(
|
||||
CollectorNodemanagerProtocolPB.class,
|
||||
clientVersion, addr, ugi, conf,
|
||||
NetUtils.getDefaultSocketFactory(conf), expireIntvl);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
|
||||
ReportNewAggregatorsInfoRequest request) throws YarnException, IOException {
|
||||
|
||||
ReportNewAggregatorsInfoRequestProto requestProto =
|
||||
((ReportNewAggregatorsInfoRequestPBImpl) request).getProto();
|
||||
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
||||
ReportNewCollectorInfoRequest request) throws YarnException, IOException {
|
||||
|
||||
ReportNewCollectorInfoRequestProto requestProto =
|
||||
((ReportNewCollectorInfoRequestPBImpl) request).getProto();
|
||||
try {
|
||||
return new ReportNewAggregatorsInfoResponsePBImpl(
|
||||
proxy.reportNewAggregatorInfo(null, requestProto));
|
||||
return new ReportNewCollectorInfoResponsePBImpl(
|
||||
proxy.reportNewCollectorInfo(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
RPCUtil.unwrapAndThrowException(e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (this.proxy != null) {
|
|
@ -19,38 +19,36 @@ package org.apache.hadoop.yarn.server.api.impl.pb.service;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoResponseProto;
|
||||
import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoResponseProto;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoResponsePBImpl;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class AggregatorNodemanagerProtocolPBServiceImpl implements
|
||||
AggregatorNodemanagerProtocolPB {
|
||||
public class CollectorNodemanagerProtocolPBServiceImpl implements
|
||||
CollectorNodemanagerProtocolPB {
|
||||
|
||||
private AggregatorNodemanagerProtocol real;
|
||||
|
||||
public AggregatorNodemanagerProtocolPBServiceImpl(AggregatorNodemanagerProtocol impl) {
|
||||
private CollectorNodemanagerProtocol real;
|
||||
|
||||
public CollectorNodemanagerProtocolPBServiceImpl(CollectorNodemanagerProtocol impl) {
|
||||
this.real = impl;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportNewAggregatorsInfoResponseProto reportNewAggregatorInfo(
|
||||
RpcController arg0, ReportNewAggregatorsInfoRequestProto proto)
|
||||
public ReportNewCollectorInfoResponseProto reportNewCollectorInfo(
|
||||
RpcController arg0, ReportNewCollectorInfoRequestProto proto)
|
||||
throws ServiceException {
|
||||
ReportNewAggregatorsInfoRequestPBImpl request =
|
||||
new ReportNewAggregatorsInfoRequestPBImpl(proto);
|
||||
ReportNewCollectorInfoRequestPBImpl request =
|
||||
new ReportNewCollectorInfoRequestPBImpl(proto);
|
||||
try {
|
||||
ReportNewAggregatorsInfoResponse response = real.reportNewAggregatorInfo(request);
|
||||
return ((ReportNewAggregatorsInfoResponsePBImpl)response).getProto();
|
||||
ReportNewCollectorInfoResponse response = real.reportNewCollectorInfo(request);
|
||||
return ((ReportNewCollectorInfoResponsePBImpl)response).getProto();
|
||||
} catch (YarnException e) {
|
||||
throw new ServiceException(e);
|
||||
} catch (IOException e) {
|
|
@ -43,11 +43,11 @@ public abstract class NodeHeartbeatRequest {
|
|||
nodeHeartbeatRequest.setNodeLabels(nodeLabels);
|
||||
return nodeHeartbeatRequest;
|
||||
}
|
||||
|
||||
|
||||
public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus,
|
||||
MasterKey lastKnownContainerTokenMasterKey,
|
||||
MasterKey lastKnownNMTokenMasterKey, Set<NodeLabel> nodeLabels,
|
||||
Map<ApplicationId, String> registeredAggregators) {
|
||||
Map<ApplicationId, String> registeredCollectors) {
|
||||
NodeHeartbeatRequest nodeHeartbeatRequest =
|
||||
Records.newRecord(NodeHeartbeatRequest.class);
|
||||
nodeHeartbeatRequest.setNodeStatus(nodeStatus);
|
||||
|
@ -56,7 +56,7 @@ public abstract class NodeHeartbeatRequest {
|
|||
nodeHeartbeatRequest
|
||||
.setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey);
|
||||
nodeHeartbeatRequest.setNodeLabels(nodeLabels);
|
||||
nodeHeartbeatRequest.setRegisteredAggregators(registeredAggregators);
|
||||
nodeHeartbeatRequest.setRegisteredCollectors(registeredCollectors);
|
||||
return nodeHeartbeatRequest;
|
||||
}
|
||||
|
||||
|
@ -78,7 +78,8 @@ public abstract class NodeHeartbeatRequest {
|
|||
public abstract void setLogAggregationReportsForApps(
|
||||
List<LogAggregationReport> logAggregationReportsForApps);
|
||||
|
||||
// This tells RM registered aggregators' address info on this node
|
||||
public abstract Map<ApplicationId, String> getRegisteredAggregators();
|
||||
public abstract void setRegisteredAggregators(Map<ApplicationId, String> appAggregatorsMap);
|
||||
// This tells RM registered collectors' address info on this node
|
||||
public abstract Map<ApplicationId, String> getRegisteredCollectors();
|
||||
public abstract void setRegisteredCollectors(Map<ApplicationId,
|
||||
String> appCollectorsMap);
|
||||
}
|
||||
|
|
|
@ -40,10 +40,10 @@ public interface NodeHeartbeatResponse {
|
|||
List<ContainerId> getContainersToBeRemovedFromNM();
|
||||
|
||||
List<ApplicationId> getApplicationsToCleanup();
|
||||
|
||||
// This tells NM the aggregators' address info of related Apps
|
||||
Map<ApplicationId, String> getAppAggregatorsMap();
|
||||
void setAppAggregatorsMap(Map<ApplicationId, String> appAggregatorsMap);
|
||||
|
||||
// This tells NM the collectors' address info of related apps
|
||||
Map<ApplicationId, String> getAppCollectorsMap();
|
||||
void setAppCollectorsMap(Map<ApplicationId, String> appCollectorsMap);
|
||||
|
||||
void setResponseId(int responseId);
|
||||
void setNodeAction(NodeAction action);
|
||||
|
|
|
@ -22,32 +22,32 @@ import java.util.Arrays;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
@Private
|
||||
public abstract class ReportNewAggregatorsInfoRequest {
|
||||
|
||||
public static ReportNewAggregatorsInfoRequest newInstance(
|
||||
List<AppAggregatorsMap> appAggregatorsList) {
|
||||
ReportNewAggregatorsInfoRequest request =
|
||||
Records.newRecord(ReportNewAggregatorsInfoRequest.class);
|
||||
request.setAppAggregatorsList(appAggregatorsList);
|
||||
public abstract class ReportNewCollectorInfoRequest {
|
||||
|
||||
public static ReportNewCollectorInfoRequest newInstance(
|
||||
List<AppCollectorsMap> appCollectorsList) {
|
||||
ReportNewCollectorInfoRequest request =
|
||||
Records.newRecord(ReportNewCollectorInfoRequest.class);
|
||||
request.setAppCollectorsList(appCollectorsList);
|
||||
return request;
|
||||
}
|
||||
|
||||
public static ReportNewAggregatorsInfoRequest newInstance(
|
||||
ApplicationId id, String aggregatorAddr) {
|
||||
ReportNewAggregatorsInfoRequest request =
|
||||
Records.newRecord(ReportNewAggregatorsInfoRequest.class);
|
||||
request.setAppAggregatorsList(
|
||||
Arrays.asList(AppAggregatorsMap.newInstance(id, aggregatorAddr)));
|
||||
|
||||
public static ReportNewCollectorInfoRequest newInstance(
|
||||
ApplicationId id, String collectorAddr) {
|
||||
ReportNewCollectorInfoRequest request =
|
||||
Records.newRecord(ReportNewCollectorInfoRequest.class);
|
||||
request.setAppCollectorsList(
|
||||
Arrays.asList(AppCollectorsMap.newInstance(id, collectorAddr)));
|
||||
return request;
|
||||
}
|
||||
|
||||
public abstract List<AppAggregatorsMap> getAppAggregatorsList();
|
||||
|
||||
public abstract void setAppAggregatorsList(
|
||||
List<AppAggregatorsMap> appAggregatorsList);
|
||||
|
||||
|
||||
public abstract List<AppCollectorsMap> getAppCollectorsList();
|
||||
|
||||
public abstract void setAppCollectorsList(
|
||||
List<AppCollectorsMap> appCollectorsList);
|
||||
|
||||
}
|
|
@ -20,12 +20,12 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
public abstract class ReportNewAggregatorsInfoResponse {
|
||||
public abstract class ReportNewCollectorInfoResponse {
|
||||
|
||||
@Private
|
||||
public static ReportNewAggregatorsInfoResponse newInstance() {
|
||||
ReportNewAggregatorsInfoResponse response =
|
||||
Records.newRecord(ReportNewAggregatorsInfoResponse.class);
|
||||
public static ReportNewCollectorInfoResponse newInstance() {
|
||||
ReportNewCollectorInfoResponse response =
|
||||
Records.newRecord(ReportNewCollectorInfoResponse.class);
|
||||
return response;
|
||||
}
|
||||
|
|
@ -34,8 +34,8 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
|||
import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeLabelsProto;
|
||||
|
@ -58,7 +58,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||
private Set<NodeLabel> labels = null;
|
||||
private List<LogAggregationReport> logAggregationReportsForApps = null;
|
||||
|
||||
Map<ApplicationId, String> registeredAggregators = null;
|
||||
Map<ApplicationId, String> registeredCollectors = null;
|
||||
|
||||
public NodeHeartbeatRequestPBImpl() {
|
||||
builder = NodeHeartbeatRequestProto.newBuilder();
|
||||
|
@ -114,8 +114,8 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||
if (this.logAggregationReportsForApps != null) {
|
||||
addLogAggregationStatusForAppsToProto();
|
||||
}
|
||||
if (this.registeredAggregators != null) {
|
||||
addRegisteredAggregatorsToProto();
|
||||
if (this.registeredCollectors != null) {
|
||||
addRegisteredCollectorsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -158,13 +158,13 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||
return ((LogAggregationReportPBImpl) value).getProto();
|
||||
}
|
||||
|
||||
private void addRegisteredAggregatorsToProto() {
|
||||
private void addRegisteredCollectorsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearRegisteredAggregators();
|
||||
for (Map.Entry<ApplicationId, String> entry : registeredAggregators.entrySet()) {
|
||||
builder.addRegisteredAggregators(AppAggregatorsMapProto.newBuilder()
|
||||
builder.clearRegisteredCollectors();
|
||||
for (Map.Entry<ApplicationId, String> entry : registeredCollectors.entrySet()) {
|
||||
builder.addRegisteredCollectors(AppCollectorsMapProto.newBuilder()
|
||||
.setAppId(convertToProtoFormat(entry.getKey()))
|
||||
.setAppAggregatorAddr(entry.getValue()));
|
||||
.setAppCollectorAddr(entry.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -248,35 +248,35 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||
builder.clearLastKnownNmTokenMasterKey();
|
||||
this.lastKnownNMTokenMasterKey = masterKey;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Map<ApplicationId, String> getRegisteredAggregators() {
|
||||
if (this.registeredAggregators != null) {
|
||||
return this.registeredAggregators;
|
||||
public Map<ApplicationId, String> getRegisteredCollectors() {
|
||||
if (this.registeredCollectors != null) {
|
||||
return this.registeredCollectors;
|
||||
}
|
||||
initRegisteredAggregators();
|
||||
return registeredAggregators;
|
||||
initRegisteredCollectors();
|
||||
return registeredCollectors;
|
||||
}
|
||||
|
||||
private void initRegisteredAggregators() {
|
||||
|
||||
private void initRegisteredCollectors() {
|
||||
NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<AppAggregatorsMapProto> list = p.getRegisteredAggregatorsList();
|
||||
this.registeredAggregators = new HashMap<ApplicationId, String> ();
|
||||
for (AppAggregatorsMapProto c : list) {
|
||||
List<AppCollectorsMapProto> list = p.getRegisteredCollectorsList();
|
||||
this.registeredCollectors = new HashMap<ApplicationId, String> ();
|
||||
for (AppCollectorsMapProto c : list) {
|
||||
ApplicationId appId = convertFromProtoFormat(c.getAppId());
|
||||
this.registeredAggregators.put(appId, c.getAppAggregatorAddr());
|
||||
this.registeredCollectors.put(appId, c.getAppCollectorAddr());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void setRegisteredAggregators(
|
||||
Map<ApplicationId, String> registeredAggregators) {
|
||||
if (registeredAggregators == null || registeredAggregators.isEmpty()) {
|
||||
public void setRegisteredCollectors(
|
||||
Map<ApplicationId, String> registeredCollectors) {
|
||||
if (registeredCollectors == null || registeredCollectors.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
maybeInitBuilder();
|
||||
this.registeredAggregators = new HashMap<ApplicationId, String>();
|
||||
this.registeredAggregators.putAll(registeredAggregators);
|
||||
this.registeredCollectors = new HashMap<ApplicationId, String>();
|
||||
this.registeredCollectors.putAll(registeredCollectors);
|
||||
}
|
||||
|
||||
private NodeStatusPBImpl convertFromProtoFormat(NodeStatusProto p) {
|
||||
|
@ -286,11 +286,11 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||
private NodeStatusProto convertToProtoFormat(NodeStatus t) {
|
||||
return ((NodeStatusPBImpl)t).getProto();
|
||||
}
|
||||
|
||||
|
||||
private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
|
||||
return new ApplicationIdPBImpl(p);
|
||||
}
|
||||
|
||||
|
||||
private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
|
||||
return ((ApplicationIdPBImpl) t).getProto();
|
||||
}
|
||||
|
|
|
@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ContainerQueui
|
|||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SystemCredentialsForAppsProto;
|
||||
|
@ -69,7 +69,7 @@ public class NodeHeartbeatResponsePBImpl extends
|
|||
private List<ApplicationId> applicationsToCleanup = null;
|
||||
private Map<ApplicationId, ByteBuffer> systemCredentials = null;
|
||||
private Resource resource = null;
|
||||
Map<ApplicationId, String> appAggregatorsMap = null;
|
||||
Map<ApplicationId, String> appCollectorsMap = null;
|
||||
|
||||
private MasterKey containerTokenMasterKey = null;
|
||||
private MasterKey nmTokenMasterKey = null;
|
||||
|
@ -127,8 +127,8 @@ public class NodeHeartbeatResponsePBImpl extends
|
|||
if (this.resource != null) {
|
||||
builder.setResource(convertToProtoFormat(this.resource));
|
||||
}
|
||||
if (this.appAggregatorsMap != null) {
|
||||
addAppAggregatorsMapToProto();
|
||||
if (this.appCollectorsMap != null) {
|
||||
addAppCollectorsMapToProto();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -142,14 +142,14 @@ public class NodeHeartbeatResponsePBImpl extends
|
|||
entry.getValue().duplicate())));
|
||||
}
|
||||
}
|
||||
|
||||
private void addAppAggregatorsMapToProto() {
|
||||
|
||||
private void addAppCollectorsMapToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearAppAggregatorsMap();
|
||||
for (Map.Entry<ApplicationId, String> entry : appAggregatorsMap.entrySet()) {
|
||||
builder.addAppAggregatorsMap(AppAggregatorsMapProto.newBuilder()
|
||||
builder.clearAppCollectorsMap();
|
||||
for (Map.Entry<ApplicationId, String> entry : appCollectorsMap.entrySet()) {
|
||||
builder.addAppCollectorsMap(AppCollectorsMapProto.newBuilder()
|
||||
.setAppId(convertToProtoFormat(entry.getKey()))
|
||||
.setAppAggregatorAddr(entry.getValue()));
|
||||
.setAppCollectorAddr(entry.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -565,14 +565,14 @@ public class NodeHeartbeatResponsePBImpl extends
|
|||
initSystemCredentials();
|
||||
return systemCredentials;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Map<ApplicationId, String> getAppAggregatorsMap() {
|
||||
if (this.appAggregatorsMap != null) {
|
||||
return this.appAggregatorsMap;
|
||||
public Map<ApplicationId, String> getAppCollectorsMap() {
|
||||
if (this.appCollectorsMap != null) {
|
||||
return this.appCollectorsMap;
|
||||
}
|
||||
initAppAggregatorsMap();
|
||||
return appAggregatorsMap;
|
||||
initAppCollectorsMap();
|
||||
return appCollectorsMap;
|
||||
}
|
||||
|
||||
private void initSystemCredentials() {
|
||||
|
@ -585,14 +585,14 @@ public class NodeHeartbeatResponsePBImpl extends
|
|||
this.systemCredentials.put(appId, byteBuffer);
|
||||
}
|
||||
}
|
||||
|
||||
private void initAppAggregatorsMap() {
|
||||
|
||||
private void initAppCollectorsMap() {
|
||||
NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<AppAggregatorsMapProto> list = p.getAppAggregatorsMapList();
|
||||
this.appAggregatorsMap = new HashMap<ApplicationId, String> ();
|
||||
for (AppAggregatorsMapProto c : list) {
|
||||
List<AppCollectorsMapProto> list = p.getAppCollectorsMapList();
|
||||
this.appCollectorsMap = new HashMap<ApplicationId, String> ();
|
||||
for (AppCollectorsMapProto c : list) {
|
||||
ApplicationId appId = convertFromProtoFormat(c.getAppId());
|
||||
this.appAggregatorsMap.put(appId, c.getAppAggregatorAddr());
|
||||
this.appCollectorsMap.put(appId, c.getAppCollectorAddr());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -606,16 +606,16 @@ public class NodeHeartbeatResponsePBImpl extends
|
|||
this.systemCredentials = new HashMap<ApplicationId, ByteBuffer>();
|
||||
this.systemCredentials.putAll(systemCredentials);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void setAppAggregatorsMap(
|
||||
Map<ApplicationId, String> appAggregatorsMap) {
|
||||
if (appAggregatorsMap == null || appAggregatorsMap.isEmpty()) {
|
||||
public void setAppCollectorsMap(
|
||||
Map<ApplicationId, String> appCollectorsMap) {
|
||||
if (appCollectorsMap == null || appCollectorsMap.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
maybeInitBuilder();
|
||||
this.appAggregatorsMap = new HashMap<ApplicationId, String>();
|
||||
this.appAggregatorsMap.putAll(appAggregatorsMap);
|
||||
this.appCollectorsMap = new HashMap<ApplicationId, String>();
|
||||
this.appCollectorsMap.putAll(appCollectorsMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,142 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
|
||||
import org.apache.hadoop.yarn.server.api.records.impl.pb.AppAggregatorsMapPBImpl;
|
||||
|
||||
public class ReportNewAggregatorsInfoRequestPBImpl extends
|
||||
ReportNewAggregatorsInfoRequest {
|
||||
|
||||
ReportNewAggregatorsInfoRequestProto proto =
|
||||
ReportNewAggregatorsInfoRequestProto.getDefaultInstance();
|
||||
|
||||
ReportNewAggregatorsInfoRequestProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private List<AppAggregatorsMap> aggregatorsList = null;
|
||||
|
||||
public ReportNewAggregatorsInfoRequestPBImpl() {
|
||||
builder = ReportNewAggregatorsInfoRequestProto.newBuilder();
|
||||
}
|
||||
|
||||
public ReportNewAggregatorsInfoRequestPBImpl(
|
||||
ReportNewAggregatorsInfoRequestProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public ReportNewAggregatorsInfoRequestProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (aggregatorsList != null) {
|
||||
addLocalAggregatorsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = ReportNewAggregatorsInfoRequestProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
private void addLocalAggregatorsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearAppAggregators();
|
||||
List<AppAggregatorsMapProto> protoList =
|
||||
new ArrayList<AppAggregatorsMapProto>();
|
||||
for (AppAggregatorsMap m : this.aggregatorsList) {
|
||||
protoList.add(convertToProtoFormat(m));
|
||||
}
|
||||
builder.addAllAppAggregators(protoList);
|
||||
}
|
||||
|
||||
private void initLocalAggregatorsList() {
|
||||
ReportNewAggregatorsInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<AppAggregatorsMapProto> aggregatorsList =
|
||||
p.getAppAggregatorsList();
|
||||
this.aggregatorsList = new ArrayList<AppAggregatorsMap>();
|
||||
for (AppAggregatorsMapProto m : aggregatorsList) {
|
||||
this.aggregatorsList.add(convertFromProtoFormat(m));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AppAggregatorsMap> getAppAggregatorsList() {
|
||||
if (this.aggregatorsList == null) {
|
||||
initLocalAggregatorsList();
|
||||
}
|
||||
return this.aggregatorsList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAppAggregatorsList(List<AppAggregatorsMap> appAggregatorsList) {
|
||||
maybeInitBuilder();
|
||||
if (appAggregatorsList == null) {
|
||||
builder.clearAppAggregators();
|
||||
}
|
||||
this.aggregatorsList = appAggregatorsList;
|
||||
}
|
||||
|
||||
private AppAggregatorsMapPBImpl convertFromProtoFormat(
|
||||
AppAggregatorsMapProto p) {
|
||||
return new AppAggregatorsMapPBImpl(p);
|
||||
}
|
||||
|
||||
private AppAggregatorsMapProto convertToProtoFormat(
|
||||
AppAggregatorsMap m) {
|
||||
return ((AppAggregatorsMapPBImpl) m).getProto();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,142 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
||||
import org.apache.hadoop.yarn.server.api.records.impl.pb.AppCollectorsMapPBImpl;
|
||||
|
||||
public class ReportNewCollectorInfoRequestPBImpl extends
|
||||
ReportNewCollectorInfoRequest {
|
||||
|
||||
ReportNewCollectorInfoRequestProto proto =
|
||||
ReportNewCollectorInfoRequestProto.getDefaultInstance();
|
||||
|
||||
ReportNewCollectorInfoRequestProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
private List<AppCollectorsMap> collectorsList = null;
|
||||
|
||||
public ReportNewCollectorInfoRequestPBImpl() {
|
||||
builder = ReportNewCollectorInfoRequestProto.newBuilder();
|
||||
}
|
||||
|
||||
public ReportNewCollectorInfoRequestPBImpl(
|
||||
ReportNewCollectorInfoRequestProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public ReportNewCollectorInfoRequestProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return getProto().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == null)
|
||||
return false;
|
||||
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto)
|
||||
maybeInitBuilder();
|
||||
mergeLocalToBuilder();
|
||||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (collectorsList != null) {
|
||||
addLocalCollectorsToProto();
|
||||
}
|
||||
}
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = ReportNewCollectorInfoRequestProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
private void addLocalCollectorsToProto() {
|
||||
maybeInitBuilder();
|
||||
builder.clearAppCollectors();
|
||||
List<AppCollectorsMapProto> protoList =
|
||||
new ArrayList<AppCollectorsMapProto>();
|
||||
for (AppCollectorsMap m : this.collectorsList) {
|
||||
protoList.add(convertToProtoFormat(m));
|
||||
}
|
||||
builder.addAllAppCollectors(protoList);
|
||||
}
|
||||
|
||||
private void initLocalCollectorsList() {
|
||||
ReportNewCollectorInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||
List<AppCollectorsMapProto> collectorsList =
|
||||
p.getAppCollectorsList();
|
||||
this.collectorsList = new ArrayList<AppCollectorsMap>();
|
||||
for (AppCollectorsMapProto m : collectorsList) {
|
||||
this.collectorsList.add(convertFromProtoFormat(m));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AppCollectorsMap> getAppCollectorsList() {
|
||||
if (this.collectorsList == null) {
|
||||
initLocalCollectorsList();
|
||||
}
|
||||
return this.collectorsList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAppCollectorsList(List<AppCollectorsMap> appCollectorsList) {
|
||||
maybeInitBuilder();
|
||||
if (appCollectorsList == null) {
|
||||
builder.clearAppCollectors();
|
||||
}
|
||||
this.collectorsList = appCollectorsList;
|
||||
}
|
||||
|
||||
private AppCollectorsMapPBImpl convertFromProtoFormat(
|
||||
AppCollectorsMapProto p) {
|
||||
return new AppCollectorsMapPBImpl(p);
|
||||
}
|
||||
|
||||
private AppCollectorsMapProto convertToProtoFormat(
|
||||
AppCollectorsMap m) {
|
||||
return ((AppCollectorsMapPBImpl) m).getProto();
|
||||
}
|
||||
|
||||
}
|
|
@ -19,33 +19,33 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoResponseProto;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoResponseProto;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class ReportNewAggregatorsInfoResponsePBImpl extends
|
||||
ReportNewAggregatorsInfoResponse {
|
||||
public class ReportNewCollectorInfoResponsePBImpl extends
|
||||
ReportNewCollectorInfoResponse {
|
||||
|
||||
ReportNewCollectorInfoResponseProto proto =
|
||||
ReportNewCollectorInfoResponseProto.getDefaultInstance();
|
||||
|
||||
ReportNewCollectorInfoResponseProto.Builder builder = null;
|
||||
|
||||
ReportNewAggregatorsInfoResponseProto proto =
|
||||
ReportNewAggregatorsInfoResponseProto.getDefaultInstance();
|
||||
|
||||
ReportNewAggregatorsInfoResponseProto.Builder builder = null;
|
||||
|
||||
boolean viaProto = false;
|
||||
|
||||
public ReportNewAggregatorsInfoResponsePBImpl() {
|
||||
builder = ReportNewAggregatorsInfoResponseProto.newBuilder();
|
||||
|
||||
public ReportNewCollectorInfoResponsePBImpl() {
|
||||
builder = ReportNewCollectorInfoResponseProto.newBuilder();
|
||||
}
|
||||
|
||||
public ReportNewAggregatorsInfoResponsePBImpl(ReportNewAggregatorsInfoResponseProto proto) {
|
||||
public ReportNewCollectorInfoResponsePBImpl(ReportNewCollectorInfoResponseProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public ReportNewAggregatorsInfoResponseProto getProto() {
|
||||
|
||||
public ReportNewCollectorInfoResponseProto getProto() {
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
return proto;
|
|
@ -1,33 +0,0 @@
|
|||
package org.apache.hadoop.yarn.server.api.records;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
|
||||
@Private
|
||||
public abstract class AppAggregatorsMap {
|
||||
|
||||
public static AppAggregatorsMap newInstance(
|
||||
ApplicationId id, String aggregatorAddr) {
|
||||
AppAggregatorsMap appAggregatorMap =
|
||||
Records.newRecord(AppAggregatorsMap.class);
|
||||
appAggregatorMap.setApplicationId(id);
|
||||
appAggregatorMap.setAggregatorAddr(aggregatorAddr);
|
||||
return appAggregatorMap;
|
||||
}
|
||||
|
||||
public abstract ApplicationId getApplicationId();
|
||||
|
||||
public abstract void setApplicationId(
|
||||
ApplicationId id);
|
||||
|
||||
public abstract String getAggregatorAddr();
|
||||
|
||||
public abstract void setAggregatorAddr(
|
||||
String addr);
|
||||
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.api.records;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
|
||||
@Private
|
||||
public abstract class AppCollectorsMap {
|
||||
|
||||
public static AppCollectorsMap newInstance(
|
||||
ApplicationId id, String collectorAddr) {
|
||||
AppCollectorsMap appCollectorsMap =
|
||||
Records.newRecord(AppCollectorsMap.class);
|
||||
appCollectorsMap.setApplicationId(id);
|
||||
appCollectorsMap.setCollectorAddr(collectorAddr);
|
||||
return appCollectorsMap;
|
||||
}
|
||||
|
||||
public abstract ApplicationId getApplicationId();
|
||||
|
||||
public abstract void setApplicationId(ApplicationId id);
|
||||
|
||||
public abstract String getCollectorAddr();
|
||||
|
||||
public abstract void setCollectorAddr(String addr);
|
||||
|
||||
}
|
|
@ -21,37 +21,37 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
||||
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProtoOrBuilder;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProtoOrBuilder;
|
||||
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
public class AppAggregatorsMapPBImpl extends AppAggregatorsMap {
|
||||
public class AppCollectorsMapPBImpl extends AppCollectorsMap {
|
||||
|
||||
AppAggregatorsMapProto proto =
|
||||
AppAggregatorsMapProto.getDefaultInstance();
|
||||
|
||||
AppAggregatorsMapProto.Builder builder = null;
|
||||
AppCollectorsMapProto proto =
|
||||
AppCollectorsMapProto.getDefaultInstance();
|
||||
|
||||
AppCollectorsMapProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
||||
|
||||
private ApplicationId appId = null;
|
||||
private String aggregatorAddr = null;
|
||||
|
||||
public AppAggregatorsMapPBImpl() {
|
||||
builder = AppAggregatorsMapProto.newBuilder();
|
||||
private String collectorAddr = null;
|
||||
|
||||
public AppCollectorsMapPBImpl() {
|
||||
builder = AppCollectorsMapProto.newBuilder();
|
||||
}
|
||||
|
||||
public AppAggregatorsMapPBImpl(AppAggregatorsMapProto proto) {
|
||||
public AppCollectorsMapPBImpl(AppCollectorsMapProto proto) {
|
||||
this.proto = proto;
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
public AppAggregatorsMapProto getProto() {
|
||||
|
||||
public AppCollectorsMapProto getProto() {
|
||||
mergeLocalToProto();
|
||||
proto = viaProto ? proto : builder.build();
|
||||
viaProto = true;
|
||||
|
@ -77,24 +77,24 @@ public class AppAggregatorsMapPBImpl extends AppAggregatorsMap {
|
|||
public String toString() {
|
||||
return TextFormat.shortDebugString(getProto());
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationId() {
|
||||
AppAggregatorsMapProtoOrBuilder p = viaProto ? proto : builder;
|
||||
AppCollectorsMapProtoOrBuilder p = viaProto ? proto : builder;
|
||||
if (this.appId == null && p.hasAppId()) {
|
||||
this.appId = convertFromProtoFormat(p.getAppId());
|
||||
}
|
||||
return this.appId;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String getAggregatorAddr() {
|
||||
AppAggregatorsMapProtoOrBuilder p = viaProto ? proto : builder;
|
||||
if (this.aggregatorAddr == null
|
||||
&& p.hasAppAggregatorAddr()) {
|
||||
this.aggregatorAddr = p.getAppAggregatorAddr();
|
||||
public String getCollectorAddr() {
|
||||
AppCollectorsMapProtoOrBuilder p = viaProto ? proto : builder;
|
||||
if (this.collectorAddr == null
|
||||
&& p.hasAppCollectorAddr()) {
|
||||
this.collectorAddr = p.getAppCollectorAddr();
|
||||
}
|
||||
return this.aggregatorAddr;
|
||||
return this.collectorAddr;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -105,31 +105,31 @@ public class AppAggregatorsMapPBImpl extends AppAggregatorsMap {
|
|||
}
|
||||
this.appId = appId;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void setAggregatorAddr(String aggregatorAddr) {
|
||||
public void setCollectorAddr(String collectorAddr) {
|
||||
maybeInitBuilder();
|
||||
if (aggregatorAddr == null) {
|
||||
builder.clearAppAggregatorAddr();
|
||||
if (collectorAddr == null) {
|
||||
builder.clearAppCollectorAddr();
|
||||
}
|
||||
this.aggregatorAddr = aggregatorAddr;
|
||||
this.collectorAddr = collectorAddr;
|
||||
}
|
||||
|
||||
|
||||
private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
|
||||
return new ApplicationIdPBImpl(p);
|
||||
}
|
||||
|
||||
|
||||
private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
|
||||
return ((ApplicationIdPBImpl) t).getProto();
|
||||
}
|
||||
|
||||
|
||||
private void maybeInitBuilder() {
|
||||
if (viaProto || builder == null) {
|
||||
builder = AppAggregatorsMapProto.newBuilder(proto);
|
||||
builder = AppCollectorsMapProto.newBuilder(proto);
|
||||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
|
||||
private void mergeLocalToProto() {
|
||||
if (viaProto) {
|
||||
maybeInitBuilder();
|
||||
|
@ -138,13 +138,13 @@ public class AppAggregatorsMapPBImpl extends AppAggregatorsMap {
|
|||
proto = builder.build();
|
||||
viaProto = true;
|
||||
}
|
||||
|
||||
|
||||
private void mergeLocalToBuilder() {
|
||||
if (this.appId != null) {
|
||||
builder.setAppId(convertToProtoFormat(this.appId));
|
||||
}
|
||||
if (this.aggregatorAddr != null) {
|
||||
builder.setAppAggregatorAddr(this.aggregatorAddr);
|
||||
if (this.collectorAddr != null) {
|
||||
builder.setAppCollectorAddr(this.collectorAddr);
|
||||
}
|
||||
}
|
||||
|
|
@ -17,13 +17,13 @@
|
|||
*/
|
||||
|
||||
option java_package = "org.apache.hadoop.yarn.proto";
|
||||
option java_outer_classname = "AggregatorNodemanagerProtocol";
|
||||
option java_outer_classname = "CollectorNodemanagerProtocol";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
||||
package hadoop.yarn;
|
||||
|
||||
import "yarn_server_common_service_protos.proto";
|
||||
|
||||
service AggregatorNodemanagerProtocolService {
|
||||
rpc reportNewAggregatorInfo (ReportNewAggregatorsInfoRequestProto) returns (ReportNewAggregatorsInfoResponseProto);
|
||||
service CollectorNodemanagerProtocolService {
|
||||
rpc reportNewCollectorInfo (ReportNewCollectorInfoRequestProto) returns (ReportNewCollectorInfoResponseProto);
|
||||
}
|
|
@ -84,7 +84,7 @@ message NodeHeartbeatRequestProto {
|
|||
optional MasterKeyProto last_known_nm_token_master_key = 3;
|
||||
optional NodeLabelsProto nodeLabels = 4;
|
||||
repeated LogAggregationReportProto log_aggregation_reports_for_apps = 5;
|
||||
repeated AppAggregatorsMapProto registered_aggregators = 6;
|
||||
repeated AppCollectorsMapProto registered_collectors = 6;
|
||||
}
|
||||
|
||||
message LogAggregationReportProto {
|
||||
|
@ -109,7 +109,7 @@ message NodeHeartbeatResponseProto {
|
|||
repeated SignalContainerRequestProto containers_to_signal = 13;
|
||||
optional ResourceProto resource = 14;
|
||||
optional ContainerQueuingLimitProto container_queuing_limit = 15;
|
||||
repeated AppAggregatorsMapProto app_aggregators_map = 16;
|
||||
repeated AppCollectorsMapProto app_collectors_map = 16;
|
||||
}
|
||||
|
||||
message ContainerQueuingLimitProto {
|
||||
|
@ -123,21 +123,21 @@ message SystemCredentialsForAppsProto {
|
|||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////
|
||||
////// From aggregator_nodemanager_protocol ////////////////////////////
|
||||
////// From collector_nodemanager_protocol ////////////////////////////
|
||||
////////////////////////////////////////////////////////////////////////
|
||||
message AppAggregatorsMapProto {
|
||||
message AppCollectorsMapProto {
|
||||
optional ApplicationIdProto appId = 1;
|
||||
optional string appAggregatorAddr = 2;
|
||||
optional string appCollectorAddr = 2;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////
|
||||
/////// aggregator_nodemanager_protocol //////////////
|
||||
/////// collector_nodemanager_protocol //////////////
|
||||
//////////////////////////////////////////////////////
|
||||
message ReportNewAggregatorsInfoRequestProto {
|
||||
repeated AppAggregatorsMapProto app_aggregators = 1;
|
||||
message ReportNewCollectorInfoRequestProto {
|
||||
repeated AppCollectorsMapProto app_collectors = 1;
|
||||
}
|
||||
|
||||
message ReportNewAggregatorsInfoResponseProto {
|
||||
message ReportNewCollectorInfoResponseProto {
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -63,10 +63,10 @@ import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
|
|||
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -76,15 +76,15 @@ public class TestRPC {
|
|||
private static final String EXCEPTION_MSG = "test error";
|
||||
private static final String EXCEPTION_CAUSE = "exception cause";
|
||||
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
||||
|
||||
public static final String ILLEGAL_NUMBER_MESSAGE =
|
||||
"aggregators' number in ReportNewAggregatorsInfoRequest is not ONE.";
|
||||
|
||||
public static final String DEFAULT_AGGREGATOR_ADDR = "localhost:0";
|
||||
|
||||
public static final ApplicationId DEFAULT_APP_ID =
|
||||
|
||||
public static final String ILLEGAL_NUMBER_MESSAGE =
|
||||
"collectors' number in ReportNewCollectorInfoRequest is not ONE.";
|
||||
|
||||
public static final String DEFAULT_COLLECTOR_ADDR = "localhost:0";
|
||||
|
||||
public static final ApplicationId DEFAULT_APP_ID =
|
||||
ApplicationId.newInstance(0, 0);
|
||||
|
||||
|
||||
@Test
|
||||
public void testUnknownCall() {
|
||||
Configuration conf = new Configuration();
|
||||
|
@ -116,17 +116,17 @@ public class TestRPC {
|
|||
server.stop();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testRPCOnAggregatorNodeManagerProtocol() throws IOException {
|
||||
public void testRPCOnCollectorNodeManagerProtocol() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
|
||||
.getName());
|
||||
YarnRPC rpc = YarnRPC.create(conf);
|
||||
String bindAddr = "localhost:0";
|
||||
InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
|
||||
Server server = rpc.getServer(AggregatorNodemanagerProtocol.class,
|
||||
new DummyNMAggregatorService(), addr, conf, null, 1);
|
||||
Server server = rpc.getServer(CollectorNodemanagerProtocol.class,
|
||||
new DummyNMCollectorService(), addr, conf, null, 1);
|
||||
server.start();
|
||||
|
||||
// Test unrelated protocol wouldn't get response
|
||||
|
@ -145,31 +145,31 @@ public class TestRPC {
|
|||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
// Test AggregatorNodemanagerProtocol get proper response
|
||||
AggregatorNodemanagerProtocol proxy = (AggregatorNodemanagerProtocol)rpc.getProxy(
|
||||
AggregatorNodemanagerProtocol.class, NetUtils.getConnectAddress(server), conf);
|
||||
// Verify request with DEFAULT_APP_ID and DEFAULT_AGGREGATOR_ADDR get
|
||||
|
||||
// Test CollectorNodemanagerProtocol get proper response
|
||||
CollectorNodemanagerProtocol proxy = (CollectorNodemanagerProtocol)rpc.getProxy(
|
||||
CollectorNodemanagerProtocol.class, NetUtils.getConnectAddress(server), conf);
|
||||
// Verify request with DEFAULT_APP_ID and DEFAULT_COLLECTOR_ADDR get
|
||||
// normally response.
|
||||
try {
|
||||
ReportNewAggregatorsInfoRequest request =
|
||||
ReportNewAggregatorsInfoRequest.newInstance(
|
||||
DEFAULT_APP_ID, DEFAULT_AGGREGATOR_ADDR);
|
||||
proxy.reportNewAggregatorInfo(request);
|
||||
ReportNewCollectorInfoRequest request =
|
||||
ReportNewCollectorInfoRequest.newInstance(
|
||||
DEFAULT_APP_ID, DEFAULT_COLLECTOR_ADDR);
|
||||
proxy.reportNewCollectorInfo(request);
|
||||
} catch (YarnException e) {
|
||||
Assert.fail("RPC call failured is not expected here.");
|
||||
}
|
||||
|
||||
// Verify empty request get YarnException back (by design in
|
||||
// DummyNMAggregatorService)
|
||||
|
||||
// Verify empty request get YarnException back (by design in
|
||||
// DummyNMCollectorService)
|
||||
try {
|
||||
proxy.reportNewAggregatorInfo(Records
|
||||
.newRecord(ReportNewAggregatorsInfoRequest.class));
|
||||
proxy.reportNewCollectorInfo(Records
|
||||
.newRecord(ReportNewCollectorInfoRequest.class));
|
||||
Assert.fail("Excepted RPC call to fail with YarnException.");
|
||||
} catch (YarnException e) {
|
||||
Assert.assertTrue(e.getMessage().contains(ILLEGAL_NUMBER_MESSAGE));
|
||||
}
|
||||
|
||||
|
||||
server.stop();
|
||||
}
|
||||
|
||||
|
@ -177,21 +177,21 @@ public class TestRPC {
|
|||
public void testHadoopProtoRPC() throws Exception {
|
||||
test(HadoopYarnProtoRPC.class.getName());
|
||||
}
|
||||
|
||||
|
||||
private void test(String rpcClass) throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
|
||||
YarnRPC rpc = YarnRPC.create(conf);
|
||||
String bindAddr = "localhost:0";
|
||||
InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
|
||||
Server server = rpc.getServer(ContainerManagementProtocol.class,
|
||||
Server server = rpc.getServer(ContainerManagementProtocol.class,
|
||||
new DummyContainerManager(), addr, conf, null, 1);
|
||||
server.start();
|
||||
RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class);
|
||||
ContainerManagementProtocol proxy = (ContainerManagementProtocol)
|
||||
rpc.getProxy(ContainerManagementProtocol.class,
|
||||
ContainerManagementProtocol proxy = (ContainerManagementProtocol)
|
||||
rpc.getProxy(ContainerManagementProtocol.class,
|
||||
NetUtils.getConnectAddress(server), conf);
|
||||
ContainerLaunchContext containerLaunchContext =
|
||||
ContainerLaunchContext containerLaunchContext =
|
||||
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
||||
|
||||
ApplicationId applicationId = ApplicationId.newInstance(0, 0);
|
||||
|
@ -255,7 +255,7 @@ public class TestRPC {
|
|||
public GetContainerStatusesResponse getContainerStatuses(
|
||||
GetContainerStatusesRequest request)
|
||||
throws YarnException {
|
||||
GetContainerStatusesResponse response =
|
||||
GetContainerStatusesResponse response =
|
||||
recordFactory.newRecordInstance(GetContainerStatusesResponse.class);
|
||||
response.setContainerStatuses(statuses);
|
||||
return response;
|
||||
|
@ -287,9 +287,9 @@ public class TestRPC {
|
|||
}
|
||||
|
||||
@Override
|
||||
public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
public StopContainersResponse stopContainers(StopContainersRequest request)
|
||||
throws YarnException {
|
||||
Exception e = new Exception(EXCEPTION_MSG,
|
||||
Exception e = new Exception(EXCEPTION_MSG,
|
||||
new Exception(EXCEPTION_CAUSE));
|
||||
throw new YarnException(e);
|
||||
}
|
||||
|
@ -332,32 +332,32 @@ public class TestRPC {
|
|||
.buildTokenService(addr).toString());
|
||||
return containerToken;
|
||||
}
|
||||
|
||||
// A dummy implementation for AggregatorNodemanagerProtocol for test purpose,
|
||||
// it only can accept one appID, aggregatorAddr pair or throw exceptions
|
||||
public class DummyNMAggregatorService
|
||||
implements AggregatorNodemanagerProtocol {
|
||||
|
||||
|
||||
// A dummy implementation for CollectorNodemanagerProtocol for test purpose,
|
||||
// it only can accept one appID, collectorAddr pair or throw exceptions
|
||||
public class DummyNMCollectorService
|
||||
implements CollectorNodemanagerProtocol {
|
||||
|
||||
@Override
|
||||
public ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
|
||||
ReportNewAggregatorsInfoRequest request)
|
||||
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
||||
ReportNewCollectorInfoRequest request)
|
||||
throws YarnException, IOException {
|
||||
List<AppAggregatorsMap> appAggregators = request.getAppAggregatorsList();
|
||||
if (appAggregators.size() == 1) {
|
||||
// check default appID and aggregatorAddr
|
||||
AppAggregatorsMap appAggregator = appAggregators.get(0);
|
||||
Assert.assertEquals(appAggregator.getApplicationId(),
|
||||
List<AppCollectorsMap> appCollectors = request.getAppCollectorsList();
|
||||
if (appCollectors.size() == 1) {
|
||||
// check default appID and collectorAddr
|
||||
AppCollectorsMap appCollector = appCollectors.get(0);
|
||||
Assert.assertEquals(appCollector.getApplicationId(),
|
||||
DEFAULT_APP_ID);
|
||||
Assert.assertEquals(appAggregator.getAggregatorAddr(),
|
||||
DEFAULT_AGGREGATOR_ADDR);
|
||||
Assert.assertEquals(appCollector.getCollectorAddr(),
|
||||
DEFAULT_COLLECTOR_ADDR);
|
||||
} else {
|
||||
throw new YarnException(ILLEGAL_NUMBER_MESSAGE);
|
||||
}
|
||||
|
||||
ReportNewAggregatorsInfoResponse response =
|
||||
recordFactory.newRecordInstance(ReportNewAggregatorsInfoResponse.class);
|
||||
|
||||
ReportNewCollectorInfoResponse response =
|
||||
recordFactory.newRecordInstance(ReportNewCollectorInfoResponse.class);
|
||||
return response;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -109,14 +109,14 @@ public class TestYarnServerApiClasses {
|
|||
original.setLastKnownNMTokenMasterKey(getMasterKey());
|
||||
original.setNodeStatus(getNodeStatus());
|
||||
original.setNodeLabels(getValidNodeLabels());
|
||||
Map<ApplicationId, String> aggregators = getAggregators();
|
||||
original.setRegisteredAggregators(aggregators);
|
||||
Map<ApplicationId, String> collectors = getCollectors();
|
||||
original.setRegisteredCollectors(collectors);
|
||||
NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl(
|
||||
original.getProto());
|
||||
assertEquals(1, copy.getLastKnownContainerTokenMasterKey().getKeyId());
|
||||
assertEquals(1, copy.getLastKnownNMTokenMasterKey().getKeyId());
|
||||
assertEquals("localhost", copy.getNodeStatus().getNodeId().getHost());
|
||||
assertEquals(aggregators, copy.getRegisteredAggregators());
|
||||
assertEquals(collectors, copy.getRegisteredCollectors());
|
||||
// check labels are coming with valid values
|
||||
Assert.assertTrue(original.getNodeLabels()
|
||||
.containsAll(copy.getNodeLabels()));
|
||||
|
@ -153,8 +153,8 @@ public class TestYarnServerApiClasses {
|
|||
original.setNextHeartBeatInterval(1000);
|
||||
original.setNodeAction(NodeAction.NORMAL);
|
||||
original.setResponseId(100);
|
||||
Map<ApplicationId, String> aggregators = getAggregators();
|
||||
original.setAppAggregatorsMap(aggregators);
|
||||
Map<ApplicationId, String> collectors = getCollectors();
|
||||
original.setAppCollectorsMap(collectors);
|
||||
|
||||
NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl(
|
||||
original.getProto());
|
||||
|
@ -164,7 +164,7 @@ public class TestYarnServerApiClasses {
|
|||
assertEquals(1, copy.getContainerTokenMasterKey().getKeyId());
|
||||
assertEquals(1, copy.getNMTokenMasterKey().getKeyId());
|
||||
assertEquals("testDiagnosticMessage", copy.getDiagnosticsMessage());
|
||||
assertEquals(aggregators, copy.getAppAggregatorsMap());
|
||||
assertEquals(collectors, copy.getAppCollectorsMap());
|
||||
assertEquals(false, copy.getAreNodeLabelsAcceptedByRM());
|
||||
}
|
||||
|
||||
|
@ -344,15 +344,15 @@ public class TestYarnServerApiClasses {
|
|||
return nodeLabels;
|
||||
}
|
||||
|
||||
private Map<ApplicationId, String> getAggregators() {
|
||||
private Map<ApplicationId, String> getCollectors() {
|
||||
ApplicationId appID = ApplicationId.newInstance(1L, 1);
|
||||
String aggregatorAddr = "localhost:0";
|
||||
Map<ApplicationId, String> aggregatorMap =
|
||||
String collectorAddr = "localhost:0";
|
||||
Map<ApplicationId, String> collectorMap =
|
||||
new HashMap<ApplicationId, String>();
|
||||
aggregatorMap.put(appID, aggregatorAddr);
|
||||
return aggregatorMap;
|
||||
collectorMap.put(appID, collectorAddr);
|
||||
return collectorMap;
|
||||
}
|
||||
|
||||
|
||||
private ContainerStatus getContainerStatus(int applicationId,
|
||||
int containerID, int appAttemptId) {
|
||||
ContainerStatus status = recordFactory
|
||||
|
|
|
@ -70,19 +70,19 @@ public interface Context {
|
|||
ConcurrentMap<ApplicationId, Application> getApplications();
|
||||
|
||||
Map<ApplicationId, Credentials> getSystemCredentialsForApps();
|
||||
|
||||
|
||||
/**
|
||||
* Get the registered aggregators that located on this NM.
|
||||
* Get the registered collectors that located on this NM.
|
||||
* @return registered
|
||||
*/
|
||||
Map<ApplicationId, String> getRegisteredAggregators();
|
||||
|
||||
Map<ApplicationId, String> getRegisteredCollectors();
|
||||
|
||||
/**
|
||||
* Return the known aggregators which get from RM for all active applications
|
||||
* Return the known collectors which get from RM for all active applications
|
||||
* running on this NM.
|
||||
* @return known aggregators.
|
||||
* @return known collectors.
|
||||
*/
|
||||
Map<ApplicationId, String> getKnownAggregators();
|
||||
Map<ApplicationId, String> getKnownCollectors();
|
||||
|
||||
ConcurrentMap<ContainerId, Container> getContainers();
|
||||
|
||||
|
|
|
@ -60,7 +60,7 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
|||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.aggregatormanager.NMAggregatorService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.collectormanager.NMCollectorService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
|
@ -99,7 +99,7 @@ public class NodeManager extends CompositeService
|
|||
private Context context;
|
||||
private AsyncDispatcher dispatcher;
|
||||
private ContainerManagerImpl containerManager;
|
||||
private NMAggregatorService nmAggregatorService;
|
||||
private NMCollectorService nmCollectorService;
|
||||
private NodeStatusUpdater nodeStatusUpdater;
|
||||
private NodeResourceMonitor nodeResourceMonitor;
|
||||
private static CompositeServiceShutdownHook nodeManagerShutdownHook;
|
||||
|
@ -184,9 +184,9 @@ public class NodeManager extends CompositeService
|
|||
metrics, dirsHandler);
|
||||
}
|
||||
}
|
||||
|
||||
protected NMAggregatorService createNMAggregatorService(Context context) {
|
||||
return new NMAggregatorService(context);
|
||||
|
||||
protected NMCollectorService createNMCollectorService(Context context) {
|
||||
return new NMCollectorService(context);
|
||||
}
|
||||
|
||||
protected WebServer createWebServer(Context nmContext,
|
||||
|
@ -379,9 +379,9 @@ public class NodeManager extends CompositeService
|
|||
metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
|
||||
|
||||
DefaultMetricsSystem.initialize("NodeManager");
|
||||
|
||||
this.nmAggregatorService = createNMAggregatorService(context);
|
||||
addService(nmAggregatorService);
|
||||
|
||||
this.nmCollectorService = createNMCollectorService(context);
|
||||
addService(nmCollectorService);
|
||||
|
||||
// StatusUpdater should be added last so that it get started last
|
||||
// so that we make sure everything is up before registering with RM.
|
||||
|
@ -474,11 +474,11 @@ public class NodeManager extends CompositeService
|
|||
|
||||
protected final ConcurrentMap<ContainerId, Container> containers =
|
||||
new ConcurrentSkipListMap<ContainerId, Container>();
|
||||
|
||||
protected Map<ApplicationId, String> registeredAggregators =
|
||||
|
||||
protected Map<ApplicationId, String> registeredCollectors =
|
||||
new ConcurrentHashMap<ApplicationId, String>();
|
||||
|
||||
protected Map<ApplicationId, String> knownAggregators =
|
||||
|
||||
protected Map<ApplicationId, String> knownCollectors =
|
||||
new ConcurrentHashMap<ApplicationId, String>();
|
||||
|
||||
protected final ConcurrentMap<ContainerId,
|
||||
|
@ -662,26 +662,26 @@ public class NodeManager extends CompositeService
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<ApplicationId, String> getRegisteredAggregators() {
|
||||
return this.registeredAggregators;
|
||||
public Map<ApplicationId, String> getRegisteredCollectors() {
|
||||
return this.registeredCollectors;
|
||||
}
|
||||
|
||||
public void addRegisteredAggregators(
|
||||
Map<ApplicationId, String> newRegisteredAggregators) {
|
||||
this.registeredAggregators.putAll(newRegisteredAggregators);
|
||||
// Update to knownAggregators as well so it can immediately be consumed by
|
||||
public void addRegisteredCollectors(
|
||||
Map<ApplicationId, String> newRegisteredCollectors) {
|
||||
this.registeredCollectors.putAll(newRegisteredCollectors);
|
||||
// Update to knownCollectors as well so it can immediately be consumed by
|
||||
// this NM's TimelineClient.
|
||||
this.knownAggregators.putAll(newRegisteredAggregators);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<ApplicationId, String> getKnownAggregators() {
|
||||
return this.knownAggregators;
|
||||
this.knownCollectors.putAll(newRegisteredCollectors);
|
||||
}
|
||||
|
||||
public void addKnownAggregators(
|
||||
Map<ApplicationId, String> knownAggregators) {
|
||||
this.knownAggregators.putAll(knownAggregators);
|
||||
@Override
|
||||
public Map<ApplicationId, String> getKnownCollectors() {
|
||||
return this.knownCollectors;
|
||||
}
|
||||
|
||||
public void addKnownCollectors(
|
||||
Map<ApplicationId, String> knownCollectors) {
|
||||
this.knownCollectors.putAll(knownCollectors);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -781,10 +781,10 @@ public class NodeManager extends CompositeService
|
|||
public Context getNMContext() {
|
||||
return this.context;
|
||||
}
|
||||
|
||||
|
||||
// For testing
|
||||
NMAggregatorService getNMAggregatorService() {
|
||||
return this.nmAggregatorService;
|
||||
NMCollectorService getNMCollectorService() {
|
||||
return this.nmCollectorService;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
|
|
|
@ -814,7 +814,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|||
NodeStatusUpdaterImpl.this.context
|
||||
.getNMTokenSecretManager().getCurrentKey(),
|
||||
nodeLabelsForHeartbeat,
|
||||
NodeStatusUpdaterImpl.this.context.getRegisteredAggregators());
|
||||
NodeStatusUpdaterImpl.this.context.getRegisteredCollectors());
|
||||
|
||||
if (logAggregationEnabled) {
|
||||
// pull log aggregation status for application running in this NM
|
||||
|
@ -907,8 +907,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|||
}
|
||||
}
|
||||
|
||||
Map<ApplicationId, String> knownAggregators = response.getAppAggregatorsMap();
|
||||
((NodeManager.NMContext)context).addKnownAggregators(knownAggregators);
|
||||
Map<ApplicationId, String> knownCollectors =
|
||||
response.getAppCollectorsMap();
|
||||
((NodeManager.NMContext)context).addKnownCollectors(knownCollectors);
|
||||
|
||||
} catch (ConnectException e) {
|
||||
//catch and throw the exception if tried MAX wait time to connect RM
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.server.nodemanager.aggregatormanager;
|
||||
package org.apache.hadoop.yarn.server.nodemanager.collectormanager;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -25,46 +25,43 @@ import java.util.Map;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.Server;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.service.CompositeService;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
||||
|
||||
public class NMAggregatorService extends CompositeService implements
|
||||
AggregatorNodemanagerProtocol {
|
||||
public class NMCollectorService extends CompositeService implements
|
||||
CollectorNodemanagerProtocol {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(NMAggregatorService.class);
|
||||
private static final Log LOG = LogFactory.getLog(NMCollectorService.class);
|
||||
|
||||
final Context context;
|
||||
|
||||
|
||||
private Server server;
|
||||
|
||||
public NMAggregatorService(Context context) {
|
||||
|
||||
super(NMAggregatorService.class.getName());
|
||||
public NMCollectorService(Context context) {
|
||||
|
||||
super(NMCollectorService.class.getName());
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceStart() throws Exception {
|
||||
Configuration conf = getConfig();
|
||||
|
||||
InetSocketAddress aggregatorServerAddress = conf.getSocketAddr(
|
||||
|
||||
InetSocketAddress collectorServerAddress = conf.getSocketAddr(
|
||||
YarnConfiguration.NM_BIND_HOST,
|
||||
YarnConfiguration.NM_AGGREGATOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_PORT);
|
||||
YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_PORT);
|
||||
|
||||
Configuration serverConf = new Configuration(conf);
|
||||
|
||||
|
@ -72,42 +69,42 @@ public class NMAggregatorService extends CompositeService implements
|
|||
YarnRPC rpc = YarnRPC.create(conf);
|
||||
|
||||
server =
|
||||
rpc.getServer(AggregatorNodemanagerProtocol.class, this,
|
||||
aggregatorServerAddress, serverConf,
|
||||
rpc.getServer(CollectorNodemanagerProtocol.class, this,
|
||||
collectorServerAddress, serverConf,
|
||||
this.context.getNMTokenSecretManager(),
|
||||
conf.getInt(YarnConfiguration.NM_AGGREGATOR_SERVICE_THREAD_COUNT,
|
||||
YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_THREAD_COUNT));
|
||||
conf.getInt(YarnConfiguration.NM_COLLECTOR_SERVICE_THREAD_COUNT,
|
||||
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT));
|
||||
|
||||
server.start();
|
||||
// start remaining services
|
||||
super.serviceStart();
|
||||
LOG.info("NMAggregatorService started at " + aggregatorServerAddress);
|
||||
LOG.info("NMCollectorService started at " + collectorServerAddress);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public void serviceStop() throws Exception {
|
||||
if (server != null) {
|
||||
server.stop();
|
||||
}
|
||||
// TODO may cleanup app aggregators running on this NM in future.
|
||||
// TODO may cleanup app collectors running on this NM in future.
|
||||
super.serviceStop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
|
||||
ReportNewAggregatorsInfoRequest request) throws IOException {
|
||||
List<AppAggregatorsMap> newAggregatorsList = request.getAppAggregatorsList();
|
||||
if (newAggregatorsList != null && !newAggregatorsList.isEmpty()) {
|
||||
Map<ApplicationId, String> newAggregatorsMap =
|
||||
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
||||
ReportNewCollectorInfoRequest request) throws IOException {
|
||||
List<AppCollectorsMap> newCollectorsList = request.getAppCollectorsList();
|
||||
if (newCollectorsList != null && !newCollectorsList.isEmpty()) {
|
||||
Map<ApplicationId, String> newCollectorsMap =
|
||||
new HashMap<ApplicationId, String>();
|
||||
for (AppAggregatorsMap aggregator : newAggregatorsList) {
|
||||
newAggregatorsMap.put(aggregator.getApplicationId(), aggregator.getAggregatorAddr());
|
||||
for (AppCollectorsMap collector : newCollectorsList) {
|
||||
newCollectorsMap.put(collector.getApplicationId(), collector.getCollectorAddr());
|
||||
}
|
||||
((NodeManager.NMContext)context).addRegisteredAggregators(newAggregatorsMap);
|
||||
((NodeManager.NMContext)context).addRegisteredCollectors(newCollectorsMap);
|
||||
}
|
||||
|
||||
return ReportNewAggregatorsInfoResponse.newInstance();
|
||||
|
||||
return ReportNewCollectorInfoResponse.newInstance();
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -496,10 +496,11 @@ public class ApplicationImpl implements Application {
|
|||
new LogHandlerAppFinishedEvent(app.appId));
|
||||
|
||||
app.context.getNMTokenSecretManager().appFinished(app.getAppId());
|
||||
// Remove aggregator info for finished apps.
|
||||
// TODO check we remove related aggregators info in failure cases (YARN-3038)
|
||||
app.context.getRegisteredAggregators().remove(app.getAppId());
|
||||
app.context.getKnownAggregators().remove(app.getAppId());
|
||||
// Remove collectors info for finished apps.
|
||||
// TODO check we remove related collectors info in failure cases
|
||||
// (YARN-3038)
|
||||
app.context.getRegisteredCollectors().remove(app.getAppId());
|
||||
app.context.getKnownCollectors().remove(app.getAppId());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -618,12 +618,12 @@ public abstract class BaseAMRMProxyTest {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<ApplicationId, String> getRegisteredAggregators() {
|
||||
public Map<ApplicationId, String> getRegisteredCollectors() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<ApplicationId, String> getKnownAggregators() {
|
||||
public Map<ApplicationId, String> getKnownCollectors() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -311,8 +311,8 @@ public class ApplicationMasterService extends AbstractService implements
|
|||
|
||||
RMApp rmApp =
|
||||
rmContext.getRMApps().get(applicationAttemptId.getApplicationId());
|
||||
// Remove aggregator address when app get finished.
|
||||
rmApp.removeAggregatorAddr();
|
||||
// Remove collector address when app get finished.
|
||||
rmApp.removeCollectorAddr();
|
||||
// checking whether the app exits in RMStateStore at first not to throw
|
||||
// ApplicationDoesNotExistInCacheException before and after
|
||||
// RM work-preserving restart.
|
||||
|
@ -573,10 +573,10 @@ public class ApplicationMasterService extends AbstractService implements
|
|||
allocateResponse.setDecreasedContainers(allocation.getDecreasedContainers());
|
||||
|
||||
allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
|
||||
|
||||
// add aggregator address for this application
|
||||
allocateResponse.setAggregatorAddr(
|
||||
this.rmContext.getRMApps().get(applicationId).getAggregatorAddr());
|
||||
|
||||
// add collector address for this application
|
||||
allocateResponse.setCollectorAddr(
|
||||
this.rmContext.getRMApps().get(applicationId).getCollectorAddr());
|
||||
|
||||
// add preemption to the allocateResponse message (if any)
|
||||
allocateResponse
|
||||
|
|
|
@ -70,7 +70,7 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppAggregatorUpdateEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppCollectorUpdateEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||
|
@ -516,11 +516,11 @@ public class ResourceTrackerService extends AbstractService implements
|
|||
return YarnServerBuilderUtils.newNodeHeartbeatResponse(NodeAction.RESYNC,
|
||||
message);
|
||||
}
|
||||
|
||||
// Check & update aggregators info from request.
|
||||
|
||||
// Check & update collectors info from request.
|
||||
// TODO make sure it won't have race condition issue for AM failed over case
|
||||
// that the older registration could possible override the newer one.
|
||||
updateAppAggregatorsMap(request);
|
||||
updateAppCollectorsMap(request);
|
||||
|
||||
// Heartbeat response
|
||||
NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
|
||||
|
@ -538,13 +538,14 @@ public class ResourceTrackerService extends AbstractService implements
|
|||
if (!systemCredentials.isEmpty()) {
|
||||
nodeHeartBeatResponse.setSystemCredentialsForApps(systemCredentials);
|
||||
}
|
||||
|
||||
// Return aggregators' map that NM needs to know
|
||||
// TODO we should optimize this to only include aggreator info that NM
|
||||
|
||||
// Return collectors' map that NM needs to know
|
||||
// TODO we should optimize this to only include collector info that NM
|
||||
// doesn't know yet.
|
||||
List<ApplicationId> keepAliveApps = remoteNodeStatus.getKeepAliveApplications();
|
||||
List<ApplicationId> keepAliveApps =
|
||||
remoteNodeStatus.getKeepAliveApplications();
|
||||
if (keepAliveApps != null) {
|
||||
setAppAggregatorsMapToResponse(keepAliveApps, nodeHeartBeatResponse);
|
||||
setAppCollectorsMapToResponse(keepAliveApps, nodeHeartBeatResponse);
|
||||
}
|
||||
|
||||
// 4. Send status to RMNode, saving the latest response.
|
||||
|
@ -589,48 +590,49 @@ public class ResourceTrackerService extends AbstractService implements
|
|||
}
|
||||
return nodeHeartBeatResponse;
|
||||
}
|
||||
|
||||
private void setAppAggregatorsMapToResponse(
|
||||
|
||||
private void setAppCollectorsMapToResponse(
|
||||
List<ApplicationId> liveApps, NodeHeartbeatResponse response) {
|
||||
Map<ApplicationId, String> liveAppAggregatorsMap = new
|
||||
Map<ApplicationId, String> liveAppCollectorsMap = new
|
||||
ConcurrentHashMap<ApplicationId, String>();
|
||||
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
||||
for (ApplicationId appId : liveApps) {
|
||||
String appAggregatorAddr = rmApps.get(appId).getAggregatorAddr();
|
||||
if (appAggregatorAddr != null) {
|
||||
liveAppAggregatorsMap.put(appId, appAggregatorAddr);
|
||||
String appCollectorAddr = rmApps.get(appId).getCollectorAddr();
|
||||
if (appCollectorAddr != null) {
|
||||
liveAppCollectorsMap.put(appId, appCollectorAddr);
|
||||
} else {
|
||||
// Log a debug info if aggregator address is not found.
|
||||
// Log a debug info if collector address is not found.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Aggregator for applicaton: " + appId + " hasn't registered yet!");
|
||||
LOG.debug("Collector for applicaton: " + appId +
|
||||
" hasn't registered yet!");
|
||||
}
|
||||
}
|
||||
}
|
||||
response.setAppAggregatorsMap(liveAppAggregatorsMap);
|
||||
response.setAppCollectorsMap(liveAppCollectorsMap);
|
||||
}
|
||||
|
||||
private void updateAppAggregatorsMap(NodeHeartbeatRequest request) {
|
||||
Map<ApplicationId, String> registeredAggregatorsMap =
|
||||
request.getRegisteredAggregators();
|
||||
if (registeredAggregatorsMap != null
|
||||
&& !registeredAggregatorsMap.isEmpty()) {
|
||||
|
||||
private void updateAppCollectorsMap(NodeHeartbeatRequest request) {
|
||||
Map<ApplicationId, String> registeredCollectorsMap =
|
||||
request.getRegisteredCollectors();
|
||||
if (registeredCollectorsMap != null
|
||||
&& !registeredCollectorsMap.isEmpty()) {
|
||||
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
||||
for (Map.Entry<ApplicationId, String> entry:
|
||||
registeredAggregatorsMap.entrySet()) {
|
||||
for (Map.Entry<ApplicationId, String> entry:
|
||||
registeredCollectorsMap.entrySet()) {
|
||||
ApplicationId appId = entry.getKey();
|
||||
String aggregatorAddr = entry.getValue();
|
||||
if (aggregatorAddr != null && !aggregatorAddr.isEmpty()) {
|
||||
String collectorAddr = entry.getValue();
|
||||
if (collectorAddr != null && !collectorAddr.isEmpty()) {
|
||||
RMApp rmApp = rmApps.get(appId);
|
||||
if (rmApp == null) {
|
||||
LOG.warn("Cannot update aggregator info because application ID: " +
|
||||
LOG.warn("Cannot update collector info because application ID: " +
|
||||
appId + " is not found in RMContext!");
|
||||
} else {
|
||||
String previousAggregatorAddr = rmApp.getAggregatorAddr();
|
||||
if (previousAggregatorAddr == null ||
|
||||
previousAggregatorAddr != aggregatorAddr) {
|
||||
// sending aggregator update event.
|
||||
RMAppAggregatorUpdateEvent event =
|
||||
new RMAppAggregatorUpdateEvent(appId, aggregatorAddr);
|
||||
String previousCollectorAddr = rmApp.getCollectorAddr();
|
||||
if (previousCollectorAddr == null ||
|
||||
previousCollectorAddr != collectorAddr) {
|
||||
// sending collector update event.
|
||||
RMAppCollectorUpdateEvent event =
|
||||
new RMAppCollectorUpdateEvent(appId, collectorAddr);
|
||||
rmContext.getDispatcher().getEventHandler().handle(event);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -175,23 +175,23 @@ public interface RMApp extends EventHandler<RMAppEvent> {
|
|||
* @return the tracking url for the application master.
|
||||
*/
|
||||
String getTrackingUrl();
|
||||
|
||||
|
||||
/**
|
||||
* The aggregator address for the application.
|
||||
* @return the address for the application's aggregator.
|
||||
* The collector address for the application.
|
||||
* @return the address for the application's collector.
|
||||
*/
|
||||
String getAggregatorAddr();
|
||||
|
||||
String getCollectorAddr();
|
||||
|
||||
/**
|
||||
* Set aggregator address for the application
|
||||
* @param aggregatorAddr the address of aggregator
|
||||
* Set collector address for the application
|
||||
* @param collectorAddr the address of collector
|
||||
*/
|
||||
void setAggregatorAddr(String aggregatorAddr);
|
||||
|
||||
void setCollectorAddr(String collectorAddr);
|
||||
|
||||
/**
|
||||
* Remove aggregator address when application is finished or killed.
|
||||
* Remove collector address when application is finished or killed.
|
||||
*/
|
||||
void removeAggregatorAddr();
|
||||
void removeCollectorAddr();
|
||||
|
||||
/**
|
||||
* The original tracking url for the application master.
|
||||
|
|
|
@ -20,17 +20,18 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
|
|||
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
|
||||
public class RMAppAggregatorUpdateEvent extends RMAppEvent {
|
||||
public class RMAppCollectorUpdateEvent extends RMAppEvent {
|
||||
|
||||
private final String appAggregatorAddr;
|
||||
|
||||
public RMAppAggregatorUpdateEvent(ApplicationId appId, String appAggregatorAddr) {
|
||||
super(appId, RMAppEventType.AGGREGATOR_UPDATE);
|
||||
this.appAggregatorAddr = appAggregatorAddr;
|
||||
private final String appCollectorAddr;
|
||||
|
||||
public RMAppCollectorUpdateEvent(ApplicationId appId,
|
||||
String appCollectorAddr) {
|
||||
super(appId, RMAppEventType.COLLECTOR_UPDATE);
|
||||
this.appCollectorAddr = appCollectorAddr;
|
||||
}
|
||||
|
||||
public String getAppAggregatorAddr(){
|
||||
return this.appAggregatorAddr;
|
||||
|
||||
public String getAppCollectorAddr(){
|
||||
return this.appCollectorAddr;
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -30,9 +30,9 @@ public enum RMAppEventType {
|
|||
|
||||
// Source: Scheduler
|
||||
APP_ACCEPTED,
|
||||
|
||||
|
||||
// TODO add source later
|
||||
AGGREGATOR_UPDATE,
|
||||
COLLECTOR_UPDATE,
|
||||
|
||||
// Source: RMAppAttempt
|
||||
ATTEMPT_REGISTERED,
|
||||
|
|
|
@ -152,7 +152,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
private long storedFinishTime = 0;
|
||||
private int firstAttemptIdInStateStore = 1;
|
||||
private int nextAttemptId = 1;
|
||||
private String aggregatorAddr;
|
||||
private String collectorAddr;
|
||||
// This field isn't protected by readlock now.
|
||||
private volatile RMAppAttempt currentAttempt;
|
||||
private String queue;
|
||||
|
@ -201,7 +201,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
.addTransition(RMAppState.NEW, RMAppState.NEW,
|
||||
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
|
||||
.addTransition(RMAppState.NEW, RMAppState.NEW,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
.addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
|
||||
RMAppEventType.START, new RMAppNewlySavingTransition())
|
||||
.addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
|
||||
|
@ -219,7 +219,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
.addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
|
||||
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
|
||||
.addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
.addTransition(RMAppState.NEW_SAVING, RMAppState.SUBMITTED,
|
||||
RMAppEventType.APP_NEW_SAVED, new AddApplicationToSchedulerTransition())
|
||||
.addTransition(RMAppState.NEW_SAVING, RMAppState.FINAL_SAVING,
|
||||
|
@ -239,7 +239,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
.addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
|
||||
RMAppEventType.MOVE, new RMAppMoveTransition())
|
||||
.addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
.addTransition(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING,
|
||||
RMAppEventType.APP_REJECTED,
|
||||
new FinalSavingTransition(
|
||||
|
@ -257,7 +257,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
.addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
|
||||
RMAppEventType.MOVE, new RMAppMoveTransition())
|
||||
.addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
.addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING,
|
||||
RMAppEventType.ATTEMPT_REGISTERED, new RMAppStateUpdateTransition(
|
||||
YarnApplicationState.RUNNING))
|
||||
|
@ -286,7 +286,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
.addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
|
||||
RMAppEventType.MOVE, new RMAppMoveTransition())
|
||||
.addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
.addTransition(RMAppState.RUNNING, RMAppState.FINAL_SAVING,
|
||||
RMAppEventType.ATTEMPT_UNREGISTERED,
|
||||
new FinalSavingTransition(
|
||||
|
@ -317,7 +317,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
RMAppEventType.APP_RUNNING_ON_NODE,
|
||||
new AppRunningOnNodeTransition())
|
||||
.addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
// ignorable transitions
|
||||
.addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
|
||||
EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.KILL,
|
||||
|
@ -330,7 +330,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
RMAppEventType.APP_RUNNING_ON_NODE,
|
||||
new AppRunningOnNodeTransition())
|
||||
.addTransition(RMAppState.FINISHING, RMAppState.FINISHING,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
// ignorable transitions
|
||||
.addTransition(RMAppState.FINISHING, RMAppState.FINISHING,
|
||||
EnumSet.of(RMAppEventType.NODE_UPDATE,
|
||||
|
@ -343,7 +343,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
RMAppEventType.APP_RUNNING_ON_NODE,
|
||||
new AppRunningOnNodeTransition())
|
||||
.addTransition(RMAppState.KILLING, RMAppState.KILLING,
|
||||
RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
|
||||
RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
|
||||
.addTransition(RMAppState.KILLING, RMAppState.FINAL_SAVING,
|
||||
RMAppEventType.ATTEMPT_KILLED,
|
||||
new FinalSavingTransition(
|
||||
|
@ -577,20 +577,20 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
public void setQueue(String queue) {
|
||||
this.queue = queue;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String getAggregatorAddr() {
|
||||
return this.aggregatorAddr;
|
||||
public String getCollectorAddr() {
|
||||
return this.collectorAddr;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void setAggregatorAddr(String aggregatorAddr) {
|
||||
this.aggregatorAddr = aggregatorAddr;
|
||||
public void setCollectorAddr(String collectorAddr) {
|
||||
this.collectorAddr = collectorAddr;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void removeAggregatorAddr() {
|
||||
this.aggregatorAddr = null;
|
||||
public void removeCollectorAddr() {
|
||||
this.collectorAddr = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -864,8 +864,8 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
|
||||
// send the ATS create Event
|
||||
sendATSCreateEvent(this, this.startTime);
|
||||
//TODO recover aggregator address.
|
||||
//this.aggregatorAddr = appState.getAggregatorAddr();
|
||||
//TODO recover collector address.
|
||||
//this.collectorAddr = appState.getCollectorAddr();
|
||||
|
||||
RMAppAttemptImpl preAttempt = null;
|
||||
for (ApplicationAttemptId attemptId :
|
||||
|
@ -938,22 +938,22 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
};
|
||||
}
|
||||
|
||||
private static final class RMAppAggregatorUpdateTransition
|
||||
private static final class RMAppCollectorUpdateTransition
|
||||
extends RMAppTransition {
|
||||
|
||||
|
||||
public void transition(RMAppImpl app, RMAppEvent event) {
|
||||
LOG.info("Updating aggregator info for app: " + app.getApplicationId());
|
||||
|
||||
RMAppAggregatorUpdateEvent appAggregatorUpdateEvent =
|
||||
(RMAppAggregatorUpdateEvent) event;
|
||||
// Update aggregator address
|
||||
app.setAggregatorAddr(appAggregatorUpdateEvent.getAppAggregatorAddr());
|
||||
|
||||
LOG.info("Updating collector info for app: " + app.getApplicationId());
|
||||
|
||||
RMAppCollectorUpdateEvent appCollectorUpdateEvent =
|
||||
(RMAppCollectorUpdateEvent) event;
|
||||
// Update collector address
|
||||
app.setCollectorAddr(appCollectorUpdateEvent.getAppCollectorAddr());
|
||||
|
||||
// TODO persistent to RMStateStore for recover
|
||||
// Save to RMStateStore
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
private static final class RMAppNodeUpdateTransition extends RMAppTransition {
|
||||
public void transition(RMAppImpl app, RMAppEvent event) {
|
||||
RMAppNodeUpdateEvent nodeUpdateEvent = (RMAppNodeUpdateEvent) event;
|
||||
|
|
|
@ -95,15 +95,15 @@ public abstract class MockAsm extends MockApps {
|
|||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
@Override
|
||||
public String getAggregatorAddr() {
|
||||
public String getCollectorAddr() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
@Override
|
||||
public void setAggregatorAddr(String aggregatorAddr) {
|
||||
public void setCollectorAddr(String collectorAddr) {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
@Override
|
||||
public void removeAggregatorAddr() {
|
||||
public void removeCollectorAddr() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
@Override
|
||||
|
|
|
@ -302,17 +302,17 @@ public class MockRMApp implements RMApp {
|
|||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
public String getAggregatorAddr() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeAggregatorAddr() {
|
||||
public String getCollectorAddr() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAggregatorAddr(String aggregatorAddr) {
|
||||
public void removeCollectorAddr() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setCollectorAddr(String collectorAddr) {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,6 +56,11 @@
|
|||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
|
|
|
@ -1,25 +1,50 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice;
|
||||
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
import org.apache.hadoop.util.ExitUtil;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeTimelineAggregatorsAuxService;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class TestTimelineServiceClientIntegration {
|
||||
private static PerNodeTimelineAggregatorsAuxService auxService;
|
||||
private static TimelineCollectorManager collectorManager;
|
||||
private static PerNodeTimelineCollectorsAuxService auxService;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupClass() throws Exception {
|
||||
try {
|
||||
auxService = PerNodeTimelineAggregatorsAuxService.launchServer(new String[0]);
|
||||
collectorManager = new MyTimelineCollectorManager();
|
||||
auxService =
|
||||
PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
|
||||
collectorManager);
|
||||
auxService.addApplication(ApplicationId.newInstance(0, 1));
|
||||
} catch (ExitUtil.ExitException e) {
|
||||
fail();
|
||||
|
@ -38,6 +63,9 @@ public class TestTimelineServiceClientIntegration {
|
|||
TimelineClient client =
|
||||
TimelineClient.createTimelineClient(ApplicationId.newInstance(0, 1));
|
||||
try {
|
||||
// set the timeline service address manually
|
||||
client.setTimelineServiceAddress(
|
||||
collectorManager.getRestServerBindAddress());
|
||||
client.init(new YarnConfiguration());
|
||||
client.start();
|
||||
TimelineEntity entity = new TimelineEntity();
|
||||
|
@ -45,10 +73,20 @@ public class TestTimelineServiceClientIntegration {
|
|||
entity.setId("test entity id");
|
||||
client.putEntities(entity);
|
||||
client.putEntitiesAsync(entity);
|
||||
} catch(Exception e) {
|
||||
fail();
|
||||
} finally {
|
||||
client.stop();
|
||||
}
|
||||
}
|
||||
|
||||
private static class MyTimelineCollectorManager extends
|
||||
TimelineCollectorManager {
|
||||
public MyTimelineCollectorManager() {
|
||||
super();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected CollectorNodemanagerProtocol getNMCollectorService() {
|
||||
return mock(CollectorNodemanagerProtocol.class);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,6 +56,11 @@
|
|||
<artifactId>hadoop-yarn-common</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-common</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
|
||||
|
@ -71,6 +76,11 @@
|
|||
<artifactId>guice</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>servlet-api</artifactId>
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
|
@ -30,12 +30,12 @@ import org.apache.hadoop.conf.Configuration;
|
|||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public class AppLevelTimelineAggregator extends TimelineAggregator {
|
||||
public class AppLevelTimelineCollector extends TimelineCollector {
|
||||
private final String applicationId;
|
||||
// TODO define key metadata such as flow metadata, user, and queue
|
||||
|
||||
public AppLevelTimelineAggregator(String applicationId) {
|
||||
super(AppLevelTimelineAggregator.class.getName() + " - " + applicationId);
|
||||
public AppLevelTimelineCollector(String applicationId) {
|
||||
super(AppLevelTimelineCollector.class.getName() + " - " + applicationId);
|
||||
this.applicationId = applicationId;
|
||||
}
|
||||
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -42,85 +42,85 @@ import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* The top-level server for the per-node timeline aggregator collection. Currently
|
||||
* The top-level server for the per-node timeline collector manager. Currently
|
||||
* it is defined as an auxiliary service to accommodate running within another
|
||||
* daemon (e.g. node manager).
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public class PerNodeTimelineAggregatorsAuxService extends AuxiliaryService {
|
||||
public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(PerNodeTimelineAggregatorsAuxService.class);
|
||||
LogFactory.getLog(PerNodeTimelineCollectorsAuxService.class);
|
||||
private static final int SHUTDOWN_HOOK_PRIORITY = 30;
|
||||
|
||||
private final TimelineAggregatorsCollection aggregatorCollection;
|
||||
private final TimelineCollectorManager collectorManager;
|
||||
|
||||
public PerNodeTimelineAggregatorsAuxService() {
|
||||
public PerNodeTimelineCollectorsAuxService() {
|
||||
// use the same singleton
|
||||
this(TimelineAggregatorsCollection.getInstance());
|
||||
this(TimelineCollectorManager.getInstance());
|
||||
}
|
||||
|
||||
@VisibleForTesting PerNodeTimelineAggregatorsAuxService(
|
||||
TimelineAggregatorsCollection aggregatorCollection) {
|
||||
super("timeline_aggregator");
|
||||
this.aggregatorCollection = aggregatorCollection;
|
||||
@VisibleForTesting PerNodeTimelineCollectorsAuxService(
|
||||
TimelineCollectorManager collectorsManager) {
|
||||
super("timeline_collector");
|
||||
this.collectorManager = collectorsManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceInit(Configuration conf) throws Exception {
|
||||
aggregatorCollection.init(conf);
|
||||
collectorManager.init(conf);
|
||||
super.serviceInit(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceStart() throws Exception {
|
||||
aggregatorCollection.start();
|
||||
collectorManager.start();
|
||||
super.serviceStart();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceStop() throws Exception {
|
||||
aggregatorCollection.stop();
|
||||
collectorManager.stop();
|
||||
super.serviceStop();
|
||||
}
|
||||
|
||||
// these methods can be used as the basis for future service methods if the
|
||||
// per-node aggregator runs separate from the node manager
|
||||
// per-node collector runs separate from the node manager
|
||||
/**
|
||||
* Creates and adds an app level aggregator for the specified application id.
|
||||
* The aggregator is also initialized and started. If the service already
|
||||
* Creates and adds an app level collector for the specified application id.
|
||||
* The collector is also initialized and started. If the service already
|
||||
* exists, no new service is created.
|
||||
*
|
||||
* @return whether it was added successfully
|
||||
*/
|
||||
public boolean addApplication(ApplicationId appId) {
|
||||
AppLevelTimelineAggregator aggregator =
|
||||
new AppLevelTimelineAggregator(appId.toString());
|
||||
return (aggregatorCollection.putIfAbsent(appId, aggregator)
|
||||
== aggregator);
|
||||
AppLevelTimelineCollector collector =
|
||||
new AppLevelTimelineCollector(appId.toString());
|
||||
return (collectorManager.putIfAbsent(appId, collector)
|
||||
== collector);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the app level aggregator for the specified application id. The
|
||||
* aggregator is also stopped as a result. If the aggregator does not exist, no
|
||||
* Removes the app level collector for the specified application id. The
|
||||
* collector is also stopped as a result. If the collector does not exist, no
|
||||
* change is made.
|
||||
*
|
||||
* @return whether it was removed successfully
|
||||
*/
|
||||
public boolean removeApplication(ApplicationId appId) {
|
||||
String appIdString = appId.toString();
|
||||
return aggregatorCollection.remove(appIdString);
|
||||
return collectorManager.remove(appIdString);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates and adds an app level aggregator for the specified application id.
|
||||
* The aggregator is also initialized and started. If the aggregator already
|
||||
* exists, no new aggregator is created.
|
||||
* Creates and adds an app level collector for the specified application id.
|
||||
* The collector is also initialized and started. If the collector already
|
||||
* exists, no new collector is created.
|
||||
*/
|
||||
@Override
|
||||
public void initializeContainer(ContainerInitializationContext context) {
|
||||
// intercept the event of the AM container being created and initialize the
|
||||
// app level aggregator service
|
||||
// app level collector service
|
||||
if (isApplicationMaster(context)) {
|
||||
ApplicationId appId = context.getContainerId().
|
||||
getApplicationAttemptId().getApplicationId();
|
||||
|
@ -129,14 +129,14 @@ public class PerNodeTimelineAggregatorsAuxService extends AuxiliaryService {
|
|||
}
|
||||
|
||||
/**
|
||||
* Removes the app level aggregator for the specified application id. The
|
||||
* aggregator is also stopped as a result. If the aggregator does not exist, no
|
||||
* Removes the app level collector for the specified application id. The
|
||||
* collector is also stopped as a result. If the collector does not exist, no
|
||||
* change is made.
|
||||
*/
|
||||
@Override
|
||||
public void stopContainer(ContainerTerminationContext context) {
|
||||
// intercept the event of the AM container being stopped and remove the app
|
||||
// level aggregator service
|
||||
// level collector service
|
||||
if (isApplicationMaster(context)) {
|
||||
ApplicationId appId = context.getContainerId().
|
||||
getApplicationAttemptId().getApplicationId();
|
||||
|
@ -154,7 +154,7 @@ public class PerNodeTimelineAggregatorsAuxService extends AuxiliaryService {
|
|||
|
||||
@VisibleForTesting
|
||||
boolean hasApplication(String appId) {
|
||||
return aggregatorCollection.containsKey(appId);
|
||||
return collectorManager.containsKey(appId);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -173,30 +173,33 @@ public class PerNodeTimelineAggregatorsAuxService extends AuxiliaryService {
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static PerNodeTimelineAggregatorsAuxService launchServer(String[] args) {
|
||||
public static PerNodeTimelineCollectorsAuxService
|
||||
launchServer(String[] args, TimelineCollectorManager collectorManager) {
|
||||
Thread
|
||||
.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
|
||||
StringUtils.startupShutdownMessage(PerNodeTimelineAggregatorsAuxService.class, args,
|
||||
LOG);
|
||||
PerNodeTimelineAggregatorsAuxService auxService = null;
|
||||
StringUtils.startupShutdownMessage(
|
||||
PerNodeTimelineCollectorsAuxService.class, args, LOG);
|
||||
PerNodeTimelineCollectorsAuxService auxService = null;
|
||||
try {
|
||||
auxService = new PerNodeTimelineAggregatorsAuxService();
|
||||
auxService = collectorManager == null ?
|
||||
new PerNodeTimelineCollectorsAuxService() :
|
||||
new PerNodeTimelineCollectorsAuxService(collectorManager);
|
||||
ShutdownHookManager.get().addShutdownHook(new ShutdownHook(auxService),
|
||||
SHUTDOWN_HOOK_PRIORITY);
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
auxService.init(conf);
|
||||
auxService.start();
|
||||
} catch (Throwable t) {
|
||||
LOG.fatal("Error starting PerNodeAggregatorServer", t);
|
||||
ExitUtil.terminate(-1, "Error starting PerNodeAggregatorServer");
|
||||
LOG.fatal("Error starting PerNodeTimelineCollectorServer", t);
|
||||
ExitUtil.terminate(-1, "Error starting PerNodeTimelineCollectorServer");
|
||||
}
|
||||
return auxService;
|
||||
}
|
||||
|
||||
private static class ShutdownHook implements Runnable {
|
||||
private final PerNodeTimelineAggregatorsAuxService auxService;
|
||||
private final PerNodeTimelineCollectorsAuxService auxService;
|
||||
|
||||
public ShutdownHook(PerNodeTimelineAggregatorsAuxService auxService) {
|
||||
public ShutdownHook(PerNodeTimelineCollectorsAuxService auxService) {
|
||||
this.auxService = auxService;
|
||||
}
|
||||
|
||||
|
@ -206,6 +209,6 @@ public class PerNodeTimelineAggregatorsAuxService extends AuxiliaryService {
|
|||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
launchServer(args);
|
||||
launchServer(args, null);
|
||||
}
|
||||
}
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -27,12 +27,12 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.service.CompositeService;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
|
||||
/**
|
||||
* Service that handles writes to the timeline service and writes them to the
|
||||
* backing storage.
|
||||
|
@ -42,12 +42,12 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public abstract class TimelineAggregator extends CompositeService {
|
||||
private static final Log LOG = LogFactory.getLog(TimelineAggregator.class);
|
||||
public abstract class TimelineCollector extends CompositeService {
|
||||
private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
|
||||
|
||||
private TimelineWriter writer;
|
||||
|
||||
public TimelineAggregator(String name) {
|
||||
public TimelineCollector(String name) {
|
||||
super(name);
|
||||
}
|
||||
|
|
@ -16,11 +16,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
@ -39,64 +42,64 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
|
||||
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Class that manages adding and removing aggregators and their lifecycle. It
|
||||
* provides thread safety access to the aggregators inside.
|
||||
* Class that manages adding and removing collectors and their lifecycle. It
|
||||
* provides thread safety access to the collectors inside.
|
||||
*
|
||||
* It is a singleton, and instances should be obtained via
|
||||
* {@link #getInstance()}.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public class TimelineAggregatorsCollection extends CompositeService {
|
||||
public class TimelineCollectorManager extends CompositeService {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(TimelineAggregatorsCollection.class);
|
||||
private static final TimelineAggregatorsCollection INSTANCE =
|
||||
new TimelineAggregatorsCollection();
|
||||
LogFactory.getLog(TimelineCollectorManager.class);
|
||||
private static final TimelineCollectorManager INSTANCE =
|
||||
new TimelineCollectorManager();
|
||||
|
||||
// access to this map is synchronized with the map itself
|
||||
private final Map<String, TimelineAggregator> aggregators =
|
||||
private final Map<String, TimelineCollector> collectors =
|
||||
Collections.synchronizedMap(
|
||||
new HashMap<String, TimelineAggregator>());
|
||||
new HashMap<String, TimelineCollector>());
|
||||
|
||||
// REST server for this aggregator collection
|
||||
// REST server for this collector manager
|
||||
private HttpServer2 timelineRestServer;
|
||||
|
||||
|
||||
private String timelineRestServerBindAddress;
|
||||
|
||||
private AggregatorNodemanagerProtocol nmAggregatorService;
|
||||
|
||||
private InetSocketAddress nmAggregatorServiceAddress;
|
||||
|
||||
static final String AGGREGATOR_COLLECTION_ATTR_KEY = "aggregator.collection";
|
||||
private CollectorNodemanagerProtocol nmCollectorService;
|
||||
|
||||
static TimelineAggregatorsCollection getInstance() {
|
||||
private InetSocketAddress nmCollectorServiceAddress;
|
||||
|
||||
static final String COLLECTOR_MANAGER_ATTR_KEY = "collector.manager";
|
||||
|
||||
static TimelineCollectorManager getInstance() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
TimelineAggregatorsCollection() {
|
||||
super(TimelineAggregatorsCollection.class.getName());
|
||||
@VisibleForTesting
|
||||
protected TimelineCollectorManager() {
|
||||
super(TimelineCollectorManager.class.getName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serviceInit(Configuration conf) throws Exception {
|
||||
this.nmAggregatorServiceAddress = conf.getSocketAddr(
|
||||
this.nmCollectorServiceAddress = conf.getSocketAddr(
|
||||
YarnConfiguration.NM_BIND_HOST,
|
||||
YarnConfiguration.NM_AGGREGATOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_PORT);
|
||||
|
||||
YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_PORT);
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void serviceStart() throws Exception {
|
||||
startWebApp();
|
||||
|
@ -112,95 +115,95 @@ public class TimelineAggregatorsCollection extends CompositeService {
|
|||
}
|
||||
|
||||
/**
|
||||
* Put the aggregator into the collection if an aggregator mapped by id does
|
||||
* Put the collector into the collection if an collector mapped by id does
|
||||
* not exist.
|
||||
*
|
||||
* @throws YarnRuntimeException if there was any exception in initializing and
|
||||
* starting the app level service
|
||||
* @return the aggregator associated with id after the potential put.
|
||||
* @return the collector associated with id after the potential put.
|
||||
*/
|
||||
public TimelineAggregator putIfAbsent(ApplicationId appId,
|
||||
TimelineAggregator aggregator) {
|
||||
public TimelineCollector putIfAbsent(ApplicationId appId,
|
||||
TimelineCollector collector) {
|
||||
String id = appId.toString();
|
||||
TimelineAggregator aggregatorInTable;
|
||||
boolean aggregatorIsNew = false;
|
||||
synchronized (aggregators) {
|
||||
aggregatorInTable = aggregators.get(id);
|
||||
if (aggregatorInTable == null) {
|
||||
TimelineCollector collectorInTable;
|
||||
boolean collectorIsNew = false;
|
||||
synchronized (collectors) {
|
||||
collectorInTable = collectors.get(id);
|
||||
if (collectorInTable == null) {
|
||||
try {
|
||||
// initialize, start, and add it to the collection so it can be
|
||||
// cleaned up when the parent shuts down
|
||||
aggregator.init(getConfig());
|
||||
aggregator.start();
|
||||
aggregators.put(id, aggregator);
|
||||
LOG.info("the aggregator for " + id + " was added");
|
||||
aggregatorInTable = aggregator;
|
||||
aggregatorIsNew = true;
|
||||
collector.init(getConfig());
|
||||
collector.start();
|
||||
collectors.put(id, collector);
|
||||
LOG.info("the collector for " + id + " was added");
|
||||
collectorInTable = collector;
|
||||
collectorIsNew = true;
|
||||
} catch (Exception e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
}
|
||||
} else {
|
||||
String msg = "the aggregator for " + id + " already exists!";
|
||||
String msg = "the collector for " + id + " already exists!";
|
||||
LOG.error(msg);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
// Report to NM if a new aggregator is added.
|
||||
if (aggregatorIsNew) {
|
||||
// Report to NM if a new collector is added.
|
||||
if (collectorIsNew) {
|
||||
try {
|
||||
reportNewAggregatorToNM(appId);
|
||||
reportNewCollectorToNM(appId);
|
||||
} catch (Exception e) {
|
||||
// throw exception here as it cannot be used if failed report to NM
|
||||
LOG.error("Failed to report a new aggregator for application: " + appId +
|
||||
" to NM Aggregator Services.");
|
||||
LOG.error("Failed to report a new collector for application: " + appId +
|
||||
" to the NM Collector Service.");
|
||||
throw new YarnRuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
return aggregatorInTable;
|
||||
|
||||
return collectorInTable;
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the aggregator for the specified id. The aggregator is also stopped
|
||||
* as a result. If the aggregator does not exist, no change is made.
|
||||
* Removes the collector for the specified id. The collector is also stopped
|
||||
* as a result. If the collector does not exist, no change is made.
|
||||
*
|
||||
* @return whether it was removed successfully
|
||||
*/
|
||||
public boolean remove(String id) {
|
||||
synchronized (aggregators) {
|
||||
TimelineAggregator aggregator = aggregators.remove(id);
|
||||
if (aggregator == null) {
|
||||
String msg = "the aggregator for " + id + " does not exist!";
|
||||
synchronized (collectors) {
|
||||
TimelineCollector collector = collectors.remove(id);
|
||||
if (collector == null) {
|
||||
String msg = "the collector for " + id + " does not exist!";
|
||||
LOG.error(msg);
|
||||
return false;
|
||||
} else {
|
||||
// stop the service to do clean up
|
||||
aggregator.stop();
|
||||
LOG.info("the aggregator service for " + id + " was removed");
|
||||
collector.stop();
|
||||
LOG.info("the collector service for " + id + " was removed");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the aggregator for the specified id.
|
||||
* Returns the collector for the specified id.
|
||||
*
|
||||
* @return the aggregator or null if it does not exist
|
||||
* @return the collector or null if it does not exist
|
||||
*/
|
||||
public TimelineAggregator get(String id) {
|
||||
return aggregators.get(id);
|
||||
public TimelineCollector get(String id) {
|
||||
return collectors.get(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the aggregator for the specified id exists in this
|
||||
* Returns whether the collector for the specified id exists in this
|
||||
* collection.
|
||||
*/
|
||||
public boolean containsKey(String id) {
|
||||
return aggregators.containsKey(id);
|
||||
return collectors.containsKey(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Launch the REST web server for this aggregator collection
|
||||
* Launch the REST web server for this collector manager
|
||||
*/
|
||||
private void startWebApp() {
|
||||
Configuration conf = getConfig();
|
||||
|
@ -211,7 +214,7 @@ public class TimelineAggregatorsCollection extends CompositeService {
|
|||
WebAppUtils.getAHSWebAppURLWithoutScheme(conf));
|
||||
this.timelineRestServerBindAddress = WebAppUtils.getResolvedAddress(
|
||||
NetUtils.createSocketAddr(bindAddress));
|
||||
LOG.info("Instantiating the per-node aggregator webapp at " +
|
||||
LOG.info("Instantiating the per-node collector webapp at " +
|
||||
timelineRestServerBindAddress);
|
||||
try {
|
||||
Configuration confForInfoServer = new Configuration(conf);
|
||||
|
@ -232,40 +235,44 @@ public class TimelineAggregatorsCollection extends CompositeService {
|
|||
options, new String[] {"/*"});
|
||||
|
||||
timelineRestServer.addJerseyResourcePackage(
|
||||
TimelineAggregatorWebService.class.getPackage().getName() + ";"
|
||||
TimelineCollectorWebService.class.getPackage().getName() + ";"
|
||||
+ GenericExceptionHandler.class.getPackage().getName() + ";"
|
||||
+ YarnJacksonJaxbJsonProvider.class.getPackage().getName(),
|
||||
"/*");
|
||||
timelineRestServer.setAttribute(AGGREGATOR_COLLECTION_ATTR_KEY,
|
||||
TimelineAggregatorsCollection.getInstance());
|
||||
timelineRestServer.setAttribute(COLLECTOR_MANAGER_ATTR_KEY,
|
||||
TimelineCollectorManager.getInstance());
|
||||
timelineRestServer.start();
|
||||
} catch (Exception e) {
|
||||
String msg = "The per-node aggregator webapp failed to start.";
|
||||
String msg = "The per-node collector webapp failed to start.";
|
||||
LOG.error(msg, e);
|
||||
throw new YarnRuntimeException(msg, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void reportNewAggregatorToNM(ApplicationId appId)
|
||||
|
||||
private void reportNewCollectorToNM(ApplicationId appId)
|
||||
throws YarnException, IOException {
|
||||
this.nmAggregatorService = getNMAggregatorService();
|
||||
ReportNewAggregatorsInfoRequest request =
|
||||
ReportNewAggregatorsInfoRequest.newInstance(appId,
|
||||
this.nmCollectorService = getNMCollectorService();
|
||||
ReportNewCollectorInfoRequest request =
|
||||
ReportNewCollectorInfoRequest.newInstance(appId,
|
||||
this.timelineRestServerBindAddress);
|
||||
LOG.info("Report a new aggregator for application: " + appId +
|
||||
" to NM Aggregator Services.");
|
||||
nmAggregatorService.reportNewAggregatorInfo(request);
|
||||
LOG.info("Report a new collector for application: " + appId +
|
||||
" to the NM Collector Service.");
|
||||
nmCollectorService.reportNewCollectorInfo(request);
|
||||
}
|
||||
|
||||
// protected for test
|
||||
protected AggregatorNodemanagerProtocol getNMAggregatorService(){
|
||||
|
||||
@VisibleForTesting
|
||||
protected CollectorNodemanagerProtocol getNMCollectorService() {
|
||||
Configuration conf = getConfig();
|
||||
final YarnRPC rpc = YarnRPC.create(conf);
|
||||
|
||||
|
||||
// TODO Security settings.
|
||||
return (AggregatorNodemanagerProtocol) rpc.getProxy(
|
||||
AggregatorNodemanagerProtocol.class,
|
||||
nmAggregatorServiceAddress, conf);
|
||||
return (CollectorNodemanagerProtocol) rpc.getProxy(
|
||||
CollectorNodemanagerProtocol.class,
|
||||
nmCollectorServiceAddress, conf);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public String getRestServerBindAddress() {
|
||||
return timelineRestServerBindAddress;
|
||||
}
|
||||
|
||||
}
|
|
@ -16,12 +16,18 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
import javax.servlet.ServletContext;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
import javax.ws.rs.*;
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.PUT;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.QueryParam;
|
||||
import javax.ws.rs.WebApplicationException;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
|
@ -36,7 +42,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.apache.hadoop.yarn.webapp.ForbiddenException;
|
||||
|
@ -53,9 +58,9 @@ import com.google.inject.Singleton;
|
|||
@Unstable
|
||||
@Singleton
|
||||
@Path("/ws/v2/timeline")
|
||||
public class TimelineAggregatorWebService {
|
||||
public class TimelineCollectorWebService {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(TimelineAggregatorWebService.class);
|
||||
LogFactory.getLog(TimelineCollectorWebService.class);
|
||||
|
||||
private @Context ServletContext context;
|
||||
|
||||
|
@ -95,12 +100,12 @@ public class TimelineAggregatorWebService {
|
|||
@Context HttpServletRequest req,
|
||||
@Context HttpServletResponse res) {
|
||||
init(res);
|
||||
return new AboutInfo("Timeline API");
|
||||
return new AboutInfo("Timeline Collector API");
|
||||
}
|
||||
|
||||
/**
|
||||
* Accepts writes to the aggregator, and returns a response. It simply routes
|
||||
* the request to the app level aggregator. It expects an application as a
|
||||
* Accepts writes to the collector, and returns a response. It simply routes
|
||||
* the request to the app level collector. It expects an application as a
|
||||
* context.
|
||||
*/
|
||||
@PUT
|
||||
|
@ -128,12 +133,12 @@ public class TimelineAggregatorWebService {
|
|||
if (appId == null) {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
TimelineAggregator service = getAggregatorService(req, appId);
|
||||
if (service == null) {
|
||||
TimelineCollector collector = getCollector(req, appId);
|
||||
if (collector == null) {
|
||||
LOG.error("Application not found");
|
||||
throw new NotFoundException(); // different exception?
|
||||
}
|
||||
service.postEntities(entities, callerUgi);
|
||||
collector.postEntities(entities, callerUgi);
|
||||
return Response.ok().build();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error putting entities", e);
|
||||
|
@ -143,8 +148,6 @@ public class TimelineAggregatorWebService {
|
|||
}
|
||||
|
||||
private String parseApplicationId(String appId) {
|
||||
// Make sure the appId is not null and is valid
|
||||
ApplicationId appID;
|
||||
try {
|
||||
if (appId != null) {
|
||||
return ConverterUtils.toApplicationId(appId.trim()).toString();
|
||||
|
@ -156,13 +159,13 @@ public class TimelineAggregatorWebService {
|
|||
}
|
||||
}
|
||||
|
||||
private TimelineAggregator
|
||||
getAggregatorService(HttpServletRequest req, String appIdToParse) {
|
||||
private TimelineCollector
|
||||
getCollector(HttpServletRequest req, String appIdToParse) {
|
||||
String appIdString = parseApplicationId(appIdToParse);
|
||||
final TimelineAggregatorsCollection aggregatorCollection =
|
||||
(TimelineAggregatorsCollection) context.getAttribute(
|
||||
TimelineAggregatorsCollection.AGGREGATOR_COLLECTION_ATTR_KEY);
|
||||
return aggregatorCollection.get(appIdString);
|
||||
final TimelineCollectorManager collectorManager =
|
||||
(TimelineCollectorManager) context.getAttribute(
|
||||
TimelineCollectorManager.COLLECTOR_MANAGER_ATTR_KEY);
|
||||
return collectorManager.get(appIdString);
|
||||
}
|
||||
|
||||
private void init(HttpServletResponse response) {
|
|
@ -65,7 +65,7 @@ public class FileSystemTimelineWriterImpl extends AbstractService
|
|||
* Stores the entire information in {@link TimelineEntity} to the
|
||||
* timeline store. Any errors occurring for individual write request objects
|
||||
* will be reported in the response.
|
||||
*
|
||||
*
|
||||
* @param data
|
||||
* a {@link TimelineEntity} object
|
||||
* @return {@link TimelineWriteResponse} object.
|
||||
|
@ -116,10 +116,10 @@ public class FileSystemTimelineWriterImpl extends AbstractService
|
|||
* Aggregates the entity information to the timeline store based on which
|
||||
* track this entity is to be rolled up to The tracks along which aggregations
|
||||
* are to be done are given by {@link TimelineAggregationTrack}
|
||||
*
|
||||
*
|
||||
* Any errors occurring for individual write request objects will be reported
|
||||
* in the response.
|
||||
*
|
||||
*
|
||||
* @param data
|
||||
* a {@link TimelineEntity} object
|
||||
* a {@link TimelineAggregationTrack} enum value
|
||||
|
|
|
@ -16,8 +16,8 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
|
||||
public class TestAppLevelTimelineAggregator {
|
||||
public class TestAppLevelTimelineCollector {
|
||||
}
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
@ -32,14 +32,15 @@ import org.apache.hadoop.util.ExitUtil;
|
|||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
|
||||
import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestPerNodeTimelineAggregatorsAuxService {
|
||||
public class TestPerNodeTimelineCollectorsAuxService {
|
||||
private ApplicationAttemptId appAttemptId;
|
||||
|
||||
public TestPerNodeTimelineAggregatorsAuxService() {
|
||||
public TestPerNodeTimelineCollectorsAuxService() {
|
||||
ApplicationId appId =
|
||||
ApplicationId.newInstance(System.currentTimeMillis(), 1);
|
||||
appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
|
||||
|
@ -47,7 +48,8 @@ public class TestPerNodeTimelineAggregatorsAuxService {
|
|||
|
||||
@Test
|
||||
public void testAddApplication() throws Exception {
|
||||
PerNodeTimelineAggregatorsAuxService auxService = createAggregatorAndAddApplication();
|
||||
PerNodeTimelineCollectorsAuxService auxService =
|
||||
createCollectorAndAddApplication();
|
||||
// auxService should have a single app
|
||||
assertTrue(auxService.hasApplication(
|
||||
appAttemptId.getApplicationId().toString()));
|
||||
|
@ -56,7 +58,7 @@ public class TestPerNodeTimelineAggregatorsAuxService {
|
|||
|
||||
@Test
|
||||
public void testAddApplicationNonAMContainer() throws Exception {
|
||||
PerNodeTimelineAggregatorsAuxService auxService = createAggregator();
|
||||
PerNodeTimelineCollectorsAuxService auxService = createCollector();
|
||||
|
||||
ContainerId containerId = getContainerId(2L); // not an AM
|
||||
ContainerInitializationContext context =
|
||||
|
@ -70,7 +72,8 @@ public class TestPerNodeTimelineAggregatorsAuxService {
|
|||
|
||||
@Test
|
||||
public void testRemoveApplication() throws Exception {
|
||||
PerNodeTimelineAggregatorsAuxService auxService = createAggregatorAndAddApplication();
|
||||
PerNodeTimelineCollectorsAuxService auxService =
|
||||
createCollectorAndAddApplication();
|
||||
// auxService should have a single app
|
||||
String appIdStr = appAttemptId.getApplicationId().toString();
|
||||
assertTrue(auxService.hasApplication(appIdStr));
|
||||
|
@ -87,7 +90,8 @@ public class TestPerNodeTimelineAggregatorsAuxService {
|
|||
|
||||
@Test
|
||||
public void testRemoveApplicationNonAMContainer() throws Exception {
|
||||
PerNodeTimelineAggregatorsAuxService auxService = createAggregatorAndAddApplication();
|
||||
PerNodeTimelineCollectorsAuxService auxService =
|
||||
createCollectorAndAddApplication();
|
||||
// auxService should have a single app
|
||||
String appIdStr = appAttemptId.getApplicationId().toString();
|
||||
assertTrue(auxService.hasApplication(appIdStr));
|
||||
|
@ -105,10 +109,11 @@ public class TestPerNodeTimelineAggregatorsAuxService {
|
|||
@Test(timeout = 60000)
|
||||
public void testLaunch() throws Exception {
|
||||
ExitUtil.disableSystemExit();
|
||||
PerNodeTimelineAggregatorsAuxService auxService = null;
|
||||
PerNodeTimelineCollectorsAuxService auxService = null;
|
||||
try {
|
||||
auxService =
|
||||
PerNodeTimelineAggregatorsAuxService.launchServer(new String[0]);
|
||||
PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
|
||||
createCollectorManager());
|
||||
} catch (ExitUtil.ExitException e) {
|
||||
assertEquals(0, e.status);
|
||||
ExitUtil.resetFirstExitException();
|
||||
|
@ -120,8 +125,9 @@ public class TestPerNodeTimelineAggregatorsAuxService {
|
|||
}
|
||||
}
|
||||
|
||||
private PerNodeTimelineAggregatorsAuxService createAggregatorAndAddApplication() {
|
||||
PerNodeTimelineAggregatorsAuxService auxService = createAggregator();
|
||||
private PerNodeTimelineCollectorsAuxService
|
||||
createCollectorAndAddApplication() {
|
||||
PerNodeTimelineCollectorsAuxService auxService = createCollector();
|
||||
// create an AM container
|
||||
ContainerId containerId = getAMContainerId();
|
||||
ContainerInitializationContext context =
|
||||
|
@ -131,15 +137,23 @@ public class TestPerNodeTimelineAggregatorsAuxService {
|
|||
return auxService;
|
||||
}
|
||||
|
||||
private PerNodeTimelineAggregatorsAuxService createAggregator() {
|
||||
TimelineAggregatorsCollection
|
||||
aggregatorsCollection = spy(new TimelineAggregatorsCollection());
|
||||
doReturn(new Configuration()).when(aggregatorsCollection).getConfig();
|
||||
PerNodeTimelineAggregatorsAuxService auxService =
|
||||
spy(new PerNodeTimelineAggregatorsAuxService(aggregatorsCollection));
|
||||
private PerNodeTimelineCollectorsAuxService createCollector() {
|
||||
TimelineCollectorManager collectorManager = createCollectorManager();
|
||||
PerNodeTimelineCollectorsAuxService auxService =
|
||||
spy(new PerNodeTimelineCollectorsAuxService(collectorManager));
|
||||
return auxService;
|
||||
}
|
||||
|
||||
private TimelineCollectorManager createCollectorManager() {
|
||||
TimelineCollectorManager collectorManager =
|
||||
spy(new TimelineCollectorManager());
|
||||
doReturn(new Configuration()).when(collectorManager).getConfig();
|
||||
CollectorNodemanagerProtocol nmCollectorService =
|
||||
mock(CollectorNodemanagerProtocol.class);
|
||||
doReturn(nmCollectorService).when(collectorManager).getNMCollectorService();
|
||||
return collectorManager;
|
||||
}
|
||||
|
||||
private ContainerId getAMContainerId() {
|
||||
return getContainerId(1L);
|
||||
}
|
|
@ -16,11 +16,12 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
||||
package org.apache.hadoop.yarn.server.timelineservice.collector;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.spy;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -30,18 +31,16 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
import com.sun.jersey.core.impl.provider.entity.XMLJAXBElementProvider;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestTimelineAggregatorsCollection {
|
||||
public class TestTimelineCollectorManager {
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testMultithreadedAdd() throws Exception {
|
||||
final TimelineAggregatorsCollection aggregatorCollection =
|
||||
spy(new TimelineAggregatorsCollection());
|
||||
doReturn(new Configuration()).when(aggregatorCollection).getConfig();
|
||||
final TimelineCollectorManager collectorManager = createCollectorManager();
|
||||
|
||||
final int NUM_APPS = 5;
|
||||
List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
|
||||
|
@ -49,9 +48,9 @@ public class TestTimelineAggregatorsCollection {
|
|||
final ApplicationId appId = ApplicationId.newInstance(0L, i);
|
||||
Callable<Boolean> task = new Callable<Boolean>() {
|
||||
public Boolean call() {
|
||||
AppLevelTimelineAggregator aggregator =
|
||||
new AppLevelTimelineAggregator(appId.toString());
|
||||
return (aggregatorCollection.putIfAbsent(appId, aggregator) == aggregator);
|
||||
AppLevelTimelineCollector collector =
|
||||
new AppLevelTimelineCollector(appId.toString());
|
||||
return (collectorManager.putIfAbsent(appId, collector) == collector);
|
||||
}
|
||||
};
|
||||
tasks.add(task);
|
||||
|
@ -67,15 +66,14 @@ public class TestTimelineAggregatorsCollection {
|
|||
}
|
||||
// check the keys
|
||||
for (int i = 0; i < NUM_APPS; i++) {
|
||||
assertTrue(aggregatorCollection.containsKey(String.valueOf(i)));
|
||||
final ApplicationId appId = ApplicationId.newInstance(0L, i);
|
||||
assertTrue(collectorManager.containsKey(appId.toString()));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultithreadedAddAndRemove() throws Exception {
|
||||
final TimelineAggregatorsCollection aggregatorCollection =
|
||||
spy(new TimelineAggregatorsCollection());
|
||||
doReturn(new Configuration()).when(aggregatorCollection).getConfig();
|
||||
final TimelineCollectorManager collectorManager = createCollectorManager();
|
||||
|
||||
final int NUM_APPS = 5;
|
||||
List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
|
||||
|
@ -83,11 +81,11 @@ public class TestTimelineAggregatorsCollection {
|
|||
final ApplicationId appId = ApplicationId.newInstance(0L, i);
|
||||
Callable<Boolean> task = new Callable<Boolean>() {
|
||||
public Boolean call() {
|
||||
AppLevelTimelineAggregator aggregator =
|
||||
new AppLevelTimelineAggregator(appId.toString());
|
||||
AppLevelTimelineCollector collector =
|
||||
new AppLevelTimelineCollector(appId.toString());
|
||||
boolean successPut =
|
||||
(aggregatorCollection.putIfAbsent(appId, aggregator) == aggregator);
|
||||
return successPut && aggregatorCollection.remove(appId.toString());
|
||||
(collectorManager.putIfAbsent(appId, collector) == collector);
|
||||
return successPut && collectorManager.remove(appId.toString());
|
||||
}
|
||||
};
|
||||
tasks.add(task);
|
||||
|
@ -103,7 +101,18 @@ public class TestTimelineAggregatorsCollection {
|
|||
}
|
||||
// check the keys
|
||||
for (int i = 0; i < NUM_APPS; i++) {
|
||||
assertFalse(aggregatorCollection.containsKey(String.valueOf(i)));
|
||||
final ApplicationId appId = ApplicationId.newInstance(0L, i);
|
||||
assertFalse(collectorManager.containsKey(appId.toString()));
|
||||
}
|
||||
}
|
||||
|
||||
private TimelineCollectorManager createCollectorManager() {
|
||||
final TimelineCollectorManager collectorManager =
|
||||
spy(new TimelineCollectorManager());
|
||||
doReturn(new Configuration()).when(collectorManager).getConfig();
|
||||
CollectorNodemanagerProtocol nmCollectorService =
|
||||
mock(CollectorNodemanagerProtocol.class);
|
||||
doReturn(nmCollectorService).when(collectorManager).getNMCollectorService();
|
||||
return collectorManager;
|
||||
}
|
||||
}
|
|
@ -27,12 +27,12 @@ import java.nio.file.Path;
|
|||
import java.nio.file.Paths;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
||||
import org.junit.Test;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
|
||||
public class TestFileSystemTimelineWriterImpl {
|
||||
|
||||
|
@ -42,9 +42,6 @@ public class TestFileSystemTimelineWriterImpl {
|
|||
*/
|
||||
@Test
|
||||
public void testWriteEntityToFile() throws Exception {
|
||||
String name = "unit_test_BaseAggregator_testWriteEntityToFile_"
|
||||
+ Long.toString(System.currentTimeMillis());
|
||||
|
||||
TimelineEntities te = new TimelineEntities();
|
||||
TimelineEntity entity = new TimelineEntity();
|
||||
String id = "hello";
|
||||
|
@ -55,25 +52,27 @@ public class TestFileSystemTimelineWriterImpl {
|
|||
entity.setModifiedTime(1425016502000L);
|
||||
te.addEntity(entity);
|
||||
|
||||
FileSystemTimelineWriterImpl fsi = new FileSystemTimelineWriterImpl();
|
||||
fsi.serviceInit(new Configuration());
|
||||
fsi.write(te);
|
||||
try (FileSystemTimelineWriterImpl fsi =
|
||||
new FileSystemTimelineWriterImpl()) {
|
||||
fsi.serviceInit(new Configuration());
|
||||
fsi.write(te);
|
||||
|
||||
String fileName = fsi.getOutputRoot() + "/" + type + "/" + id
|
||||
+ FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
|
||||
Path path = Paths.get(fileName);
|
||||
File f = new File(fileName);
|
||||
assertTrue(f.exists() && !f.isDirectory());
|
||||
List<String> data = Files.readAllLines(path, StandardCharsets.UTF_8);
|
||||
// ensure there's only one entity + 1 new line
|
||||
assertTrue(data.size() == 2);
|
||||
String d = data.get(0);
|
||||
// confirm the contents same as what was written
|
||||
assertEquals(d, TimelineUtils.dumpTimelineRecordtoJSON(entity));
|
||||
String fileName = fsi.getOutputRoot() + "/" + type + "/" + id
|
||||
+ FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
|
||||
Path path = Paths.get(fileName);
|
||||
File f = new File(fileName);
|
||||
assertTrue(f.exists() && !f.isDirectory());
|
||||
List<String> data = Files.readAllLines(path, StandardCharsets.UTF_8);
|
||||
// ensure there's only one entity + 1 new line
|
||||
assertTrue(data.size() == 2);
|
||||
String d = data.get(0);
|
||||
// confirm the contents same as what was written
|
||||
assertEquals(d, TimelineUtils.dumpTimelineRecordtoJSON(entity));
|
||||
|
||||
// delete the directory
|
||||
File outputDir = new File(fsi.getOutputRoot());
|
||||
FileUtils.deleteDirectory(outputDir);
|
||||
assertTrue(!(f.exists()));
|
||||
// delete the directory
|
||||
File outputDir = new File(fsi.getOutputRoot());
|
||||
FileUtils.deleteDirectory(outputDir);
|
||||
assertTrue(!(f.exists()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue