diff --git hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java index 3fa42fe..c878b60 100644 --- hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java +++ hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java @@ -670,9 +670,16 @@ public TaskAttemptImpl(TaskId taskId, int i, this.resourceCapability.setVirtualCores( getCpuRequired(conf, taskId.getTaskType())); + float bwLimit = getBandwithLimit(conf, taskId.getTaskType()); + this.resourceCapability.setHdfsBandwidthEnforcement(bwLimit); + + LOG.info("ResourceCapability: memory " + resourceCapability.getMemory() + + ", vcore: " + resourceCapability.getVirtualCores() + + ", bwLimit: " + resourceCapability.getHdfsBandwidthEnforcement()); + this.dataLocalHosts = resolveHosts(dataLocalHosts); RackResolver.init(conf); - this.dataLocalRacks = new HashSet(); + this.dataLocalRacks = new HashSet(); for (String host : this.dataLocalHosts) { this.dataLocalRacks.add(RackResolver.resolve(host).getNetworkLocation()); } @@ -704,6 +711,19 @@ private int getCpuRequired(Configuration conf, TaskType taskType) { return vcores; } + private float getBandwithLimit(Configuration conf, TaskType taskType) { + float limit = 0; + if (taskType == TaskType.MAP) { + limit = conf.getFloat(MRJobConfig.MAP_HDFS_BW_LIMIT_MBPS, 0); + LOG.debug("MAP Hdfs bandwidth limit: " + limit); + } else if (taskType == TaskType.REDUCE) { + limit = conf.getFloat(MRJobConfig.REDUCE_HDFS_BW_LIMIT_MBPS, 0); + LOG.debug("REDUCE Hdfs bandwidth limit: " + limit); + } + + return limit; + } + /** * Create a {@link LocalResource} record with all the given parameters. */ diff --git hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java index 1aeee2c..5916e6b 100644 --- hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java +++ hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java @@ -372,6 +372,8 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) { reqEvent.getCapability().setMemory(mapResourceRequest.getMemory()); reqEvent.getCapability().setVirtualCores( mapResourceRequest.getVirtualCores()); + reqEvent.getCapability().setHdfsBandwidthEnforcement( + mapResourceRequest.getHdfsBandwidthEnforcement()); scheduledRequests.addMap(reqEvent);//maps are immediately scheduled } else { if (reduceResourceRequest.equals(Resources.none())) { @@ -399,7 +401,9 @@ protected synchronized void handleEvent(ContainerAllocatorEvent event) { // set the resources reqEvent.getCapability().setMemory(reduceResourceRequest.getMemory()); reqEvent.getCapability().setVirtualCores( - reduceResourceRequest.getVirtualCores()); + reduceResourceRequest.getVirtualCores()); + reqEvent.getCapability().setHdfsBandwidthEnforcement( + reduceResourceRequest.getHdfsBandwidthEnforcement()); if (reqEvent.getEarlierAttemptFailed()) { //add to the front of queue for fail fast pendingReduces.addFirst(new ContainerRequest(reqEvent, diff --git hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java index 59b887d..2410457 100644 --- hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java +++ hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java @@ -933,4 +933,8 @@ public static final int DEFAULT_MR_ENCRYPTED_INTERMEDIATE_DATA_BUFFER_KB = 128; + String MAP_HDFS_BW_LIMIT_MBPS = "mapreduce.map.bandwithlimit.mbps"; + + String REDUCE_HDFS_BW_LIMIT_MBPS = "mapreduce.reduce.bandwithlimit.mbps"; + } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java index 88b57f1..29c1355 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java @@ -40,7 +40,12 @@ * very coarse. A complementary axis for CPU requests that represents processing * power will likely be added in the future to enable finer-grained resource * configuration.

- * + * + *

In order to manage HDFS bandwidth requests, the new field has been added + * to the Resource model (hdfs_bandwidth_enforcement). + * The unit for bandwidth is megabits. The 0 value of bandwidth indicates + * that no limitation is set on bandwidth usage

+ * *

Typically, applications request Resource of suitable * capability to run their component tasks.

* @@ -105,6 +110,30 @@ public static Resource newInstance(int memory, int vCores) { @Evolving public abstract void setVirtualCores(int vCores); + /** + * Get HDFS bandwidth enforcement of the resource. + * + * HDFS bandwidth enforcement defines the limit bandwidth which + * the container can use to communicate with local HDFS datanode. + * + * @return HDFS bandwidth enforcement of the resource + */ + @Public + @Evolving + public abstract float getHdfsBandwidthEnforcement(); + + /** + * Set HDFS bandwidth enforcement of the resource. + * + * HDFS bandwidth enforcement defines the limit bandwidth which + * the container can use to communicate with local HDFS DataNode. + * + * @param hdfsBandwidth HDFS bandwidth enforcement of the resource + */ + @Public + @Evolving + public abstract void setHdfsBandwidthEnforcement(float hdfsBandwidth); + @Override public int hashCode() { final int prime = 263167; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 6b660f7..e157162 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -1894,7 +1894,7 @@ private static void addDeprecatedKeys() { public static final String YARN_HTTP_POLICY_KEY = YARN_PREFIX + "http.policy"; public static final String YARN_HTTP_POLICY_DEFAULT = HttpConfig.Policy.HTTP_ONLY .name(); - + /** * Node-labels configurations */ @@ -1934,6 +1934,46 @@ public static boolean isDistributedNodeLabelConfiguration(Configuration conf) { NODELABEL_CONFIGURATION_TYPE, DEFAULT_NODELABEL_CONFIGURATION_TYPE)); } + /** + * HDFS Bandwidth Enforcement configurations. + */ + public static final String NM_HDFS_BE_ENABLE = + NM_PREFIX + "hdfs-bandwidth-enforcement.enable"; + public static final boolean DEFAULT_NM_HDFS_BE_ENABLE = false; + public static final String NM_HDFS_BE_CLIENT_MODE = + NM_PREFIX + "hdfs-bandwidth-enforcement.client_mode"; + public static final String NM_HDFS_BE_REPORT_SERVICE_CLASS = + NM_PREFIX + "hdfs-bandwidth-enforcement.container_report_service.class"; + public static final String NM_HDFS_BE_ENABLE_SUBMITTER_ONLY = + NM_PREFIX + "hdfs-bandwidth-enforcement.enable-submitter-only"; + public static final String NM_HDFS_BE_CONTAINER_PLUGINS = + NM_PREFIX + "hdfs-bandwidth-enforcement.container-plugins"; + public static final String NM_HDFS_PORT = + NM_PREFIX + "hdfs-bandwidth-enforcement.port"; + public static final int DEFAULT_NM_HDFS_PORT = 50010; + public static final String NM_HDFS_BE_DEVICES = + NM_PREFIX + "hdfs-bandwidth-enforcement.devices"; + public static final String DEFAULT_NM_HDFS_BE_DEVICES = "lo"; + public static final String NM_HDFS_BE_CHECK_TC_INTERVAL_MS = + NM_PREFIX + "hdfs-bandwidth-enforcement.check-tc-config-interval"; + public static final String NM_HDFS_BE_MIN_RATE = + NM_PREFIX + "hdfs-bandwidth-enforcement.minimum-rate-mbps"; + public static final String NM_HDFS_BE_MAX_RATE = + NM_PREFIX + "hdfs-bandwidth-enforcement.maximum-rate-mbps"; + public static final String NM_HDFS_BE_EXECUTE_SUDO_SS = + NM_PREFIX + "hdfs-bandwidth-enforcement.execute-sudo-ss"; + public static final String NM_HDFS_BE_SUBMITTER_CLASS = + NM_PREFIX + "hdfs-bandwidth-enforcement.submitter.class"; + public static final String NM_HDFS_BE_COLLECTOR_CLASS = + NM_PREFIX + "hdfs-bandwidth-enforcement.collector.class"; + public static final long DEFAULT_NM_HDFS_BE_CHECK_TC_INTERVAL_MS = 1000; + public static final String NM_HDFS_BE_CONFIG_ROOT_DIR = + NM_PREFIX + "hdfs-bandwidth-enforcement.config-root.path"; + public static final String NM_HDFS_BE_CONTAINER_DATA_LOCAL_PATH = + NM_PREFIX + "hdfs-bandwidth-enforcement.container-local-data.path"; + public static final String NM_HDFS_BE_ZK_SERVER_ADDRESS = + NM_PREFIX + "hdfs-bandwidth-enforcement.zk-server.address"; + public YarnConfiguration() { super(); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto index f801409..e694e8c 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto @@ -56,6 +56,7 @@ message ContainerIdProto { message ResourceProto { optional int32 memory = 1; optional int32 virtual_cores = 2; + optional float hdfs_bandwidth_enforcement = 3; } message ResourceOptionProto { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java index a28c6ed..8fd0d34 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java @@ -80,6 +80,18 @@ public void setVirtualCores(int vCores) { } @Override + public float getHdfsBandwidthEnforcement() { + ResourceProtoOrBuilder p = viaProto ? proto : builder; + return (p.getHdfsBandwidthEnforcement()); + } + + @Override + public void setHdfsBandwidthEnforcement(float bandwidth) { + maybeInitBuilder(); + builder.setHdfsBandwidthEnforcement(bandwidth); + } + + @Override public int compareTo(Resource other) { int diff = this.getMemory() - other.getMemory(); if (diff == 0) { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java index 2ee95ce..18aa6ef 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java @@ -153,17 +153,15 @@ public Resource divideAndCeil(Resource numerator, int denominator) { public Resource normalize(Resource r, Resource minimumResource, Resource maximumResource, Resource stepFactor) { int normalizedMemory = Math.min( - roundUp( - Math.max(r.getMemory(), minimumResource.getMemory()), - stepFactor.getMemory()), - maximumResource.getMemory()); + roundUp( + Math.max(r.getMemory(), minimumResource.getMemory()), + stepFactor.getMemory()), maximumResource.getMemory()); int normalizedCores = Math.min( - roundUp( - Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()), - stepFactor.getVirtualCores()), - maximumResource.getVirtualCores()); - return Resources.createResource(normalizedMemory, - normalizedCores); + roundUp( + Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()), + stepFactor.getVirtualCores()), maximumResource.getVirtualCores()); + return Resources.createResource(normalizedMemory, normalizedCores, + r.getHdfsBandwidthEnforcement()); } @Override diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java index 472811a..87f0c31 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java @@ -51,6 +51,16 @@ public void setVirtualCores(int cores) { } @Override + public float getHdfsBandwidthEnforcement() { + return 0; + } + + @Override + public void setHdfsBandwidthEnforcement(float bandwidth) { + throw new RuntimeException("NONE cannot be modified!"); + } + + @Override public int compareTo(Resource o) { int diff = 0 - o.getMemory(); if (diff == 0) { @@ -84,6 +94,16 @@ public void setVirtualCores(int cores) { } @Override + public float getHdfsBandwidthEnforcement() { + return Float.MAX_VALUE; + } + + @Override + public void setHdfsBandwidthEnforcement(float bandwidth) { + throw new RuntimeException("UNBOUNDED cannot be modified!"); + } + + @Override public int compareTo(Resource o) { int diff = Integer.MAX_VALUE - o.getMemory(); if (diff == 0) { @@ -105,6 +125,15 @@ public static Resource createResource(int memory, int cores) { return resource; } + public static Resource createResource(int memory, int cores, + float hdfsBandwidth) { + Resource resource = Records.newRecord(Resource.class); + resource.setMemory(memory); + resource.setVirtualCores(cores); + resource.setHdfsBandwidthEnforcement(hdfsBandwidth); + return resource; + } + public static Resource none() { return NONE; } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml index babed8e..2e92de8 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml @@ -164,6 +164,17 @@ org.fusesource.leveldbjni leveldbjni-all + + org.apache.hadoop + hadoop-hdfs + test + + + org.apache.hadoop + hadoop-hdfs + test-jar + test + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java index 3721b0e..262688e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java @@ -79,7 +79,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider; -import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.TrafficController; import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils; import org.apache.hadoop.yarn.util.YarnVersionInfo; @@ -139,6 +139,9 @@ private final NodeLabelsProvider nodeLabelsProvider; private final boolean hasNodeLabelsProvider; + // HDFS Bandwidth Enforcement + private TrafficController tcController; + public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher, NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) { this(context, dispatcher, healthChecker, metrics, null); @@ -202,6 +205,9 @@ protected void serviceInit(Configuration conf) throws Exception { LOG.debug(YARN_NODEMANAGER_DURATION_TO_TRACK_STOPPED_CONTAINERS + " :" + durationToTrackStoppedContainers); } + + tcController = new TrafficController(conf, true); + super.serviceInit(conf); LOG.info("Initialized nodemanager with :" + " physical-memory=" + memoryMb + " virtual-memory=" + virtualMemoryMb + @@ -227,6 +233,12 @@ protected void serviceStart() throws Exception { registerWithRM(); super.serviceStart(); startStatusUpdater(); + String hostId = null; + if (this.nodeId != null) { + hostId = this.nodeId.getHost(); + } + tcController.initialize(hostId); + tcController.start(); } catch (Exception e) { String errorMessage = "Unexpected error starting NodeStatusUpdater"; LOG.error(errorMessage, e); @@ -245,6 +257,7 @@ protected void serviceStop() throws Exception { // Interrupt the updater. this.isStopped = true; stopRMProxy(); + tcController.stop(); super.serviceStop(); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java index 289d6d0..2903b2e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java @@ -505,10 +505,11 @@ private void sendContainerMonitorStartEvent() { int cpuVcores = getResource().getVirtualCores(); long localizationDuration = containerLaunchStartTime - containerLocalizationStartTime; + float bandwidthEnforcement = getResource().getHdfsBandwidthEnforcement(); dispatcher.getEventHandler().handle( new ContainerStartMonitoringEvent(containerId, vmemBytes, pmemBytes, cpuVcores, launchDuration, - localizationDuration)); + localizationDuration, bandwidthEnforcement)); } private void addDiagnostics(String... diags) { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java index c09bebf..774553c 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java @@ -27,16 +27,18 @@ private final int cpuVcores; private final long launchDuration; private final long localizationDuration; + private final float bandwidthEnforcement; public ContainerStartMonitoringEvent(ContainerId containerId, long vmemLimit, long pmemLimit, int cpuVcores, long launchDuration, - long localizationDuration) { + long localizationDuration, float bandwidthEnforcement) { super(containerId, ContainersMonitorEventType.START_MONITORING_CONTAINER); this.vmemLimit = vmemLimit; this.pmemLimit = pmemLimit; this.cpuVcores = cpuVcores; this.launchDuration = launchDuration; this.localizationDuration = localizationDuration; + this.bandwidthEnforcement = bandwidthEnforcement; } public long getVmemLimit() { @@ -58,4 +60,8 @@ public long getLaunchDuration() { public long getLocalizationDuration() { return this.localizationDuration; } + + public float getBandwidthEnforcement() { + return bandwidthEnforcement; + } } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index b5f154d..0a72f1a 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -38,9 +38,11 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionHandler; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.YarnContainerService; import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils; -import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; +import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import com.google.common.base.Preconditions; @@ -78,6 +80,10 @@ private static final long UNKNOWN_MEMORY_LIMIT = -1L; private int nodeCpuPercentageForYARN; + // HDFS Bandwidth Enforcement + private ProcBasedConnectionHandler connectionHandler; + private YarnContainerService yarnContainerRegister; + public ContainersMonitorImpl(ContainerExecutor exec, AsyncDispatcher dispatcher, Context context) { super("containers-monitor"); @@ -171,6 +177,16 @@ protected void serviceInit(Configuration conf) throws Exception { 1) + "). Thrashing might happen."); } } + + yarnContainerRegister = + YarnContainerService.loadYarnContainerService(conf); + if(yarnContainerRegister != null && + !yarnContainerRegister.isClientMode()) { + connectionHandler = new ProcBasedConnectionHandler(conf); + LOG.info("Register container service plugin: " + yarnContainerRegister); + connectionHandler.registerContainerService(yarnContainerRegister); + } + super.serviceInit(conf); } @@ -185,7 +201,8 @@ private boolean isEnabled() { + this.getClass().getName() + " is disabled."); return false; } - if (!(isPmemCheckEnabled() || isVmemCheckEnabled())) { + if (!(isPmemCheckEnabled() || isVmemCheckEnabled() + || yarnContainerRegister != null)) { LOG.info("Neither virutal-memory nor physical-memory monitoring is " + "needed. Not running the monitor-thread"); return false; @@ -198,6 +215,20 @@ private boolean isEnabled() { protected void serviceStart() throws Exception { if (this.isEnabled()) { this.monitoringThread.start(); + String hostName = context.getNodeId().getHost(); + try { + if(yarnContainerRegister != null && + yarnContainerRegister.isClientMode()) { + yarnContainerRegister.initialize(hostName); + yarnContainerRegister.start(); + } + if(connectionHandler != null) { + connectionHandler.initialize(hostName); + connectionHandler.start(); + } + } catch(Exception e) { + LOG.error("Error occured when starting connectionHandler", e); + } } super.serviceStart(); } @@ -205,6 +236,17 @@ protected void serviceStart() throws Exception { @Override protected void serviceStop() throws Exception { if (this.isEnabled()) { + try { + if(connectionHandler != null) { + connectionHandler.stop(); + } + if(yarnContainerRegister != null && + yarnContainerRegister.isClientMode()) { + yarnContainerRegister.stop(); + } + } catch(Exception e) { + LOG.error("Cannot stop connectionHandler: " + e.getMessage(), e); + } this.monitoringThread.interrupt(); try { this.monitoringThread.join(); @@ -212,6 +254,7 @@ protected void serviceStop() throws Exception { ; } } + super.serviceStop(); } @@ -408,6 +451,9 @@ public void run() { // or if the container's pid is removed from ContainerExecutor LOG.debug("Tracking ProcessTree " + pId + " for the first time"); + if(yarnContainerRegister != null){ + yarnContainerRegister.registerPid(containerId, pId); + } ResourceCalculatorProcessTree pt = ResourceCalculatorProcessTree.getResourceCalculatorProcessTree(pId, processTreeClass, conf); @@ -640,11 +686,18 @@ public void handle(ContainersMonitorEvent monitoringEvent) { startEvent.getVmemLimit(), startEvent.getPmemLimit(), startEvent.getCpuVcores()); this.containersToBeAdded.put(containerId, processTreeInfo); + if(yarnContainerRegister != null){ + yarnContainerRegister.addMonitoringContainer(containerId, + startEvent.getBandwidthEnforcement()); + } } break; case STOP_MONITORING_CONTAINER: synchronized (this.containersToBeRemoved) { this.containersToBeRemoved.add(containerId); + if(yarnContainerRegister != null){ + yarnContainerRegister.stopMonitoringContainer(containerId); + } } break; default: diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractAsyncTCDataCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractAsyncTCDataCollector.java new file mode 100644 index 0000000..3ea65e4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractAsyncTCDataCollector.java @@ -0,0 +1,146 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; + +/** + * Collect TC settings from back-end storages. In this case TrafficControl will + * not start the monitoring thread to collect the update from the back-end + * storage. + * + */ +public abstract class AbstractAsyncTCDataCollector extends + AbstractTCDataCollector { + + private static final Log LOG = LogFactory + .getLog(AbstractAsyncTCDataCollector.class); + + private AbstractTrafficController tcController; + + private ExecutorService executor = Executors.newSingleThreadExecutor(); + + private UpdateRequest updateRequest = null; + + public AbstractAsyncTCDataCollector(Configuration conf) { + super(conf); + } + + /** + * Register TrafficController as callback. + * + * @param callBack + */ + public final void registerCallback(AbstractTrafficController callBack) { + this.tcController = callBack; + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * AbstractTCDataCollector#stop() + */ + @Override + public void stop() { + try { + executor.shutdownNow(); + } catch (Exception e) { + ; + } + super.stop(); + } + + /** + * This function should be called when update is found. Only one update can be + * in progress, and one can be queued. + */ + public final synchronized void notifyUpdate() { + if (!isReady()) { + return; + } + + LOG.debug("notifyUpdate()"); + synchronized (AbstractAsyncTCDataCollector.this) { + if (updateRequest == null) { + updateRequest = new UpdateRequest(new Runnable() { + @Override + public void run() { + try { + LOG.debug("Calling update()..."); + tcController.update(); + } catch (RuntimeException e) { + LOG.error("notifyUpdate error: " + + e.getMessage(), e); + } + } + }); + } + + if (!updateRequest.inProgress) { + updateRequest.submit(); + } else { + updateRequest.queued = true; + } + } + } + + private final class UpdateRequest implements Runnable { + private final Runnable item; + private boolean queued; + private boolean inProgress; + + private UpdateRequest(Runnable item) { + this.item = item; + this.queued = true; + this.inProgress = false; + } + + // Caller must have a lock + private void submit() { + LOG.debug("Submit update"); + executor.submit(this); + } + + public void run() { + try { + synchronized (AbstractAsyncTCDataCollector.this) { + assert queued; + queued = false; + inProgress = true; + } + item.run(); + } finally { + synchronized (AbstractAsyncTCDataCollector.this) { + inProgress = false; + if (queued) { + // another submit for this job is requested while we + // were doing the update. Do it again. + submit(); + } + } + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractConnectionHandler.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractConnectionHandler.java new file mode 100644 index 0000000..d0bb017 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractConnectionHandler.java @@ -0,0 +1,53 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +/** + * Collects HDFS connections on host and submit to the persistent storage. + * + */ +public abstract class AbstractConnectionHandler implements ConnectionMonitor, + TrafficControlDataSubmitter { + + static final Log LOG = LogFactory.getLog(AbstractConnectionHandler.class); + + public synchronized void process() { + + Map> partialConnections = collect(); + if (partialConnections.isEmpty()) { + return; + } + + long start = System.currentTimeMillis(); + for (Entry> entry : partialConnections + .entrySet()) { + submit(entry.getKey(), entry.getValue()); + } + LOG.debug("submit time elapsed: " + + (System.currentTimeMillis() - start) + " ms"); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractService.java new file mode 100644 index 0000000..16df1de --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractService.java @@ -0,0 +1,44 @@ +/** + * 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.nodemanager.trafficcontrol; + +/** + * The interface for managing services. + */ +public interface AbstractService { + + /** + * Post initialization before starting the service. + * + * @param localNodeId + * the name of the local host. + */ + void initialize(String localNodeId); + + /** + * Start the service. + */ + void start(); + + /** + * Stop the service and release the resources. + */ + void stop(); + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTCDataCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTCDataCollector.java new file mode 100644 index 0000000..51170cf --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTCDataCollector.java @@ -0,0 +1,48 @@ +/** + * 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.nodemanager.trafficcontrol; + +import org.apache.hadoop.conf.Configuration; + +/** + * Collect TC settings from back-end storages. + * + */ +public abstract class AbstractTCDataCollector implements + TrafficControlDataCollector, AbstractService { + + public AbstractTCDataCollector(Configuration conf) { + return; + } + + public void start() { + return; + } + + public void stop() { + return; + } + + /** + * Indicate whether the plugin is ready for collecting data. + * + * @return true if the plugin is ready + */ + public abstract boolean isReady(); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTCDataSubmitter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTCDataSubmitter.java new file mode 100644 index 0000000..c4079af --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTCDataSubmitter.java @@ -0,0 +1,41 @@ +/** + * 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.nodemanager.trafficcontrol; + +import org.apache.hadoop.conf.Configuration; + +/** + * Base class for submitting TC settings to the storage. + * + */ +public abstract class AbstractTCDataSubmitter implements + TrafficControlDataSubmitter, AbstractService { + + public AbstractTCDataSubmitter(Configuration conf) { + return; + } + + public void start() { + return; + } + + public void stop() { + return; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTrafficController.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTrafficController.java new file mode 100644 index 0000000..a521f4a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractTrafficController.java @@ -0,0 +1,53 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TrafficControlEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +/** + * Retry {@link TrafficControlEvent} from persistence the execute appropriate + * traffic control setting. + * + */ +public abstract class AbstractTrafficController implements + TrafficControlExecutor, TrafficControlDataCollector, LTCEventBuilder { + + /** + * Collect the updated hosts and their connections, then synchronize with + * stored database and the current list of connections. Finally it will + * generate the list of TC events to be executed by devices. It is called + * periodically in case of sync update or through the notifyUpdate() of async + * update. + */ + public synchronized void update() { + Map> partialConnections = + collectData(); + if (partialConnections != null + && !partialConnections.isEmpty()) { + List events = buildTCEvents(partialConnections); + if (!events.isEmpty()) { + execute(events); + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractYarnContainerReportService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractYarnContainerReportService.java new file mode 100644 index 0000000..02457d0 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/AbstractYarnContainerReportService.java @@ -0,0 +1,33 @@ +/** + * 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.nodemanager.trafficcontrol; + +import org.apache.hadoop.conf.Configuration; + +/** + * Abstract class for report plugins of Yarn containers in case + * HdfsTrafficControl runs as a standalone application. + * + */ +public abstract class AbstractYarnContainerReportService implements + ContainerService, AbstractService { + public AbstractYarnContainerReportService(Configuration conf) { + return; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/ConnectionMonitor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/ConnectionMonitor.java new file mode 100644 index 0000000..1ee8a11 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/ConnectionMonitor.java @@ -0,0 +1,38 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +/** + * Collect tc events from the storage. + * + */ +public interface ConnectionMonitor { + + /** + * Collect connections data to all DataNodes, where changes are detected. + * + * @return map of DataNode <-> connections grouped by containers + */ + Map> collect(); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/ContainerService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/ContainerService.java new file mode 100644 index 0000000..d1cce82 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/ContainerService.java @@ -0,0 +1,54 @@ +/** + * 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.nodemanager.trafficcontrol; + +/** + * Interface for managing monitoring containers. + * + */ +public interface ContainerService { + + /** + * Add new container to monitor. The rate must be specified and cannot be + * zero. + * + * @param containerId + * the unique id of the container + * @param rateInMbps + * limit rate in mbps + * @return true if the container is added to monitor, otherwise false. + */ + boolean addMonitoringContainer(String containerId, float rateInMbps); + + /** + * Register the id of the first process spawned by the container. + * + * @param containerId + * @param pid + */ + boolean registerPid(String containerId, int pid); + + /** + * Stop monitoring the container with given id. + * + * @param containerId + */ + void stopMonitoringContainer(String containerId); + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/HdfsTrafficControl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/HdfsTrafficControl.java new file mode 100644 index 0000000..767caa0 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/HdfsTrafficControl.java @@ -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.nodemanager.trafficcontrol; + +import java.io.IOException; +import java.net.InetAddress; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionHandler; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.TrafficController; + +/** + * Main class for executing as a standalon application. + * + */ +public final class HdfsTrafficControl implements AbstractService { + + static final Log LOG = LogFactory.getLog(HdfsTrafficControl.class); + + private ProcBasedConnectionHandler connectionHandler; + private TrafficController trafficController; + + private Configuration conf; + + private HdfsTrafficControl(boolean runExecutorOnly) throws IOException { + if (!FactoryHelper.getInstance().isTCApplicable()) { + return; + } + + conf = new Configuration(true); + conf.addResource("yarn-site.xml"); + + trafficController = new TrafficController(conf); + // controllerThread = new TrafficControllerTherad(); + + if (!runExecutorOnly) { + connectionHandler = new ProcBasedConnectionHandler(conf); + // monitoringThread = new ConnectionHandlerTherad(); + } else { + LOG.warn("Only start Traffic Control Executor!"); + } + } + + public static void main(String[] args) throws IOException { + boolean onlyExecutor = false; + InetAddress iAddress = InetAddress.getLocalHost(); + // To get the Canonical host name + String canonicalHostName = iAddress.getCanonicalHostName(); + LOG.info("Hostname is: " + + canonicalHostName); + + if (args != null) { + for (String arg : args) { + if (arg.equalsIgnoreCase("-onlyExecutor") + || arg.equalsIgnoreCase("onlyExecutor")) { + onlyExecutor = true; + break; + } + } + } + + LOG.info("only start TCExecutor: " + + onlyExecutor); + + HdfsTrafficControl app = new HdfsTrafficControl(onlyExecutor); + app.initialize(canonicalHostName); + app.start(); + } + + private void addShutdownHook() { + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { + + shutdown(); + try { + Thread.sleep(1000L); + } catch (InterruptedException e) { + ; + } + } + }); + } + + @Override + public void initialize(String localNodeId) { + if (connectionHandler != null) { + connectionHandler.initialize(localNodeId); + } + + if (trafficController != null) { + trafficController.initialize(localNodeId); + } + + addShutdownHook(); + + } + + @Override + public void start() { + if (connectionHandler != null) { + connectionHandler.start(); + } + + if (trafficController != null) { + trafficController.start(); + } + } + + @Override + public void stop() { + if (connectionHandler != null) { + connectionHandler.stop(); + } + + if (trafficController != null) { + trafficController.stop(); + } + } + + public void shutdown() { + stop(); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/LTCEventBuilder.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/LTCEventBuilder.java new file mode 100644 index 0000000..762fd7a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/LTCEventBuilder.java @@ -0,0 +1,41 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TrafficControlEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +/** + * Interface for constructing LTC settings. + * + */ +public interface LTCEventBuilder { + /** + * Construct TC events based on the input connections data. + * + * @param connections + * list of connections of the updated remote hosts + * @return list of TC events to be executed + */ + List buildTCEvents( + Map> connections); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlDataCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlDataCollector.java new file mode 100644 index 0000000..e19a33b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlDataCollector.java @@ -0,0 +1,38 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +/** + * Collect TC settings from storages. + * + */ +public interface TrafficControlDataCollector { + + /** + * Collect all connections from updated remote hosts only. + * + * @return a map of connections grouped by containers + */ + Map> collectData(); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlDataSubmitter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlDataSubmitter.java new file mode 100644 index 0000000..024f969 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlDataSubmitter.java @@ -0,0 +1,40 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.List; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +/** + * Interface for submitting containers based grouping connections. + * + */ +public interface TrafficControlDataSubmitter { + + /** + * Save connections data of a given host to the storage. + * + * @param remoteHost + * the updated remote DataNode + * @param connections + * list of connections to the given DataNode + */ + boolean submit(String remoteHost, List connections); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlExecutor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlExecutor.java new file mode 100644 index 0000000..5bbfd11 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TrafficControlExecutor.java @@ -0,0 +1,38 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.List; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TrafficControlEvent; + +/** + * Traffic control actions executor: ADD/DEL class, filters. + * + */ +public interface TrafficControlExecutor { + + /** + * Apply TC settings based on the input events. + * + * @param events + * list of TC events to be executed + */ + void execute(List events); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCClassEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCClassEvent.java new file mode 100644 index 0000000..c0362a7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCClassEvent.java @@ -0,0 +1,41 @@ +/** + * 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.nodemanager.trafficcontrol.event; + +/** + * Data class for an LTC class request. + * + */ +public class TCClassEvent extends TrafficControlEvent { + + private String rate; + + public TCClassEvent(String containerId, TCEventType action, String rate, + boolean isLoopback) { + super(containerId, action, isLoopback); + this.rate = rate; + } + + /** + * @return the rate + */ + public String getRate() { + return rate; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCEnumHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCEnumHelper.java new file mode 100644 index 0000000..49f5626 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCEnumHelper.java @@ -0,0 +1,80 @@ +/** + * 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.nodemanager.trafficcontrol.event; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * + * Helper class for custom enum type. + * + * @param + * custom type. + */ +public class TCEnumHelper { + + private List names; + + private Map nameMap; + + public TCEnumHelper(Type[] elements) { + + names = new ArrayList(); + nameMap = new HashMap(); + + if (elements == null) { + return; + } + for (Type element : elements) { + names.add(element.toString()); + nameMap.put(element.toString().toUpperCase(), element); + } + } + + public List getNames() { + return (names); + } + + public void putElement(String name, Type element) { + nameMap.put(name, element); + } + + public Type getElement(String name) { + if (name == null) { + return null; + } + return nameMap.get(name.toUpperCase()); + } + + public Type getElement(String name, Type defaultValue) { + if (name == null) { + return defaultValue; + } + + Type value = nameMap.get(name.toUpperCase()); + if (value == null) { + return defaultValue; + } + return value; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCEventType.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCEventType.java new file mode 100644 index 0000000..7ffb09f --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCEventType.java @@ -0,0 +1,57 @@ +/** + * 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.nodemanager.trafficcontrol.event; + +import java.util.List; + +/** + * + * Define the type of LTC settings. + * + */ +public enum TCEventType { + + ADD_FILTER("ADD_FILTER"), DEL_FILTER("DEL_FILTER"), ADD_CLASS("ADD_CLASS"), + DEL_CLASS("DEL_CLASS"), CHANGE_CLASS("CHANGE_CLASS"), UNDEF("UNDEF"); + + private static final TCEnumHelper DB = + new TCEnumHelper(TCEventType.values()); + private String name; + + TCEventType(String name) { + this.name = name; + } + + public String toString() { + return name; + } + + public static List getNames() { + return DB.getNames(); + } + + public static List getAllowedNames() { + return getNames(); + } + + public static TCEventType getTCEventType(String name) { + return DB.getElement(name.toUpperCase()); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCFilterEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCFilterEvent.java new file mode 100644 index 0000000..35b404d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TCFilterEvent.java @@ -0,0 +1,71 @@ +/** + * 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.nodemanager.trafficcontrol.event; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; + +/** + * Data class for an LTC filter request. + * + */ +public class TCFilterEvent extends TrafficControlEvent { + private int localPort; + private String remoteHost; + private int remotePort; + + public TCFilterEvent(String containerId, TCEventType eventType, + Connection connection) { + super(containerId, eventType, connection.isLoopback()); + this.localPort = connection.getSrcPort(); + this.remoteHost = connection.getDstHost(); + this.remotePort = connection.getDstPort(); + } + + /** + * @return the localPort + */ + public int getLocalPort() { + return localPort; + } + + /** + * @return the remoteHost + */ + public String getRemoteHost() { + return remoteHost; + } + + /** + * @return the remotePort + */ + public int getRemotePort() { + return remotePort; + } + + public static TCFilterEvent createDelFilterEvent(String containerId, + Connection connection) { + return new TCFilterEvent(containerId, TCEventType.DEL_FILTER, connection); + } + + public static TCFilterEvent createAddFilterEvent(String containerId, + Connection connection) { + return new TCFilterEvent(containerId, TCEventType.ADD_FILTER, connection); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TrafficControlEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TrafficControlEvent.java new file mode 100644 index 0000000..7d8543e --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/event/TrafficControlEvent.java @@ -0,0 +1,67 @@ +/** + * 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.nodemanager.trafficcontrol.event; + +/** + * TrafficControlEvent contains information of how LINUX traffic control + * subsystem should be set. + *

+ * TrafficControlEvent can be collected automatically from system information or + * defined by enforcement handling entity. + *

+ * TrafficControlEvent will be handled by TrafficControlDevice to set LTC on + * concerned host + */ +public abstract class TrafficControlEvent { + + private String containerId; + private TCEventType eventType; + private boolean isLoopback; + + public TrafficControlEvent(String containerId, TCEventType eventType, + boolean isLoopback) { + this.containerId = containerId; + this.eventType = eventType; + this.isLoopback = isLoopback; + } + + /** + * @return the containerId + */ + public String getContainerId() { + return containerId; + } + + /** + * @return the eventType + */ + public TCEventType getEventType() { + return eventType; + } + + /** + * Only important for setting TC. + * + * @return the isLoopback + */ + public boolean isLoopback() { + return isLoopback; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/AbstractContainerService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/AbstractContainerService.java new file mode 100644 index 0000000..1cc0084 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/AbstractContainerService.java @@ -0,0 +1,177 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractService; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.ContainerService; + +/** + * + * Abstract class for providing container service through plugins. These plugins + * must extend this class. + * + */ +public abstract class AbstractContainerService implements ContainerService, + AbstractService { + + private static final Log LOG = LogFactory + .getLog(AbstractContainerService.class); + + public static final Pattern CONTAINER_NAME_FORMAT = Pattern + .compile("^[a-zA-Z][\\w-]+[a-zA-Z0-9]$"); + + private ContainerService callBack; + + private Map containerMap = new HashMap(); + + public AbstractContainerService(Configuration conf) { + + } + + private boolean validateContainerId(String containerId) { + if (containerId == null) { + return false; + } + + if (containerId.length() > 100) { + return false; + } + + return CONTAINER_NAME_FORMAT.matcher(containerId).matches(); + } + + /** + * It should be called from the connection handler only. + * + * @param callBack + */ + final void setCallBack(ContainerService callBack) { + this.callBack = callBack; + } + + final ContainerService getCallBack() { + return callBack; + } + + protected abstract float normalize(float rate); + + @Override + public final boolean addMonitoringContainer(String containerId, + float rateInMbps) { + if (!validateContainerId(containerId)) { + LOG.warn(String.format("containerId is invalid: %s", containerId)); + return false; + } + + float rate = normalize(rateInMbps); + if (rate <= 0) { + LOG.debug(String.format("The limit rate is 0! containerId: %s", + containerId)); + return false; + } + + boolean added = false; + synchronized (containerMap) { + if (callBack != null + && !containerMap.containsKey(containerId)) { + added = callBack.addMonitoringContainer(containerId, rate); + if (added) { + containerMap.put(containerId, rate); + } + } + } + + return added; + } + + @Override + public final boolean registerPid(String containerId, int pid) { + if (!validateContainerId(containerId)) { + LOG.warn("containerId is invalid: " + + containerId); + return false; + } + + if (pid <= 1) { + LOG.warn("PID is invalid: " + + pid + " for " + containerId); + return false; + } + + synchronized (containerMap) { + if (callBack != null + && containerMap.containsKey(containerId)) { + return callBack.registerPid(containerId, pid); + } + } + + return false; + } + + @Override + public final void stopMonitoringContainer(String containerId) { + if (!validateContainerId(containerId)) { + LOG.warn("containerId is invalid: " + + containerId); + return; + } + synchronized (containerMap) { + if (callBack != null + && containerMap.containsKey(containerId)) { + callBack.stopMonitoringContainer(containerId); + } + } + } + + /** + * It should be called from the connection handler only. + * + * @param containerId + */ + final void deleteCachedData(String containerId) { + synchronized (containerMap) { + containerMap.remove(containerId); + } + } + + /** + * Return the unmodifiable view of the current containerMap. + **/ + public Map getUnModifableMapOfContainers() { + return Collections.unmodifiableMap(containerMap); + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#toString() + */ + @Override + public String toString() { + return this.getClass().getName(); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FactoryHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FactoryHelper.java new file mode 100644 index 0000000..09c3524 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FactoryHelper.java @@ -0,0 +1,174 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.lang.reflect.Constructor; +import java.nio.charset.StandardCharsets; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Shell; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataSubmitter; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.executor.TCCommand; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.executor.TrafficControlDeviceExecutor; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; + +/** + * Helper class. + * + */ +public class FactoryHelper { + + static final Log LOG = LogFactory.getLog(FactoryHelper.class); + private static FactoryHelper instance = null; + + private String procFsRoot = "/proc"; + + private FactoryHelper() { + return; + } + + public static FactoryHelper getInstance() { + if (instance == null) { + instance = new FactoryHelper(); + } + + return instance; + } + + /** + * @return the procFsRoot + */ + public String getProcFsRoot() { + return procFsRoot; + } + + /** + * @param procFsRoot + * the procFsRoot to set + */ + public void setProcFsRoot(String procFsRoot) { + this.procFsRoot = procFsRoot; + } + + public AbstractTCDataSubmitter getTCDataSubmitter(Configuration pConf) { + + Class submitterClazz = + pConf.getClass(YarnConfiguration.NM_HDFS_BE_SUBMITTER_CLASS, + HdfsTCDataSubmitter.class, AbstractTCDataSubmitter.class); + + if (submitterClazz != null) { + try { + Constructor c = + submitterClazz.getConstructor(Configuration.class); + AbstractTCDataSubmitter pSubmitter = c.newInstance(pConf); + LOG.info("Loaded plugin " + + pSubmitter.getClass().getName()); + return pSubmitter; + } catch (Exception e) { + LOG.error("Cannot initialize submitter: " + + e.getMessage(), e); + } + } + return null; + } + + public TrafficControlDeviceExecutor getDevice(String deviceName, + int monitoringPort, FactoryHelper helper) { + return new TrafficControlDeviceExecutor(deviceName, monitoringPort, helper); + } + + public AbstractTCDataCollector getTCDataCollector(Configuration pConf) { + Class collectorClazz = + pConf.getClass(YarnConfiguration.NM_HDFS_BE_COLLECTOR_CLASS, + HdfsTCDataCollector.class, AbstractTCDataCollector.class); + + if (collectorClazz != null) { + try { + Constructor c = + collectorClazz.getConstructor(Configuration.class); + AbstractTCDataCollector pCollector = c.newInstance(pConf); + LOG.info("Loaded plugin " + + pCollector.getClass().getName()); + return pCollector; + } catch (Exception e) { + LOG.error("Cannot initialize collector: " + + e.getMessage(), e); + } + } + + return null; + } + + public ConnectionCollector getConnectionCollector(int monitoringPort, + boolean isDataNode) { + return new ConnectionCollector(monitoringPort, isDataNode); + + } + + public int exec(TCCommand tcCmd) { + Process p; + try { + p = Runtime.getRuntime().exec(tcCmd.toString()); + p.waitFor(); + return p.exitValue(); + } catch (IOException e) { + LOG.error("Error occurred when executing command: " + + tcCmd, e); + } catch (InterruptedException e) { + ; + } + return -1; + } + + public BufferedReader getBufferedReader(TCCommand tcCmd) throws IOException { + Process p = Runtime.getRuntime().exec(tcCmd.toString()); + BufferedReader input = + new BufferedReader(new InputStreamReader(p.getInputStream(), + StandardCharsets.UTF_8)); + + return input; + } + + /** + * Checks if Linux Traffic Control can be applied on this system. + * + * @return true if the system is Linux. False otherwise. + */ + public boolean isTCApplicable() { + try { + if (!Shell.LINUX) { + LOG.info("HDFS bandwidth enforcement currently is supported only on " + + "Linux."); + return false; + } + } catch (SecurityException se) { + LOG.warn("Failed to get Operating System name. " + + se.getMessage(), se); + return false; + } + return true; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FileBasedContainerService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FileBasedContainerService.java new file mode 100644 index 0000000..d9e04a1 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FileBasedContainerService.java @@ -0,0 +1,315 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.DirectoryStream; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.nio.file.WatchEvent.Kind; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.HashMap; +import java.util.List; +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.yarn.conf.YarnConfiguration; + +import com.google.common.base.Strings; + +/** + * File based plug-in of the container service. It monitoring a given local + * folder for new containers. + * + */ +public class FileBasedContainerService extends AbstractContainerService { + + static final Log LOG = LogFactory.getLog(FileBasedContainerService.class); + + private static final String WATCHER_DIR_NAME = "monitoring_containers"; + + private Path monitoringDir; + private MonitorThread monitorThread; + + private boolean initialized = false; + private volatile boolean done = false; + private boolean isEnabled = false; + + private Map modifiedTimes = new HashMap(); + + public FileBasedContainerService(Configuration conf) { + super(conf); + monitoringDir = getStorageDir(conf); + } + + public static Path getStorageDir(Configuration conf) { + Path storageDir = null; + final String localPath = + conf.get(YarnConfiguration.NM_HDFS_BE_CONTAINER_DATA_LOCAL_PATH, null); + if (!Strings.isNullOrEmpty(localPath)) { + storageDir = Paths.get(localPath.trim()); + } else { + storageDir = + Paths.get(System.getProperty("java.io.tmpdir"), WATCHER_DIR_NAME); + } + + return storageDir; + } + + @Override + protected float normalize(float rate) { + return rate; + } + + @Override + public void initialize(String localNodeId) { + try { + LOG.info("FileBasedContainerService: Folder to check containers is " + + monitoringDir); + if (!Files.exists(monitoringDir)) { + Files.createDirectory(monitoringDir); + } + + monitorThread = new MonitorThread(); + isEnabled = true; + + } catch (IOException e) { + LOG.error("Cannot create watch service: " + + e.getMessage(), e); + } + } + + private void apply(Path containerFile, String clsId) throws IOException { + if (!Files.isRegularFile(containerFile)) { + return; + } + + long lastModifiedTime = 0; + if (modifiedTimes.containsKey(clsId)) { + lastModifiedTime = modifiedTimes.get(clsId); + } + + if (Files.getLastModifiedTime(containerFile).toMillis() + - lastModifiedTime < 1000) { + return; + } + + modifiedTimes.put(clsId, Files.getLastModifiedTime(containerFile) + .toMillis()); + + float rate = 0; + int pid = 0; + try { + List lines = + Files.readAllLines(containerFile, StandardCharsets.UTF_8); + if (lines != null) { + for (String line : lines) { + String trimmedLine = line.replaceAll("\\s", ""); + if (trimmedLine.startsWith("rate=")) { + rate = getRate(trimmedLine.substring(5)); + } + + if (trimmedLine.startsWith("pid=")) { + pid = getPid(trimmedLine.substring(4)); + } + } + } + } catch (IOException e) { + LOG.error(e); + } + + if (rate > 0 + && pid >= 0) { + + if (initialized + || pid > 1) { + addMonitoringContainer(clsId, rate); + } + if (pid > 1) { + registerPid(clsId, pid); + } + } + } + + private int getPid(String pid) { + try { + return Integer.parseInt(pid); + } catch ( + NumberFormatException | NullPointerException e) { + return -1; + } + } + + private float getRate(String rate) { + try { + return Float.parseFloat(rate); + } catch ( + NumberFormatException | NullPointerException e) { + return -1; + } + } + + @Override + public void start() { + if (!isEnabled) { + return; + } + LOG.info("Starting monitorThread"); + monitorThread.start(); + } + + @Override + public void stop() { + LOG.info("Stopping monitorThread"); + done = true; + } + + private void init() { + LOG.info("Walking the directory: " + + monitoringDir); + try (DirectoryStream directoryStream = + Files.newDirectoryStream(monitoringDir)) { + for (Path containerFile : directoryStream) { + String clsId = containerFile.getFileName().toString(); + Path fullChildPath = monitoringDir.resolve(containerFile); + if (!Files.isDirectory(fullChildPath)) { + LOG.info("Found existing file: " + + fullChildPath); + apply(fullChildPath, clsId); + } + } + } catch (IOException e) { + LOG.warn("Error occurred when reading " + + monitoringDir, e); + } + + initialized = true; + } + + @SuppressWarnings("unchecked") + private void process() throws IOException { + // we register three events. i.e. whenever a file is created, deleted or + // modified the watcher gets informed + try (WatchService ws = FileSystems.getDefault().newWatchService();) { + WatchKey key = + monitoringDir.register(ws, StandardWatchEventKinds.ENTRY_CREATE, + StandardWatchEventKinds.ENTRY_DELETE, + StandardWatchEventKinds.ENTRY_MODIFY); + + // we can poll for events in an infinite loop + while (!done) { + try { + // the take method waits till watch service receives a + // notification + key = ws.take(); + } catch (InterruptedException e) { + ; + } + + // once a key is obtained, we poll for events on that key + modifiedTimes.clear(); + List> keys = key.pollEvents(); + + for (WatchEvent watchEvent : keys) { + WatchEvent ev = (WatchEvent) watchEvent; + Path filename = ev.context(); + Path fullChildPath = monitoringDir.resolve(filename); + if (Files.isDirectory(fullChildPath)) { + continue; + } + + final String containerId = filename.toString(); + Kind eventType = watchEvent.kind(); + if (eventType == StandardWatchEventKinds.OVERFLOW) { + continue; + } + + try { + if (eventType == StandardWatchEventKinds.ENTRY_CREATE) { + LOG.info("New container detected: " + + containerId); + apply(fullChildPath, containerId); + } else if (eventType == StandardWatchEventKinds.ENTRY_MODIFY) { + LOG.info("Changes detected for container " + + containerId); + apply(fullChildPath, containerId); + } else if (eventType == StandardWatchEventKinds.ENTRY_DELETE) { + LOG.info("Container is stopped: " + + containerId); + stopMonitoringContainer(containerId); + } + } catch (Exception e) { + continue; + } + } + + // we need to reset the key so the further key events may be + // polled + if (!key.reset()) { + break; + } + } + } + // close the watcher service + // watchService.close(); + } + + class MonitorThread extends Thread { + + /* + * (non-Javadoc) + * + * @see java.lang.Thread#run() + */ + @Override + public void run() { + + init(); + while (!done) { + try { + process(); + } catch (IOException e) { + LOG.error("Error occured: " + + e.getMessage(), e); + } + + if (!done + && !Files.exists(monitoringDir)) { + LOG.warn(monitoringDir + + " is deleted. Recreate it and continue."); + try { + Files.createDirectory(monitoringDir); + } catch (IOException e1) { + LOG.error("Error occured: " + + e1.getMessage(), e1); + break; + } + } + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FileBasedYarnContainerReporter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FileBasedYarnContainerReporter.java new file mode 100644 index 0000000..6a88a5c --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/FileBasedYarnContainerReporter.java @@ -0,0 +1,128 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractYarnContainerReportService; + +/** + * In the client mode, Yarn can use this plug-in to cooperate with + * {@link FileBasedContainerService}. + * + */ +public class FileBasedYarnContainerReporter extends + AbstractYarnContainerReportService { + + static final Log LOG = LogFactory + .getLog(FileBasedYarnContainerReporter.class); + + private Path storagePath; + + public FileBasedYarnContainerReporter(Configuration conf) { + super(conf); + storagePath = FileBasedContainerService.getStorageDir(conf); + } + + @Override + public boolean addMonitoringContainer(String containerId, float rateInMbps) { + String content = String.format("rate=%.2f%n", rateInMbps); + Path containerFile = storagePath.resolve(containerId); + try { + Files.write(containerFile, content.getBytes(StandardCharsets.UTF_8)); + } catch (IOException e) { + LOG.error("Cannot create file " + + containerFile + " with line: " + content, e); + return false; + } + return true; + } + + @Override + public boolean registerPid(String containerId, int pid) { + Path containerFile = storagePath.resolve(containerId); + if (Files.exists(containerFile)) { + try { + List lines = + Files.readAllLines(containerFile, StandardCharsets.UTF_8); + if (lines != null) { + for (String line : lines) { + if (line.startsWith("rate=")) { + String content = String.format("%s%npid=%d%n", line, pid); + Files.write(containerFile, + content.getBytes(StandardCharsets.UTF_8)); + return true; + } + } + } + + LOG.warn("We shoudn't reach here: No rate is specifed for container: " + + containerId + " in " + containerFile); + + } catch (IOException e) { + LOG.error("Error occured when processing file " + + containerFile, e); + } + } else { + LOG.error("File " + + containerFile + " is not existed. Nothing to do!"); + } + return false; + } + + @Override + public void stopMonitoringContainer(String containerId) { + Path containerFile = storagePath.resolve(containerId); + try { + Files.deleteIfExists(containerFile); + } catch (IOException e) { + LOG.error("Cannot delete file " + + containerFile, e); + } + + } + + @Override + public void initialize(String localNodeId) { + if (!Files.exists(storagePath)) { + try { + Files.createDirectory(storagePath); + } catch (IOException e) { + ; + } + } + } + + @Override + public void start() { + return; + } + + @Override + public void stop() { + return; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/HdfsTCDataCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/HdfsTCDataCollector.java new file mode 100644 index 0000000..517cc93 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/HdfsTCDataCollector.java @@ -0,0 +1,155 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import net.minidev.json.parser.JSONParser; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +import com.google.common.base.Strings; + +/** + * Collect TC settings from HDFS storages. + * + */ +public class HdfsTCDataCollector extends AbstractTCDataCollector { + + static final Log LOG = LogFactory.getLog(HdfsTCDataCollector.class); + + private HashMap hostCheckedTimestamp; + private FileSystem fs; + private Path rootHdfsDir; + private String configRootPath; + + private JSONParser parser; + private Map cachePaths = new HashMap(); + + public HdfsTCDataCollector(Configuration conf) throws IOException { + super(conf); + + hostCheckedTimestamp = new HashMap(); + this.fs = FileSystem.get(conf); + this.configRootPath = + conf.get(YarnConfiguration.NM_HDFS_BE_CONFIG_ROOT_DIR, + HdfsTCDataSubmitter.TC_HDFS_DIR); + parser = new JSONParser(JSONParser.MODE_PERMISSIVE); + } + + public void initialize(String nodeId) { + rootHdfsDir = new Path(configRootPath, nodeId); + try { + if (!fs.exists(rootHdfsDir)) { + fs.mkdirs(rootHdfsDir); + } + } catch (IOException e) { + LOG.warn("Cannot check folder: " + + rootHdfsDir, e); + } + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * AbstractTCDataCollector#isReady() + */ + @Override + public boolean isReady() { + return true; + } + + @Override + public synchronized Map> collectData() { + Map> hostConnections = new HashMap<>(); + Set deletedHosts = + new HashSet(hostCheckedTimestamp.keySet()); + try { + FileStatus[] connectedHosts = fs.listStatus(rootHdfsDir); + for (FileStatus nmHostStatus : connectedHosts) { + if (!nmHostStatus.isFile()) { + continue; + } + + final Path tcConfigFile = nmHostStatus.getPath(); + String nmHostName = + StorageUtil.getHostName(tcConfigFile.getName(), cachePaths); + + if (Strings.isNullOrEmpty(nmHostName)) { + continue; + } + + boolean needUpdate = false; + long lastModifiedTime = nmHostStatus.getModificationTime(); + deletedHosts.remove(nmHostName); + + if (!hostCheckedTimestamp.containsKey(nmHostName)) { + needUpdate = true; + } else { + Long preTime = hostCheckedTimestamp.get(nmHostName); + needUpdate = lastModifiedTime > preTime; + } + + if (needUpdate) { + if (LOG.isDebugEnabled()) { + LOG.debug("Handling connections from " + + nmHostName); + } + try { + FSDataInputStream is = fs.open(tcConfigFile); + hostConnections.put(nmHostName, StorageUtil.decodeHostConnections( + parser, new InputStreamReader(is, StandardCharsets.UTF_8))); + + hostCheckedTimestamp.put(nmHostName, lastModifiedTime); + } catch (Exception e) { + LOG.warn("Can not read " + + nmHostStatus.getPath() + ". Caused by: " + e, e); + } + } + } + } catch (IOException e) { + LOG.error(e); + } + // Deleted hosts should be also passed to the handler (same as hosts with + // the empty content) + for (String nmHost : deletedHosts) { + hostConnections.put(nmHost, new ArrayList()); + } + + return hostConnections; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/HdfsTCDataSubmitter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/HdfsTCDataSubmitter.java new file mode 100644 index 0000000..ee645c4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/HdfsTCDataSubmitter.java @@ -0,0 +1,115 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataSubmitter; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +/** + * This class submit TC settings to HDFS storage. + * + */ +public class HdfsTCDataSubmitter extends AbstractTCDataSubmitter { + + static final Log LOG = LogFactory.getLog(HdfsTCDataSubmitter.class); + + public static final String TC_HDFS_DIR = "hdfs-bandwidth-enforcement"; + public static final String TC_CONFIG_FILE = "tconfig"; + public static final String TC_CONFIG_TIMESTAMP = "timestamp"; + + private FileSystem fs; + private String configRootPath = ""; + + private String tcConfigName; + + private Set connectedDatanodes = new HashSet(); + + public HdfsTCDataSubmitter(Configuration conf) throws IOException { + super(conf); + this.fs = FileSystem.get(conf); + + this.configRootPath = + conf.get(YarnConfiguration.NM_HDFS_BE_CONFIG_ROOT_DIR, TC_HDFS_DIR); + + // Check and create a folder /user/{user_name}/hdfs-bandwidth-enforcement + Path tcRootHdfsDir = new Path(configRootPath); + if (!fs.exists(tcRootHdfsDir)) { + if (!fs.mkdirs(tcRootHdfsDir)) { + throw new IOException("Cannot create directory " + + tcRootHdfsDir); + } + } + } + + public void initialize(String localNodeId) { + if (localNodeId != null) { + tcConfigName = String.format("__%s__", localNodeId); + } + } + + public synchronized boolean submit(String remoteHost, + List hostConnections) { + + try { + String remoteHostName = NetUtils.getHostNameOfIP(remoteHost); + if (remoteHostName == null) { + LOG.warn("Cannot get hostname for " + + remoteHost); + remoteHostName = remoteHost; + } + + // Only submit to HDFS if the dedicated DN already to collect it! + Path remoteConfigPath = new Path(configRootPath, remoteHostName); + if (!fs.exists(remoteConfigPath)) { + LOG.warn(remoteConfigPath + + " is not existed. Skip this round!"); + return false; + } + + // config file + Path tcConfig = new Path(remoteConfigPath, tcConfigName); + LOG.info("Submitting to " + + tcConfig); + FSDataOutputStream os = fs.create(tcConfig, true); + os.writeBytes(StorageUtil.encodeHostConnections(hostConnections)); + os.flush(); + os.close(); + + connectedDatanodes.add(remoteHost); + return true; + } catch (IOException e) { + LOG.error(e); + } + + return false; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ProcBasedConnectionHandler.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ProcBasedConnectionHandler.java new file mode 100644 index 0000000..3fcd98a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ProcBasedConnectionHandler.java @@ -0,0 +1,553 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractConnectionHandler; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractService; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataSubmitter; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.ContainerService; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.CommonProcessTree; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.NMContainerConnections; +import org.jboss.netty.util.internal.ConcurrentHashMap; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; + +/** + * Collect HDFS connections of containers and submit them to the back-end + * storage. + * + */ +public class ProcBasedConnectionHandler extends AbstractConnectionHandler + implements AbstractService { + static final Log LOG = LogFactory.getLog(ProcBasedConnectionHandler.class); + + static final Long WAIT_FOR_FISRT_PID = 10 * 60 * 1000L; + + private HashMap containersMap; + private List finishedContainers; + private Lock lock = new ReentrantLock(); + private FactoryHelper helper; + + private boolean isEnabled = true; + + private int monitoringPort; + private ConnectionMonitorThread monitorThread; + + private volatile boolean done = false; + + private List containerServiceList; + private ContainerRegister containerRegister; + + private Map registerTime = new ConcurrentHashMap<>(); + + /* Use for stored unsuccessful updated connections */ + + private BiMap containerPidMap = HashBiMap.create(); + + private AbstractTCDataSubmitter submitter; + private ProcBasedConnectionMonitor connectionMonitor; + + private Configuration conf; + + public ProcBasedConnectionHandler(Configuration conf) throws IOException { + this(conf, FactoryHelper.getInstance()); + } + + @VisibleForTesting + public ProcBasedConnectionHandler(Configuration conf, FactoryHelper helper) + throws IOException { + this.helper = helper; + this.conf = conf; + if (!this.helper.isTCApplicable()) { + isEnabled = false; + return; + } + + submitter = this.helper.getTCDataSubmitter(conf); + if (submitter == null) { + LOG.warn("Submitter is null. Disable HDFS bandwidth enforcement!"); + isEnabled = false; + return; + } + + this.monitoringPort = + conf.getInt(YarnConfiguration.NM_HDFS_PORT, + YarnConfiguration.DEFAULT_NM_HDFS_PORT); + + containersMap = new HashMap(); + finishedContainers = new ArrayList(); + containerServiceList = new ArrayList(); + + monitorThread = new ConnectionMonitorThread(conf); + containerRegister = new ContainerRegister(); + + registerContainerServices(conf); + } + + /** + * @return the lock + */ + Lock getLock() { + return lock; + } + + private void registerContainerServices(Configuration conf) { + String containerServicesConf = + conf.get(YarnConfiguration.NM_HDFS_BE_CONTAINER_PLUGINS, ""); + + String[] filters = containerServicesConf.replaceAll("\\s", "").split(","); + for (String filterName : filters) { + if (Strings.isNullOrEmpty(filterName)) { + continue; + } + LOG.info("Registering container detector plugin: " + + filterName); + try { + Class clazz = conf.getClassByName(filterName); + if (clazz != null + && AbstractContainerService.class.isAssignableFrom(clazz)) { + Class filterClazz = + clazz.asSubclass(AbstractContainerService.class); + Constructor constructor = + filterClazz.getConstructor(Configuration.class); + registerContainerService(constructor.newInstance(conf)); + } + } catch (Exception e) { + LOG.error("Cannot initialize container detector service: " + + filterName, e); + } + } + + } + + public boolean isEnabled() { + return isEnabled; + } + + public void + registerContainerService(AbstractContainerService containerService) { + if (isEnabled) { + containerService.setCallBack(containerRegister); + containerServiceList.add(containerService); + } else { + LOG.warn("Cannot register " + + containerService + " as HDFS bandwidth enforcement is disable"); + } + } + + /** + * @return the monitoringPort + */ + public int getMonitoringPort() { + return monitoringPort; + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * TrafficControlDataSubmitter#start() + */ + @Override + public void start() { + if (isEnabled) { + LOG.info("Starting HDFS ConnectionHandler"); + submitter.start(); + monitorThread.start(); + for (AbstractContainerService containerService : containerServiceList) { + try { + containerService.start(); + } catch (Exception e) { + LOG.error("Error occurred when starting ContainerService" + + containerService, e); + } + } + } + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * TrafficControlDataSubmitter#stop() + */ + @Override + public void stop() { + if (!isEnabled) { + return; + } + + LOG.info("HDFS ConnectionHandler stopping..."); + done = true; + + for (AbstractContainerService containerService : containerServiceList) { + try { + containerService.stop(); + } catch (Exception e) { + LOG.error("Error occurred when stopping ContainerService" + + containerService, e); + } + } + + try { + submitter.stop(); + } catch (Exception e) { + LOG.warn("Errors occured when stopping submitter: " + + e.getMessage(), e); + } + + LOG.info("HDFS ConnectionHandler stopped"); + } + + public void initialize(final String localNodeId) { + if (!isEnabled) { + return; + } + + if (containerServiceList.isEmpty()) { + LOG.fatal("At least one valid containerService " + + " must be specify in property: " + + YarnConfiguration.NM_HDFS_BE_CONTAINER_PLUGINS); + isEnabled = false; + return; + } + + String nodeId = localNodeId; + if (nodeId == null + || nodeId.isEmpty()) { + try { + InetAddress iAddress = InetAddress.getLocalHost(); + // To get the Canonical host name + nodeId = iAddress.getCanonicalHostName(); + LOG.info("Change hostname to: " + + nodeId); + } catch (UnknownHostException e) { + LOG.warn("Inactive HDFS ConnectionHandler component: " + + e.getMessage(), e); + isEnabled = false; + return; + } + } + + LOG.info("Initializing HDFS ConnectionHandler for " + + nodeId); + + if (connectionMonitor == null) { + setConnectionMonitor(new ProcBasedConnectionMonitor(conf)); + } + + for (AbstractContainerService containerService : containerServiceList) { + try { + containerService.initialize(nodeId); + } catch (Exception e) { + LOG.error("Error occurred when initializing ContainerService" + + containerService, e); + } + } + + submitter.initialize(nodeId); + } + + private boolean addTcClassView(String containerId, float rateInMbps) { + if (rateInMbps > 0) { + String rateStr = rateInMbps + + "mbps"; + return addTcClassView(new NMContainerConnections(containerId, rateStr, + this.monitoringPort)); + } else { + LOG.info("No limit set for HDFS bandwidth of container " + + containerId); + } + + return false; + } + + public boolean addTcClassView(NMContainerConnections view) { + try { + lock.lock(); + final String containerId = view.getContainerId(); + if (!containersMap.containsKey(containerId)) { + LOG.info("Add new container to track: " + + containerId + ", rate is " + view.getRate()); + containersMap.put(containerId, view); + registerTime.put(containerId, System.currentTimeMillis()); + + return true; + } + + return false; + + } finally { + lock.unlock(); + } + } + + public boolean registerContainer(String containerId, String pid) { + try { + lock.lock(); + if (!containerPidMap.containsKey(containerId)) { + Path pidFile = + Paths.get(FactoryHelper.getInstance().getProcFsRoot(), pid); + if (Files.exists(pidFile)) { + LOG.info(String.format("Register container: %s, pid: %s", + containerId, pid)); + containerPidMap.put(containerId, pid); + NMContainerConnections con = containersMap.get(containerId); + if (con != null) { + con.initialized(); + CommonProcessTree pTree = connectionMonitor.getProcfsTree(pid); + if (pTree != null) { + con.updateProcessTree(pTree); + } + } + return true; + } else { + LOG.warn(String.format( + "Invalid pid %s is reported for container: %s." + + " Stop monitoring the container!", pid, containerId)); + removeTcClassView(containerId); + } + } + } finally { + lock.unlock(); + } + return false; + } + + @Override + public synchronized Map> collect() { + return connectionMonitor.collect(); + } + + @Override + public boolean submit(String host, List data) { + try { + lock.lock(); + if (done) { + return false; + } + boolean ok = submitter.submit(host, data); + if (ok) { + connectionMonitor.getConnectionsData().remove(host); + } + + return true; + } finally { + lock.unlock(); + } + } + + public void removeTcClassView(String containerId) { + try { + lock.lock(); + LOG.info("Stop monitoring container: " + + containerId); + NMContainerConnections tcClass = containersMap.get(containerId); + if (tcClass != null) { + tcClass.stopTrackContainer(); + finishedContainers.add(tcClass); + containersMap.remove(containerId); + } + containerPidMap.remove(containerId); + registerTime.remove(containerId); + + for (AbstractContainerService service : containerServiceList) { + service.deleteCachedData(containerId); + } + } finally { + lock.unlock(); + } + } + + /** + * @param monitor + * the connectionMonitor to set + */ + public void setConnectionMonitor(ProcBasedConnectionMonitor monitor) { + if (this.connectionMonitor == null) { + this.connectionMonitor = monitor; + this.connectionMonitor.setConnectionHandler(this); + } + } + + /** + * @return the containersMap + */ + public Map getContainersMap() { + return containersMap; + } + + /** + * @return the finishedContainers + */ + public List getFinishedContainers() { + return finishedContainers; + } + + public void buildContainerTrees( + final Map processTree) { + Iterator> entries = + containersMap.entrySet().iterator(); + while (entries.hasNext()) { + Entry entry = entries.next(); + String containerId = entry.getKey(); + NMContainerConnections nmContainer = entry.getValue(); + String pid = containerPidMap.get(containerId); + if (pid != null) { + nmContainer.updateProcessTree(processTree.get(pid)); + } else { + nmContainer.setProcessTree(null); + } + } + } + + public void removeInvalidPid(String invalidPid) { + final String containerId = containerPidMap.inverse().get(invalidPid); + if (!Strings.isNullOrEmpty(containerId)) { + removeTcClassView(containerId); + } + } + + class ConnectionMonitorThread extends Thread { + private long monitoringInterval; + private long checkOrphanContainersInterval = 5 * 60 * 1000L; + private long checkOrphanContainerTime = checkOrphanContainersInterval; + + public ConnectionMonitorThread(Configuration conf) { + super("Collector thread for collecting TC data"); + this.monitoringInterval = + conf.getLong(YarnConfiguration.NM_HDFS_BE_CHECK_TC_INTERVAL_MS, + YarnConfiguration.DEFAULT_NM_HDFS_BE_CHECK_TC_INTERVAL_MS); + if (checkOrphanContainersInterval < 2 * monitoringInterval) { + checkOrphanContainersInterval = 2 * monitoringInterval; + } + + checkOrphanContainerTime = checkOrphanContainersInterval; + } + + @Override + public void run() { + while (true) { + try { + Thread.sleep(monitoringInterval); + if (done + || Thread.interrupted()) { + break; + } + + if (containersMap.isEmpty() + && finishedContainers.isEmpty()) { + continue; + } + + process(); + checkOrphanContainerTime -= monitoringInterval; + + if (checkOrphanContainerTime <= 0) { + checkOrphanContainerTime = checkOrphanContainersInterval; + checkInvalidContainers(); + } + + } catch (InterruptedException e) { + LOG.warn(ConnectionMonitorThread.class.getName() + + " is interrupted."); + } catch (Exception e) { + LOG.warn("Something wrong when processing live connections", e); + } + } + } // end run + + private void checkInvalidContainers() { + LOG.debug("Check orphan containers (without PID after 10 minutes)"); + long currentTime = System.currentTimeMillis(); + Set invalidContainers = new HashSet(); + for (Entry entry : containersMap + .entrySet()) { + String cID = entry.getKey(); + NMContainerConnections con = entry.getValue(); + if (!con.isInitialized() + && registerTime.containsKey(cID)) { + if (currentTime + - registerTime.get(cID) >= WAIT_FOR_FISRT_PID) { + LOG.warn(cID + + " doesn't have pid for at least 10 minutes. Remove it"); + invalidContainers.add(cID); + } + } + } + for (String cID : invalidContainers) { + removeTcClassView(cID); + } + } + } // end ConnectionMonitorSubmitThread + + private class ContainerRegister implements ContainerService { + + @Override + public boolean addMonitoringContainer(String containerId, float rate) { + if (isEnabled) { + return addTcClassView(containerId, rate); + } + + return false; + } + + @Override + public boolean registerPid(String containerId, int pid) { + if (isEnabled) { + return registerContainer(containerId, Integer.toString(pid)); + } + + return false; + } + + @Override + public void stopMonitoringContainer(String containerId) { + if (isEnabled) { + removeTcClassView(containerId); + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ProcBasedConnectionMonitor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ProcBasedConnectionMonitor.java new file mode 100644 index 0000000..a6811ce --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ProcBasedConnectionMonitor.java @@ -0,0 +1,340 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.BufferedReader; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.ConnectionMonitor; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.CommonProcessTree; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.NMContainerConnections; +import org.jboss.netty.util.internal.ConcurrentHashMap; + +import com.google.common.base.Joiner; +import com.google.common.collect.Maps; + +/** + * This class monitors the /proc file system and collect network data for + * containers. + * + */ +public class ProcBasedConnectionMonitor implements ConnectionMonitor { + + static final Log LOG = LogFactory.getLog(ProcBasedConnectionMonitor.class); + + static final String CONNECTED_DN_FILE_NAME = "connected_dn_nodes.dat"; + + static final Pattern PROCFS_STAT_FILE_FORMAT = Pattern + .compile("^([0-9-]+)\\s+\\(([^\\s]+)\\)\\s[^\\s]\\s([0-9-]+)\\s.+"); + + static final Pattern VALID_PID = Pattern.compile("^(\\d+)$"); + + private Set connectedHosts = new HashSet(); + private Map pidMap = Maps.newHashMap(); + + private Joiner joiner = Joiner.on(","); + private Path backupConnectedDNsFile = Paths.get( + System.getProperty("java.io.tmpdir"), CONNECTED_DN_FILE_NAME); + + private Map> connectionsData = + new ConcurrentHashMap>(); + + private Map procfsTree = + new HashMap(); + + private ConnectionCollector collector; + private ProcBasedConnectionHandler connectionHandler; + private Lock lock; + private Configuration conf; + + public ProcBasedConnectionMonitor(Configuration conf) { + this.conf = conf; + } + + public void + setConnectionHandler(ProcBasedConnectionHandler connectionHandler) { + if (this.connectionHandler == null) { + this.connectionHandler = connectionHandler; + init(); + } + } + + /** + * @param collector + * the collector to set + */ + public void setCollector(ConnectionCollector collector) { + if (this.collector == null) { + this.collector = collector; + } + } + + public void setBackupConnectedDNsFile(String path) { + backupConnectedDNsFile = Paths.get(path); + } + + public void init() { + lock = connectionHandler.getLock(); + if (collector == null) { + int monitoringPort = connectionHandler.getMonitoringPort(); + collector = new ConnectionCollector(monitoringPort, false); + boolean canExecuteSudoSS = + conf.getBoolean(YarnConfiguration.NM_HDFS_BE_EXECUTE_SUDO_SS, false); + if (canExecuteSudoSS) { + collector.sudoIsAvailabe(); + } + } + + try { + if (Files.exists(backupConnectedDNsFile)) { + String connectedHostIpList = readFirstLine(backupConnectedDNsFile); + connectedHostIpList = connectedHostIpList.replaceAll("\\s", ""); + String[] hosts = connectedHostIpList.split(","); + for (String host : hosts) { + if (!host.isEmpty() + && !connectionsData.containsKey(host)) { + connectionsData.put(host, new ArrayList()); + } + } + } + } catch (Exception e) { + ; + } + } + + private String readFirstLine(Path file) throws IOException { + try (BufferedReader reader = + Files.newBufferedReader(file, StandardCharsets.UTF_8)) { + return reader.readLine(); + } + } + + /** + * @return the connectionsData + */ + public Map> getConnectionsData() { + return connectionsData; + } + + @Override + public Map> collect() { + + buildConnectionDB(); + + Set savedConnectedHosts = new HashSet(connectedHosts); + + buildUpdateData(); + + if (!connectionsData.isEmpty()) { + if (!savedConnectedHosts.containsAll(connectedHosts) + || !connectedHosts.containsAll(savedConnectedHosts)) { + saveConnectedHosts(connectedHosts); + } + } + + return connectionsData; + } + + public void saveConnectedHosts(Set connectedHosts) { + try { + Files.write(backupConnectedDNsFile, + joiner.join(connectedHosts).getBytes(StandardCharsets.UTF_8)); + } catch (IOException e) { + LOG.error("Cannot save the list of connected DataNodes", e); + } + } + + private void buildUpdateData() { + try { + lock.lock(); + connectedHosts.clear(); + // First collect all updated hosts + Set submitHostList = + new HashSet(connectionsData.keySet()); + connectedHosts.addAll(submitHostList); + + Map containersMap = + connectionHandler.getContainersMap(); + List finishedContainers = + connectionHandler.getFinishedContainers(); + + for (NMContainerConnections nmContainers : finishedContainers) { + nmContainers.collectUpdatedHosts(submitHostList, connectedHosts); + } + for (NMContainerConnections nmContainers : containersMap.values()) { + nmContainers.collectUpdatedHosts(submitHostList, connectedHosts); + } + + // track changes + for (NMContainerConnections nmContainers : finishedContainers) { + nmContainers.commitChanges(connectionsData, submitHostList); + } + + for (NMContainerConnections nmContainers : containersMap.values()) { + nmContainers.commitChanges(connectionsData, submitHostList); + } + + finishedContainers.clear(); + } finally { + lock.unlock(); + } + } + + public void buildConnectionDB() { + Map connections = new HashMap(); + Map> pidInodes = new HashMap>(); + // Collect first time + collector.collectConnections(false, connections); + + try { + lock.lock(); + + Map containersMap = + connectionHandler.getContainersMap(); + // Get PID tree + walkPIDTree(FactoryHelper.getInstance().getProcFsRoot()); + + // Collect all inodes for each container. + for (NMContainerConnections nmCon : containersMap.values()) { + nmCon.collectInodes(); + } + + // Collect again! + collector.collectConnections(false, connections); + collector.collectConnections(true, connections, pidInodes); + + // Group connections by containers + for (NMContainerConnections nmCon : containersMap.values()) { + nmCon.collectConnections(connections, pidInodes); + } + } finally { + lock.unlock(); + } + } + + /** + * @return the procfsTree + */ + public CommonProcessTree getProcfsTree(String pid) { + return procfsTree.get(pid); + } + + private void walkPIDTree(String procRootPath) { + Set oldPids = new HashSet<>(pidMap.keySet()); + + boolean foundNewProcess = false; + try (DirectoryStream directoryStream = + Files.newDirectoryStream(Paths.get(procRootPath))) { + for (Path pidDir : directoryStream) { + if (Files.isDirectory(pidDir)) { + final String pid = pidDir.getFileName().toString(); + Matcher pidMatcher = VALID_PID.matcher(pid); + if (pidMatcher.matches()) { + if (oldPids.contains(pid)) { + oldPids.remove(pid); + continue; + } + + try { + final String statContent = readFirstLine(pidDir.resolve("stat")); + Matcher m = PROCFS_STAT_FILE_FORMAT.matcher(statContent); + if (m.find()) { + String ppid = m.group(3); + updateProcFsTree(pid, ppid, procfsTree); + pidMap.put(pid, ppid); + } + foundNewProcess = true; + } catch (NoSuchFileException e) { + ; + } catch (Exception e) { + LOG.error("Cannot walk the directory: " + + pidDir, e); + } + } + } + } + } catch ( + IOException | SecurityException e) { + LOG.error(e, e); + } + + for (String pid : oldPids) { + removeOldPid(pid, procfsTree); + pidMap.remove(pid); + connectionHandler.removeInvalidPid(pid); + } + + if (foundNewProcess + || !oldPids.isEmpty()) { + connectionHandler.buildContainerTrees(procfsTree); + } + } + + public static void removeOldPid(final String pid, + Map treeDB) { + CommonProcessTree pTree = treeDB.get(pid); + if (pTree != null) { + String ppid = pTree.getParentPID(); + if (ppid != null) { + treeDB.get(ppid).getChildren().remove(pTree); + } + } + + treeDB.remove(pid); + } + + public static void updateProcFsTree(final String pid, final String ppid, + Map treeDB) { + + if (!treeDB.containsKey(ppid)) { + // future parent process + treeDB.put(ppid, new CommonProcessTree(ppid, null)); + } + + if (!treeDB.containsKey(pid)) { + treeDB.put(pid, new CommonProcessTree(pid, ppid)); + } else {// it is a parent of other process. + treeDB.get(pid).setParentPID(ppid); + } + // Update parent + treeDB.get(ppid).getChildren().add(treeDB.get(pid)); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/StorageUtil.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/StorageUtil.java new file mode 100644 index 0000000..fef9fe3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/StorageUtil.java @@ -0,0 +1,217 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.Reader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import net.minidev.json.JSONArray; +import net.minidev.json.JSONObject; +import net.minidev.json.JSONStyle; +import net.minidev.json.JSONValue; +import net.minidev.json.parser.JSONParser; +import net.minidev.json.parser.ParseException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +import com.google.common.base.Strings; + +/** + * Helper class. + * + */ +public final class StorageUtil { + + static final Log LOG = LogFactory.getLog(StorageUtil.class); + static final Pattern TC_CONFIG_NAME_FORMAT = Pattern.compile("^__(.+)__$"); + + private StorageUtil() { + return; + } + + /** + * Encode the list of container connections of a DataNode to JSON format. + * + * @param containerSnapshot + * list of containers have connections with the DataNode + * @return JSON output + */ + public static String encodeHostConnections( + List containerSnapshot) { + JSONObject obj = new JSONObject(); + obj.put("containers", containerSnapshot); + + return JSONValue.toJSONString(obj, JSONStyle.MAX_COMPRESS); + } + + /** + * Construct a list of containers (have at least one connection) from a JSON + * content. + * + * @param parser + * JSON parser + * + * @param jsonContent + * JSON input + * @return list of containers have connections with the DataNode + */ + public static List decodeHostConnections( + JSONParser parser, String jsonContent) { + try { + return buildHostConnections((JSONObject) parser.parse(jsonContent)); + } catch (ParseException e) { + LOG.warn("Invalid json content: " + + e.getMessage(), e); + } + + return new ArrayList(); + } + + /** + * Construct a list of containers (have at least one connection) from a JSON + * content. + * + * @param parser + * JSON parser + * + * @param reader + * reader of JSON input + * @return list of containers have connections with the DataNode + */ + public static List decodeHostConnections( + JSONParser parser, Reader reader) { + try { + return buildHostConnections((JSONObject) parser.parse(reader)); + } catch (ParseException e) { + LOG.warn("Invalid json content: " + + e.getMessage(), e); + } + + return new ArrayList(); + } + + /** + * Return a list of containers have at least one connection with DN. + * + * @param obj + * Json data + * @return list of containers + */ + private static List buildHostConnections( + JSONObject obj) { + List containers = + new ArrayList(); + try { + JSONArray containersObj = (JSONArray) obj.get("containers"); + for (int i = 0; i < containersObj.size(); i++) { + JSONObject containerObj = (JSONObject) containersObj.get(i); + String clsId = (String) containerObj.get("containerId"); + String rate = (String) containerObj.get("rate"); + DNContainerConnections container = + new DNContainerConnections(clsId, rate); + + JSONArray connectionsObj = (JSONArray) containerObj.get("connections"); + for (int j = 0; j < connectionsObj.size(); j++) { + JSONObject connectionObj = (JSONObject) connectionsObj.get(j); + + String srcHost = (String) connectionObj.get("srcHost"); + Integer srcPort = (Integer) connectionObj.get("srcPort"); + String dstHost = (String) connectionObj.get("dstHost"); + Integer dstPort = (Integer) connectionObj.get("dstPort"); + + container.getConnections().add( + new Connection(srcHost, srcPort, dstHost, dstPort)); + } + // Add only if it has at least one connection. + if (!container.getConnections().isEmpty()) { + containers.add(container); + } + } + } catch (Exception e) { + LOG.warn("Error occured when parsing JSON output: " + + e.getMessage(), e); + } + + return containers; + } + + public static void createTimestampFile(FileSystem fs, Path hdfsPath, + Long timestamp) { + try (FSDataOutputStream os = fs.create(hdfsPath, true);) { + os.writeBytes(String.valueOf(timestamp)); + } catch (IOException e) { + LOG.error("Can not create timestamp file at " + + hdfsPath, e); + } + } + + public static long getTimestamp(FileSystem fs, Path timestampFile) { + long ts = 0; + try (FSDataInputStream is = fs.open(timestampFile); + BufferedReader br = + new BufferedReader( + new InputStreamReader(is, StandardCharsets.UTF_8))) { + + final String line = br.readLine(); + if (line != null + && !line.isEmpty()) { + ts = Long.parseLong(line.trim()); + } + } catch ( + IOException | NumberFormatException e) { + ; + } + + return ts; + } + + public static String getHostName(String encodedHostName, + Map cacheNodeNames) { + String nodeName = cacheNodeNames.get(encodedHostName); + if (Strings.isNullOrEmpty(nodeName)) { + nodeName = encodedHostName; + Matcher m = TC_CONFIG_NAME_FORMAT.matcher(nodeName); + boolean validTCConfig = m.find(); + if (validTCConfig) { + nodeName = m.group(1); + if (!Strings.isNullOrEmpty(nodeName)) { + cacheNodeNames.put(encodedHostName, nodeName); + } + } + + nodeName = cacheNodeNames.get(encodedHostName); + } + return nodeName; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/TrafficController.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/TrafficController.java new file mode 100644 index 0000000..278b332 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/TrafficController.java @@ -0,0 +1,545 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractAsyncTCDataCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractService; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTrafficController; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TCClassEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TCEventType; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TCFilterEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TrafficControlEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.executor.TrafficControlDeviceExecutor; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Predicate; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Table; + +/** + * This class collects TC settings from HDFS storage and apply them to the given + * device. + * + */ +public class TrafficController extends AbstractTrafficController implements + AbstractService { + + static final Log LOG = LogFactory.getLog(TrafficController.class); + + private List deviceExecutors; + private AbstractTCDataCollector dataCollector; + private CollectorThread collectorThread = null; + private ConnectionCollector connectionCollector; + + private String localNodeId; + private int syncLoops = 5; + private int monitoringPort = 50010; + + private volatile boolean done = false; + private boolean isEnabled = true; + + private boolean asyncTCDataCollectorIsUsed = false; + private FactoryHelper helper = null; + + private Table connectionDb; + private Map connectionCache = + new HashMap(); + + public TrafficController(Configuration conf) { + this(conf, false, FactoryHelper.getInstance(), 5); + } + + public TrafficController(Configuration conf, boolean runInNodeManager) { + this(conf, runInNodeManager, FactoryHelper.getInstance(), 5); + } + + @VisibleForTesting + public TrafficController(Configuration conf, boolean runInNodeManager, + FactoryHelper helper, int syncLoops) { + if (runInNodeManager + && !conf.getBoolean(YarnConfiguration.NM_HDFS_BE_ENABLE, + YarnConfiguration.DEFAULT_NM_HDFS_BE_ENABLE)) { + LOG.info("Traffic Control executor for Hdfs" + + " bandwidth enforcement is not enabled"); + isEnabled = false; + return; + } + + this.helper = helper; + isEnabled = this.helper.isTCApplicable(); + if (!isEnabled) { + return; + } + + this.syncLoops = syncLoops; + + if (runInNodeManager) { + if (conf.getBoolean(YarnConfiguration.NM_HDFS_BE_CLIENT_MODE, false)) { + LOG.warn("HDFS bandwidth enforcement is run in client mode." + + " Do not start Traffic Control executor!"); + isEnabled = false; + return; + } + + LOG.info("Traffic Control executor for Hdfs " + + "bandwidth enforcement enabled:" + isEnabled); + + if (conf.getBoolean(YarnConfiguration.NM_HDFS_BE_ENABLE_SUBMITTER_ONLY, + false)) { + LOG.info("Only Submitter for Hdfs bandwidth enforcement is enabled!"); + isEnabled = false; + return; + } + } + + dataCollector = helper.getTCDataCollector(conf); + if (dataCollector == null) { + LOG.warn("Collector is null. Disable HDFS bandwidth enforcement!"); + isEnabled = false; + return; + } + + if (dataCollector instanceof AbstractAsyncTCDataCollector) { + asyncTCDataCollectorIsUsed = true; + } + + this.monitoringPort = + conf.getInt(YarnConfiguration.NM_HDFS_PORT, + YarnConfiguration.DEFAULT_NM_HDFS_PORT); + this.connectionCollector = + this.helper.getConnectionCollector(monitoringPort, true); + + this.deviceExecutors = new ArrayList(); + String deviceList = + conf.get(YarnConfiguration.NM_HDFS_BE_DEVICES, + YarnConfiguration.DEFAULT_NM_HDFS_BE_DEVICES); + deviceList = deviceList.replaceAll("\\s", ""); + List deviceNames = Arrays.asList(deviceList.split(",")); + for (String deviceName : deviceNames) { + deviceExecutors.add(this.helper.getDevice(deviceName, monitoringPort, + helper)); + } + + if (!asyncTCDataCollectorIsUsed) { + collectorThread = new CollectorThread(conf); + } + + connectionDb = HashBasedTable.create(); + } + + /** + * @return the monitoringPort + */ + public int getMonitoringPort() { + return monitoringPort; + } + + @Override + public void start() { + if (!isEnabled) { + return; + } + LOG.info("Starting HDFS TrafficController"); + dataCollector.start(); + + if (collectorThread != null) { + collectorThread.start(); + } + } + + @Override + public void stop() { + if (!isEnabled) { + return; + } + LOG.info("HDFS TrafficController stopping..."); + done = true; + + try { + dataCollector.stop(); + } catch (Exception e) { + LOG.warn("Error occurred when stopping collector: " + + e.getMessage(), e); + } + + LOG.info("HDFS TrafficController stopped"); + } + + /** + * @return the isEnabled + */ + public boolean isEnabled() { + return isEnabled; + } + + @Override + public void initialize(final String nodeId) { + if (!isEnabled) { + return; + } + + this.localNodeId = nodeId; + if (localNodeId == null + || localNodeId.isEmpty()) { + try { + InetAddress iAddress = InetAddress.getLocalHost(); + // To get the Canonical host name + localNodeId = iAddress.getCanonicalHostName(); + LOG.info("Change hostname to: " + + localNodeId); + } catch (UnknownHostException e) { + LOG.warn("Inactive HDFS TrafficController component: " + + e.getMessage(), e); + isEnabled = false; + return; + } + } + + LOG.info("Initializing HDFS TrafficController for " + + localNodeId); + + Iterator iterator = + deviceExecutors.listIterator(); + + while (iterator.hasNext()) { + TrafficControlDeviceExecutor device = iterator.next(); + try { + device.initDevice(); + } catch (IOException e) { + LOG.error("Cannot init the device: " + + device.getName() + ". Do not use this device!", e); + iterator.remove(); + } + } + + if (deviceExecutors.isEmpty()) { + LOG.error("Cannot init any devices. Exit!"); + isEnabled = false; + return; + } + + if (asyncTCDataCollectorIsUsed) { + ((AbstractAsyncTCDataCollector) dataCollector) + .registerCallback((AbstractTrafficController) this); + } + + dataCollector.initialize(localNodeId); + } + + @Override + public void execute(List events) { + if (!events.isEmpty()) { + for (TrafficControlDeviceExecutor device : deviceExecutors) { + device.update(events); + } + } + } + + @Override + public Map> collectData() { + if (!done) { + return dataCollector.collectData(); + } + + return null; + } + + @Override + public List buildTCEvents( + Map> connections) { + List eventList = new ArrayList(); + + if (connections.isEmpty()) { + return eventList; + } + List delClassList = new ArrayList(); + List addClassList = new ArrayList(); + List changeClassList = new ArrayList(); + List delFilterList = new ArrayList(); + List addFilterList = new ArrayList(); + + final Collection currentNMConnections = + collectCurrentConnections(); + + Predicate validConnection = new Predicate() { + @Override + public boolean apply(Connection connection) { + return currentNMConnections == null + || currentNMConnections.contains(connection); + } + }; + + synchronized (connectionDb) { + + for (Entry> entry : connections + .entrySet()) { + final String remoteNMHost = entry.getKey(); + + boolean isLoopback = localNodeId.equals(remoteNMHost); + + List nmContainers = entry.getValue(); + Map dbContainers = + connectionDb.row(remoteNMHost); + + if (nmContainers == null + || nmContainers.isEmpty()) { + // Disconnected host. Delete its containers! + for (DNContainerConnections delContainer : dbContainers.values()) { + deleteTCClass(remoteNMHost, delContainer, delClassList, + delFilterList); + } + } else if (dbContainers.isEmpty()) { // New hosts + for (DNContainerConnections newContainer : nmContainers) { + final String containerId = newContainer.getContainerId(); + if (newContainer.getConnections().isEmpty()) { + LOG.warn(containerId + + " hasn't got any connections. Do not add it!"); + continue; + } + + addNewTCClass(remoteNMHost, containerId, newContainer, + validConnection, addClassList, addFilterList); + } + } else { // NM hosts with updated connections + Set storedContainerIds = + new HashSet(dbContainers.keySet()); + for (DNContainerConnections nmContainer : nmContainers) { + final String containerId = nmContainer.getContainerId(); + List updatedConnections = nmContainer.getConnections(); + + if (!storedContainerIds.contains(containerId)) { // new container + addNewTCClass(remoteNMHost, containerId, nmContainer, + validConnection, addClassList, addFilterList); + } else { + // this is existing class. Synchronize its connections + boolean hasConnections = false; + storedContainerIds.remove(containerId); + List storedConnections = + dbContainers.get(containerId).getConnections(); + + // Synchronize connections + Iterator iter = updatedConnections.iterator(); + while (iter.hasNext()) { + Connection connection = iter.next(); + if (validConnection.apply(connection)) { + // new connection + if (!storedConnections.contains(connection)) { + // Create addFilter events + addFilterList.add(TCFilterEvent.createAddFilterEvent( + containerId, connection)); + } else { // This is existed. + storedConnections.remove(connection); + } + + hasConnections = true; + } else { + iter.remove(); + } + } + + // Create delFilter events for disconnected ones + for (Connection connection : storedConnections) { + delFilterList.add(TCFilterEvent.createDelFilterEvent( + containerId, connection)); + } + + if (!hasConnections) { + // Create delClass event + delClassList.add(new TCClassEvent(containerId, + TCEventType.DEL_CLASS, "", isLoopback)); + // Delete from DB + connectionDb.remove(remoteNMHost, containerId); + } else { + String newRate = dbContainers.get(containerId).getRate(); + if (!newRate.equals(nmContainer.getRate())) { + changeClassList.add(new TCClassEvent(containerId, + TCEventType.CHANGE_CLASS, nmContainer.getRate(), + isLoopback)); + } + // Update the DB. + connectionDb.put(remoteNMHost, containerId, nmContainer); + } + } + } + + // Deleted containers + for (String containerId : storedContainerIds) { + deleteTCClass(remoteNMHost, dbContainers.get(containerId), + delClassList, delFilterList); + } + } + } + + eventList.addAll(delFilterList); + eventList.addAll(delClassList); + eventList.addAll(changeClassList); + eventList.addAll(addClassList); + eventList.addAll(addFilterList); + } + return eventList; + } + + /** + * Delete TC class ralated to a given container and all their filters. + * + * @param remoteNMHost + * @param deletedContainer + * @param delClassList + * @param delFilterList + */ + private void deleteTCClass(final String remoteNMHost, + DNContainerConnections deletedContainer, List delClassList, + List delFilterList) { + final String containerId = deletedContainer.getContainerId(); + + // Create delClass event + delClassList.add(new TCClassEvent(containerId, TCEventType.DEL_CLASS, "", + localNodeId.equals(remoteNMHost))); + + // Create delFilter events + for (Connection connection : deletedContainer.getConnections()) { + delFilterList.add(TCFilterEvent.createDelFilterEvent(containerId, + connection)); + } + + // Delete from DB + connectionDb.remove(remoteNMHost, containerId); + } + + /** + * Create new TC class and its filter events for new container. + * + * @param remoteNMHost + * @param containerId + * @param container + * @param validConnection + * @param addClassList + * @param addFilterList + */ + private void addNewTCClass(final String remoteNMHost, + final String containerId, DNContainerConnections container, + Predicate validConnection, List addClassList, + List addFilterList) { + boolean hasConnections = false; + + List connections = container.getConnections(); + + // Create addFilter events + for (Iterator iter = connections.iterator(); iter.hasNext();) { + Connection connection = iter.next(); + if (validConnection.apply(connection)) { + addFilterList.add(TCFilterEvent.createAddFilterEvent(containerId, + connection)); + hasConnections = true; + } else { + iter.remove(); + } + } + + // Create addClass event + if (hasConnections) { + addClassList.add(new TCClassEvent(containerId, TCEventType.ADD_CLASS, + container.getRate(), localNodeId.equals(remoteNMHost))); + // Put to the DB. + connectionDb.put(remoteNMHost, containerId, container); + } + } + + /** + * Collect all connections from port of DataNode. + * + * @return list of connections from HDFS port + */ + @VisibleForTesting + public Collection collectCurrentConnections() { + if (syncLoops > 0) { + syncLoops--; + if (syncLoops == 0) { + return null; + } + connectionCollector.collectConnections(false, connectionCache); + try { + Thread.sleep(10); + } catch (InterruptedException e) { + ; + } + connectionCollector.collectConnections(false, connectionCache); + connectionCollector.collectConnections(true, connectionCache); + + return connectionCache.values(); + } + + return null; + } + + class CollectorThread extends Thread { + private Long monitoringInterval = 500L; + + public CollectorThread(Configuration conf) { + super("Collector thread for collecting TC data"); + this.monitoringInterval = + conf.getLong(YarnConfiguration.NM_HDFS_BE_CHECK_TC_INTERVAL_MS, + YarnConfiguration.DEFAULT_NM_HDFS_BE_CHECK_TC_INTERVAL_MS); + } + + @Override + public void run() { + while (true) { + try { + Thread.sleep(monitoringInterval); + if (done + || Thread.interrupted()) { + break; + } + + if (dataCollector.isReady()) { + update(); + } + } catch (InterruptedException e) { + LOG.warn(CollectorThread.class.getName() + + " is interrupted."); + } catch (Exception e) { + ; + } + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/YarnContainerService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/YarnContainerService.java new file mode 100644 index 0000000..b70efdf --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/YarnContainerService.java @@ -0,0 +1,202 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.lang.reflect.Constructor; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractYarnContainerReportService; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; + +/** + * This plug-in provides the container service for + * {@link ProcBasedConnectionHandler}. It is called from + * {@link ContainersMonitorImpl}. + * + */ +public class YarnContainerService extends AbstractContainerService { + + private static final Log LOG = LogFactory.getLog(YarnContainerService.class); + + private float minRate = 0; + private float maxRate = 1_000_000; // 1000Gb + + private boolean isClientMode = false; + + public YarnContainerService(Configuration conf, boolean isClientMode) { + super(conf); + float configuredMinRate = + conf.getFloat(YarnConfiguration.NM_HDFS_BE_MIN_RATE, 0); + minRate = Math.min(configuredMinRate, 1_000); + maxRate = conf.getFloat(YarnConfiguration.NM_HDFS_BE_MAX_RATE, 1_000_000); + + if (maxRate + - minRate < 1) { + maxRate = 1_000_000; + } + + LOG.info(String.format( + "Min HDFS rate allocation for YARN containers: %.2fmbps", minRate)); + LOG.info(String.format( + "Max HDFS rate allocation for YARN containers: %.2fmbps", maxRate)); + + this.isClientMode = isClientMode; + } + + public static YarnContainerService + loadYarnContainerService(Configuration conf) { + return loadYarnContainerService(conf, FactoryHelper.getInstance()); + } + + @VisibleForTesting + public static YarnContainerService loadYarnContainerService( + Configuration conf, FactoryHelper helper) { + boolean isEnabled = + conf.getBoolean(YarnConfiguration.NM_HDFS_BE_ENABLE, + YarnConfiguration.DEFAULT_NM_HDFS_BE_ENABLE) + && helper.isTCApplicable(); + + LOG.info("HDFS bandwidth enforcement: " + + isEnabled); + + if (!isEnabled) { + return null; + } + + if (!conf.getBoolean(YarnConfiguration.NM_HDFS_BE_CLIENT_MODE, false)) { + return new YarnContainerService(conf, false); + } + + String reporterClazzName = + conf.get(YarnConfiguration.NM_HDFS_BE_REPORT_SERVICE_CLASS); + if (!Strings.isNullOrEmpty(reporterClazzName)) { + LOG.info("Register container report plugin: " + + reporterClazzName); + + try { + Class reportClazz = + conf.getClass(YarnConfiguration.NM_HDFS_BE_REPORT_SERVICE_CLASS, + null, AbstractYarnContainerReportService.class); + + if (reportClazz == null) { + LOG.error("Cannot load the report plugin defined in property: " + + YarnConfiguration.NM_HDFS_BE_REPORT_SERVICE_CLASS); + return null; + } + + Constructor constructor = + reportClazz.getConstructor(Configuration.class); + YarnContainerService containerService = + new YarnContainerService(conf, true); + containerService.setCallBack(constructor.newInstance(conf)); + return containerService; + } catch (Exception e) { + LOG.error("Cannot set callback service: " + + e.getMessage(), e); + } + } + + return null; + } + + /** + * @return the isClientMode + */ + public boolean isClientMode() { + return isClientMode; + } + + public void stopMonitoringContainer(ContainerId con) { + stopMonitoringContainer(con.toString()); + } + + public void addMonitoringContainer(ContainerId con, float rateInMbps) { + addMonitoringContainer(con.toString(), rateInMbps); + } + + public void registerPid(ContainerId containerId, String pId) { + registerPid(containerId.toString(), Integer.parseInt(pId)); + } + + @Override + protected float normalize(float rate) { + float newRate = Math.min(Math.max(rate, minRate), maxRate); + if (newRate != rate) { + LOG.info("Normalized HDFS read bandwidth limit: " + + newRate + " mbps"); + } + return newRate; + } + + @Override + public void initialize(String localNodeId) { + if (isClientMode) { + ((AbstractYarnContainerReportService) getCallBack()) + .initialize(localNodeId); + } + } + + @Override + public void start() { + if (isClientMode) { + ((AbstractYarnContainerReportService) getCallBack()).start(); + } + } + + @Override + public void stop() { + + try { + Set containerIds = + new HashSet(getUnModifableMapOfContainers().keySet()); + for (String containerId : containerIds) { + stopMonitoringContainer(containerId); + } + } catch (Exception e) { + ; + } + + if (isClientMode) { + ((AbstractYarnContainerReportService) getCallBack()).stop(); + } + } + + /** + * @return the minRate + */ + public float getMinRate() { + return minRate; + } + + /** + * @return the maxRate + */ + public float getMaxRate() { + return maxRate; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkAsyncTCDataCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkAsyncTCDataCollector.java new file mode 100644 index 0000000..1cddb0e --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkAsyncTCDataCollector.java @@ -0,0 +1,213 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import net.minidev.json.parser.JSONParser; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.CloseableUtils; +import org.apache.curator.utils.ZKPaths; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractAsyncTCDataCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +import com.google.common.base.Strings; + +/** + * Retrieve connection data from ZooKeeper storage and immediately report them + * to {@link TrafficController}. + * + */ +public class ZkAsyncTCDataCollector extends AbstractAsyncTCDataCollector { + + static final Log LOG = LogFactory.getLog(ZkAsyncTCDataCollector.class); + private CuratorFramework zkClient; + private PathChildrenCache cache = null; + private JSONParser parser; + private String nodeId; + private volatile boolean isReady = false; + + private String dataRootPath; + private Map cachePaths = new HashMap(); + + private Map> updatedContainers = + new HashMap<>(); + + public ZkAsyncTCDataCollector(Configuration conf) { + super(conf); + String zkServerHostPort = + conf.get(YarnConfiguration.NM_HDFS_BE_ZK_SERVER_ADDRESS, null); + + zkClient = + CuratorFrameworkFactory.newClient(zkServerHostPort, + new ExponentialBackoffRetry(1000, 5)); + + parser = new JSONParser(JSONParser.MODE_PERMISSIVE); + } + + @Override + public void initialize(String nodeId) { + this.nodeId = nodeId; + this.dataRootPath = ZKPaths.makePath(ZkTCDataSubmitter.ZK_ROOT, nodeId); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * AbstractTCDataCollector#isReady() + */ + @Override + public boolean isReady() { + return isReady; + } + + @Override + public void start() { + super.start(); + + try { + zkClient.start(); + + // Check for root path and create if needed + if (zkClient.checkExists().forPath(dataRootPath) == null) { + zkClient.create().creatingParentsIfNeeded().inBackground() + .forPath(dataRootPath, nodeId.getBytes(StandardCharsets.UTF_8)); + } + + // Start cache + cache = new PathChildrenCache(zkClient, dataRootPath, true); + cache.getListenable().addListener(new CollectorListener()); + cache.start(StartMode.POST_INITIALIZED_EVENT); + } catch (Exception e) { + LOG.error("Error happened: " + + e.getMessage(), e); + } + } + + @Override + public void stop() { + CloseableUtils.closeQuietly(cache); + CloseableUtils.closeQuietly(zkClient); + super.stop(); + } + + private void processsEventData(ChildData data, boolean isRemoved) { + final String nodePath = data.getPath(); + if (LOG.isDebugEnabled()) { + LOG.debug("Handling connections from " + + nodePath); + } + synchronized (updatedContainers) { + final String zkNodeName = + StorageUtil + .getHostName(ZKPaths.getNodeFromPath(nodePath), cachePaths); + if (!Strings.isNullOrEmpty(zkNodeName)) { + if (isRemoved) { + updatedContainers.put(zkNodeName, + new ArrayList()); + } else { + updatedContainers.put( + zkNodeName, + StorageUtil.decodeHostConnections(parser, + new String(data.getData(), StandardCharsets.UTF_8))); + } + + if (isReady) { + notifyUpdate(); + } + } + } + } + + @Override + public Map> collectData() { + Map> connectionMap = new HashMap<>(); + synchronized (updatedContainers) { + + Iterator>> entries = + updatedContainers.entrySet().iterator(); + while (entries.hasNext()) { + List list = + new ArrayList(); + Entry> entry = entries.next(); + for (DNContainerConnections dnCon : entry.getValue()) { + list.add(dnCon.cloneContainer()); + } + connectionMap.put(entry.getKey(), list); + } + + if (!updatedContainers.isEmpty()) { + updatedContainers.clear(); + } + } + + return connectionMap; + } + + class CollectorListener implements PathChildrenCacheListener { + + @Override + public void + childEvent(CuratorFramework client, PathChildrenCacheEvent event) + throws Exception { + + switch (event.getType()) { + case INITIALIZED: { + LOG.info("Cache is populated!"); + isReady = true; + notifyUpdate(); + break; + } + case CHILD_ADDED: + case CHILD_UPDATED: { + processsEventData(event.getData(), false); + break; + } + case CHILD_REMOVED: { + processsEventData(event.getData(), true); + break; + } + default: { + LOG.warn("An unexpected event occured: " + + event.getType()); + break; + } + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkTCDataCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkTCDataCollector.java new file mode 100644 index 0000000..52b2625 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkTCDataCollector.java @@ -0,0 +1,209 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import net.minidev.json.parser.JSONParser; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.CloseableUtils; +import org.apache.curator.utils.ZKPaths; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; + +import com.google.common.base.Strings; + +/** + * Retrieve connection data from ZooKeeper storage and report them to + * {@link TrafficController} when there is a request from + * {@link TrafficController}. + * + */ +public class ZkTCDataCollector extends AbstractTCDataCollector { + + static final Log LOG = LogFactory.getLog(ZkTCDataCollector.class); + private CuratorFramework zkClient; + private PathChildrenCache cache = null; + private JSONParser parser; + private String nodeId; + + private volatile boolean isReady = false; + + private String dataRootPath; + private Map cachePaths = new HashMap(); + + private Map> updatedContainers = + new HashMap<>(); + + public ZkTCDataCollector(Configuration conf) { + super(conf); + String zkServerHostPort = + conf.get(YarnConfiguration.NM_HDFS_BE_ZK_SERVER_ADDRESS, null); + + zkClient = + CuratorFrameworkFactory.newClient(zkServerHostPort, + new ExponentialBackoffRetry(1000, 5)); + + parser = new JSONParser(JSONParser.MODE_PERMISSIVE); + } + + @Override + public void initialize(String nodeId) { + this.nodeId = nodeId; + this.dataRootPath = ZKPaths.makePath(ZkTCDataSubmitter.ZK_ROOT, nodeId); + } + + @Override + public void start() { + super.start(); + try { + zkClient.start(); + + // Check for root path and create if needed + if (zkClient.checkExists().forPath(dataRootPath) == null) { + zkClient.create().creatingParentsIfNeeded().inBackground() + .forPath(dataRootPath, nodeId.getBytes(StandardCharsets.UTF_8)); + } + + // Start cache + cache = new PathChildrenCache(zkClient, dataRootPath, true); + cache.getListenable().addListener(new CollectorListener()); + cache.start(StartMode.POST_INITIALIZED_EVENT); + } catch (Exception e) { + LOG.error("Error happened: " + + e.getMessage(), e); + } + } + + @Override + public void stop() { + CloseableUtils.closeQuietly(cache); + CloseableUtils.closeQuietly(zkClient); + super.stop(); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * AbstractTCDataCollector#isReady() + */ + @Override + public boolean isReady() { + return isReady; + } + + private void processsEventData(ChildData data, boolean isRemoved) { + final String nodePath = data.getPath(); + if (LOG.isDebugEnabled()) { + LOG.debug("Handling connections from " + + nodePath); + } + synchronized (updatedContainers) { + final String zkNodeName = + StorageUtil + .getHostName(ZKPaths.getNodeFromPath(nodePath), cachePaths); + if (!Strings.isNullOrEmpty(zkNodeName)) { + if (isRemoved) { + updatedContainers.put(zkNodeName, + new ArrayList()); + } else { + updatedContainers.put( + zkNodeName, + StorageUtil.decodeHostConnections(parser, + new String(data.getData(), StandardCharsets.UTF_8))); + } + } + } + } + + @Override + public Map> collectData() { + Map> connectionMap = new HashMap<>(); + synchronized (updatedContainers) { + + Iterator>> entries = + updatedContainers.entrySet().iterator(); + while (entries.hasNext()) { + List list = + new ArrayList(); + Entry> entry = entries.next(); + for (DNContainerConnections dnCon : entry.getValue()) { + list.add(dnCon.cloneContainer()); + } + connectionMap.put(entry.getKey(), list); + } + + if (!updatedContainers.isEmpty()) { + updatedContainers.clear(); + } + } + + return connectionMap; + } + + class CollectorListener implements PathChildrenCacheListener { + + @Override + public void + childEvent(CuratorFramework client, PathChildrenCacheEvent event) + throws Exception { + + switch (event.getType()) { + case INITIALIZED: { + LOG.info("Cache is populated!"); + isReady = true; + break; + } + case CHILD_ADDED: + case CHILD_UPDATED: { + processsEventData(event.getData(), false); + break; + } + case CHILD_REMOVED: { + processsEventData(event.getData(), true); + break; + } + default: { + LOG.warn("An unexpected event occured: " + + event.getType()); + break; + } + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkTCDataSubmitter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkTCDataSubmitter.java new file mode 100644 index 0000000..ceec4ba --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/ZkTCDataSubmitter.java @@ -0,0 +1,141 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import java.nio.charset.StandardCharsets; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.CloseableUtils; +import org.apache.curator.utils.ZKPaths; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.AbstractTCDataSubmitter; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; +import org.apache.zookeeper.KeeperException.NoNodeException; + +/** + * Submit connection data to ZooKeeper storage when there is a request from + * {@link ProcBasedConnectionHandler}. + * + */ +public class ZkTCDataSubmitter extends AbstractTCDataSubmitter { + + static final Log LOG = LogFactory.getLog(ZkTCDataSubmitter.class); + public static final String ZK_ROOT = "tcData"; + + private CuratorFramework zkClient; + + private String tcConfigName; + + public ZkTCDataSubmitter(Configuration conf) { + super(conf); + String zkServerHostPort = + conf.get(YarnConfiguration.NM_HDFS_BE_ZK_SERVER_ADDRESS, null); + + zkClient = + CuratorFrameworkFactory.newClient(zkServerHostPort, + new ExponentialBackoffRetry(1000, 5)); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * TrafficControlDataSubmitter#start() + */ + @Override + public void start() { + zkClient.start(); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * TrafficControlDataSubmitter#stop() + */ + @Override + public void stop() { + CloseableUtils.closeQuietly(zkClient); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * TrafficControlDataSubmitter + * #initialize(org.apache.hadoop.yarn.server.nodemanager.Context) + */ + @Override + public void initialize(String localNodeId) { + if (localNodeId != null) { + tcConfigName = String.format("__%s__", localNodeId); + } + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.nodemanager.trafficcontrol. + * TrafficControlDataSubmitter#submit(java.lang.String, java.util.List) + */ + @Override + public boolean submit(String remoteHost, + List hostConnections) { + try { + + String remoteHostName = NetUtils.getHostNameOfIP(remoteHost); + if (remoteHostName == null) { + LOG.warn("Cannot get hostname for " + + remoteHost); + remoteHostName = remoteHost; + } + + final String remoteZKPath = ZKPaths.makePath(ZK_ROOT, remoteHostName); + if (zkClient.checkExists().forPath(remoteZKPath) == null) { + LOG.debug(remoteZKPath + + " is not exist. Skip this round"); + return false; + } + + final String zkPath = ZKPaths.makePath(remoteZKPath, tcConfigName); + byte[] data = + StorageUtil.encodeHostConnections(hostConnections).getBytes( + StandardCharsets.UTF_8); + try { + zkClient.setData().forPath(zkPath, data); + } catch (NoNodeException e) { + zkClient.create().forPath(zkPath, data); + } + + return true; + } catch (Exception e) { + LOG.error("Cannot submit data to " + + remoteHost, e); + } + + return false; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCClassExecutor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCClassExecutor.java new file mode 100644 index 0000000..cd4a14a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCClassExecutor.java @@ -0,0 +1,167 @@ +/** + * 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.nodemanager.trafficcontrol.impl.executor; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TCFilterEvent; + +/** + * Executor class for LTC class settings. + * + */ +public class TCClassExecutor { + + static final Log LOG = LogFactory.getLog(TCClassExecutor.class); + + private String parent; + private int itemIndex; + private String rate; + private TrafficControlDeviceExecutor device; + private String minor; + private String burst; + + private List filters; + + public TCClassExecutor(TrafficControlDeviceExecutor tcDevice, String parent, + int itemIndex, String rate, String burst) { + this(tcDevice, parent, itemIndex, null, rate, burst); + } + + TCClassExecutor(TrafficControlDeviceExecutor tcDevice, String parent, + int itemIndex, String minor, String rate, String burst) { + this.device = tcDevice; + this.parent = parent; + this.itemIndex = itemIndex; + this.rate = rate; + this.burst = burst; + filters = new ArrayList(); + if (minor != null) { + this.minor = minor; + } else { + this.minor = TCHandlerUtils.getClassMinorFromIndex(itemIndex); + } + } + + public String getClassIdentifier() { + return String.format("%s:%s", parent, minor); + } + + public String getRate() { + return rate; + } + + private TCCommand addClassCmd(String action) { + TCCommand cmd = new TCCommand(); + cmd.add(String.format("class %s dev %s parent %s: classid %s htb rate %s", + action, device.getName(), parent, getClassIdentifier(), rate)); + if (this.burst != null) { + cmd.add(String.format("burst %s cburst %s", burst, burst)); + } + return cmd; + } + + public int doAddClass(boolean newClassMinor) { + int ret; + TCCommand cmd; + if (newClassMinor) { + cmd = addClassCmd("add"); + ret = device.exec(cmd); + if (ret == 0) { + cmd = new TCCommand(); + cmd.add(String.format("qdisc add dev %s parent %s handle %s: sfq", + device.getName(), getClassIdentifier(), minor)); + ret = device.exec(cmd); + } + if (ret != 0) { + LOG.warn("Cannot add new class: " + + cmd.toString()); + device.freeClassItemIndex(itemIndex); + } else { + LOG.debug("Add new class: " + + cmd.toString()); + } + } else { + cmd = addClassCmd("change"); + ret = device.exec(cmd); + if (ret != 0) { + LOG.warn("Can not change class: " + + cmd.toString()); + } else { + LOG.debug("Reuse class"); + } + } + return ret; + } + + /** + * Delete all filters belonged to this class. However the class is stored for + * later use. + * + * @return 0 if the deletion is success. + */ + public int doDelClass() { + int result = 0; + for (TCFilterExecutor filter : filters) { + TCCommand cmd = filter.delFilterCmd(); + result = device.exec(cmd); + device.freeFilterItemIndex(filter.getItemIndex()); + device.removeFilterFromList(filter.getHandle()); + } + + device.freeClassItemIndex(itemIndex); + filters.clear(); + return result; + } + + public TCFilterExecutor addFilter(TCFilterEvent event) { + int pItemIndex = device.getFreeFilterItem(); + TCFilterExecutor filter = + new TCFilterExecutor(device.getName(), parent, pItemIndex, + getClassIdentifier()); + filter.setMatch(event.getLocalPort(), event.getRemoteHost(), + event.getRemotePort()); + if (device.exec(filter.addFilterCmd()) == 0) { + LOG.debug("Success added filter " + + filter.toString()); + filter.setOwnClass(this); + filters.add(filter); + return filter; + } else { + LOG.warn("Cannot execute command: " + + filter.addFilterCmd()); + device.freeFilterItemIndex(pItemIndex); + } + return null; + } + + public boolean removeFilter(TCFilterExecutor filter) { + return filters.remove(filter); + } + + @Override + public String toString() { + return String.format("[TrafficControlClass -> Device: %s, parent: %s," + + " classId: %s, rate: %s, burst: %s , cburst %s]", device.getName(), + parent, getClassIdentifier(), rate, burst, burst); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCCommand.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCCommand.java new file mode 100644 index 0000000..81a8f2e --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCCommand.java @@ -0,0 +1,61 @@ +/** + * 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.nodemanager.trafficcontrol.impl.executor; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import com.google.common.base.Joiner; + +/** + * Basic class for manipulating LTC commands. + * + */ +public class TCCommand { + + private static final String[] TC_CMD = new String[] {"sudo", "tc"}; + + private static Joiner joiner = Joiner.on(" "); + + private List cmdList; + + public TCCommand() { + cmdList = new ArrayList(Arrays.asList(TC_CMD)); + } + + public void add(String cmd) { + cmdList.add(cmd); + } + + public void addAll(String[] arrays) { + for (String s : arrays) { + cmdList.add(s); + } + } + + public void clear() { + cmdList.clear(); + cmdList.addAll(Arrays.asList(TC_CMD)); + } + + public String toString() { + return joiner.join(cmdList); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCFilterExecutor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCFilterExecutor.java new file mode 100644 index 0000000..8fa1f81 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCFilterExecutor.java @@ -0,0 +1,171 @@ +/** + * 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.nodemanager.trafficcontrol.impl.executor; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +/** + * Executor class for LTC filter settings. + * + */ +public class TCFilterExecutor { + static final Log LOG = LogFactory.getLog(TCFilterExecutor.class); + + static final String HASH_TABLE = "800"; + + private int itemIndex; + + private int localPort; + private String remoteHost; + private int remotePort; + + private String device; + private String parent; + private String flowId; + private String handle; + private int priority = 1; + + private boolean isEnable = false; + + private TCClassExecutor ownClass; + + TCFilterExecutor(String device, String parent, int itemIndex, String flowId, + String handle, int priority) { + this.device = device; + this.parent = parent; + this.itemIndex = itemIndex; + this.flowId = flowId; + this.priority = priority; + this.handle = handle; + if (handle != null) { + isEnable = true; + } + } + + public TCFilterExecutor(String device, String parent, int itemIndex, + String flowId) { + this(device, parent, itemIndex, flowId, null, 1); + } + + public void setOwnClass(TCClassExecutor ownClass) { + this.ownClass = ownClass; + } + + public TCClassExecutor getOwnClass() { + return ownClass; + } + + public void setMatch(int localPort, String remoteHost, int remotePort) { + isEnable = true; + this.localPort = localPort; + this.remoteHost = remoteHost; + this.remotePort = remotePort; + } + + public boolean match(int localPort, String remoteHost, int remotePort) { + return this.localPort == localPort + && this.remoteHost.equals(remoteHost) && this.remotePort == remotePort; + } + + private String protocolMatch() { + if (remoteHost != null) { + return String.format( + "match ip sport %d 0xffff match ip dst %s match ip dport %d 0xffff", + localPort, remoteHost, remotePort); + } else { + return String.format("match ip sport %d 0xffff match ip dport %d 0xffff", + localPort, remotePort); + } + } + + public int getItemIndex() { + return itemIndex; + } + + private String getItem() { + return TCHandlerUtils.convertToHex(itemIndex, + TCHandlerUtils.TC_FILTER_BASE_MINOR); + } + + public String getHandle() { + if (handle != null) { + return handle; + } + return String.format("%s::%s", HASH_TABLE, getItem()); + } + + public String getHt() { + return HASH_TABLE; + } + + public TCCommand addFilterCmd() { + if (!isEnable) { + return null; + } + + TCCommand cmd = new TCCommand(); + cmd.add("filter add"); + cmd.add(String.format("dev %s parent %s: prio %s", device, parent, + priority)); + cmd.add(String.format("handle ::%s protocol ip u32 ht %s::", getItem(), + getHt())); + cmd.add(protocolMatch()); + cmd.add(String.format("flowid %s", flowId)); + LOG.debug(cmd.toString()); + return cmd; + } + + public TCCommand delFilterCmd() { + if (!isEnable) { + return null; + } + + TCCommand cmd = new TCCommand(); + cmd.add("filter del"); + cmd.add(String.format("dev %s parent %s: prio %s", device, parent, + priority)); + cmd.add(String.format("handle %s protocol ip u32", getHandle())); + LOG.debug(cmd.toString()); + return cmd; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof TCFilterExecutor) { + TCFilterExecutor o = (TCFilterExecutor) obj; + return localPort == o.localPort + && remoteHost.equals(o.remoteHost) && remotePort == o.remotePort; + } + return false; + } + + @Override + public String toString() { + return String.format( + "[TC filter -> Device: %s, parent: %s, handle: %s, u32: %s]", device, + parent, getHandle(), protocolMatch()); + } + + @Override + public int hashCode() { + assert false : "hashCode not designed"; + return 42; // any arbitrary constant will do + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCHandlerUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCHandlerUtils.java new file mode 100644 index 0000000..ed56d1f --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TCHandlerUtils.java @@ -0,0 +1,49 @@ +/** + * 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.nodemanager.trafficcontrol.impl.executor; + +/** + * Utilities for handling LTC. + * + */ +public final class TCHandlerUtils { + + static final int TC_CLASS_BASE_MINOR = 1000; + static final int TC_FILTER_BASE_MINOR = 1000; + + private TCHandlerUtils() { + } + + public static String getClassMinorFromIndex(int index) { + return convertToHex(index, TC_CLASS_BASE_MINOR); + } + + public static TCCommand newTcCommand() { + return new TCCommand(); + } + + public static String convertToHex(int itemIndex, int base) { + return Integer.toHexString(itemIndex + + base); + } + + public static int convertToDecimal(String hexStr) { + return Integer.parseInt(hexStr, 16); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TrafficControlDeviceExecutor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TrafficControlDeviceExecutor.java new file mode 100644 index 0000000..c8ccaa2 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/executor/TrafficControlDeviceExecutor.java @@ -0,0 +1,582 @@ +/** + * 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.nodemanager.trafficcontrol.impl.executor; + +import java.io.BufferedReader; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TCClassEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TCEventType; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TCFilterEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.event.TrafficControlEvent; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; + +import com.google.common.annotations.VisibleForTesting; + +/** + * + * Executor class for network devices. + * + */ +public class TrafficControlDeviceExecutor { + + private static final Log LOG = LogFactory + .getLog(TrafficControlDeviceExecutor.class); + + private static final String DEFAULT_QDISC_HANDLE = "10"; + private static final String TC_QDISC_TYPE = "htb"; + + public static final Pattern TC_ROOT_QDISC_SHOW = Pattern + .compile("^qdisc\\s+htb\\s+([0-9a-fA-F]+):\\s+root"); + + public static final Pattern TC_HDFS_CLASS_SHOW = Pattern + .compile("^class\\s+htb\\s+([0-9a-fA-F]+:([0-9a-fA-F]+))\\s+(.+?)" + + "rate\\s+([0-9]+(\\.\\d+)?)([mMkK]b?(it)?|B|b)\\s+"); + + public static final Pattern TC_HDFS_FILTER_INFO = Pattern + .compile("^filter\\s+protocol\\s+ip\\s+pref\\s+(\\d+)\\s+(.+)" + + "fh\\s+([0-9a-fA-F]+::([0-9a-fA-F]+))\\s+(.+)" + + "flowid\\s+([0-9a-fA-F]+:[0-9a-fA-F]+)"); + + public static final Pattern TC_OTHER_FILTER_INFO = Pattern + .compile("^filter\\s+(.+)fh\\s+([0-9a-fA-F]*)" + + ":([0-9a-fA-F]*):([0-9a-fA-F]+)"); + + public static final Pattern TC_COMMON_FILTER_INFO = Pattern + .compile("^filter\\s+(.+)flowid\\s+([0-9a-fA-F]+:[0-9a-fA-F]+)"); + + static final Pattern TC_FILTER_MATCH_PORT = Pattern + .compile("\\s*match\\s+([0-9a-fA-F]{4})([0-9a-fA-F]{4})/ffffffff" + + "\\s+at\\s+20"); + + static final Pattern TC_FILTER_MATCH_IP = Pattern + .compile("\\s*match\\s+([0-9a-fA-F]{2})([0-9a-fA-F]{2})" + + "([0-9a-fA-F]{2})([0-9a-fA-F]{2})/ffffffff\\s+at\\s+16"); + + private String deviceName; + private String qDiscHandle; + private List clsItemPool; + private List filterItemPool; + + private boolean initialized = false; + + private FactoryHelper helper; + + // Helper map to quickly find TcClass by identifier + private Map tcClasses; + + // key = filterHandle + private Map tcFilters; + + private Map oldTcClasses = new HashMap<>(); + private List oldTcFilters = new ArrayList<>(); + private Set reservedFilterItems = new HashSet(); + private Set reservedClassItems = new HashSet(); + private Set otherClassidList = new HashSet(); + + private boolean isLoopBack = false; + private int monitoringPort; + + @VisibleForTesting + private String burst; + + public TrafficControlDeviceExecutor(String deviceName, int monitoringPort, + FactoryHelper helper) { + this(deviceName, null, monitoringPort, helper); + } + + public TrafficControlDeviceExecutor(String deviceName, String burst, + int monitoringPort, FactoryHelper helper) { + this.deviceName = deviceName; + this.monitoringPort = monitoringPort; + clsItemPool = new ArrayList(); + filterItemPool = new ArrayList(); + tcClasses = new HashMap(); + tcFilters = new HashMap(); + if (this.deviceName.equals("lo")) { + isLoopBack = true; + } + + this.burst = burst; + this.helper = helper; + } + + public void initDevice() throws IOException { + LOG.info("Initialize Qdisc for " + + deviceName); + TCCommand tcCommand = new TCCommand(); + tcCommand.add(String.format("qdisc show dev %s", deviceName)); + + BufferedReader input = helper.getBufferedReader(tcCommand); + String line; + boolean hasContent = false; + while ((line = input.readLine()) != null) { + // Cannot find device + if (line.contains("Cannot find device")) { + throw new IOException("Cannot find device: " + + deviceName); + } + + hasContent = true; + Matcher m = TC_ROOT_QDISC_SHOW.matcher(line); + if (m.find()) { + qDiscHandle = m.group(1); + LOG.info("Found existing root qdisc with handler " + + qDiscHandle + ". Reuse it!"); + collectHdfsTcClasses(); + if (!oldTcClasses.isEmpty()) { + collectHdfsTcFilters(); + } else { + initialized = true; + } + + return; + } + } + + if (hasContent) { + tcCommand.clear(); + tcCommand.add(String.format("qdisc del dev %s root", deviceName)); + if (exec(tcCommand) != 0) { + LOG.warn("Cannot execute qdisc command for device " + + deviceName + ": " + tcCommand); + } + } + + qDiscHandle = DEFAULT_QDISC_HANDLE; + + // not found any, create new + tcCommand.clear(); + tcCommand.add(String.format("qdisc add dev %s root handle %s: %s", + deviceName, qDiscHandle, TC_QDISC_TYPE)); + + if (exec(tcCommand) != 0) { + throw new IOException("Can not execute command " + + tcCommand); + } + } + + /** + * @return the initialized + */ + public boolean isInitialized() { + return initialized; + } + + int exec(TCCommand cmd) { + return helper.exec(cmd); + } + + /** + * Collect all maybe HDFS class with minor number is greater than 1000. + * + * @throws IOException + */ + private void collectHdfsTcClasses() throws IOException { + TCCommand tcCommand = new TCCommand(); + tcCommand.add(String.format("class show dev %s parent %s:", deviceName, + qDiscHandle)); + BufferedReader input = helper.getBufferedReader(tcCommand); + String line; + + while ((line = input.readLine()) != null) { + line = line.trim(); + LOG.debug(line); + Matcher m = TC_HDFS_CLASS_SHOW.matcher(line); + if (m.find()) { + String classid = m.group(1); + String classHexMinor = m.group(2); + + int minor = TCHandlerUtils.convertToDecimal(classHexMinor) + - TCHandlerUtils.TC_CLASS_BASE_MINOR; + if (minor < 0) { + otherClassidList.add(classid); + continue; + } else { + reservedClassItems.add(minor); + } + + String unit = m.group(6).toLowerCase(); + String newUnit = unit; + switch (unit) { + case "mb": + case "kb": + newUnit += "ps"; + break; + case "m": + case "k": + newUnit += "bps"; + break; + case "b": + newUnit = "bps"; + break; + default: + break; + } + + String rate = String.format("%s%s", m.group(4), newUnit); + oldTcClasses.put(classid, new TCClassExecutor(this, qDiscHandle, -1, + classHexMinor, rate, burst)); + } + } + + if (!otherClassidList.isEmpty()) { + LOG.debug(String.format("Some TC classes %s may not be ours.", + otherClassidList)); + } + + if (!oldTcClasses.isEmpty()) { + LOG.debug(String.format("Old TC classes: %s", oldTcClasses)); + } + } + + private void collectHdfsTcFilters() throws IOException { + TCCommand tcCommand = new TCCommand(); + tcCommand.add(String.format("filter show dev %s parent %s:", deviceName, + qDiscHandle)); + BufferedReader input = helper.getBufferedReader(tcCommand); + String line; + int filterMinor = -1; + String currentFilterFlowid = null; + boolean mayBeOurFilter = false; + boolean foundIp = false; + boolean foundSrcHdfsPort = false; + TCFilterExecutor tcFilterExecutor = null; + while ((line = input.readLine()) != null) { + line = line.trim(); + LOG.debug(line); + if (line.startsWith("filter")) { + if (tcFilterExecutor != null) { + if (foundIp + && foundSrcHdfsPort) { + LOG.debug("Add old filter: " + + tcFilterExecutor); + oldTcFilters.add(tcFilterExecutor); + } else if (currentFilterFlowid != null) { + oldTcClasses.remove(currentFilterFlowid); + } + } + + filterMinor = -1; + tcFilterExecutor = null; + foundIp = false; + foundSrcHdfsPort = false; + currentFilterFlowid = null; + mayBeOurFilter = false; + Matcher m = TC_COMMON_FILTER_INFO.matcher(line); + // Valid filter + if (m.find()) { + currentFilterFlowid = m.group(2); + // Not our filter + if (!oldTcClasses.containsKey(currentFilterFlowid)) { + currentFilterFlowid = null; + continue; + } + + Matcher m1 = TC_HDFS_FILTER_INFO.matcher(line); + // May be our filter + if (m1.find()) { + mayBeOurFilter = true; + filterMinor = TCHandlerUtils.convertToDecimal(m1.group(4)) + - TCHandlerUtils.TC_FILTER_BASE_MINOR; + if (filterMinor >= 0) { + reservedFilterItems.add(filterMinor); + tcFilterExecutor = + new TCFilterExecutor(deviceName, qDiscHandle, -1, + currentFilterFlowid, m1.group(3), Integer.parseInt(m1 + .group(1))); + LOG.debug("Found old filter: " + + tcFilterExecutor); + } + } else { + currentFilterFlowid = null; + Matcher m2 = TC_OTHER_FILTER_INFO.matcher(line); + if (m2.find()) { + filterMinor = TCHandlerUtils.convertToDecimal(m2.group(4)) + - TCHandlerUtils.TC_FILTER_BASE_MINOR; + if (filterMinor >= 0) { + reservedFilterItems.add(filterMinor); + } + } + } + } + } else if (line.startsWith("match") + && mayBeOurFilter) { + Matcher m3 = TC_FILTER_MATCH_PORT.matcher(line); + if (m3.matches()) { + if (Integer.parseInt(m3.group(1), 16) == monitoringPort) { + foundSrcHdfsPort = true; + } + } else { + Matcher m4 = TC_FILTER_MATCH_IP.matcher(line); + if (m4.find()) { + foundIp = true; + } + } + } + } + + if (tcFilterExecutor != null) { + if (foundIp + && foundSrcHdfsPort) { + oldTcFilters.add(tcFilterExecutor); + } else if (currentFilterFlowid != null) { + oldTcClasses.remove(currentFilterFlowid); + } + } + + if (!oldTcFilters.isEmpty()) { + LOG.debug("Old tc filters: " + + oldTcFilters); + } + } + + private void removeOldTCSettings() { + LOG.debug("Remove old tc settings of " + + deviceName); + TCCommand tcCommand = new TCCommand(); + for (TCFilterExecutor tcFilter : oldTcFilters) { + tcCommand = tcFilter.delFilterCmd(); + if (exec(tcCommand) != 0) { + LOG.warn("Can not execute command " + + tcCommand); + } + } + + for (Entry entry : oldTcClasses.entrySet()) { + tcCommand.clear(); + tcCommand.add(String.format("class del dev %s classid %s", deviceName, + entry.getKey())); + LOG.debug("Delete old LTC class: " + + tcCommand); + if (exec(tcCommand) != 0) { + LOG.warn("Can not execute command " + + tcCommand); + } + } + + // oldTcClasses.clear(); + // oldTcFilters.clear(); + initialized = true; + LOG.debug("Remove old tc settings - DONE for " + + deviceName); + } + + /** + * @return the deviceName + */ + public String getName() { + return deviceName; + } + + int getFreeFilterItem() { + return firstFreeItemIndex(filterItemPool, reservedFilterItems); + } + + private int firstFreeItemIndex(List itemPool, + Set reservedIndexes) { + int i; + for (i = 0; i < itemPool.size(); i++) { + if (itemPool.get(i)) { + break; + } + } + + if (i == itemPool.size()) { + if (reservedIndexes.contains(i)) { + reservedIndexes.remove(i); + itemPool.add(false); + i++; + } + itemPool.add(false); + } else { + itemPool.set(i, false); + } + + return i; + } + + private void freeItem(List itemPool, int i) { + if (i >= 0) { + itemPool.set(i, true); + } + } + + void freeClassItemIndex(int i) { + freeItem(clsItemPool, i); + } + + void freeFilterItemIndex(int i) { + freeItem(filterItemPool, i); + } + + public void update(List events) { + if (!initialized) { + try { + removeOldTCSettings(); + } catch (Exception e) { + LOG.error("Error occured when removing old tc settings", e); + } + } + for (TrafficControlEvent event : events) { + if (canApply(event)) { + update(event); + } + } + } + + private boolean canApply(TrafficControlEvent event) { + return event.isLoopback() == isLoopBack; + } + + private void update(TrafficControlEvent event) { + if (event.getEventType() == TCEventType.ADD_FILTER) { + doAddFilter((TCFilterEvent) event); + } else if (event.getEventType() == TCEventType.DEL_FILTER) { + doDelFilter((TCFilterEvent) event); + } else if (event.getEventType() == TCEventType.ADD_CLASS) { + doAddClass((TCClassEvent) event); + } else if (event.getEventType() == TCEventType.DEL_CLASS) { + doDelClass((TCClassEvent) event); + } else if (event.getEventType() == TCEventType.CHANGE_CLASS) { + doChangeClass((TCClassEvent) event); + } + } + + private void doAddClass(TCClassEvent event) { + final String clsId = event.getContainerId(); + if (!tcClasses.containsKey(clsId)) { + // get class id + int i; + boolean newClassMinor = false; + for (i = 0; i < clsItemPool.size(); i++) { + if (clsItemPool.get(i)) { + break; + } + } + + if (i == clsItemPool.size()) { + if (reservedClassItems.contains(i)) { + reservedClassItems.remove(i); + clsItemPool.add(false); + i++; + } + newClassMinor = true; + clsItemPool.add(false); + + } else { + clsItemPool.set(i, false); + } + + TCClassExecutor cls = + new TCClassExecutor(this, qDiscHandle, i, event.getRate(), burst); + + if (cls.doAddClass(newClassMinor) == 0) { + tcClasses.put(clsId, cls); + LOG.debug("ClassId: " + + clsId + " -> add class " + cls.toString()); + } else { + LOG.error("Failed to create tcClass"); + return; + } + } + } + + private void doChangeClass(TCClassEvent event) { + final String clsId = event.getContainerId(); + final TCClassExecutor cls = tcClasses.get(clsId); + + if (cls != null) { + String oldRate = cls.getRate(); + if (cls.doAddClass(false) == 0) { + LOG.debug("ClassId: " + + clsId + " -> change rate from " + oldRate + " to: " + + cls.toString()); + } else { + LOG.error("Failed to change rate of tcClass"); + return; + } + } + } + + private void doDelClass(TCClassEvent event) { + final String containerId = event.getContainerId(); + final TCClassExecutor cls = tcClasses.get(containerId); + if (cls != null) { + if (cls.doDelClass() == 0) { + LOG.debug("ClassId" + + containerId + " -> del class " + cls.toString()); + } + tcClasses.remove(containerId); + } + } + + private void doAddFilter(TCFilterEvent event) { + final String clsId = event.getContainerId(); + TCClassExecutor cls = tcClasses.get(clsId); + if (cls != null) { + TCFilterExecutor filter = cls.addFilter(event); + if (filter != null) { + tcFilters.put(filter.getHandle(), filter); + } + } + } + + private void doDelFilter(TCFilterEvent event) { + TCFilterExecutor filter = findFilter(event); + if (filter == null) { + return; + } + + TCCommand cmd = filter.delFilterCmd(); + if (exec(cmd) == 0) { + filter.getOwnClass().removeFilter(filter); + freeFilterItemIndex(filter.getItemIndex()); + tcFilters.remove(filter.getHandle()); + } else { + LOG.warn("Cannot del filter: " + + filter.toString()); + } + } + + private TCFilterExecutor findFilter(TCFilterEvent event) { + for (TCFilterExecutor filter : tcFilters.values()) { + if (filter.match(event.getLocalPort(), event.getRemoteHost(), + event.getRemotePort())) { + return filter; + } + } + return null; + } + + void removeFilterFromList(String filterHandler) { + tcFilters.remove(filterHandler); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/CommonProcessTree.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/CommonProcessTree.java new file mode 100644 index 0000000..aeacf55 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/CommonProcessTree.java @@ -0,0 +1,119 @@ +/** + * 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.nodemanager.trafficcontrol.view; + +import java.util.HashSet; +import java.util.Set; + +/** + * Data class for storing the structure of /proc. + * + */ +public class CommonProcessTree { + private String pid; + private String parentPID; + + private Set children = new HashSet<>(); + + public CommonProcessTree(String pid, String ppid) { + this.pid = pid; + this.parentPID = ppid; + } + + /** + * @return the ppid + */ + public String getParentPID() { + return parentPID; + } + + /** + * @param ppid + * the ppid to set + */ + public void setParentPID(String ppid) { + this.parentPID = ppid; + } + + /** + * @return the pid + */ + public String getPid() { + return pid; + } + + /** + * @return the children + */ + public Set getChildren() { + return children; + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#hashCode() + */ + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime + * result + ((pid == null) ? 0 : pid.hashCode()); + return result; + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#equals(java.lang.Object) + */ + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + CommonProcessTree other = (CommonProcessTree) obj; + if (pid == null) { + if (other.pid != null) { + return false; + } + } else if (!pid.equals(other.pid)) { + return false; + } + return true; + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#toString() + */ + @Override + public String toString() { + return "CommonProcessTree [pid=" + + pid + ", ppid=" + parentPID + ", children=" + children + "]"; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/Connection.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/Connection.java new file mode 100644 index 0000000..2d55613 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/Connection.java @@ -0,0 +1,106 @@ +/** + * 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.nodemanager.trafficcontrol.view; + +/** + * Data class for an IPv4 connection. + * + */ +public class Connection { + private String srcHost; + private int srcPort; + private String dstHost; + private int dstPort; + + public Connection(String srcHost, int srcPort, String dstHost, int dstPort) { + this.srcHost = srcHost; + this.srcPort = srcPort; + this.dstHost = dstHost; + this.dstPort = dstPort; + } + + public Connection reverseConnection() { + return new Connection(dstHost, dstPort, srcHost, srcPort); + } + + public Connection cloneConnection() { + return new Connection(srcHost, srcPort, dstHost, dstPort); + } + + /** + * @return the srcHost + */ + public String getSrcHost() { + return srcHost; + } + + /** + * @return the srcPort + */ + public int getSrcPort() { + return srcPort; + } + + /** + * @return the dstHost + */ + public String getDstHost() { + return dstHost; + } + + /** + * @return the dstPort + */ + public int getDstPort() { + return dstPort; + } + + public boolean isLoopback() { + return srcHost.equals(dstHost); + } + + public String formatConnection() { + return String.format("%s:%d->%s:%d", srcHost, srcPort, dstHost, dstPort); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof Connection) { + Connection other = (Connection) obj; + return this.srcHost.equals(other.srcHost) + && this.srcPort == other.srcPort + && this.dstHost.equals(other.dstHost) + && this.dstPort == other.dstPort; + } + + return false; + } + + @Override + public String toString() { + return String.format("Connection: %s:%d->%s:%d", srcHost, srcPort, dstHost, + dstPort); + } + + @Override + public int hashCode() { + assert false : "hashCode not designed"; + return 42; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/ConnectionCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/ConnectionCollector.java new file mode 100644 index 0000000..b582eb0 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/ConnectionCollector.java @@ -0,0 +1,366 @@ +/** + * 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.nodemanager.trafficcontrol.view; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.net.Inet6Address; +import java.net.InetAddress; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; + +import com.google.common.net.InetAddresses; + +/** + * This class is responsible for retrieve network data for processes. + * + */ +public class ConnectionCollector { + + static final Log LOG = LogFactory.getLog(ConnectionCollector.class); + + public static final Pattern SS_ESTABLISHED_LINE_FORMAT = Pattern + .compile("\\s*\\d+\\s+\\d+\\s+" // recv-send data + + "([:0-9a-fA-F\\.]+):([0-9]+)\\s+" // source address + + "([:0-9a-fA-F\\.]+):([0-9]+)\\s+" // destination address + + "(timer:\\S+\\s+)?" // timer:(keepalive,120min,0) + + "(users:\\(\\(.+,(\\d+),\\d+\\)\\)\\s+)?" // + // users:(("java",14551,246)) + + "uid:(\\d+)\\s+" + "ino:(\\d+)" + ".+"); + + public static final Pattern PROC_NET_TCP_LINE_FORMAT = Pattern + .compile("\\s*\\d+:\\s+" // sl + + "([0-9A-F]+):" // 4 bytes of local ip + + "([0-9A-F]{4})" // Port of local ip + + "\\s+([0-9A-F]+):" // 4 bytes of remote ip + + "([0-9A-F]{4})" // Port of remote ip + + "\\s+([0-9A-F]{2})" // state + + "\\s+(\\d+):(\\d+)" // tx_queue:rx_queue + + "\\s+(\\d+):(\\d+)" // tr:tm->when + + "\\s+(\\d+)" // retrnsmt + + "\\s+(\\d+)" // uid + + "\\s+(\\d+)" // timeout + + "\\s+(\\d+)" // inode + + "(.+)"); + + static final String[] STATES = new String[] {"UNDEF", "ESTABLISHED", + "SYN_SENT", "SYN_RECV", "FIN_WAIT1", "FIN_WAIT2", "TIME_WAIT", "CLOSE", + "CLOSE_WAIT", "LAST_ACK", "LISTEN", "CLOSING"}; + + private String ssCommand; + private int monitoringPort = 0; + private boolean isDataNode = false; + + public ConnectionCollector(int monitoringPort, boolean isDataNode) { + this.monitoringPort = monitoringPort; + this.isDataNode = isDataNode; + if (isDataNode) { + ssCommand = + String.format("ss -noept state established src *:%d", monitoringPort); + } else { + ssCommand = + String.format("ss -noept state established dst *:%d", monitoringPort); + } + + LOG.debug("ss command: " + + ssCommand); + } + + public void sudoIsAvailabe() { + if (isDataNode) { + return; + } + ssCommand = String.format("sudo %s", ssCommand); + LOG.debug("ss command: " + + ssCommand); + } + + /** + * Collect connections from the /proc file system or the output of the ss + * program. + * + * @param useSS + * @param connections + * @param inodes + */ + public void collectConnections(boolean useSS, + Map connections, Map> inodes) { + if (useSS) { + connections.putAll(readEstablishedConsWithSS(inodes)); + } else { + connections.putAll(readProcNetTCP(FactoryHelper.getInstance() + .getProcFsRoot())); + } + } + + /** + * Collect all established from or to the monitoring port. + * + * @param useSS + * @param connections + */ + public void collectConnections(boolean useSS, + Map connections) { + collectConnections(useSS, connections, null); + } + + /** + * @return the monitoringPort + */ + public int getMonitoringPort() { + return monitoringPort; + } + + public Map readEstablishedConsWithSS( + Map> pidInodes) { + Map connections = new HashMap(); + try (BufferedReader reader = executeCommand(ssCommand)) { + String line; + while ((line = reader.readLine()) != null) { + Matcher m = SS_ESTABLISHED_LINE_FORMAT.matcher(line); + if (m.matches()) { + // LOG.info("Matched: " + line); + final String localIp = getIPv4Address(m.group(1)); + final String remoteIp = getIPv4Address(m.group(3)); + + Connection connection = + new Connection(localIp, Integer.parseInt(m.group(2)), remoteIp, + Integer.parseInt(m.group(4))); + + final String inode = m.group(9); + connections.put(inode, connection); + + if (pidInodes != null) { + try { + String pidStr = m.group(7); + if (Integer.parseInt(pidStr) > 0) { + if (!pidInodes.containsKey(pidStr)) { + pidInodes.put(pidStr, new HashSet()); + } + pidInodes.get(pidStr).add(inode); + } + } catch (NumberFormatException e) { + ; + } + } + } + } + } catch (Exception e) { + LOG.error(e, e); + } + + return connections; + } + + private static String getIPv4Address(String ipAddressStr) { + String ip = ipAddressStr; + if (ipAddressStr.contains(":")) { + ip = checkAndConvertIPv6to4Address(ipAddressStr); + } + + return ip; + } + + private BufferedReader executeCommand(String cmd) throws IOException, + InterruptedException { + final String[] cmdargs = new String[] {"/bin/sh", "-c", cmd}; + ProcessBuilder builder = new ProcessBuilder(cmdargs); + builder.redirectErrorStream(true); + Process process = builder.start(); + process.waitFor(); + return new BufferedReader(new InputStreamReader(process.getInputStream(), + StandardCharsets.UTF_8)); + } + + public Map readProcNetTCP(String procfsDir) { + Map connections = new HashMap(); + connections.putAll(parseProcNetTCP(Paths.get(procfsDir, "net", "tcp"), + isDataNode, monitoringPort)); + + Path path = Paths.get(procfsDir, "net", "tcp6"); + if (Files.exists(path)) { + connections.putAll(parseProcNetTCP(path, isDataNode, monitoringPort)); + } + + return connections; + } + + /** + * Collect connections from or to a given monitoring port. + * + * @param path + * file to read + * @param isDatanode + * indicates whether it is a DataNode or a NodeManager + * @param portToCheck + * the monitoring port + * @return the map of connections and the related inode + */ + public Map parseProcNetTCP(Path path, boolean isDatanode, + int portToCheck) { + String line; + Map connections = new HashMap(); + try (BufferedReader reader = + Files.newBufferedReader(path, StandardCharsets.UTF_8)) { + while ((line = reader.readLine()) != null) { + Object[] objs = parseProcNetTCPLine(line, portToCheck, isDatanode); + if (objs != null) { + connections.put((String) objs[0], (Connection) objs[1]); + } + } + } catch (IOException e) { + LOG.error(e, e); + } + return connections; + } + + private static Object[] parseProcNetTCPLine(String line, int portToCheck, + boolean isDatanode) { + Matcher m = PROC_NET_TCP_LINE_FORMAT.matcher(line); + + if (m.matches()) { + if (!getState(m.group(5)).equals("ESTABLISHED")) { + return null; + } + int localPort = getPortFromHex(m.group(2)); + int remotePort = getPortFromHex(m.group(4)); + if (isDatanode + && localPort != portToCheck) { + return null; + } else if (!isDatanode + && remotePort != portToCheck) { + return null; + } + + final String inode = m.group(13); + String localIP = getIPv4AddressFromHex(m.group(1)); + String remoteIP = getIPv4AddressFromHex(m.group(3)); + + return new Object[] {inode, + new Connection(localIP, localPort, remoteIP, remotePort)}; + } + + return null; + } + + private static String getIPv4AddressFromHex(String hexIpRecord) { + String ip = null; + if (hexIpRecord.length() == 8) { + ip = toIPv4AddrString(hexIpRecord); + } else { + ip = checkAndConvertIPv6to4Address(toIPv6AddrString(hexIpRecord)); + } + return ip; + } + + /** + * Convert a record of IP (has length of 8 bytes) in /proc/net/tcp. + * + * @param hexStr + * record of IPv4 in the hex format + * @return the valid representation of IPv4 + */ + public static String toIPv4AddrString(String hexStr) { + long ipa = Long.parseLong(hexStr, 16); + StringBuilder b = new StringBuilder(); + b.append(Long.toString(0x000000ff & (ipa))); + b.append("."); + b.append(Long.toString(0x000000ff & (ipa >> 8))); + b.append("."); + b.append(Long.toString(0x000000ff & (ipa >> 16))); + b.append("."); + b.append(Long.toString(0x000000ff & (ipa >> 24))); + return b.toString(); + } + + private static int getPortFromHex(String hexPort) { + return hex2Int(hexPort); + } + + private static String getState(String hexPort) { + int index = hex2Int(hexPort); + if (index >= STATES.length) { + index = 0; + } + return STATES[index]; + } + + private static int hex2Int(String hexStr) { + return Integer.parseInt(hexStr, 16); + } + + /** + * Convert the record of IPv6 (has length of 32 bytes) in /proc/net/tcp6 to + * the form of the IPv6 address. + * + * @param rawNetTcpIpStr + * the record of IPv6 in the hex format + * @return the valid representation of IPv6 + */ + public static String toIPv6AddrString(String rawNetTcpIpStr) { + StringBuilder result = new StringBuilder(); + final char[] hexChars = rawNetTcpIpStr.trim().toCharArray(); + for (int i = 0; i < 4; i++) { + for (int j = 3; j >= 0; j--) { + int index = (8 * i + 2 * j); + result.append(hexChars[index]).append(hexChars[index + 1]); + result.append((j == 2) ? ":" : ""); + } + result.append((i < 3) ? ":" : ""); + } + return result.toString(); + } + + /** + * Convert an IPv6 into the IPv4 format. + * + * @param ipv6AddrStr + * the IPv6 record + * @return an IPv4 address + */ + public static String checkAndConvertIPv6to4Address(String ipv6AddrStr) { + InetAddress inetAddr = InetAddresses.forString(ipv6AddrStr); + if (InetAddresses.isMappedIPv4Address(ipv6AddrStr)) { + return InetAddresses.toAddrString(inetAddr); + } else if (inetAddr instanceof Inet6Address) { + Inet6Address inet6Addr = (Inet6Address) inetAddr; + if (InetAddresses.hasEmbeddedIPv4ClientAddress(inet6Addr)) { + return InetAddresses.getEmbeddedIPv4ClientAddress(inet6Addr) + .getHostAddress(); + } + + return InetAddresses.toAddrString(inetAddr); + } + + return null; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/ContainerProcessTree.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/ContainerProcessTree.java new file mode 100644 index 0000000..ef9e953 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/ContainerProcessTree.java @@ -0,0 +1,150 @@ +/** + * 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.nodemanager.trafficcontrol.view; + +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; + +/** + * This class contains process data (process id, child processes and inodes) of + * a NM container. + * + */ +public class ContainerProcessTree { + + private static final Log LOG = LogFactory.getLog(ContainerProcessTree.class); + + private String containerId; + + private CommonProcessTree processTree; + + private Map containerPIDTree = new HashMap<>(); + + private Set inodes = new HashSet(); + + public ContainerProcessTree(String containerId, CommonProcessTree tree) { + this.processTree = tree; + this.containerId = containerId; + } + + /** + * @return the processTree + */ + public CommonProcessTree getProcessTree() { + return processTree; + } + + public void buildPIDTree(boolean excludeRootPID) { + containerPIDTree.clear(); + inodes.clear(); + + LinkedList allChildrenQueue = new LinkedList<>(); + allChildrenQueue.add(processTree); + + while (!allChildrenQueue.isEmpty()) { + CommonProcessTree child = allChildrenQueue.remove(); + String childPid = child.getPid(); + + if (!containerPIDTree.containsKey(childPid)) { + containerPIDTree.put(childPid, child); + } + allChildrenQueue.addAll(child.getChildren()); + } + } + + public Set getCurrentPIDList() { + return containerPIDTree.keySet(); + } + + /** + * Collect inodes of this process tree. Note that only /proc/PID/fd can only + * walked by the process' owner. + */ + public void collectInodes() { + inodes.clear(); + for (String pid : containerPIDTree.keySet()) { + inodes.addAll(collectInodesOfPID(pid)); + } + } + + /** + * Collect all socket inodes of the given PID. + * + * @param pid + * the porocess id + * @return the list of socket inodes of the given pid + */ + public Set collectInodesOfPID(String pid) { + Set inodes = new HashSet(); + try (DirectoryStream directoryStream = + Files.newDirectoryStream(Paths.get(FactoryHelper.getInstance() + .getProcFsRoot(), pid, "fd"))) { + for (Path link : directoryStream) { + // if (Files.isSymbolicLink(link)) { + String inode = Files.readSymbolicLink(link).toString(); + // socket:[21831257] + if (inode.startsWith("socket")) { + inodes.add(inode.replace("socket:[", "").replace("]", "")); + } + // } + } + } catch ( + NoSuchFileException | AccessDeniedException e) { + ;// When process is complete + } catch (IOException ex) { + LOG.error("Cannot read inoded of process " + + pid, ex); + } + + return inodes; + } + + /** + * @return the inodes + */ + public Set getInodes() { + return inodes; + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#toString() + */ + @Override + public String toString() { + return "ContainerProcessTree [containerId=" + + containerId + ", pTree=" + processTree + ", childPids=" + + containerPIDTree.keySet() + "]"; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/DNContainerConnections.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/DNContainerConnections.java new file mode 100644 index 0000000..df96654b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/DNContainerConnections.java @@ -0,0 +1,127 @@ +/** + * 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.nodemanager.trafficcontrol.view; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +/** + * This class storing the connections between one NM container and one DN host. + * + */ +public class DNContainerConnections { + static final Log LOG = LogFactory.getLog(DNContainerConnections.class); + + private String containerId; + private String rate; + + private List connections; + + public DNContainerConnections(String containerId, String rate) { + this.containerId = containerId; + this.rate = rate; + this.connections = new ArrayList(); + } + + /** + * @return the containerId + */ + public String getContainerId() { + return containerId; + } + + /** + * @return the rate + */ + public String getRate() { + return rate; + } + + /** + * @return the connections + */ + public List getConnections() { + return connections; + } + + public DNContainerConnections cloneContainer() { + DNContainerConnections newOne = + new DNContainerConnections(containerId, rate); + for (Connection con : connections) { + newOne.getConnections().add(con.cloneConnection()); + } + + return newOne; + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#hashCode() + */ + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime + * result + ((containerId == null) ? 0 : containerId.hashCode()); + return result; + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#equals(java.lang.Object) + */ + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + DNContainerConnections other = (DNContainerConnections) obj; + if (containerId == null) { + if (other.containerId != null) { + return false; + } + } else if (!containerId.equals(other.containerId)) { + return false; + } + return true; + } + + /* + * (non-Javadoc) + * + * @see java.lang.Object#toString() + */ + @Override + public String toString() { + return "DNContainerConnections [containerId=" + + containerId + ", rate=" + rate + ", connections=" + connections + "]"; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/NMContainerConnections.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/NMContainerConnections.java new file mode 100644 index 0000000..a435682 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/view/NMContainerConnections.java @@ -0,0 +1,315 @@ +/** + * 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.nodemanager.trafficcontrol.view; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; + +/** + * Class defined to handle connections of a given container and its child + * processes. The TC events (class, filter) will be constructed based on the + * collected connections. + * + * E.g.: tc class add dev eth0 parent 1:1 classid 1:20 htb rate 3mbit + * + * + */ +public class NMContainerConnections { + + static final Log LOG = LogFactory.getLog(NMContainerConnections.class); + + private String containerId; + private int monitoringPort; + private String rate = null; + + private boolean initialized = false; + + // Cache all traced actual connections + private List connections; + + private Map dnConnectionsMap; + private Set updatedHostsList; + + private ContainerProcessTree processTree; + + private boolean isCompleted = false; + + public NMContainerConnections(String containerId, String rate, int port) { + this.containerId = containerId; + this.rate = rate; + this.monitoringPort = port; + + connections = new ArrayList(); + dnConnectionsMap = Maps.newHashMap(); + updatedHostsList = new HashSet(); + } + + public String getContainerId() { + return containerId; + } + + /** + * @param processTree + * the processTree to set + */ + public void setProcessTree(ContainerProcessTree processTree) { + this.processTree = processTree; + if (this.processTree != null) { + this.processTree.buildPIDTree(false); + } + } + + public boolean updateProcessTree(CommonProcessTree commonProcessTree) { + boolean hasChanges = false; + if (commonProcessTree == null) { + processTree = null; + return false; + } + + if (processTree == null + || processTree.getProcessTree() == null + || processTree.getProcessTree() != commonProcessTree) { + processTree = new ContainerProcessTree(containerId, commonProcessTree); + hasChanges = true; + } + + final Set oldPidsList = + new HashSet(processTree.getCurrentPIDList()); + + processTree.buildPIDTree(false); + + if (!oldPidsList.containsAll(processTree.getCurrentPIDList()) + || !processTree.getCurrentPIDList().containsAll(oldPidsList)) { + hasChanges = true; + } + if (hasChanges + && LOG.isDebugEnabled()) { + LOG.debug(containerId + + ": pidsList: " + + Joiner.on(",").join(processTree.getCurrentPIDList())); + } + return hasChanges; + } + + /** + * @return the rate (may be null) + */ + public String getRate() { + return rate; + } + + /** + * @param rate + * the rate to set + */ + public void setRate(String rate) { + this.rate = rate; + } + + /** + * @return the isCompleted + */ + public boolean isCompleted() { + return isCompleted; + } + + /** + * @return the initialized + */ + public boolean isInitialized() { + return initialized; + } + + /** + * Indicate that the container is fully initialized. + */ + public void initialized() { + this.initialized = true; + } + + public void collectInodes() { + if (processTree != null) { + processTree.collectInodes(); + } + } + + @VisibleForTesting + public void collectConnections(Map allConnections) { + collectConnections(allConnections, null); + } + + /** + * @return the processTree + */ + public ContainerProcessTree getProcessTree() { + return processTree; + } + + public void collectConnections(Map allConnections, + Map> pidInodes) { + + List myConnections = new ArrayList(); + + if (getProcessTree() == null) { + if (initialized) { + resetContainerConnections(); + } + return; + } + + if (Strings.isNullOrEmpty(rate) + || isCompleted) { + return; + } + + Set inodes = getProcessTree().getInodes(); + if (pidInodes != null) { + for (String pid : getProcessTree().getCurrentPIDList()) { + if (pidInodes.containsKey(pid)) { + inodes.addAll(pidInodes.get(pid)); + } + } + } + + for (String inode : inodes) { + if (allConnections.containsKey(inode)) { + myConnections.add(allConnections.get(inode).cloneConnection()); + } + } + syncConnections(myConnections); + } + + private void syncConnections(List parsedConnections) { + List currentConnections = new ArrayList(); + synchronized (this.dnConnectionsMap) { + Set disconnectedHosts = + new HashSet(dnConnectionsMap.keySet()); + dnConnectionsMap.clear(); + + for (Connection connection : parsedConnections) { + final String remoteHost = connection.getDstHost(); + disconnectedHosts.remove(remoteHost); + + if (!dnConnectionsMap.containsKey(remoteHost)) { + dnConnectionsMap.put(remoteHost, new DNContainerConnections( + containerId, rate)); + } + + // The connection is already traced + if (connections.contains(connection)) { + connections.remove(connection); + } else { + + updatedHostsList.add(remoteHost); + if (LOG.isDebugEnabled()) { + LOG.debug(containerId + + ": Detect new connection " + connection.formatConnection()); + } + } + + dnConnectionsMap.get(remoteHost).getConnections() + .add(connection.reverseConnection()); + + currentConnections.add(connection); + } + + for (Connection disconnected : connections) { + updatedHostsList.add(disconnected.getDstHost()); + + if (LOG.isDebugEnabled()) { + LOG.debug(containerId + + ": Detect disconnected connection " + + disconnected.formatConnection()); + } + } + + updatedHostsList.addAll(disconnectedHosts); + connections = new ArrayList(currentConnections); + } + } + + private void resetContainerConnections() { + synchronized (this.dnConnectionsMap) { + updatedHostsList = new HashSet(dnConnectionsMap.keySet()); + dnConnectionsMap.clear(); + connections.clear(); + } + } + + public void stopTrackContainer() { + resetContainerConnections(); + isCompleted = true; + } + + public void collectUpdatedHosts(Set updatedHosts, + Set connectedHosts) { + updatedHosts.addAll(updatedHostsList); + connectedHosts.addAll(dnConnectionsMap.keySet()); + } + + public void commitChanges(Map> dnData, + Set submitHostList) { + synchronized (this.dnConnectionsMap) { + for (String updatedDNHost : submitHostList) { + + if (!dnData.containsKey(updatedDNHost)) { + dnData.put(updatedDNHost, new ArrayList()); + } + + DNContainerConnections updatedSnapshot = + dnConnectionsMap.get(updatedDNHost); + List list = dnData.get(updatedDNHost); + + // remove from cache + if (updatedSnapshot == null + || isCompleted) { + list.remove(new DNContainerConnections(containerId, rate)); + } else { + DNContainerConnections cloneSnapshot = + updatedSnapshot.cloneContainer(); + int index = list.indexOf(updatedSnapshot); + if (index > -1) { + list.set(index, cloneSnapshot); + } else { + list.add(cloneSnapshot); + } + } + } + + updatedHostsList.clear(); + } + } + + public String toString() { + return String.format( + "[NMContainerConnections] clsId: %s, rate: %s, hdfsPort: %d]", + containerId, rate, monitoringPort); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestConnectionCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestConnectionCollector.java new file mode 100644 index 0000000..ea0c8b3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestConnectionCollector.java @@ -0,0 +1,78 @@ +/** + * 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.nodemanager.trafficcontrol; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; +import org.junit.Assert; +import org.junit.Test; + +public class TestConnectionCollector { + + private String testIP = "10.10.0.116"; + + @Test + public void testParsingIPv6MappedFormat() { + String ipAddr = "::ffff:" + + testIP; + Assert.assertTrue(ipAddr + + " is IPv4 mapped", + ConnectionCollector.checkAndConvertIPv6to4Address(ipAddr) + .equals(testIP)); + } + + @Test + public void testParsingIPv6CompatibleFormat() { + String ipAddr = "::" + + testIP; + Assert.assertTrue(ipAddr + + " is IPv4 compatible", ConnectionCollector + .checkAndConvertIPv6to4Address(ipAddr).equals(testIP)); + } + + @Test + public void testParsingIPv6to4Format() { + String ipAddr = "2002:0a0a:0074::"; + Assert.assertTrue(ipAddr + + " is IPv4 compatible", ConnectionCollector + .checkAndConvertIPv6to4Address(ipAddr).equals(testIP)); + } + + @Test + public void testParsingIPOfNetTCP() { + String hexStr = "74000A0A"; + Assert.assertTrue(hexStr + + " hex string should be converted to " + testIP, ConnectionCollector + .toIPv4AddrString(hexStr).equals(testIP)); + } + + @Test + public void testParsingIPOfNetTCP6() { + String hexStr = "0000000000000000FFFF000074000A0A"; + String expectedIPv6 = "0000:0000:0000:0000:0000:FFFF:0A0A:0074"; + String convertedIP = ConnectionCollector.toIPv6AddrString(hexStr); + Assert.assertTrue(hexStr + + " hex string should be converted to " + expectedIPv6, + convertedIP.equals(expectedIPv6)); + + Assert.assertTrue("The IP should be " + + testIP, ConnectionCollector + .checkAndConvertIPv6to4Address(convertedIP).equals(testIP)); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestContainerProcessTree.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestContainerProcessTree.java new file mode 100644 index 0000000..eba16c7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestContainerProcessTree.java @@ -0,0 +1,93 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionMonitor; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.CommonProcessTree; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.NMContainerConnections; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class TestContainerProcessTree { + + private Map procFsTree = new HashMap<>(); + + @Test + public void testRegisterContainerWithExistedPid() { + procFsTree.clear(); + + String firstPid = "100"; + + ProcBasedConnectionMonitor.updateProcFsTree(firstPid, "1", procFsTree); + ProcBasedConnectionMonitor.updateProcFsTree("101", firstPid, procFsTree); + ProcBasedConnectionMonitor.updateProcFsTree("1001", "101", procFsTree); + ProcBasedConnectionMonitor.updateProcFsTree("1002", "101", procFsTree); + ProcBasedConnectionMonitor.updateProcFsTree("102", firstPid, procFsTree); + + String containerId = "clsId_1"; + NMContainerConnections nmView = + new NMContainerConnections(containerId, "30mbps", 50010); + + boolean updated = false; + Set pidList; + + // We should have 5 pids for this container + updated = nmView.updateProcessTree(procFsTree.get(firstPid)); + pidList = nmView.getProcessTree().getCurrentPIDList(); + Assert.assertTrue(updated); + Assert.assertTrue(pidList.size() == 5); + + // NMContainer must remove invalid pids when the pids are not existed + // anymore + ProcBasedConnectionMonitor.removeOldPid("1001", procFsTree); + updated = nmView.updateProcessTree(procFsTree.get(firstPid)); + pidList = nmView.getProcessTree().getCurrentPIDList(); + Assert.assertTrue(updated); + Assert.assertTrue(pidList.size() == 4 + && !pidList.contains("1001")); + + // Removing pid 101 must also remove its children + ProcBasedConnectionMonitor.removeOldPid("101", procFsTree); + updated = nmView.updateProcessTree(procFsTree.get(firstPid)); + pidList = nmView.getProcessTree().getCurrentPIDList(); + Assert.assertTrue(updated); + Assert.assertTrue("", pidList.size() == 2 + && pidList.contains(firstPid) && pidList.contains("102")); + + // Nothing changes + updated = nmView.updateProcessTree(procFsTree.get(firstPid)); + Assert.assertFalse(updated); + + // New pids must be detected + ProcBasedConnectionMonitor.updateProcFsTree("103", firstPid, procFsTree); + ProcBasedConnectionMonitor.updateProcFsTree("2002", "102", procFsTree); + updated = nmView.updateProcessTree(procFsTree.get(firstPid)); + pidList = nmView.getProcessTree().getCurrentPIDList(); + Assert.assertTrue(updated); + Assert.assertTrue("", pidList.size() == 4 + && pidList.contains("2002") && pidList.contains("103")); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestContainerService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestContainerService.java new file mode 100644 index 0000000..029084a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestContainerService.java @@ -0,0 +1,178 @@ +/** + * 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.nodemanager.trafficcontrol; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.DummyContainerService; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionHandler; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionMonitor; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.NMContainerConnections; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class TestContainerService { + + @Mock + FactoryHelper helper; + + @Mock + AbstractTCDataSubmitter submitter; + + DummyContainerService containerService; + + @Rule + public TemporaryFolder procFsRoot = new TemporaryFolder(); + + Configuration conf; + String cid1 = "cls1111"; + String cid2 = "cls2222"; + String invalidClsId = "23Invalid-"; + + ProcBasedConnectionHandler connectionHandler; + + @Before + public void setup() throws IOException { + conf = spy(new Configuration()); + containerService = new DummyContainerService(conf); + + when(helper.isTCApplicable()).thenReturn(true, true, true); + when(helper.getTCDataSubmitter(any(Configuration.class))).thenReturn( + submitter); + + connectionHandler = new ProcBasedConnectionHandler(conf, helper); + connectionHandler.registerContainerService(containerService); + connectionHandler + .setConnectionMonitor(new ProcBasedConnectionMonitor(conf)); + } + + @Test + public void testInvalidContainerShouldNotBeAdded() { + boolean added = containerService.addMonitoringContainer(invalidClsId, 50); + Assert.assertFalse("Container is not added", added); + } + + @Test + public void testContainerWithoutRateShouldNotBeAdded() { + boolean added = containerService.addMonitoringContainer(cid1, 0); + Assert.assertFalse("Container is not added", added); + } + + @Test + public void testAddValidContainer() { + boolean added = containerService.addMonitoringContainer(cid1, 50); + Assert.assertTrue("Container must be added", added); + Assert.assertTrue(containerService.getUnModifableMapOfContainers() + .containsKey(cid1)); + } + + @Test + public void testAddAlreadyAddedContainerByOther() { + connectionHandler.addTcClassView(new NMContainerConnections(cid1, "50mbps", + 50010)); + + boolean added = containerService.addMonitoringContainer(cid1, 50); + Assert.assertFalse("Container cannot be added if the same" + + " containerId is already registered by other frameworks", added); + Assert.assertTrue(containerService.getUnModifableMapOfContainers() + .isEmpty()); + } + + @Test + public void testAddTwiceValidContainerShouldFail() { + containerService.addMonitoringContainer(cid1, 50); + + // Add twice + boolean added = containerService.addMonitoringContainer(cid1, 70); + Assert.assertFalse("Container cannot be added twice", added); + } + + @Test + public void testRegisterPidOfNonAddedContainer() { + boolean ok = containerService.registerPid(cid1, 12); + Assert.assertFalse("Pid of none regsistered container cannot be added", ok); + } + + @Test + public void testRegisterPidWithInvalidData() { + boolean ok = containerService.registerPid(invalidClsId, 12); + Assert.assertFalse("Pid of an invalid container cannot be added", ok); + + ok = containerService.registerPid(cid1, 1); + Assert.assertFalse("Invalid Pid cannot be added", ok); + } + + @Test + public void testRegisterPidOfAnAddedContainer() throws IOException { + containerService.addMonitoringContainer(cid1, 50); + + procFsRoot.newFile("12"); + FactoryHelper.getInstance().setProcFsRoot(procFsRoot.getRoot().getPath()); + + boolean ok = containerService.registerPid(cid1, 12); + Assert.assertTrue("Pid should be added", ok); + + ok = containerService.registerPid(cid1, 13); + Assert.assertFalse("We cannot report pid twice for one container", ok); + } + + @Test + public void testStopMonitoringContainerByExternalPlugin() throws IOException { + containerService.addMonitoringContainer(cid1, 50); + Assert.assertTrue(containerService.getUnModifableMapOfContainers() + .containsKey(cid1)); + containerService.stopMonitoringContainer(cid1); + Assert.assertTrue(containerService.getUnModifableMapOfContainers() + .isEmpty()); + } + + @Test + public void testStopMonitoringContainerByConnectionHandler() + throws IOException { + containerService.addMonitoringContainer(cid1, 50); + Assert.assertTrue(containerService.getUnModifableMapOfContainers() + .containsKey(cid1)); + + connectionHandler.removeTcClassView(cid1); + Assert.assertTrue(containerService.getUnModifableMapOfContainers() + .isEmpty()); + + // It is safe to remove once more + containerService.stopMonitoringContainer(cid1); + } + + @After + public void tearDown() { + FactoryHelper.getInstance().setProcFsRoot("/proc"); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestParsingProcNetTCP.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestParsingProcNetTCP.java new file mode 100644 index 0000000..c50a843 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestParsingProcNetTCP.java @@ -0,0 +1,172 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import com.google.common.base.Joiner; + +public class TestParsingProcNetTCP { + + private Joiner joiner = Joiner.on(" ").skipNulls(); + + @Rule + public TemporaryFolder procFsRoot = new TemporaryFolder(); + + private int hdfsPort = 50010; + + private String procRootDir; + + private Path procNetTCPPath; + + @Before + public void setup() throws IOException { + procFsRoot.newFolder("net"); + procRootDir = procFsRoot.getRoot().getPath(); + procNetTCPPath = Paths.get(procRootDir, "net", "tcp"); + + StringBuffer bf = new StringBuffer(""); + bf.append(joiner.join(" sl local_address rem_address", + "st tx_queue rx_queue tr tm->when retrnsmt", "uid timeout inode\n")); + bf.append(joiner.join(" 0: 00000000:C396 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "1001 0 14595 1 0000000000000000 100 0 0 10 0\n")); + + bf.append(joiner.join(" 1: 00000000:0016 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "0 0 10095 1 0000000000000000 100 0 0 10 0\n")); + + bf.append(joiner.join(" 2: 00000000:1F98 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "1001 0 22466511 1 0000000000000000 100 0 0 10 0\n")); + + bf.append(joiner.join(" 3: 00000000:27D8 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "1001 0 26014 1 0000000000000000 100 0 0 10 0\n")); + bf.append(joiner.join(" 4: 00000000:34FA 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "1001 0 22474669 1 0000000000000000 100 0 0 10 0\n")); + bf.append(joiner.join(" 5: 00000000:C35A 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "1001 0 14858 1 0000000000000000 100 0 0 10 0\n")); + bf.append(joiner.join(" 6: 00000000:C39B 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "1001 0 14889 1 0000000000000000 100 0 0 10 0\n")); + + bf.append(joiner.join(" 19: 00000000:C3AA 00000000:0000 0A", + "00000000:00000000 00:00000000 00000000", + "1001 0 11095 1 0000000000000000 100 0 0 10 0\n")); + bf.append(joiner.join(" 20: 74000A0A:2328 74000A0A:90B0 01", + "00000000:00000000 02:000AFFB2 00000000", + "1001 0 22486882 2 0000000000000000 20 4 27 10 7\n")); + bf.append(joiner.join(" 21: 74000A0A:2328 74000A0A:E34C 01", + "00000000:00000000 02:000AD94B 00000000", + "1001 0 17267 2 0000000000000000 20 4 27 10 -1 7\n")); + bf.append(joiner.join(" 23: 74000A0A:1F5F 74000A0A:A509 01", + "00000000:00000000 02:000A8C7E 00000000", + "1001 0 22474671 4 0000000000000000 20 4 33 10 7\n")); + bf.append(joiner.join(" 24: 74000A0A:C35A 74000A0A:BC1F 01", + "00000000:00000000 00:00000000 00000000", + "1001 0 22483476 1 0000000000000000 20\n")); + bf.append(joiner.join(" 22: 74000A0A:A509 74000A0A:1F5F 01", + "00000000:00000000 02:000A8C7E 00000000", + "1001 0 22475090 3 0000000000000000 20 4 30 10 4 20 10 7\n")); + bf.append(joiner.join(" 25: 74000A0A:BC18 74000A0A:C35A 08", + "00000000:00000001 00:00000000 00000000", + "1001 0 22477456 1 0000000000000000 20 4 16 10 7\n")); + bf.append(joiner.join(" 26: 74000A0A:2328 74000A0A:9071 01", + "00000000:00000000 02:000A8C7E 00000000", + "1001 0 22474805 2 0000000000000000 22 4 29 10 7\n")); + bf.append(joiner.join(" 27: 74000A0A:0016 50000A0A:CD9B 01", + "00001240:00000000 01:00000001 00000000", + "0 0 22180097 4 0000000000000000 20 4 15 8 6\n")); + bf.append(joiner.join(" 28: 74000A0A:BC1B 74000A0A:C35A 08", + "00000000:00000001 00:00000000 00000000", + "1001 0 22477509 1 0000000000000000 20 4 16 10 7\n")); + bf.append(joiner.join(" 29: 74000A0A:9071 74000A0A:2328 01", + "00000000:00000000 02:000A8C7E 00000000", + "1001 0 22474154 2 0000000000000000 20 4 28 10 7\n")); + bf.append(joiner.join(" 30: 74000A0A:1F98 50000A0A:D07C 01", + "00000000:00000000 00:00000000 00000000", + "1001 0 22486876 1 0000000000000000 20 4 30 39 20\n")); + bf.append(joiner.join(" 31: 74000A0A:0016 5A00000A:DC0D 01", + "00000000:00000000 02:0008A618 00000000", + "0 0 22443426 2 0000000000000000 21 4 17 10 -1\n")); + bf.append(joiner.join(" 32: 74000A0A:0016 50000A0A:CCF2 01", + "00000000:00000000 02:0009F2E5 00000000", + "0 0 22156060 2 0000000000000000 20 4 1 10 7\n")); + bf.append(joiner.join(" 33: 74000A0A:0016 5A00000A:DC0B 01", + "00000000:00000000 02:00080C7E 00000000", + "0 0 22438554 2 0000000000000000 21 4 23 10 -1\n")); + bf.append(joiner.join(" 34: 74000A0A:90AF 74000A0A:2328 06", + "00000000:00000000 03:000000B2 00000000", + "0 0 0 3 0000000000000000\n")); + bf.append(joiner.join(" 35: 74000A0A:0016 13000A0A:C833 01", + "00000000:00000000 02:0008194B 00000000", + "0 0 22438656 2 0000000000000000 22 4 9 18 17\n")); + bf.append(joiner.join(" 36: 74000A0A:BC1F 74000A0A:C35A 01", + "00000000:00000000 00:00000000 00000000", + "1001 0 22481365 1 0000000000000000 22 4 29 10 7\n")); + bf.append(joiner.join(" 37: 74000A0A:BC1C 74000A0A:C35A 08", + "00000000:00000001 00:00000000 00000000", + "1001 0 22482484 1 0000000000000000 21 4 20 10 7\n")); + bf.append(joiner.join(" 38: 74000A0A:90B0 74000A0A:2328 01", + "00000000:00000000 02:000AFFB2 00000000", + "1001 0 22488131 2 0000000000000000 20 4 30 10 7\n")); + bf.append(joiner.join(" 39: 74000A0A:E34C 74000A0A:2328 01", + "00000000:00000000 02:000AD94B 00000000", + "1001 0 14898 2 0000000000000000 20 4 30 10 -1\n")); + + bf.append(joiner.join(" 40: 0000000000000000FFFF000074000A0A:0885", + "0000000000000000FFFF000074000A0A:E78A 01", + "00000000:00000000 00:00000000 00000000", + "1001 0 171593978 1 0000000000000000 20 4 31 10 -1\n")); + + Files.write(procNetTCPPath, bf.toString().getBytes()); + } + + @Test + public void parseProcNetTCPForOutgoingConnections() throws IOException { + ConnectionCollector collector = new ConnectionCollector(hdfsPort, false); + Map cons = + collector.parseProcNetTCP(procNetTCPPath, false, hdfsPort); + + Assert.assertTrue("We have one HDFS connection", cons.size() == 1); + } + + @Test + public void parseProcNetTCPForIncommingConnections() throws IOException { + ConnectionCollector collector = new ConnectionCollector(hdfsPort, true); + Map cons = + collector.parseProcNetTCP(procNetTCPPath, true, hdfsPort); + Assert.assertTrue("We have one HDFS connection", cons.size() == 1); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestParsingSSContent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestParsingSSContent.java new file mode 100644 index 0000000..30282b6 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestParsingSSContent.java @@ -0,0 +1,130 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.util.regex.Matcher; + +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; +import org.junit.Assert; +import org.junit.Test; + +import com.google.common.base.Joiner; + +public class TestParsingSSContent { + + // 0 3 10.10.0.116:48843 10.10.0.116:50010 timer:(on,200ms,0) + // users:(("java",14882,279)) uid:1001 ino:22626218 sk:ffff880199aedb00 + // <-> + // 0 0 10.10.0.116:48839 10.10.0.116:50010 users:(("java",19747,204)) + // uid:1001 ino:22626890 sk:ffff8804056ce900 <-> + // 0 0 10.10.0.116:50010 10.10.0.116:48805 users:(("java",2358,275)) + // uid:1001 ino:22618314 sk:ffff880199aef000 <-> + + private Joiner joiner = Joiner.on(" ").skipNulls(); + + @Test + public void testParsingHeader() { + String[] header = + new String[] { "Recv-Q", "Send-Q", "Local", "Address:Port", "Peer", + "Address:Port" }; + + Matcher m = match(joiner.join(header)); + Assert.assertFalse("Header should be ignored", m.matches()); + } + + @Test + public void testParsingFullSSLine() { + String line = + String.format("0 0 %s:%d %s:%d\t%s %s %s ino:1234 %s", + "10.10.0.116", 1234, "10.10.0.80", 50010, "timer:(on,200ms,0)", + "users:((\"java\",14882,279))", "uid:1001", + "sk:ffff880199aedb00 <->"); + + Matcher m = match(line); + boolean find = m.matches(); + Assert.assertTrue(find); + Assert.assertTrue("The process id should be 14882", + Integer.parseInt(m.group(7)) == 14882); + + } + + @Test + public void testParsingSSLineOfOtherUser() { + String line = + String.format("0 0 %s:%d %s:%d\t %s ino:1234 %s", "10.10.0.116", + 1234, "10.10.0.80", 50010, "uid:1001", "sk:ffff880199aedb00 <->"); + + Matcher m = match(line); + boolean find = m.matches(); + Assert.assertTrue(find); + + Assert.assertTrue("Inode should be found!", + Integer.parseInt(m.group(9)) == 1234); + } + + @Test + public void testParsingSSLineOfOtherUserWithIPv6() { + String line = + String.format("41503 0 %s:%d %s:%d\t %s ino:1234 %s", + "::ffff:10.10.0.116", 46361, "::10.10.0.116", 50010, "uid:1003", + "sk:ffff880406143e00 <->"); + + Matcher m = match(line); + boolean find = m.matches(); + Assert.assertTrue(find); + + Assert.assertTrue("Inode should be found!", + Integer.parseInt(m.group(9)) == 1234); + } + + @Test + public void testParsingPartialSSLineWithUsers() { + String line = + String.format("0 0 %s:%d %s:%d\t%s %s ino:1234 %s", "10.10.0.116", + 1234, "10.10.0.80", 50010, "users:((\"java\",19747,204))", + "uid:1001", "sk:ffff880199aedb00 <->"); + + Matcher m = match(line); + boolean find = m.matches(); + Assert.assertTrue(find); + + Assert.assertTrue("User id should be found!", + Integer.parseInt(m.group(8)) == 1001); + } + + @Test + public void testParsingPartialSSLineWithTimers() { + String line = + String.format("0 0 %s:%d %s:%d\t%s %s ino:1234 %s", "10.10.0.116", + 1234, "10.10.0.80", 50010, "timer:(on,200ms,0)", "uid:1001", + "sk:ffff880199aedb00 <->"); + + Matcher m = match(line); + boolean find = m.matches(); + Assert.assertTrue(find); + + Assert.assertTrue("Dst port must be 50010", + Integer.parseInt(m.group(4)) == 50010); + } + + private Matcher match(String line) { + return ConnectionCollector.SS_ESTABLISHED_LINE_FORMAT.matcher(line); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestProcBasedConnectionHandlerWithHDFS.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestProcBasedConnectionHandlerWithHDFS.java new file mode 100644 index 0000000..a394db3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestProcBasedConnectionHandlerWithHDFS.java @@ -0,0 +1,200 @@ +/** + * 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.nodemanager.trafficcontrol; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.BaseTestConnectionHandler; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.HdfsTCDataSubmitter; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.StorageUtil; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; +import org.junit.Assert; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class TestProcBasedConnectionHandlerWithHDFS extends + BaseTestConnectionHandler { + static final Log LOG = LogFactory + .getLog(TestProcBasedConnectionHandlerWithHDFS.class); + + MiniDFSCluster cluster; + FileSystem fs; + + Path hdfsDir; + + Path nm1PathOnDN1; + Path nm1PathOnDN2; + + Path dnDir1; + Path dnDir2; + + private void printContent(FileSystem fs, Path file) { + try (FSDataInputStream is = fs.open(file); + BufferedReader br = new BufferedReader(new InputStreamReader(is));) { + String line; + while ((line = br.readLine()) != null) { + LOG.info(line); + } + } catch (IOException e) { + LOG.info("can not get content", e); + } + } + + private List readContent(FileSystem fs, Path file) { + try (FSDataInputStream is = fs.open(file)) { + return StorageUtil.decodeHostConnections(parser, + new InputStreamReader(is)); + } catch (IOException e) { + LOG.info("can not get content", e); + } + + return null; + } + + private void listDir(FileSystem fs, Path dir) { + FileStatus[] files; + try { + files = fs.listStatus(dir); + for (FileStatus file : files) { + LOG.info("LISTING: " + + file.getPath().toString()); + } + } catch (Exception e) { + e.printStackTrace(); + LOG.info("can not list dir"); + } + + } + + @Override + protected void initBackendStorage() { + LOG.info("Mock HDFS cluster"); + conf = new HdfsConfiguration(); + + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + LOG.info("Get FileSystem"); + fs = cluster.getFileSystem(); + + fs.mkdirs(new Path(HdfsTCDataSubmitter.TC_HDFS_DIR)); + fs.mkdirs(new Path(HdfsTCDataSubmitter.TC_HDFS_DIR, host1)); + fs.mkdirs(new Path(HdfsTCDataSubmitter.TC_HDFS_DIR, host2)); + + hdfsDir = new Path(HdfsTCDataSubmitter.TC_HDFS_DIR); + + dnDir1 = new Path(HdfsTCDataSubmitter.TC_HDFS_DIR, host1); + dnDir2 = new Path(HdfsTCDataSubmitter.TC_HDFS_DIR, host2); + + nm1PathOnDN1 = new Path(dnDir1, nm1TCConfigName); + LOG.info("nm1ConfigOnDN1: " + + nm1PathOnDN1); + + nm1PathOnDN2 = new Path(dnDir2, nm1TCConfigName); + LOG.info("nm1ConfigOnDN2: " + + nm1PathOnDN2); + } catch (IOException e) { + LOG.error(e); + } + } + + @Override + protected void shutdown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Override + protected void validateRound1() throws IOException { + Assert.assertTrue(fs.exists(hdfsDir)); + listDir(fs, hdfsDir); + + // 0.0.0.181 + // |_ __0.0.0.181__ + LOG.info("dir1: " + + dnDir1); + + Assert.assertTrue(fs.exists(dnDir1)); + listDir(fs, dnDir1); + Assert.assertTrue(fs.exists(nm1PathOnDN1)); + + // check content + LOG.info("\nCheck content on " + + nm1PathOnDN1); + printContent(fs, nm1PathOnDN1); + List dnContainers = readContent(fs, nm1PathOnDN1); + Assert.assertTrue(dnContainers.size() == 1); + DNContainerConnections dn = dnContainers.get(0); + Assert.assertTrue(dn.getConnections().size() == 2); + + // 0.0.0.190 + // |_ 0.0.0.181 + Assert.assertTrue(fs.exists(dnDir2)); + Assert.assertTrue(fs.exists(nm1PathOnDN2)); + // check content + LOG.info("Check content on tcConfig2"); + printContent(fs, nm1PathOnDN2); + dnContainers = readContent(fs, nm1PathOnDN2); + Assert.assertTrue(dnContainers.size() == 1); + Assert.assertTrue(dnContainers.get(0).getConnections().size() == 1); + + } + + @Override + protected void validateRound2() { + // check content + LOG.info("\nCheck content on tcConfig1\n"); + printContent(fs, nm1PathOnDN1); + List dnContainers = readContent(fs, nm1PathOnDN1); + Assert.assertTrue(dnContainers.size() == 1); + Assert.assertTrue(dnContainers.get(0).getConnections().size() == 1); + + LOG.info("\nCheck content on tcConfig2\n"); + printContent(fs, nm1PathOnDN2); + dnContainers = readContent(fs, nm1PathOnDN2); + Assert.assertTrue(dnContainers.isEmpty()); + } + + @Override + protected void validateRound3() { + LOG.info("\nCheck content on tcConfig1\n"); + printContent(fs, nm1PathOnDN1); + List dnContainers = readContent(fs, nm1PathOnDN1); + Assert.assertTrue(dnContainers.isEmpty()); + + LOG.info("\nCheck content on tcConfig2\n"); + printContent(fs, nm1PathOnDN2); + dnContainers = readContent(fs, nm1PathOnDN2); + Assert.assertTrue(dnContainers.isEmpty()); + + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestProcBasedConnectionHandlerWithMock.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestProcBasedConnectionHandlerWithMock.java new file mode 100644 index 0000000..840b7b2 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestProcBasedConnectionHandlerWithMock.java @@ -0,0 +1,363 @@ +/** + * 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.nodemanager.trafficcontrol; + +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyListOf; +import static org.mockito.Matchers.anySetOf; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.AbstractContainerService; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.DummyContainerService; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionHandler; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionMonitor; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.CommonProcessTree; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ContainerProcessTree; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.NMContainerConnections; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import org.mortbay.log.Log; + +@RunWith(MockitoJUnitRunner.class) +public class TestProcBasedConnectionHandlerWithMock { + + Configuration conf; + + @Mock + AbstractTCDataSubmitter submitter; + + @Mock + FactoryHelper helper; + + @Mock + ConnectionCollector connectionCollector; + + ProcBasedConnectionMonitor connectionMonitor; + + AbstractContainerService containerService; + + ProcBasedConnectionHandler connectionHandler; + + @Rule + public TemporaryFolder procFsRoot = new TemporaryFolder(); + + NMContainerConnections con; + NMContainerConnections con1; + + ContainerProcessTree spyTree1; + ContainerProcessTree spyTree2; + + Set inodeList; + Set inodeList1; + Map connectionList; + + private String host1 = "0.0.0.181", host2 = "0.0.0.190"; + private String clsId1 = "cls1", clsId2 = "cls2", yarnPID1 = "1234", + yarnPID2 = "1235"; + private int hdfsPort = 50010; + + @Before + public void setup() throws IOException { + + conf = spy(new Configuration()); + + when(helper.isTCApplicable()).thenReturn(true); + + when(helper.getTCDataSubmitter(any(Configuration.class))).thenReturn( + submitter); + + connectionHandler = new ProcBasedConnectionHandler(conf, helper); + containerService = new DummyContainerService(conf); + connectionHandler.registerContainerService(containerService); + + // connectionCollector = spy(new ConnectionCollector(hdfsPort, false)); + connectionMonitor = spy(new ProcBasedConnectionMonitor(conf)); + connectionMonitor.setCollector(connectionCollector); + connectionMonitor.setBackupConnectedDNsFile(procFsRoot.newFile().getPath()); + connectionHandler.setConnectionMonitor(connectionMonitor); + + procFsRoot.newFile(yarnPID1); + procFsRoot.newFile(yarnPID2); + FactoryHelper.getInstance().setProcFsRoot(procFsRoot.getRoot().getPath()); + + inodeList = new HashSet(Arrays.asList("1", "2", "3", "4", "5")); + inodeList1 = + new HashSet(Arrays.asList("11", "12", "13", "14", "15")); + + spyTree1 = + spy(new ContainerProcessTree(clsId1, new CommonProcessTree(yarnPID1, + "2345"))); + spyTree2 = + spy(new ContainerProcessTree(clsId2, new CommonProcessTree(yarnPID2, + "22345"))); + + con = + new NMContainerConnections(clsId1, "60mbs", + connectionHandler.getMonitoringPort()); + con.setProcessTree(spyTree1); + + con1 = + new NMContainerConnections(clsId2, "70mbs", + connectionHandler.getMonitoringPort()); + con1.setProcessTree(spyTree2); + + connectionList = new HashMap(); + connectionList.put("1", new Connection(host1, 11111, host1, hdfsPort)); + connectionList.put("2", new Connection(host1, 11112, host1, hdfsPort)); + connectionList.put("5", new Connection(host1, 11115, host2, hdfsPort)); + + when(spyTree1.getInodes()).thenReturn(inodeList); + when(spyTree2.getInodes()).thenReturn(inodeList1); + + doNothing().when(connectionMonitor).buildConnectionDB(); + } + + @Test + public void verifyCreation() { + + verify(helper, times(1)).getTCDataSubmitter(any(Configuration.class)); + + assertTrue("Default monitor port is " + + hdfsPort, connectionHandler.getMonitoringPort() == hdfsPort); + } + + @Test + public void nothingToSubmitWithNewEmptyContainer() { + containerService.addMonitoringContainer(clsId1, 60); + + Map> dns = connectionHandler.collect(); + + assertTrue("Submitter shouldn't see newly added empty containers", + dns.isEmpty()); + + verify(connectionMonitor, never()).saveConnectedHosts( + anySetOf(String.class)); + + } + + @Test + public void updateTcClassViewThenCollect() { + + connectionHandler.addTcClassView(con); + + con.collectConnections(connectionList); + + Map> dns = connectionHandler.collect(); + + assertTrue("We should have 2 datanodes", dns.keySet().size() == 2); + assertTrue(dns.keySet().containsAll(Arrays.asList(host1, host2))); + + when(submitter.submit(anyString(), anyListOf(DNContainerConnections.class))) + .thenReturn(true, true); + + connectionHandler.process(); + verify(submitter, times(2)).submit(anyString(), + anyListOf(DNContainerConnections.class)); + + verify(connectionMonitor, times(1)).saveConnectedHosts( + anySetOf(String.class)); + + assertTrue("connectionsData cache is empty", dns.isEmpty()); + } + + @Test + public void updateTcClassViewThenSubmitOK() { + + when(submitter.submit(anyString(), anyListOf(DNContainerConnections.class))) + .thenReturn(true, true); + + connectionHandler.addTcClassView(con); + // Get the ref of connectionsData cache + Map> dns = connectionHandler.collect(); + con.collectConnections(connectionList); + + connectionHandler.process(); + + verify(submitter, times(2)).submit(anyString(), + anyListOf(DNContainerConnections.class)); + assertTrue("connectionsData cache is empty", dns.isEmpty()); + } + + @Test + public void submitTwoContainersTwiceAndAlwaysOK() { + + when(submitter.submit(eq(host1), anyListOf(DNContainerConnections.class))) + .thenReturn(true, true); + + when(submitter.submit(eq(host2), anyListOf(DNContainerConnections.class))) + .thenReturn(true, true); + + connectionHandler.addTcClassView(con); + + // 1. round + con.collectConnections(connectionList); + Map> dns = connectionHandler.collect(); + connectionHandler.process(); + + assertTrue("connectionsData cache is empty", dns.isEmpty()); + + // 2. round + connectionHandler.addTcClassView(con1); + connectionList.put("11", new Connection(host1, 11121, host1, hdfsPort)); + connectionList.put("12", new Connection(host1, 11122, host1, hdfsPort)); + // connectionList.put("15", new Connection(host1, 11125, host2, hdfsPort)); + + con.collectConnections(connectionList); + con1.collectConnections(connectionList); + + dns = connectionHandler.collect(); + + System.out.println(dns); + + assertTrue("Changes are in " + + host1 + " only", dns.size() == 1 + && dns.containsKey(host1)); + + int consNum = 0; + for (DNContainerConnections dn : dns.get(host1)) { + consNum += dn.getConnections().size(); + } + + assertTrue(host1 + + " has 4 HDFS connections", consNum == 4); + + connectionHandler.process(); + assertTrue("connectionsData cache is empty", dns.isEmpty()); + } + + @Test + public void submitTwoContainersTwiceWithSomeFailed() { + + when(submitter.submit(eq(host1), anyListOf(DNContainerConnections.class))) + .thenReturn(true, true); + + when(submitter.submit(eq(host2), anyListOf(DNContainerConnections.class))) + .thenReturn(false, true); + + connectionHandler.addTcClassView(con); + + // 1. round + con.collectConnections(connectionList); + Map> dns = connectionHandler.collect(); + connectionHandler.process(); + + assertTrue("connectionsData must contain " + + host2, dns.size() == 1 + && dns.containsKey(host2)); + + // 2. round + connectionHandler.addTcClassView(con1); + connectionList.put("11", new Connection(host1, 11121, host1, hdfsPort)); + connectionList.put("12", new Connection(host1, 11122, host1, hdfsPort)); + + con.collectConnections(connectionList); + con1.collectConnections(connectionList); + + dns = connectionHandler.collect(); + assertTrue("Changes are in both host", dns.size() == 2); + + int consNum = 0; + for (DNContainerConnections dn : dns.get(host1)) { + consNum += dn.getConnections().size(); + } + + assertTrue(host1 + + " has 4 HDFS connections", consNum == 4); + + assertTrue(host2 + + " has 1 HDFS connections", dns.get(host2).get(0).getConnections() + .size() == 1); + + connectionHandler.process(); + assertTrue("connectionsData cache is empty", dns.isEmpty()); + } + + @Test + public void removeDisconnectedHost() { + + when(submitter.submit(anyString(), anyListOf(DNContainerConnections.class))) + .thenReturn(true, true); + + connectionHandler.addTcClassView(con); + + // 1. update + con.collectConnections(connectionList); + connectionHandler.process(); + + // 2. update + connectionList.remove("5"); + con.collectConnections(connectionList); + + // Get the ref of connectionsData cache + Map> dns = connectionHandler.collect(); + + Log.info("dns: " + + dns); + assertTrue("connectionsData has " + + host2 + " with empty list", dns.get(host2).isEmpty()); + } + + @Test + public void updateTcClassViewThenSubmitFailed() throws IOException, + InterruptedException { + + when(submitter.submit(anyString(), anyListOf(DNContainerConnections.class))) + .thenReturn(true, false); + + connectionHandler.addTcClassView(con); + // Get the ref of connectionsData cache + Map> dns = connectionHandler.collect(); + con.collectConnections(connectionList); + connectionHandler.process(); + + assertTrue("connectionsData cache is not empty", dns.keySet().size() == 1); + } + + @After + public void tearDown() { + FactoryHelper.getInstance().setProcFsRoot("/proc"); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestTrafficControllerWithMock.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestTrafficControllerWithMock.java new file mode 100644 index 0000000..ae79fa7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestTrafficControllerWithMock.java @@ -0,0 +1,179 @@ +/** + * 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.nodemanager.trafficcontrol; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.TrafficController; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.executor.TCCommand; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.executor.TrafficControlDeviceExecutor; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.DNContainerConnections; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class TestTrafficControllerWithMock { + Configuration conf; + + @Mock + AbstractTCDataCollector collector; + + @Mock + FactoryHelper helper; + + TrafficControlDeviceExecutor loopbackDevice; + + TrafficController tcController; + + private String host1 = "0.0.0.181"; + private String clsId1 = "cls1", clsId2 = "cls2"; + private int hdfsPort = 50010; + + @Before + public void setup() throws IOException { + + conf = spy(new Configuration()); + + loopbackDevice = + spy(new TrafficControlDeviceExecutor("lo", hdfsPort, helper)); + + when(helper.isTCApplicable()).thenReturn(true); + when(helper.getTCDataCollector(any(Configuration.class))).thenReturn( + collector); + + when(helper.getDevice(anyString(), anyInt(), any(FactoryHelper.class))) + .thenReturn(loopbackDevice); + + tcController = new TrafficController(conf, false, helper, 0); + } + + @Test + public void testCreation() throws IOException { + + Assert.assertTrue("Default monitoring port is " + + hdfsPort, hdfsPort == tcController.getMonitoringPort()); + verify(helper, times(1)).getDevice(eq("lo"), eq(hdfsPort), eq(helper)); + when(helper.getBufferedReader(any(TCCommand.class))).thenReturn( + getQdiscShowOfLoopback()); + tcController.initialize(host1); + Assert.assertTrue("Loopback device should be initialized properly", + loopbackDevice.isInitialized()); + } + + @Test + public void testCreationWithFail() throws IOException { + + when(helper.getBufferedReader(any(TCCommand.class))).thenReturn( + new BufferedReader(new StringReader("Cannot find device \"eth1\""))); + + tcController.initialize(host1); + Assert.assertFalse("Creation should be failed", tcController.isEnabled()); + } + + @Test + public void testUpdateLoopback() throws IOException { + when(helper.getBufferedReader(any(TCCommand.class))).thenReturn( + getQdiscShowOfLoopback()); + tcController.initialize(host1); + + Map> connections = new HashMap<>(); + connections.put(host1, new ArrayList()); + + DNContainerConnections dn1 = new DNContainerConnections(clsId1, "50mbps"); + dn1.getConnections().add(new Connection(host1, 11111, host1, hdfsPort)); + dn1.getConnections().add(new Connection(host1, 11112, host1, hdfsPort)); + + when(collector.collectData()).thenReturn(connections); + connections.get(host1).add(dn1); + + when(helper.exec(any(TCCommand.class))).thenReturn(0); + + tcController.update(); + + // We should have 4 tc commands: 1 class, 1 qdisc and 2 filters + verify(helper, times(4)).exec(any(TCCommand.class)); + + // 2. round: Update container + connections = new HashMap<>(); + dn1 = new DNContainerConnections(clsId1, "50mbps"); + dn1.getConnections().add(new Connection(host1, 11111, host1, hdfsPort)); + connections.put(host1, new ArrayList()); + connections.get(host1).add(dn1); + + when(collector.collectData()).thenReturn(connections); + + tcController.update(); + + // 1 filter must be deleted + verify(helper, times(5)).exec(any(TCCommand.class)); + + // 3. round: remove container + connections = new HashMap<>(); + connections.put(host1, new ArrayList()); + when(collector.collectData()).thenReturn(connections); + tcController.update(); + + // the last filter must be deleted, but the class is remained for latter + verify(helper, times(6)).exec(any(TCCommand.class)); + + // 4. round: New container + DNContainerConnections dn2 = new DNContainerConnections(clsId2, "70mbps"); + dn2.getConnections().add(new Connection(host1, 11121, host1, hdfsPort)); + connections = new HashMap<>(); + connections.put(host1, new ArrayList()); + connections.get(host1).add(dn2); + when(collector.collectData()).thenReturn(connections); + tcController.update(); + + // we should have more 2 tc events: change_class and add_filter + verify(helper, times(8)).exec(any(TCCommand.class)); + + } + + private BufferedReader getQdiscShowOfLoopback() { + String s = + "qdisc htb 10: root refcnt 2 r2q 10 " + + " default 0 direct_packets_stat 30614827 direct_qlen 2\n" + + "qdisc sfq 3e9: parent 10:3e9 limit 127p " + + " quantum 64Kb depth 127 divisor 1024\n"; + return new BufferedReader(new StringReader(s)); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestYarnContainerService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestYarnContainerService.java new file mode 100644 index 0000000..f0054a2 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/TestYarnContainerService.java @@ -0,0 +1,133 @@ +/** + * 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.nodemanager.trafficcontrol; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.when; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FactoryHelper; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.FileBasedYarnContainerReporter; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.ProcBasedConnectionHandler; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.impl.YarnContainerService; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class TestYarnContainerService { + + @Mock + FactoryHelper helper; + + @Mock + AbstractTCDataSubmitter submitter; + + ProcBasedConnectionHandler connectionHandler; + YarnContainerService yarnContainerService; + Configuration conf; + String cid1 = "cls1111"; + + float minRate = 40.0f; + float maxRate = 100.0f; + + @Before + public void setup() throws IOException { + conf = new Configuration(); + + conf.set(YarnConfiguration.NM_HDFS_BE_ENABLE, "true"); + conf.set(YarnConfiguration.NM_HDFS_BE_MIN_RATE, Float.toString(minRate)); + conf.set(YarnConfiguration.NM_HDFS_BE_MAX_RATE, Float.toString(maxRate)); + + when(helper.isTCApplicable()).thenReturn(true, true, true); + } + + @Test + public void testFeatureIsDisableInDefault() { + conf.unset(YarnConfiguration.NM_HDFS_BE_ENABLE); + yarnContainerService = + YarnContainerService.loadYarnContainerService(conf, helper); + Assert.assertNull("yarnContainerService is disable in default", + yarnContainerService); + } + + @Test + public void testLoadYarnContainerServiceInEmbeddedMode() { + + yarnContainerService = + YarnContainerService.loadYarnContainerService(conf, helper); + + Assert.assertNotNull("yarnContainerService should not be null", + yarnContainerService); + Assert.assertFalse(yarnContainerService.isClientMode()); + + Assert.assertEquals(minRate, yarnContainerService.getMinRate(), 0.0f); + Assert.assertEquals(maxRate, yarnContainerService.getMaxRate(), 0.0f); + } + + @Test + public void testRateShouldBeNormalized() throws IOException { + + yarnContainerService = + YarnContainerService.loadYarnContainerService(conf, helper); + when(helper.getTCDataSubmitter(any(Configuration.class))).thenReturn( + submitter); + connectionHandler = new ProcBasedConnectionHandler(conf, helper); + connectionHandler.registerContainerService(yarnContainerService); + + yarnContainerService.addMonitoringContainer(cid1, 20); + Assert.assertTrue(yarnContainerService.getUnModifableMapOfContainers().get( + cid1) == minRate); + } + + @Test + public void + testCannotLoadYarnContainerServiceInClientModeWithoutReportPlugin() { + + conf.set(YarnConfiguration.NM_HDFS_BE_CLIENT_MODE, "true"); + yarnContainerService = + YarnContainerService.loadYarnContainerService(conf, helper); + + Assert.assertNull( + "YarnContainerService need a report plugin in the client mode", + yarnContainerService); + } + + @Test + public void testLoadYarnContainerServiceInClientModeWithReportPlugin() { + + conf.set(YarnConfiguration.NM_HDFS_BE_CLIENT_MODE, "true"); + conf.set(YarnConfiguration.NM_HDFS_BE_REPORT_SERVICE_CLASS, + FileBasedYarnContainerReporter.class.getName()); + + yarnContainerService = + YarnContainerService.loadYarnContainerService(conf, helper); + + Assert.assertNotNull("YarnContainerService should be loaded", + yarnContainerService); + + Assert.assertTrue(yarnContainerService.isClientMode()); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/BaseTestConnectionHandler.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/BaseTestConnectionHandler.java new file mode 100644 index 0000000..8ae18f9 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/BaseTestConnectionHandler.java @@ -0,0 +1,180 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import net.minidev.json.parser.JSONParser; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.CommonProcessTree; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.Connection; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ConnectionCollector; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.ContainerProcessTree; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.view.NMContainerConnections; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mock; + +public abstract class BaseTestConnectionHandler { + private static final Log LOG = LogFactory + .getLog(BaseTestConnectionHandler.class); + + protected JSONParser parser = new JSONParser(JSONParser.MODE_PERMISSIVE); + protected int hdfsPort = 50010; + + protected Configuration conf; + ProcBasedConnectionHandler connectionHandler; + + protected Map connectionList; + protected Set inodeList; + protected ContainerProcessTree spyTree; + protected ContainerId containerId = BuilderUtils.newContainerId(12345, 1, 0, + 1); + + @Mock + ConnectionCollector connectionCollector; + + protected String host1 = "0.0.0.181", host2 = "0.0.0.190"; + protected String pid = "1234", ppid = "123"; + + protected String nm1TCConfigName = String.format("__%s__", host1); + protected String nm2TCConfigName = String.format("__%s__", host2); + + @Rule + public TemporaryFolder procFsRoot = new TemporaryFolder(); + + @SuppressWarnings("unchecked") + @Before + public void setup() throws IOException { + Assume.assumeFalse(System.getProperty("os.name").toLowerCase() + .startsWith("win")); + + connectionList = new HashMap(); + connectionList.put("1", new Connection(host1, 12345, host1, hdfsPort)); + connectionList.put("2", new Connection(host1, 23456, host2, hdfsPort)); + connectionList.put("3", new Connection(host1, 45678, host1, hdfsPort)); + + inodeList = new HashSet(Arrays.asList("1", "2", "3", "4", "5")); + spyTree = + spy(new ContainerProcessTree(containerId.toString(), + new CommonProcessTree(pid, ppid))); + when(spyTree.getInodes()).thenReturn(inodeList, inodeList, inodeList, + inodeList, inodeList, inodeList); + } + + protected abstract void initBackendStorage(); + + protected abstract void shutdown(); + + protected abstract void validateRound1() throws IOException; + + protected abstract void validateRound2() throws IOException; + + protected abstract void validateRound3() throws IOException; + + @Test(timeout = 60000) + public void testTcClass() { + + initBackendStorage(); + + try { + + Thread.sleep(1000); + + LOG.info("init Monitor"); + conf.set(YarnConfiguration.NM_HDFS_BE_CONTAINER_PLUGINS, + DummyContainerService.class.getName()); + connectionHandler = new ProcBasedConnectionHandler(conf); + + ProcBasedConnectionMonitor connectionMonitor = + spy(new ProcBasedConnectionMonitor(conf)); + connectionMonitor.setCollector(connectionCollector); + connectionMonitor.setBackupConnectedDNsFile(procFsRoot.newFile() + .getPath()); + connectionHandler.setConnectionMonitor(connectionMonitor); + + doNothing().when(connectionMonitor).buildConnectionDB(); + + connectionHandler.initialize(host1); + + LOG.info("addTcClassViewr"); + NMContainerConnections tcClass = + new NMContainerConnections(containerId.toString(), "10mbit", hdfsPort); + tcClass.setProcessTree(spyTree); + + connectionHandler.addTcClassView(tcClass); + + LOG.info("call update [1]"); + tcClass.collectConnections(connectionList); + connectionHandler.process(); + + validateRound1(); + Thread.sleep(1000); + + // round2 + LOG.info("call update [2]"); + connectionList.remove("2"); + connectionList.remove("3"); + tcClass.collectConnections(connectionList); + connectionHandler.process(); + + validateRound2(); + + Thread.sleep(1000); + + // round 3: delete class + LOG.info("call update [3]"); + tcClass.stopTrackContainer(); + connectionHandler.process(); + + validateRound3(); + + Thread.sleep(1000); + + } catch (Exception e) { + LOG.error(e.getMessage(), e); + } finally { + shutdown(); + } + } + + @After + public void tearDown() { + FactoryHelper.getInstance().setProcFsRoot("/proc"); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/DummyContainerService.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/DummyContainerService.java new file mode 100644 index 0000000..3794ebe --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/trafficcontrol/impl/DummyContainerService.java @@ -0,0 +1,53 @@ +/** + * 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.nodemanager.trafficcontrol.impl; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.server.nodemanager.trafficcontrol.ContainerService; + +public class DummyContainerService extends AbstractContainerService { + + public DummyContainerService(Configuration conf) { + super(conf); + } + + @Override + public void initialize(String localNodeId) { + return; + } + + @Override + public void start() { + return; + } + + @Override + public void stop() { + return; + } + + @Override + protected float normalize(float rate) { + return rate; + } + + public void setMyCallBack(ContainerService callback) { + setCallBack(callback); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java index 0e25360..67fa112 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java @@ -327,4 +327,50 @@ public ApplicationAttemptId getApplicationAttemptId() { } return conts; } + + public void addRequests(String[] hosts, int memory, float hdfsBandwidthEnforcement, + int priority,int containers) throws Exception { + requests.addAll(createReq(hosts, memory, hdfsBandwidthEnforcement, priority, containers)); + } + + public List createReq(String[] hosts, int memory, float hdfsBandwidthEnforcement,int priority, + int containers) throws Exception { + List reqs = new ArrayList(); + for (String host : hosts) { + ResourceRequest hostReq = createResourceReq(host, memory, hdfsBandwidthEnforcement, + priority, containers); + reqs.add(hostReq); + ResourceRequest rackReq = createResourceReq("/default-rack", memory, + hdfsBandwidthEnforcement, priority, containers); + reqs.add(rackReq); + } + + ResourceRequest offRackReq = createResourceReq(ResourceRequest.ANY, memory,hdfsBandwidthEnforcement, + priority, containers); + reqs.add(offRackReq); + return reqs; + } + + public ResourceRequest createResourceReq(String resource, int memory, float hdfsBandwidthEnforcement, + int priority, int containers) throws Exception { + ResourceRequest req = Records.newRecord(ResourceRequest.class); + req.setResourceName(resource); + req.setNumContainers(containers); + + Priority pri = Records.newRecord(Priority.class); + pri.setPriority(priority); + req.setPriority(pri); + + Resource capability = Records.newRecord(Resource.class); + capability.setMemory(memory); + capability.setHdfsBandwidthEnforcement(hdfsBandwidthEnforcement); + + req.setCapability(capability); + return req; + } + + public List getRequests() { + return requests; + } + } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestHdfsBandwidthEnforcement.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestHdfsBandwidthEnforcement.java new file mode 100644 index 0000000..7095576 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestHdfsBandwidthEnforcement.java @@ -0,0 +1,293 @@ +/** + * 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.resourcemanager.scheduler; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NMToken; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto; +import org.apache.hadoop.yarn.server.resourcemanager.MockAM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; +import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.junit.Assert; +import org.junit.Test; + +public class TestHdfsBandwidthEnforcement { + private static final Log LOG = LogFactory + .getLog(TestHdfsBandwidthEnforcement.class); + + private final int GB = 1024; + + private void printTestName(String testName) { + LOG.info(String.format("**** %s ***", testName)); + } + + private void print(String type, Resource r) { + LOG.info(String.format("%s ", type, + r.getMemory(), r.getVirtualCores(), r.getHdfsBandwidthEnforcement())); + } + + @Test + public void testAllocateRequest() { + + printTestName("Test Allocate Request"); + List resourceAsk = new ArrayList(); + + float bandwidth = 10f; + Resource resource = Resources.createResource(1024, 1, bandwidth); + + ResourceRequest rr = Records.newRecord(ResourceRequest.class); + rr.setCapability(resource); + + print("ResourceRequest", rr.getCapability()); + + resourceAsk.add(rr); + + AllocateRequest req = + AllocateRequest.newInstance(0, 0, resourceAsk, null, null); + + AllocateRequestProto p = ((AllocateRequestPBImpl) req).getProto(); + req = new AllocateRequestPBImpl(p); + + List ask = req.getAskList(); + + Assert.assertEquals(ask.size(), 1); + Assert.assertTrue(bandwidth == ask.get(0).getCapability() + .getHdfsBandwidthEnforcement()); + + print("ProtoBuffer ResourceRequest", ask.get(0).getCapability()); + + } + + @Test + public void testAllocateResponse() { + printTestName(" Test Allocate Response"); + + List containers = new ArrayList(); + + float bandwidth = 10f; + Resource resource = Resources.createResource(1024, 1, bandwidth); + + Container container = + Container.newInstance(null, null, null, resource, null, null); + + print("ContainerResource", container.getResource()); + + containers.add(container); + + AllocateResponse r = + AllocateResponse.newInstance(3, new ArrayList(), + containers, new ArrayList(), null, null, 3, null, + new ArrayList(), null, null); + + // serde + AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto(); + r = new AllocateResponsePBImpl(p); + + List allocatedContainers = r.getAllocatedContainers(); + Assert.assertTrue(allocatedContainers.size() == 1); + Assert.assertTrue(allocatedContainers.get(0).getResource() + .getHdfsBandwidthEnforcement() == bandwidth); + + print("ProtocolBuffer ContainerResource", allocatedContainers.get(0) + .getResource()); + } + + @Test + public void testNormalizeResourceUsingDefaultResourceCalculator() { + printTestName("Test Normalize Resource Using DefaultResourceCalculator"); + + ResourceCalculator rc = new DefaultResourceCalculator(); + final int minMemory = 1024; + final int maxMemory = 8192; + Resource minResource = Resources.createResource(minMemory, 0); + Resource maxResource = Resources.createResource(maxMemory, 0); + + ResourceRequest ask = Records.newRecord(ResourceRequest.class); + float bandwidth = 10f; + Resource capability = Resources.createResource(minMemory, 1, bandwidth); + + ask.setCapability(capability); + + print("ResourceRequest", ask.getCapability()); + + SchedulerUtils.normalizeRequest(ask, rc, null, minResource, maxResource); + + print("Normalized ResourceRequest", ask.getCapability()); + + // DefaultResourceCalculator unset HDFS Bandwidth Enforcement + Assert.assertTrue(0 == ask.getCapability().getHdfsBandwidthEnforcement()); + + } + + @Test + public void testNormalizeResourceUsingDominantResourceCalculator() { + + printTestName("Test Normalize Resource Using DominantResourceCalculator"); + + ResourceCalculator rc = new DominantResourceCalculator(); + final int minMemory = 1024; + final int maxMemory = 8192; + Resource minResource = Resources.createResource(minMemory, 0); + Resource maxResource = Resources.createResource(maxMemory, 0); + + ResourceRequest ask = Records.newRecord(ResourceRequest.class); + float bandwidth = 10f; + Resource capability = Resources.createResource(minMemory, 1, bandwidth); + + ask.setCapability(capability); + + print("ResourceRequest", ask.getCapability()); + + SchedulerUtils.normalizeRequest(ask, rc, null, minResource, maxResource); + + print("Normalized ResourceRequest", ask.getCapability()); + + Assert.assertTrue(bandwidth == ask.getCapability() + .getHdfsBandwidthEnforcement()); + } + + @Test(timeout = 3000000) + public void testContainerAllocationWithCapacityScheduler() throws Exception { + + printTestName("Test Container Allocation With Capacity Scheduler"); + YarnConfiguration conf = new YarnConfiguration(); + // use Capacity Scheduler + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + + // use Dominant Resource Calculator + conf.setClass(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, + DominantResourceCalculator.class, ResourceCalculator.class); + + try (MockRM rm = new MockRM(conf);) { + rm.start(); + // Register node1 + MockNM nm1 = rm.registerNode("127.0.0.1:1234", 2 * GB, 4); + MockNM nm2 = rm.registerNode("127.0.0.1:2234", 3 * GB, 4); + + nm1.nodeHeartbeat(true); + nm2.nodeHeartbeat(true); + + // wait.. + int waitCount = 20; + int size = rm.getRMContext().getRMNodes().size(); + while ((size = rm.getRMContext().getRMNodes().size()) != 2 + && waitCount-- > 0) { + LOG.info("Waiting for node managers to register : " + + size); + Thread.sleep(100); + } + Assert.assertEquals(2, rm.getRMContext().getRMNodes().size()); + // Submit an application + RMApp app1 = rm.submitApp(128); + + // kick the scheduling + nm1.nodeHeartbeat(true); + RMAppAttempt attempt1 = app1.getCurrentAppAttempt(); + MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId()); + am1.registerAppAttempt(); + + LOG.info("sending container requests "); + + float bandwidth = 10f; + am1.addRequests(new String[] { "*" }, 3 * GB, bandwidth, 1, 1); + + List requests = am1.getRequests(); + if (requests != null + && requests.size() > 0) { + print("ResourceRequest", requests.get(0).getCapability()); + } else { + LOG.info("ERROR: can not get ResourceRequest"); + } + + AllocateResponse alloc1Response = am1.schedule(); // send the request + + // kick the scheduler + nm1.nodeHeartbeat(true); + int waitCounter = 20; + LOG.info("heartbeating nm1"); + while (alloc1Response.getAllocatedContainers().size() < 1 + && waitCounter-- > 0) { + LOG.info("Waiting for containers to be created for app 1..."); + Thread.sleep(500); + alloc1Response = am1.schedule(); + } + LOG.info("received container : " + + alloc1Response.getAllocatedContainers().size()); + + // No container should be allocated. + // Internally it should not been reserved. + Assert.assertTrue(alloc1Response.getAllocatedContainers().size() == 0); + + LOG.info("heartbeating nm2"); + waitCounter = 20; + nm2.nodeHeartbeat(true); + while (alloc1Response.getAllocatedContainers().size() < 1 + && waitCounter-- > 0) { + LOG.info("Waiting for containers to be created for app 1..."); + Thread.sleep(500); + alloc1Response = am1.schedule(); + } + LOG.info("received container : " + + alloc1Response.getAllocatedContainers().size()); + Assert.assertTrue(alloc1Response.getAllocatedContainers().size() == 1); + + List allocatedContainers = + alloc1Response.getAllocatedContainers(); + for (Container container : allocatedContainers) { + LOG.info("Allocated container resource: " + + container.getResource()); + if (container.getResource() != null) { + float b = container.getResource().getHdfsBandwidthEnforcement(); + Assert.assertTrue(bandwidth == b); + + print("Allocated container resource", container.getResource()); + } else { + LOG.info("Can not get resource of the container"); + } + } + + rm.stop(); + } + } +}