Updating relese notes and scheduler documentation (#69)

* Updated RELEASE-NOTES with information about the new flag which allows operators to enable SLA aware killing for non-production tasks.
* Fixed a typo in SchedulerMain.
* Updated documentation to reflect the output of the scheduler when run with the `-help` flag.
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index f7449fa..5ad6f85 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -27,6 +27,9 @@
   CPU, memory, and disk metrics for observed tasks. This is useful in setups where metrics cannot be
   gathered reliable (e.g. when using PID namespaces) or when it is expensive due to hundreds of
   active tasks per host.
+- Added flag `-sla_aware_kill_non_prod` which allows operators to enable SLA aware killing
+  for non-production jobs. Jobs are considered non-production when they are preemptable and/or
+  revocable.
 
 ### Deprecations and removals:
 
diff --git a/docs/reference/scheduler-configuration.md b/docs/reference/scheduler-configuration.md
index 3f8cba8..c7b8063 100644
--- a/docs/reference/scheduler-configuration.md
+++ b/docs/reference/scheduler-configuration.md
@@ -7,266 +7,470 @@
 to properly set the most important options.
 
 ```
-$ aurora-scheduler -help
--------------------------------------------------------------------------
--h or -help to print this help message
-
-Required flags:
--backup_dir [not null]
-	Directory to store backups under. Will be created if it does not exist.
--cluster_name [not null]
-	Name to identify the cluster being served.
--framework_authentication_file
-	Properties file which contains framework credentials to authenticate with Mesosmaster. Must contain the properties 'aurora_authentication_principal' and 'aurora_authentication_secret'.
--ip
-	The ip address to listen. If not set, the scheduler will listen on all interfaces.
--mesos_master_address [not null]
-	Address for the mesos master, can be a socket address or zookeeper path.
--mesos_role
-	The Mesos role this framework will register as. The default is to left this empty, and the framework will register without any role and only receive unreserved resources in offer.
--serverset_path [not null, must be non-empty]
-	ZooKeeper ServerSet path to register at.
--shiro_after_auth_filter
-	Fully qualified class name of the servlet filter to be applied after the shiro auth filters are applied.
--thermos_executor_path
-	Path to the thermos executor entry point.
--tier_config [file must be readable]
-	Configuration file defining supported task tiers, task traits and behaviors.
--webhook_config [file must exist, file must be readable]
-	Path to webhook configuration file.
--zk_endpoints [must have at least 1 item]
-	Endpoint specification for the ZooKeeper servers.
-
-Optional flags:
--allow_container_volumes (default false)
-	Allow passing in volumes in the job. Enabling this could pose a privilege escalation threat.
--allow_docker_parameters (default false)
-	Allow to pass docker container parameters in the job.
--allow_gpu_resource (default false)
-	Allow jobs to request Mesos GPU resource.
--allowed_container_types (default [MESOS])
-	Container types that are allowed to be used by jobs.
--allowed_job_environments (default ^(prod|devel|test|staging\d*)$)
-	Regular expression describing the environments that are allowed to be used by jobs.
--async_slot_stat_update_interval (default (1, mins))
-	Interval on which to try to update open slot stats.
--async_task_stat_update_interval (default (1, hrs))
-	Interval on which to try to update resource consumption stats.
--async_worker_threads (default 8)
-	The number of worker threads to process async task operations with.
--backup_interval (default (1, hrs))
-	Minimum interval on which to write a storage backup.
--cron_scheduler_num_threads (default 10)
-	Number of threads to use for the cron scheduler thread pool.
--cron_scheduling_max_batch_size (default 10) [must be > 0]
-	The maximum number of triggered cron jobs that can be processed in a batch.
--cron_start_initial_backoff (default (5, secs))
-	Initial backoff delay while waiting for a previous cron run to be killed.
--cron_start_max_backoff (default (1, mins))
-	Max backoff delay while waiting for a previous cron run to be killed.
--cron_timezone (default GMT)
-	TimeZone to use for cron predictions.
--custom_executor_config [file must exist, file must be readable]
-	Path to custom executor settings configuration file.
--default_docker_parameters (default {})
-	Default docker parameters for any job that does not explicitly declare parameters.
--dlog_max_entry_size (default (512, KB))
-	Specifies the maximum entry size to append to the log. Larger entries will be split across entry Frames.
--dlog_shutdown_grace_period (default (2, secs))
-	Specifies the maximum time to wait for scheduled checkpoint and snapshot actions to complete before forcibly shutting down.
--dlog_snapshot_interval (default (1, hrs))
-	Specifies the frequency at which snapshots of local storage are taken and written to the log.
--enable_cors_for
-	List of domains for which CORS support should be enabled.
--enable_mesos_fetcher (default false)
-	Allow jobs to pass URIs to the Mesos Fetcher. Note that enabling this feature could pose a privilege escalation threat.
--enable_preemptor (default true)
-	Enable the preemptor and preemption
--enable_revocable_cpus (default true)
-	Treat CPUs as a revocable resource.
--enable_revocable_ram (default false)
-	Treat RAM as a revocable resource.
--executor_user (default root)
-	User to start the executor. Defaults to "root". Set this to an unprivileged user if the mesos master was started with "--no-root_submissions". If set to anything other than "root", the executor will ignore the "role" setting for jobs since it can't use setuid() anymore. This means that all your jobs will run under the specified user and the user has to exist on the Mesos agents.
--first_schedule_delay (default (1, ms))
-	Initial amount of time to wait before first attempting to schedule a PENDING task.
--flapping_task_threshold (default (5, mins))
-	A task that repeatedly runs for less than this time is considered to be flapping.
--framework_announce_principal (default false)
-	When 'framework_authentication_file' flag is set, the FrameworkInfo registered with the mesos master will also contain the principal. This is necessary if you intend to use mesos authorization via mesos ACLs. The default will change in a future release. Changing this value is backwards incompatible. For details, see MESOS-703.
--framework_failover_timeout (default (21, days))
-	Time after which a framework is considered deleted.  SHOULD BE VERY HIGH.
--framework_name (default Aurora)
-	Name used to register the Aurora framework with Mesos.
--global_container_mounts (default [])
-	A comma separated list of mount points (in host:container form) to mount into all (non-mesos) containers.
--history_max_per_job_threshold (default 100)
-	Maximum number of terminated tasks to retain in a job history.
--history_min_retention_threshold (default (1, hrs))
-	Minimum guaranteed time for task history retention before any pruning is attempted.
--history_prune_threshold (default (2, days))
-	Time after which the scheduler will prune terminated task history.
--host_maintenance_polling_interval (default (1, minute))
-	Interval between polling for pending host maintenance requests.
--hostname
-	The hostname to advertise in ZooKeeper instead of the locally-resolved hostname.
--http_authentication_mechanism (default NONE)
-	HTTP Authentication mechanism to use.
--http_port (default 0)
-	The port to start an HTTP server on.  Default value will choose a random port.
--initial_flapping_task_delay (default (30, secs))
-	Initial amount of time to wait before attempting to schedule a flapping task.
--initial_schedule_penalty (default (1, secs))
-	Initial amount of time to wait before attempting to schedule a task that has failed to schedule.
--initial_task_kill_retry_interval (default (5, secs))
-	When killing a task, retry after this delay if mesos has not responded, backing off up to transient_task_state_timeout
--job_update_history_per_job_threshold (default 10)
-	Maximum number of completed job updates to retain in a job update history.
--job_update_history_pruning_interval (default (15, mins))
-	Job update history pruning interval.
--job_update_history_pruning_threshold (default (30, days))
-	Time after which the scheduler will prune completed job update history.
--kerberos_debug (default false)
-	Produce additional Kerberos debugging output.
--kerberos_server_keytab
-	Path to the server keytab.
--kerberos_server_principal
-	Kerberos server principal to use, usually of the form HTTP/aurora.example.com@EXAMPLE.COM
--max_flapping_task_delay (default (5, mins))
-	Maximum delay between attempts to schedule a flapping task.
--max_leading_duration (default (1, days))
-	After leading for this duration, the scheduler should commit suicide.
--max_parallel_coordinated_maintenance (default 10)
-	Maximum number of coordinators that can be contacted in parallel.
--max_registration_delay (default (1, mins))
-	Max allowable delay to allow the driver to register before aborting
--max_reschedule_task_delay_on_startup (default (30, secs))
-	Upper bound of random delay for pending task rescheduling on scheduler startup.
--max_saved_backups (default 48)
-	Maximum number of backups to retain before deleting the oldest backups.
--max_schedule_attempts_per_sec (default 40.0)
-	Maximum number of scheduling attempts to make per second.
--max_schedule_penalty (default (1, mins))
-	Maximum delay between attempts to schedule a PENDING tasks.
--max_sla_duration_secs (default (2, hrs))
-	Maximum duration window for which SLA requirements are to be satisfied. This does not apply to jobs that have a CoordinatorSlaPolicy.
--max_status_update_batch_size (default 1000) [must be > 0]
-	The maximum number of status updates that can be processed in a batch.
--max_task_event_batch_size (default 300) [must be > 0]
-	The maximum number of task state change events that can be processed in a batch.
--max_tasks_per_job (default 4000) [must be > 0]
-	Maximum number of allowed tasks in a single job.
--max_tasks_per_schedule_attempt (default 5) [must be > 0]
-	The maximum number of tasks to pick in a single scheduling attempt.
--max_update_instance_failures (default 20000) [must be > 0]
-	Upper limit on the number of failures allowed during a job update. This helps cap potentially unbounded entries into storage.
--min_offer_hold_time (default (5, mins))
-	Minimum amount of time to hold a resource offer before declining.
--min_required_instances_for_sla_check (default 20)
-	Minimum number of instances required for a job to be eligible for SLA check. This does not apply to jobs that have a CoordinatorSlaPolicy.
--native_log_election_retries (default 20)
-	The maximum number of attempts to obtain a new log writer.
--native_log_election_timeout (default (15, secs))
-	The timeout for a single attempt to obtain a new log writer.
--native_log_file_path
-	Path to a file to store the native log data in.  If the parent directory doesnot exist it will be created.
--native_log_quorum_size (default 1)
-	The size of the quorum required for all log mutations.
--native_log_read_timeout (default (5, secs))
-	The timeout for doing log reads.
--native_log_write_timeout (default (3, secs))
-	The timeout for doing log appends and truncations.
--native_log_zk_group_path
-	A zookeeper node for use by the native log to track the master coordinator.
--offer_filter_duration (default (5, secs))
-	Duration after which we expect Mesos to re-offer unused resources. A short duration improves scheduling performance in smaller clusters, but might lead to resource starvation for other frameworks if you run many frameworks in your cluster.
--offer_hold_jitter_window (default (1, mins))
-	Maximum amount of random jitter to add to the offer hold time window.
--offer_reservation_duration (default (3, mins))
-	Time to reserve a agent's offers while trying to satisfy a task preempting another.
--offer_set_module (default [class org.apache.aurora.scheduler.offers.OfferSetModule])
-  Guice module for replacing offer holding and scheduling logic.
--partition_aware (default false)
-  Whether or not to integrate with the partition-aware Mesos capabilities.
--populate_discovery_info (default false)
-	If true, Aurora populates DiscoveryInfo field of Mesos TaskInfo.
--preemption_delay (default (3, mins))
-	Time interval after which a pending task becomes eligible to preempt other tasks
--preemption_slot_finder_modules (default [class org.apache.aurora.scheduler.preemptor.PendingTaskProcessorModule, class org.apache.aurora.scheduler.preemptor.PreemptionVictimFilterModule])
-  Guice modules for replacing preemption logic.
--preemption_slot_hold_time (default (5, mins))
-	Time to hold a preemption slot found before it is discarded.
--preemption_slot_search_interval (default (1, mins))
-	Time interval between pending task preemption slot searches.
--receive_revocable_resources (default false)
-	Allows receiving revocable resource offers from Mesos.
--reconciliation_explicit_batch_interval (default (5, secs))
-	Interval between explicit batch reconciliation requests.
--reconciliation_explicit_batch_size (default 1000) [must be > 0]
-	Number of tasks in a single batch request sent to Mesos for explicit reconciliation.
--reconciliation_explicit_interval (default (60, mins))
-	Interval on which scheduler will ask Mesos for status updates of all non-terminal tasks known to scheduler.
--reconciliation_implicit_interval (default (60, mins))
-	Interval on which scheduler will ask Mesos for status updates of all non-terminal tasks known to Mesos.
--reconciliation_initial_delay (default (1, mins))
-	Initial amount of time to delay task reconciliation after scheduler start up.
--reconciliation_schedule_spread (default (30, mins))
-	Difference between explicit and implicit reconciliation intervals intended to create a non-overlapping task reconciliation schedule.
--require_docker_use_executor (default true)
-	If false, Docker tasks may run without an executor (EXPERIMENTAL)
--scheduling_max_batch_size (default 3) [must be > 0]
-	The maximum number of scheduling attempts that can be processed in a batch.
--serverset_endpoint_name (default http)
-	Name of the scheduler endpoint published in ZooKeeper.
--shiro_ini_path
-	Path to shiro.ini for authentication and authorization configuration.
--shiro_realm_modules (default [class org.apache.aurora.scheduler.http.api.security.IniShiroRealmModule])
-	Guice modules for configuring Shiro Realms.
--sla_aware_action_max_batch_size (default 300) [must be > 0]
-	The maximum number of sla aware update actions that can be processed in a batch.
--sla_aware_kill_retry_min_delay (default (1, min)) [must be > 0]
-	The minimum amount of time to wait before retrying an SLA-aware kill (using a truncated binary backoff).
--sla_aware_kill_retry_max_delay (default (5, min)) [must be > 0]
-	The maximum amount of time to wait before retrying an SLA-aware kill (using a truncated binary backoff).
--sla_coordinator_timeout (default (1, min)) [must be > 0]
-	Timeout interval for communicating with Coordinator.
--sla_non_prod_metrics (default [])
-	Metric categories collected for non production tasks.
--sla_prod_metrics (default [JOB_UPTIMES, PLATFORM_UPTIME, MEDIANS])
-	Metric categories collected for production tasks.
--sla_stat_refresh_interval (default (1, mins))
-	The SLA stat refresh interval.
--stat_retention_period (default (1, hrs))
-	Time for a stat to be retained in memory before expiring.
--stat_sampling_interval (default (1, secs))
-	Statistic value sampling interval.
--task_assigner_modules (default [class org.apache.aurora.scheduler.scheduling.TaskAssignerImplModule])
-  Guice modules for replacing task assignment logic.
--thermos_executor_cpu (default 0.25)
-	The number of CPU cores to allocate for each instance of the executor.
--thermos_executor_flags
-	Extra arguments to be passed to the thermos executor
--thermos_executor_ram (default (128, MB))
-	The amount of RAM to allocate for each instance of the executor.
--thermos_executor_resources (default [])
-	A comma separated list of additional resources to copy into the sandbox.Note: if thermos_executor_path is not the thermos_executor.pex file itself, this must include it.
--thermos_home_in_sandbox (default false)
-	If true, changes HOME to the sandbox before running the executor. This primarily has the effect of causing the executor and runner to extract themselves into the sandbox.
--thrift_method_interceptor_modules (default [])
-	Additional Guice modules for intercepting Thrift method calls.
--transient_task_state_timeout (default (5, mins))
-	The amount of time after which to treat a task stuck in a transient state as LOST.
--viz_job_url_prefix (default )
-	URL prefix for job container stats.
--zk_chroot_path
-	chroot path to use for the ZooKeeper connections
--zk_digest_credentials
-	user:password to use when authenticating with ZooKeeper.
--zk_in_proc (default false)
-	Launches an embedded zookeeper server for local testing causing -zk_endpoints to be ignored if specified.
--zk_session_timeout (default (4, secs))
-	The ZooKeeper session timeout.
--zk_use_curator (default true)
-	DEPRECATED: Uses Apache Curator as the zookeeper client; otherwise a copy of Twitter commons/zookeeper (the legacy library) is used.
--------------------------------------------------------------------------
+Usage: org.apache.aurora.scheduler.app.SchedulerMain [options]
+  Options:
+    -allow_container_volumes
+      Allow passing in volumes in the job. Enabling this could pose a
+      privilege escalation threat.
+      Default: false
+    -allow_docker_parameters
+      Allow to pass docker container parameters in the job.
+      Default: false
+    -allow_gpu_resource
+      Allow jobs to request Mesos GPU resource.
+      Default: false
+    -allowed_container_types
+      Container types that are allowed to be used by jobs.
+      Default: [MESOS]
+    -allowed_job_environments
+      Regular expression describing the environments that are allowed to be
+      used by jobs.
+      Default: ^(prod|devel|test|staging\d*)$
+    -async_slot_stat_update_interval
+      Interval on which to try to update open slot stats.
+      Default: (1, mins)
+    -async_task_stat_update_interval
+      Interval on which to try to update resource consumption stats.
+      Default: (1, hrs)
+    -async_worker_threads
+      The number of worker threads to process async task operations with.
+      Default: 8
+  * -backup_dir
+      Directory to store backups under. Will be created if it does not exist.
+    -backup_interval
+      Minimum interval on which to write a storage backup.
+      Default: (1, hrs)
+  * -cluster_name
+      Name to identify the cluster being served.
+    -cron_scheduler_num_threads
+      Number of threads to use for the cron scheduler thread pool.
+      Default: 10
+    -cron_scheduling_max_batch_size
+      The maximum number of triggered cron jobs that can be processed in a
+      batch.
+      Default: 10
+    -cron_start_initial_backoff
+      Initial backoff delay while waiting for a previous cron run to be
+      killed.
+      Default: (5, secs)
+    -cron_start_max_backoff
+      Max backoff delay while waiting for a previous cron run to be killed.
+      Default: (1, mins)
+    -cron_timezone
+      TimeZone to use for cron predictions.
+      Default: GMT
+    -custom_executor_config
+      Path to custom executor settings configuration file.
+    -default_docker_parameters
+      Default docker parameters for any job that does not explicitly declare
+      parameters.
+      Default: []
+    -dlog_max_entry_size
+      Specifies the maximum entry size to append to the log. Larger entries
+      will be split across entry Frames.
+      Default: (512, KB)
+    -dlog_snapshot_interval
+      Specifies the frequency at which snapshots of local storage are taken
+      and written to the log.
+      Default: (1, hrs)
+    -enable_cors_for
+      List of domains for which CORS support should be enabled.
+    -enable_mesos_fetcher
+      Allow jobs to pass URIs to the Mesos Fetcher. Note that enabling this
+      feature could pose a privilege escalation threat.
+      Default: false
+    -enable_preemptor
+      Enable the preemptor and preemption
+      Default: true
+    -enable_revocable_cpus
+      Treat CPUs as a revocable resource.
+      Default: true
+    -enable_revocable_ram
+      Treat RAM as a revocable resource.
+      Default: false
+    -enable_update_affinity
+      Enable best-effort affinity of task updates.
+      Default: false
+    -executor_user
+      User to start the executor. Defaults to "root". Set this to an
+      unprivileged user if the mesos master was started with
+      "--no-root_submissions". If set to anything other than "root", the
+      executor will ignore the "role" setting for jobs since it can't use
+      setuid() anymore. This means that all your jobs will run under the
+      specified user and the user has to exist on the Mesos agents.
+      Default: root
+    -first_schedule_delay
+      Initial amount of time to wait before first attempting to schedule a
+      PENDING task.
+      Default: (1, ms)
+    -flapping_task_threshold
+      A task that repeatedly runs for less than this time is considered to be
+      flapping.
+      Default: (5, mins)
+    -framework_announce_principal
+      When 'framework_authentication_file' flag is set, the FrameworkInfo
+      registered with the mesos master will also contain the principal. This
+      is necessary if you intend to use mesos authorization via mesos ACLs.
+      The default will change in a future release. Changing this value is
+      backwards incompatible. For details, see MESOS-703.
+      Default: false
+    -framework_authentication_file
+      Properties file which contains framework credentials to authenticate
+      with Mesosmaster. Must contain the properties
+      'aurora_authentication_principal' and 'aurora_authentication_secret'.
+    -framework_failover_timeout
+      Time after which a framework is considered deleted.  SHOULD BE VERY
+      HIGH.
+      Default: (21, days)
+    -framework_name
+      Name used to register the Aurora framework with Mesos.
+      Default: Aurora
+    -global_container_mounts
+      A comma separated list of mount points (in host:container form) to mount
+      into all (non-mesos) containers.
+      Default: []
+    -history_max_per_job_threshold
+      Maximum number of terminated tasks to retain in a job history.
+      Default: 100
+    -history_min_retention_threshold
+      Minimum guaranteed time for task history retention before any pruning is
+      attempted.
+      Default: (1, hrs)
+    -history_prune_threshold
+      Time after which the scheduler will prune terminated task history.
+      Default: (2, days)
+    -hold_offers_forever
+      Hold resource offers indefinitely, disabling automatic offer decline
+      settings.
+      Default: false
+    -host_maintenance_polling_interval
+      Interval between polling for pending host maintenance requests.
+      Default: (1, mins)
+    -hostname
+      The hostname to advertise in ZooKeeper instead of the locally-resolved
+      hostname.
+    -http_authentication_mechanism
+      HTTP Authentication mechanism to use.
+      Default: NONE
+      Possible Values: [NONE, BASIC, NEGOTIATE]
+    -http_port
+      The port to start an HTTP server on.  Default value will choose a random
+      port.
+      Default: 0
+    -initial_flapping_task_delay
+      Initial amount of time to wait before attempting to schedule a flapping
+      task.
+      Default: (30, secs)
+    -initial_schedule_penalty
+      Initial amount of time to wait before attempting to schedule a task that
+      has failed to schedule.
+      Default: (1, secs)
+    -initial_task_kill_retry_interval
+      When killing a task, retry after this delay if mesos has not responded,
+      backing off up to transient_task_state_timeout
+      Default: (15, secs)
+    -ip
+      The ip address to listen. If not set, the scheduler will listen on all
+      interfaces.
+    -job_update_history_per_job_threshold
+      Maximum number of completed job updates to retain in a job update
+      history.
+      Default: 10
+    -job_update_history_pruning_interval
+      Job update history pruning interval.
+      Default: (15, mins)
+    -job_update_history_pruning_threshold
+      Time after which the scheduler will prune completed job update history.
+      Default: (30, days)
+    -kerberos_debug
+      Produce additional Kerberos debugging output.
+      Default: false
+    -kerberos_server_keytab
+      Path to the server keytab.
+    -kerberos_server_principal
+      Kerberos server principal to use, usually of the form
+      HTTP/aurora.example.com@EXAMPLE.COM
+    -max_flapping_task_delay
+      Maximum delay between attempts to schedule a flapping task.
+      Default: (5, mins)
+    -max_leading_duration
+      After leading for this duration, the scheduler should commit suicide.
+      Default: (1, days)
+    -max_parallel_coordinated_maintenance
+      Maximum number of coordinators that can be contacted in parallel.
+      Default: 10
+    -max_registration_delay
+      Max allowable delay to allow the driver to register before aborting
+      Default: (1, mins)
+    -max_reschedule_task_delay_on_startup
+      Upper bound of random delay for pending task rescheduling on scheduler
+      startup.
+      Default: (30, secs)
+    -max_saved_backups
+      Maximum number of backups to retain before deleting the oldest backups.
+      Default: 48
+    -max_schedule_attempts_per_sec
+      Maximum number of scheduling attempts to make per second.
+      Default: 40.0
+    -max_schedule_penalty
+      Maximum delay between attempts to schedule a PENDING tasks.
+      Default: (1, mins)
+    -max_sla_duration_secs
+      Maximum duration window for which SLA requirements are to be
+      satisfied.This does not apply to jobs that have a CoordinatorSlaPolicy.
+      Default: (2, hrs)
+    -max_status_update_batch_size
+      The maximum number of status updates that can be processed in a batch.
+      Default: 1000
+    -max_task_event_batch_size
+      The maximum number of task state change events that can be processed in
+      a batch.
+      Default: 300
+    -max_tasks_per_job
+      Maximum number of allowed tasks in a single job.
+      Default: 4000
+    -max_tasks_per_schedule_attempt
+      The maximum number of tasks to pick in a single scheduling attempt.
+      Default: 5
+    -max_update_instance_failures
+      Upper limit on the number of failures allowed during a job update. This
+      helps cap potentially unbounded entries into storage.
+      Default: 20000
+    -mesos_driver
+      Which Mesos Driver to use
+      Default: SCHEDULER_DRIVER
+      Possible Values: [SCHEDULER_DRIVER, V0_DRIVER, V1_DRIVER]
+  * -mesos_master_address
+      Address for the mesos master, can be a socket address or zookeeper path.
+    -mesos_role
+      The Mesos role this framework will register as. The default is to left
+      this empty, and the framework will register without any role and only
+      receive unreserved resources in offer.
+    -min_offer_hold_time
+      Minimum amount of time to hold a resource offer before declining.
+      Default: (5, mins)
+    -min_required_instances_for_sla_check
+      Minimum number of instances required for a job to be eligible for SLA
+      check. This does not apply to jobs that have a CoordinatorSlaPolicy.
+      Default: 20
+    -native_log_election_retries
+      The maximum number of attempts to obtain a new log writer.
+      Default: 20
+    -native_log_election_timeout
+      The timeout for a single attempt to obtain a new log writer.
+      Default: (15, secs)
+    -native_log_file_path
+      Path to a file to store the native log data in.  If the parent directory
+      doesnot exist it will be created.
+    -native_log_quorum_size
+      The size of the quorum required for all log mutations.
+      Default: 1
+    -native_log_read_timeout
+      The timeout for doing log reads.
+      Default: (5, secs)
+    -native_log_write_timeout
+      The timeout for doing log appends and truncations.
+      Default: (3, secs)
+    -native_log_zk_group_path
+      A zookeeper node for use by the native log to track the master
+      coordinator.
+    -offer_filter_duration
+      Duration after which we expect Mesos to re-offer unused resources. A
+      short duration improves scheduling performance in smaller clusters, but
+      might lead to resource starvation for other frameworks if you run many
+      frameworks in your cluster.
+      Default: (5, secs)
+    -offer_hold_jitter_window
+      Maximum amount of random jitter to add to the offer hold time window.
+      Default: (1, mins)
+    -offer_order
+      Iteration order for offers, to influence task scheduling. Multiple
+      orderings will be compounded together. E.g. CPU,MEMORY,RANDOM would sort
+      first by cpus offered, then memory and finally would randomize any equal
+      offers.
+      Default: [RANDOM]
+    -offer_reservation_duration
+      Time to reserve a agent's offers while trying to satisfy a task
+      preempting another.
+      Default: (3, mins)
+    -offer_set_module
+      Custom Guice module to provide a custom OfferSet.
+      Default: class org.apache.aurora.scheduler.offers.OfferManagerModule$OfferSetModule
+    -offer_static_ban_cache_max_size
+      The number of offers to hold in the static ban cache. If no value is
+      specified, the cache will grow indefinitely. However, entries will
+      expire within 'min_offer_hold_time' + 'offer_hold_jitter_window' of
+      being written.
+      Default: 9223372036854775807
+    -partition_aware
+      Enable paritition-aware status updates.
+      Default: false
+    -populate_discovery_info
+      If true, Aurora populates DiscoveryInfo field of Mesos TaskInfo.
+      Default: false
+    -preemption_delay
+      Time interval after which a pending task becomes eligible to preempt
+      other tasks
+      Default: (3, mins)
+    -preemption_reservation_max_batch_size
+      The maximum number of reservations for a task group to be made in a
+      batch.
+      Default: 5
+    -preemption_slot_finder_modules
+      Guice modules for custom preemption slot searching for pending tasks.
+      Default: [class org.apache.aurora.scheduler.preemptor.PendingTaskProcessorModule, class org.apache.aurora.scheduler.preemptor.PreemptionVictimFilterModule]
+    -preemption_slot_hold_time
+      Time to hold a preemption slot found before it is discarded.
+      Default: (5, mins)
+    -preemption_slot_search_initial_delay
+      Initial amount of time to delay preemption slot searching after
+      scheduler start up.
+      Default: (3, mins)
+    -preemption_slot_search_interval
+      Time interval between pending task preemption slot searches.
+      Default: (1, mins)
+    -receive_revocable_resources
+      Allows receiving revocable resource offers from Mesos.
+      Default: false
+    -reconciliation_explicit_batch_interval
+      Interval between explicit batch reconciliation requests.
+      Default: (5, secs)
+    -reconciliation_explicit_batch_size
+      Number of tasks in a single batch request sent to Mesos for explicit
+      reconciliation.
+      Default: 1000
+    -reconciliation_explicit_interval
+      Interval on which scheduler will ask Mesos for status updates of
+      allnon-terminal tasks known to scheduler.
+      Default: (60, mins)
+    -reconciliation_implicit_interval
+      Interval on which scheduler will ask Mesos for status updates of
+      allnon-terminal tasks known to Mesos.
+      Default: (60, mins)
+    -reconciliation_initial_delay
+      Initial amount of time to delay task reconciliation after scheduler
+      start up.
+      Default: (1, mins)
+    -reconciliation_schedule_spread
+      Difference between explicit and implicit reconciliation intervals
+      intended to create a non-overlapping task reconciliation schedule.
+      Default: (30, mins)
+    -require_docker_use_executor
+      If false, Docker tasks may run without an executor (EXPERIMENTAL)
+      Default: true
+    -scheduling_max_batch_size
+      The maximum number of scheduling attempts that can be processed in a
+      batch.
+      Default: 3
+    -serverset_endpoint_name
+      Name of the scheduler endpoint published in ZooKeeper.
+      Default: http
+  * -serverset_path
+      ZooKeeper ServerSet path to register at.
+    -shiro_after_auth_filter
+      Fully qualified class name of the servlet filter to be applied after the
+      shiro auth filters are applied.
+    -shiro_credentials_matcher
+      The shiro credentials matcher to use (will be constructed by Guice).
+      Default: class org.apache.shiro.authc.credential.SimpleCredentialsMatcher
+    -shiro_ini_path
+      Path to shiro.ini for authentication and authorization configuration.
+    -shiro_realm_modules
+      Guice modules for configuring Shiro Realms.
+      Default: [class org.apache.aurora.scheduler.http.api.security.IniShiroRealmModule]
+    -sla_aware_action_max_batch_size
+      The maximum number of sla aware update actions that can be processed in
+      a batch.
+      Default: 300
+    -sla_aware_kill_non_prod
+      Enables SLA awareness for drain and and update for non-production tasks
+      Default: false
+    -sla_aware_kill_retry_max_delay
+      Maximum amount of time to wait between attempting to perform an
+      SLA-Aware kill on a task.
+      Default: (5, mins)
+    -sla_aware_kill_retry_min_delay
+      Minimum amount of time to wait between attempting to perform an
+      SLA-Aware kill on a task.
+      Default: (1, mins)
+    -sla_coordinator_timeout
+      Timeout interval for communicating with Coordinator.
+      Default: (1, mins)
+    -sla_non_prod_metrics
+      Metric categories collected for non production tasks.
+      Default: []
+    -sla_prod_metrics
+      Metric categories collected for production tasks.
+      Default: [JOB_UPTIMES, PLATFORM_UPTIME, MEDIANS]
+    -sla_stat_refresh_interval
+      The SLA stat refresh interval.
+      Default: (1, mins)
+    -stat_retention_period
+      Time for a stat to be retained in memory before expiring.
+      Default: (1, hrs)
+    -stat_sampling_interval
+      Statistic value sampling interval.
+      Default: (1, secs)
+    -task_assigner_modules
+      Guice modules for customizing task assignment.
+      Default: [class org.apache.aurora.scheduler.scheduling.TaskAssignerImplModule]
+    -thermos_executor_cpu
+      The number of CPU cores to allocate for each instance of the executor.
+      Default: 0.25
+    -thermos_executor_flags
+      Extra arguments to be passed to the thermos executor
+    -thermos_executor_path
+      Path to the thermos executor entry point.
+    -thermos_executor_ram
+      The amount of RAM to allocate for each instance of the executor.
+      Default: (128, MB)
+    -thermos_executor_resources
+      A comma separated list of additional resources to copy into the
+      sandbox.Note: if thermos_executor_path is not the thermos_executor.pex
+      file itself, this must include it.
+      Default: []
+    -thermos_home_in_sandbox
+      If true, changes HOME to the sandbox before running the executor. This
+      primarily has the effect of causing the executor and runner to extract
+      themselves into the sandbox.
+      Default: false
+    -thrift_method_interceptor_modules
+      Custom Guice module(s) to provide additional Thrift method interceptors.
+      Default: []
+    -tier_config
+      Configuration file defining supported task tiers, task traits and
+      behaviors.
+    -transient_task_state_timeout
+      The amount of time after which to treat a task stuck in a transient
+      state as LOST.
+      Default: (5, mins)
+    -unavailability_threshold
+      Threshold time, when running tasks should be drained from a host, before
+      a host becomes unavailable. Should be greater than min_offer_hold_time +
+      offer_hold_jitter_window.
+      Default: (6, mins)
+    -update_affinity_reservation_hold_time
+      How long to wait for a reserved agent to reoffer freed up resources.
+      Default: (3, mins)
+    -viz_job_url_prefix
+      URL prefix for job container stats.
+      Default: <empty string>
+    -webhook_config
+      Path to webhook configuration file.
+    -zk_chroot_path
+      chroot path to use for the ZooKeeper connections
+    -zk_connection_timeout
+      The ZooKeeper connection timeout.
+      Default: (10, secs)
+    -zk_digest_credentials
+      user:password to use when authenticating with ZooKeeper.
+  * -zk_endpoints
+      Endpoint specification for the ZooKeeper servers.
+    -zk_in_proc
+      Launches an embedded zookeeper server for local testing causing
+      -zk_endpoints to be ignored if specified.
+      Default: false
+    -zk_session_timeout
+      The ZooKeeper session timeout.
+      Default: (15, secs)
 ```
diff --git a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
index 3ce9bc2..e58b79b 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
@@ -105,7 +105,7 @@
     public enum DriverKind {
       // TODO(zmanji): Remove this option once V0_DRIVER has been proven out in production.
       // This is the original driver that libmesos shipped with. Uses unversioned protobufs, and has
-      // minimal backwards compatability guarantees.
+      // minimal backwards compatibility guarantees.
       SCHEDULER_DRIVER,
       // These are the new drivers that libmesos ships with. They use versioned (V1) protobufs for
       // the Java API.
@@ -113,7 +113,7 @@
       // the V1 API (ie mesos maintenance).
       V0_DRIVER,
       // V1 Driver offers the V1 API over a full HTTP API implementation. It allows for maintenance
-      // primatives and other new features.
+      // primitives and other new features.
       V1_DRIVER,
     }