瀏覽代碼

YARN-3069. Document missing properties in yarn-default.xml. Contributed by Ray Chiang.

Akira Ajisaka 10 年之前
父節點
當前提交
d6675606dc

+ 1 - 0
hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md

@@ -276,6 +276,7 @@ The following table lists the configuration property names that are deprecated i
 | user.name | mapreduce.job.user.name |
 | user.name | mapreduce.job.user.name |
 | webinterface.private.actions | mapreduce.jobtracker.webinterface.trusted |
 | webinterface.private.actions | mapreduce.jobtracker.webinterface.trusted |
 | yarn.app.mapreduce.yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts | yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts |
 | yarn.app.mapreduce.yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts | yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts |
+| yarn.client.app-submission.poll-interval | yarn.client.application-client-protocol.poll-timeout-ms |
 
 
 The following table lists additional changes to some configuration properties:
 The following table lists additional changes to some configuration properties:
 
 

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -329,6 +329,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3800. Reduce storage footprint for ReservationAllocation. (Anubhav Dhoot
     YARN-3800. Reduce storage footprint for ReservationAllocation. (Anubhav Dhoot
     via curino)
     via curino)
 
 
+    YARN-3069. Document missing properties in yarn-default.xml.
+    (Ray Chiang via aajisaka)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

+ 447 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -24,7 +24,8 @@
 
 
 <configuration>
 <configuration>
 
 
-  <!-- IPC Configs -->
+  <!-- IPC Configuration -->
+
   <property>
   <property>
     <description>Factory to create client IPC classes.</description>
     <description>Factory to create client IPC classes.</description>
     <name>yarn.ipc.client.factory.class</name>
     <name>yarn.ipc.client.factory.class</name>
@@ -46,7 +47,8 @@
     <value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value>
     <value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value>
   </property>
   </property>
   
   
-  <!-- Resource Manager Configs -->
+  <!-- Resource Manager Configuration -->
+
   <property>
   <property>
     <description>The hostname of the RM.</description>
     <description>The hostname of the RM.</description>
     <name>yarn.resourcemanager.hostname</name>
     <name>yarn.resourcemanager.hostname</name>
@@ -142,6 +144,32 @@
     <value>${yarn.resourcemanager.hostname}:8090</value>
     <value>${yarn.resourcemanager.hostname}:8090</value>
   </property>
   </property>
 
 
+  <property>
+    <description>
+    The Kerberos keytab file to be used for spnego filter for the RM web
+    interface.
+    </description>
+    <name>yarn.resourcemanager.webapp.spnego-keytab-file</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    The Kerberos principal to be used for spnego filter for the RM web
+    interface.
+    </description>
+    <name>yarn.resourcemanager.webapp.spnego-principal</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Add button to kill application in the RM Application view.
+    </description>
+    <name>yarn.resourcemanager.webapp.ui-actions.enabled</name>
+    <value>true</value>
+  </property>
+
   <property>
   <property>
     <name>yarn.resourcemanager.resource-tracker.address</name>
     <name>yarn.resourcemanager.resource-tracker.address</name>
     <value>${yarn.resourcemanager.hostname}:8031</value>
     <value>${yarn.resourcemanager.hostname}:8031</value>
@@ -280,7 +308,16 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <description>Enable RM to recover state after starting. If true, then 
+    <description>
+    Used by node labels.  If set to true, the port should be included in the
+    node name.  Only usable if your scheduler supports node labels.
+    </description>
+    <name>yarn.scheduler.include-port-in-node-name</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>Enable RM to recover state after starting. If true, then
       yarn.resourcemanager.store.class must be specified. </description>
       yarn.resourcemanager.store.class must be specified. </description>
     <name>yarn.resourcemanager.recovery.enabled</name>
     <name>yarn.resourcemanager.recovery.enabled</name>
     <value>false</value>
     <value>false</value>
@@ -672,6 +709,16 @@
     <!-- <value>org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider</value> -->
     <!-- <value>org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider</value> -->
   </property>
   </property>
 
 
+  <property>
+    <description>
+    The value specifies the file system (e.g. HDFS) path where ResourceManager
+    loads configuration if yarn.resourcemanager.configuration.provider-class
+    is set to org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider.
+    </description>
+    <name>yarn.resourcemanager.configuration.file-system-based-store</name>
+    <value>/yarn/conf</value>
+  </property>
+
   <property>
   <property>
     <description>The setting that controls whether yarn system metrics is
     <description>The setting that controls whether yarn system metrics is
     published on the timeline server or not by RM.</description>
     published on the timeline server or not by RM.</description>
@@ -695,6 +742,119 @@
   </property>
   </property>
 
 
   <!-- Node Manager Configs -->
   <!-- Node Manager Configs -->
+
+  <property>
+    <description>
+    RM DelegationTokenRenewer thread count
+    </description>
+    <name>yarn.resourcemanager.delegation-token-renewer.thread-count</name>
+    <value>50</value>
+  </property>
+
+  <property>
+    <description>
+    RM secret key update interval in ms
+    </description>
+    <name>yarn.resourcemanager.delegation.key.update-interval</name>
+    <value>86400000</value>
+  </property>
+
+  <property>
+    <description>
+    RM delegation token maximum lifetime in ms
+    </description>
+    <name>yarn.resourcemanager.delegation.token.max-lifetime</name>
+    <value>604800000</value>
+  </property>
+
+  <property>
+    <description>
+    RM delegation token update interval in ms
+    </description>
+    <name>yarn.resourcemanager.delegation.token.renew-interval</name>
+    <value>86400000</value>
+  </property>
+
+  <property>
+    <description>
+    Thread pool size for RMApplicationHistoryWriter.
+    </description>
+    <name>yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size</name>
+    <value>10</value>
+  </property>
+
+  <property>
+    <description>
+    Comma-separated list of values (in minutes) for schedule queue related
+    metrics.
+    </description>
+    <name>yarn.resourcemanager.metrics.runtime.buckets</name>
+    <value>60,300,1440</value>
+  </property>
+
+  <property>
+    <description>
+    Interval for the roll over for the master key used to generate
+    NodeManager tokens.  It is expected to be set to a value much larger
+    than yarn.nm.liveness-monitor.expiry-interval-ms.
+    </description>
+    <name>yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs</name>
+    <value>86400</value>
+  </property>
+
+  <property>
+    <description>
+    Flag to enable the ResourceManager reservation system.
+    </description>
+    <name>yarn.resourcemanager.reservation-system.enable</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    The Java class to use as the ResourceManager reservation system.
+    By default, is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacityReservationSystem
+    when using CapacityScheduler and is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.FairReservationSystem
+    when using FairScheduler.
+    </description>
+    <name>yarn.resourcemanager.reservation-system.class</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    The plan follower policy class name to use for the ResourceManager
+    reservation system.
+    By default, is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacitySchedulerPlanFollower
+    is used when using CapacityScheduler, and is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.FairSchedulerPlanFollower
+    when using FairScheduler.
+    </description>
+    <name>yarn.resourcemanager.reservation-system.plan.follower</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Step size of the reservation system in ms
+    </description>
+    <name>yarn.resourcemanager.reservation-system.planfollower.time-step</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>
+    The expiry interval for a container
+    </description>
+    <name>yarn.resourcemanager.rm.container-allocation.expiry-interval-ms</name>
+    <value>600000</value>
+  </property>
+
+  <!-- Node Manager Configuration -->
+
   <property>
   <property>
     <description>The hostname of the NM.</description>
     <description>The hostname of the NM.</description>
     <name>yarn.nodemanager.hostname</name>
     <name>yarn.nodemanager.hostname</name>
@@ -727,7 +887,7 @@
   <property>
   <property>
     <description>Environment variables that containers may override rather than use NodeManager's default.</description>
     <description>Environment variables that containers may override rather than use NodeManager's default.</description>
     <name>yarn.nodemanager.env-whitelist</name>
     <name>yarn.nodemanager.env-whitelist</name>
-    <value>JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME</value>
+    <value>JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME</value>
   </property>
   </property>
 
 
   <property>
   <property>
@@ -840,7 +1000,14 @@
 
 
   <property>
   <property>
     <description>
     <description>
-      Where to store container logs. An application's localized log directory 
+    </description>
+    <name>yarn.nodemanager.container-localizer.java.opts</name>
+    <value>-Xmx256m</value>
+  </property>
+
+  <property>
+    <description>
+      Where to store container logs. An application's localized log directory
       will be found in ${yarn.nodemanager.log-dirs}/application_${appid}.
       will be found in ${yarn.nodemanager.log-dirs}/application_${appid}.
       Individual containers' log directories will be below this, in directories 
       Individual containers' log directories will be below this, in directories 
       named container_{$contid}. Each container directory will contain the files
       named container_{$contid}. Each container directory will contain the files
@@ -879,6 +1046,17 @@
     <value>-1</value>
     <value>-1</value>
   </property>
   </property>
 
 
+  <property>
+    <description>
+    How long for ResourceManager to wait for NodeManager to report its
+    log aggregation status. If waiting time of which the log aggregation
+    status is reported from NodeManager exceeds the configured value, RM
+    will report log aggregation status for this NodeManager as TIME_OUT
+    </description>
+    <name>yarn.log-aggregation-status.time-out.ms</name>
+    <value>600000</value>
+  </property>
+
   <property>
   <property>
     <description>Time in seconds to retain user logs. Only applicable if
     <description>Time in seconds to retain user logs. Only applicable if
     log aggregation is disabled
     log aggregation is disabled
@@ -1005,6 +1183,32 @@
     <value>${yarn.nodemanager.hostname}:8042</value>
     <value>${yarn.nodemanager.hostname}:8042</value>
   </property>
   </property>
 
 
+  <property>
+    <description>
+    The https adddress of the NM web application.
+    </description>
+    <name>yarn.nodemanager.webapp.https.address</name>
+    <value>0.0.0.0:8044</value>
+  </property>
+
+  <property>
+    <description>
+    The Kerberos keytab file to be used for spnego filter for the NM web
+    interface.
+    </description>
+    <name>yarn.nodemanager.webapp.spnego-keytab-file</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    The Kerberos principal to be used for spnego filter for the NM web
+    interface.
+    </description>
+    <name>yarn.nodemanager.webapp.spnego-principal</name>
+    <value></value>
+  </property>
+
   <property>
   <property>
     <description>How often to monitor containers.</description>
     <description>How often to monitor containers.</description>
     <name>yarn.nodemanager.container-monitor.interval-ms</name>
     <name>yarn.nodemanager.container-monitor.interval-ms</name>
@@ -1112,6 +1316,12 @@
     <name>yarn.nodemanager.linux-container-executor.cgroups.mount-path</name>
     <name>yarn.nodemanager.linux-container-executor.cgroups.mount-path</name>
   </property>
   </property>
 
 
+  <property>
+    <description>Delay in ms between attempts to remove linux cgroup</description>
+    <name>yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms</name>
+    <value>20</value>
+  </property>
+
   <property>
   <property>
     <description>This determines which of the two modes that LCE should use on
     <description>This determines which of the two modes that LCE should use on
       a non-secure cluster.  If this value is set to true, then all containers
       a non-secure cluster.  If this value is set to true, then all containers
@@ -1165,6 +1375,23 @@
     <value>false</value>
     <value>false</value>
   </property>
   </property>
 
 
+  <property>
+    <description>
+    Interval of time the linux container executor should try cleaning up
+    cgroups entry when cleaning up a container.
+    </description>
+    <name>yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>
+    The UNIX group that the linux-container-executor should run as.
+    </description>
+    <name>yarn.nodemanager.linux-container-executor.group</name>
+    <value></value>
+  </property>
+
   <property>
   <property>
     <description>T-file compression types used to compress aggregated logs.</description>
     <description>T-file compression types used to compress aggregated logs.</description>
     <name>yarn.nodemanager.log-aggregation.compression-type</name>
     <name>yarn.nodemanager.log-aggregation.compression-type</name>
@@ -1272,6 +1499,66 @@
 
 
   <!--Docker configuration-->
   <!--Docker configuration-->
 
 
+  <property>
+    <description>
+    Adjustment to the container OS scheduling priority.  In Linux, passed
+    directly to the nice command.
+    </description>
+    <name>yarn.nodemanager.container-executor.os.sched.priority.adjustment</name>
+    <value>0</value>
+  </property>
+
+  <property>
+    <description>
+    Flag to enable container metrics
+    </description>
+    <name>yarn.nodemanager.container-metrics.enable</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <description>
+    Container metrics flush period in ms.  Set to -1 for flush on completion.
+    </description>
+    <name>yarn.nodemanager.container-metrics.period-ms</name>
+    <value>-1</value>
+  </property>
+
+  <property>
+    <description>
+    Class used to calculate current container resource utilization.
+    </description>
+    <name>yarn.nodemanager.container-monitor.process-tree.class</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Flag to enable NodeManager disk health checker
+    </description>
+    <name>yarn.nodemanager.disk-health-checker.enable</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <description>
+    Number of threads to use in NM log cleanup.  Used when log aggregation
+    is disabled.
+    </description>
+    <name>yarn.nodemanager.log.deletion-threads-count</name>
+    <value>4</value>
+  </property>
+
+  <property>
+    <description>
+    The Windows group that the windows-container-executor should run as.
+    </description>
+    <name>yarn.nodemanager.windows-secure-container-executor.group</name>
+    <value></value>
+  </property>
+
+  <!-- Docker Configuration -->
+
   <property>
   <property>
     <name>yarn.nodemanager.docker-container-executor.exec-name</name>
     <name>yarn.nodemanager.docker-container-executor.exec-name</name>
     <value>/usr/bin/docker</value>
     <value>/usr/bin/docker</value>
@@ -1280,24 +1567,28 @@
     </description>
     </description>
   </property>
   </property>
 
 
-  <!--Map Reduce configuration-->
   <property>
   <property>
-    <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
-    <value>org.apache.hadoop.mapred.ShuffleHandler</value>
+    <description>
+    The Docker image name to use for DockerContainerExecutor
+    </description>
+    <name>yarn.nodemanager.docker-container-executor.image-name</name>
+    <value></value>
   </property>
   </property>
 
 
-  <property>
-    <name>mapreduce.job.jar</name>
-    <value/>
-  </property>
+  <!-- Map Reduce Configuration -->
 
 
   <property>
   <property>
     <name>mapreduce.job.hdfs-servers</name>
     <name>mapreduce.job.hdfs-servers</name>
     <value>${fs.defaultFS}</value>
     <value>${fs.defaultFS}</value>
   </property>
   </property>
 
 
-  <!-- WebAppProxy Configuration-->
-  
+  <property>
+    <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
+    <value>org.apache.hadoop.mapred.ShuffleHandler</value>
+  </property>
+
+  <!-- WebAppProxy Configuration -->
+
   <property>
   <property>
     <description>The kerberos principal for the proxy, if the proxy is not
     <description>The kerberos principal for the proxy, if the proxy is not
     running as part of the RM.</description>
     running as part of the RM.</description>
@@ -1318,8 +1609,8 @@
      <value/>
      <value/>
   </property>
   </property>
 
 
-  <!-- Applications' Configuration-->
-  
+  <!-- Applications' Configuration -->
+
   <property>
   <property>
     <description>
     <description>
       CLASSPATH for YARN applications. A comma-separated list
       CLASSPATH for YARN applications. A comma-separated list
@@ -1346,7 +1637,7 @@
     <value></value>
     <value></value>
   </property>
   </property>
 
 
-  <!-- Timeline Service's Configuration-->
+  <!-- Timeline Service Configuration -->
 
 
   <property>
   <property>
     <description>Indicate to clients whether timeline service is enabled or not.
     <description>Indicate to clients whether timeline service is enabled or not.
@@ -1530,6 +1821,7 @@
   </property>
   </property>
 
 
   <!--  Shared Cache Configuration -->
   <!--  Shared Cache Configuration -->
+
   <property>
   <property>
     <description>Whether the shared cache is enabled</description>
     <description>Whether the shared cache is enabled</description>
     <name>yarn.sharedcache.enabled</name>
     <name>yarn.sharedcache.enabled</name>
@@ -1671,7 +1963,99 @@
     <value>20</value>
     <value>20</value>
   </property>
   </property>
 
 
-  <!-- Other configuration -->
+  <property>
+    <description>
+    ACL protocol for use in the Timeline server.
+    </description>
+    <name>security.applicationhistory.protocol.acl</name>
+    <value></value>
+  </property>
+
+  <!-- Minicluster Configuration (for testing only!) -->
+
+  <property>
+    <description>
+    Set to true for MiniYARNCluster unit tests
+    </description>
+    <name>yarn.is.minicluster</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Set for MiniYARNCluster unit tests to control resource monitoring
+    </description>
+    <name>yarn.minicluster.control-resource-monitoring</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Set to false in order to allow MiniYARNCluster to run tests without
+    port conflicts.
+    </description>
+    <name>yarn.minicluster.fixed.ports</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Set to false in order to allow the NodeManager in MiniYARNCluster to
+    use RPC to talk to the RM.
+    </description>
+    <name>yarn.minicluster.use-rpc</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    As yarn.nodemanager.resource.memory-mb property but for the NodeManager
+    in a MiniYARNCluster.
+    </description>
+    <name>yarn.minicluster.yarn.nodemanager.resource.memory-mb</name>
+    <value>4096</value>
+  </property>
+
+  <!-- Node Labels Configuration -->
+
+  <property>
+    <description>
+    Enable node labels feature
+    </description>
+    <name>yarn.node-labels.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Retry policy used for FileSystem node label store. The policy is
+    specified by N pairs of sleep-time in milliseconds and number-of-retries
+    "s1,n1,s2,n2,...".
+    </description>
+    <name>yarn.node-labels.fs-store.retry-policy-spec</name>
+    <value>2000, 500</value>
+  </property>
+
+  <property>
+    <description>
+    URI for NodeLabelManager.  The default value is
+    /tmp/hadoop-yarn-${user}/node-labels/ in the local filesystem.
+    </description>
+    <name>yarn.node-labels.fs-store.root-dir</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Set configuration type for node labels. Administrators can specify
+    "centralized" or "distributed".
+    </description>
+    <name>yarn.node-labels.configuration-type</name>
+    <value>centralized</value>
+  </property>
+
+  <!-- Other Configuration -->
+
   <property>
   <property>
     <description>The interval that the yarn client library uses to poll the
     <description>The interval that the yarn client library uses to poll the
     completion status of the asynchronous API of application client protocol.
     completion status of the asynchronous API of application client protocol.
@@ -1681,7 +2065,16 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <description>RSS usage of a process computed via 
+    <description>
+    The duration (in ms) the YARN client waits for an expected state change
+    to occur.  -1 means unlimited wait time.
+    </description>
+    <name>yarn.client.application-client-protocol.poll-timeout-ms</name>
+    <value>-1</value>
+  </property>
+
+  <property>
+    <description>RSS usage of a process computed via
     /proc/pid/stat is not very accurate as it includes shared pages of a
     /proc/pid/stat is not very accurate as it includes shared pages of a
     process. /proc/pid/smaps provides useful information like
     process. /proc/pid/smaps provides useful information like
     Private_Dirty, Private_Clean, Shared_Dirty, Shared_Clean which can be used
     Private_Dirty, Private_Clean, Shared_Dirty, Shared_Clean which can be used
@@ -1693,6 +2086,30 @@
     <value>false</value>
     <value>false</value>
   </property>
   </property>
 
 
+  <property>
+    <description>
+    URL for log aggregation server
+    </description>
+    <name>yarn.log.server.url</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    RM Application Tracking URL
+    </description>
+    <name>yarn.tracking.url.generator</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Class to be used for YarnAuthorizationProvider
+    </description>
+    <name>yarn.authorization-provider</name>
+    <value></value>
+  </property>
+
   <property>
   <property>
     <description>Defines how often NMs wake up to upload log files.
     <description>Defines how often NMs wake up to upload log files.
     The default value is -1. By default, the logs will be uploaded when
     The default value is -1. By default, the logs will be uploaded when
@@ -1703,4 +2120,15 @@
     <name>yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds</name>
     <name>yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds</name>
     <value>-1</value>
     <value>-1</value>
   </property>
   </property>
+
+  <property>
+    <description>
+    Enable/disable intermediate-data encryption at YARN level. For now,
+    this only is used by the FileSystemRMStateStore to setup right
+    file-system security attributes.
+    </description>
+    <name>yarn.intermediate-data-encryption.enable</name>
+    <value>false</value>
+  </property>
+
 </configuration>
 </configuration>

+ 37 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java

@@ -33,18 +33,19 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
 
+  @SuppressWarnings("deprecation")
   @Override
   @Override
   public void initializeMemberVariables() {
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
     xmlFilename = new String("yarn-default.xml");
     configurationClasses = new Class[] { YarnConfiguration.class };
     configurationClasses = new Class[] { YarnConfiguration.class };
-        
 
 
     // Allocate for usage
     // Allocate for usage
     configurationPropsToSkipCompare = new HashSet<String>();
     configurationPropsToSkipCompare = new HashSet<String>();
+    configurationPrefixToSkipCompare = new HashSet<String>();
 
 
     // Set error modes
     // Set error modes
     errorIfMissingConfigProps = true;
     errorIfMissingConfigProps = true;
-    errorIfMissingXmlProps = false;
+    errorIfMissingXmlProps = true;
 
 
     // Specific properties to skip
     // Specific properties to skip
     configurationPropsToSkipCompare
     configurationPropsToSkipCompare
@@ -79,6 +80,37 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
     configurationPropsToSkipCompare
         .add(YarnConfiguration
         .add(YarnConfiguration
 	    .YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
 	    .YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_SCM_STORE_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_SCM_APP_CHECKER_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_SHARED_CACHE_CHECKSUM_ALGO_IMPL);
+
+    // Ignore all YARN Application Timeline Service (version 1) properties
+    configurationPrefixToSkipCompare.add("yarn.timeline-service.");
+
+    // Used as Java command line properties, not XML
+    configurationPrefixToSkipCompare.add("yarn.app.container");
+
+    // Ignore NodeManager "work in progress" variables
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_ENABLED);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_INTERFACE);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_YARN_MBIT);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_DISK_RESOURCE_ENABLED);
+
+    // Set by container-executor.cfg
+    configurationPrefixToSkipCompare.add(YarnConfiguration.NM_USER_HOME_DIR);
+
+    // Ignore deprecated properties
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS);
 
 
     // Allocate for usage
     // Allocate for usage
     xmlPropsToSkipCompare = new HashSet<String>();
     xmlPropsToSkipCompare = new HashSet<String>();
@@ -94,13 +126,11 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
 
     // Used in the XML file as a variable reference internal to the XML file
     // Used in the XML file as a variable reference internal to the XML file
     xmlPropsToSkipCompare.add("yarn.nodemanager.hostname");
     xmlPropsToSkipCompare.add("yarn.nodemanager.hostname");
-    xmlPropsToSkipCompare.add("yarn.timeline-service.hostname");
 
 
-    // Currently defined in TimelineAuthenticationFilterInitializer
-    xmlPrefixToSkipCompare.add("yarn.timeline-service.http-authentication");
+    // Ignore all YARN Application Timeline Service (version 1) properties
+    xmlPrefixToSkipCompare.add("yarn.timeline-service");
 
 
-    // Currently defined in RegistryConstants
+    // Currently defined in RegistryConstants/core-site.xml
     xmlPrefixToSkipCompare.add("hadoop.registry");
     xmlPrefixToSkipCompare.add("hadoop.registry");
   }
   }
-
 }
 }