Переглянути джерело

Branching for 0.20.205 releases

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-205@1169684 13f79535-47bb-0310-9956-ffa450edef68
Matthew Foley 14 роки тому
батько
коміт
d32dcd3406
100 змінених файлів з 9558 додано та 1478 видалено
  1. 277 55
      CHANGES.txt
  2. 80 41
      bin/hadoop
  3. 1 5
      bin/hadoop-config.sh
  4. 12 2
      bin/hadoop-daemon.sh
  5. 5 1
      bin/hadoop-daemons.sh
  6. 5 1
      bin/rcc
  7. 5 1
      bin/slaves.sh
  8. 5 1
      bin/start-all.sh
  9. 5 1
      bin/start-balancer.sh
  10. 5 1
      bin/start-dfs.sh
  11. 5 1
      bin/start-jobhistoryserver.sh
  12. 5 1
      bin/start-mapred.sh
  13. 5 1
      bin/stop-all.sh
  14. 5 1
      bin/stop-balancer.sh
  15. 5 1
      bin/stop-dfs.sh
  16. 5 1
      bin/stop-jobhistoryserver.sh
  17. 5 1
      bin/stop-mapred.sh
  18. 33 6
      build.xml
  19. 12 0
      conf/fair-scheduler.xml.template
  20. 30 0
      conf/hadoop-metrics2.properties
  21. 17 0
      ivy.xml
  22. 5 1
      ivy/libraries.properties
  23. 9 17
      src/c++/task-controller/impl/task-controller.c
  24. 18 5
      src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacitySchedulerQueue.java
  25. 3 2
      src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java
  26. 166 3
      src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java
  27. 253 0
      src/contrib/fairscheduler/designdoc/fair_scheduler_design_doc.tex
  28. 8 0
      src/contrib/fairscheduler/ivy.xml
  29. 18 1
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/CapBasedLoadManager.java
  30. 13 4
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/DefaultTaskSelector.java
  31. 506 427
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
  32. 142 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerEventLog.java
  33. 113 86
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java
  34. 2 1
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FifoJobComparator.java
  35. 185 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/JobSchedulable.java
  36. 22 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/LoadManager.java
  37. 65 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/LocalityLevel.java
  38. 1 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/NewJobWeightBooster.java
  39. 41 1
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/Pool.java
  40. 228 23
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java
  41. 221 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolSchedulable.java
  42. 171 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/Schedulable.java
  43. 209 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/SchedulingAlgorithms.java
  44. 26 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/SchedulingMode.java
  45. 2 1
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/TaskSelector.java
  46. 1 0
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/WeightAdjuster.java
  47. 124 0
      src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/FakeSchedulable.java
  48. 150 0
      src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestCapBasedLoadManager.java
  49. 184 0
      src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestComputeFairShares.java
  50. 719 224
      src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java
  51. 199 0
      src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerSystem.java
  52. 3 0
      src/contrib/hdfsproxy/build.xml
  53. 9 5
      src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/TestHdfsProxy.java
  54. 5 0
      src/core/core-default.xml
  55. 13 0
      src/core/org/apache/hadoop/fs/FileSystem.java
  56. 95 0
      src/core/org/apache/hadoop/fs/FileUtil.java
  57. 384 196
      src/core/org/apache/hadoop/fs/HarFileSystem.java
  58. 8 65
      src/core/org/apache/hadoop/fs/RawLocalFileSystem.java
  59. 20 3
      src/core/org/apache/hadoop/http/HttpServer.java
  60. 24 0
      src/core/org/apache/hadoop/io/IOUtils.java
  61. 7 0
      src/core/org/apache/hadoop/io/SequenceFile.java
  62. 3 1
      src/core/org/apache/hadoop/io/nativeio/NativeIO.java
  63. 75 22
      src/core/org/apache/hadoop/ipc/Client.java
  64. 54 14
      src/core/org/apache/hadoop/ipc/RPC.java
  65. 2 2
      src/core/org/apache/hadoop/ipc/Server.java
  66. 287 0
      src/core/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
  67. 95 0
      src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaConf.java
  68. 99 0
      src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaMetricVisitor.java
  69. 198 0
      src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink30.java
  70. 104 0
      src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink31.java
  71. 24 3
      src/core/org/apache/hadoop/metrics2/util/MetricsCache.java
  72. 43 20
      src/core/org/apache/hadoop/net/CachedDNSToSwitchMapping.java
  73. 7 0
      src/core/org/apache/hadoop/net/DNSToSwitchMapping.java
  74. 47 1
      src/core/org/apache/hadoop/net/NetUtils.java
  75. 1 1
      src/core/org/apache/hadoop/net/ScriptBasedMapping.java
  76. 75 0
      src/core/org/apache/hadoop/security/AuthenticationFilterInitializer.java
  77. 47 16
      src/core/org/apache/hadoop/security/SecurityUtil.java
  78. 15 0
      src/core/org/apache/hadoop/security/authentication/README.txt
  79. 274 0
      src/core/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
  80. 50 0
      src/core/org/apache/hadoop/security/authentication/client/AuthenticationException.java
  81. 39 0
      src/core/org/apache/hadoop/security/authentication/client/Authenticator.java
  82. 270 0
      src/core/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
  83. 74 0
      src/core/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java
  84. 404 0
      src/core/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
  85. 89 0
      src/core/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
  86. 226 0
      src/core/org/apache/hadoop/security/authentication/server/AuthenticationToken.java
  87. 310 0
      src/core/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
  88. 134 0
      src/core/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java
  89. 395 0
      src/core/org/apache/hadoop/security/authentication/util/KerberosName.java
  90. 100 0
      src/core/org/apache/hadoop/security/authentication/util/Signer.java
  91. 31 0
      src/core/org/apache/hadoop/security/authentication/util/SignerException.java
  92. 112 0
      src/core/org/apache/hadoop/security/token/Token.java
  93. 65 0
      src/core/org/apache/hadoop/security/token/TokenRenewer.java
  94. 58 0
      src/core/org/apache/hadoop/util/ChecksumUtil.java
  95. 2 0
      src/core/org/apache/hadoop/util/DiskChecker.java
  96. 200 0
      src/docs/releasenotes.html
  97. 88 0
      src/docs/relnotes.py
  98. 124 0
      src/docs/src/documentation/content/xdocs/HttpAuthentication.xml
  99. 434 211
      src/docs/src/documentation/content/xdocs/fair_scheduler.xml
  100. 4 0
      src/docs/src/documentation/content/xdocs/site.xml

+ 277 - 55
CHANGES.txt

@@ -2,77 +2,210 @@ Hadoop Change Log
 
 
 Release 0.20.205.0 - unreleased
 Release 0.20.205.0 - unreleased
 
 
+  NEW FEATURES
+
+    HDFS-2202. Add a new DFSAdmin command to set balancer bandwidth of
+    datanodes without restarting.  (Eric Payne via szetszwo)
+
+    HDFS-200. Support append and sync for hadoop 0.20 branch. (dhruba)
+
+    HDFS-826. Allow a mechanism for an application to detect that 
+    datanode(s) have died in the write pipeline. (dhruba)
+
+    HDFS-142. Blocks that are being written by a client are stored in the
+    blocksBeingWritten directory. 
+    (Dhruba Borthakur, Nicolas Spiegelberg, Todd Lipcon via dhruba)
+
+    HDFS-630. Client can exclude specific nodes in the write pipeline.
+    (Nicolas Spiegelberg via dhruba)
+
+    HDFS-895. Allow hflush/sync to occur in parallel with new writes to
+    the file. (Todd Lipcon via hairong)
+
+    HDFS-1520. Lightweight NameNode operation recoverLease to trigger 
+    lease recovery. (Hairong Kuang via dhruba)
+
+    MAPREDUCE-2764. Allow JobTracker to renew and cancel arbitrary token types,
+    including delegation tokens obtained via hftp. (omalley)
+
+    HADOOP-7119 add Kerberos HTTP SPNEGO authentication support to 
+    Hadoop JT/NN/DN/TT web-consoles backport from Trunk (sanjay)
+
+    HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http
+    access to HDFS.  (szetszwo)
+
   BUG FIXES
   BUG FIXES
 
 
-    HDFS-1878. TestHDFSServerPorts unit test failure - race condition 
-    in FSNamesystem.close() causes NullPointerException without serious
-    consequence. (mattf)
+    MAPREDUCE-2324. Removed usage of broken
+    ResourceEstimator.getEstimatedReduceInputSize to check against usable
+    disk-space on TaskTracker. (Robert Evans via acmurthy) 
 
 
-    MAPREDUCE-2452. Moves the cancellation of delegation tokens to a separate
-    thread. (ddas)
+    MAPREDUCE-2729. Ensure jobs with reduces which can't be launched due to
+    slow-start do not count for user-limits. (Sherry Chen via acmurthy) 
 
 
-    HDFS-1377. Quota bug for partial blocks allows quotas to be violated. (eli)
+    HADOOP-6833. IPC leaks call parameters when exceptions thrown.
+    (Todd Lipcon via eli)
+  
+    HADOOP-7400. Fix HdfsProxyTests fails when the -Dtest.build.dir 
+    and -Dbuild.test is set a dir other than build dir (gkesavan).
 
 
-    MAPREDUCE-2451. Log the details from health check script at the
-    JobTracker. (Thomas Graves via cdouglas)
+    MAPREDUCE-2650. back-port MAPREDUCE-2238 to 0.20-security. 
+    (Sherry Chen via mahadev)
 
 
-    MAPREDUCE-2456. Log the reduce taskID and associated TaskTrackers with
-    failed fetch notifications in the JobTracker log.
-    (Jeffrey Naisbitt via cdouglas)
+    HDFS-2053. Bug in INodeDirectory#computeContentSummary warning
+    (Michael Noll via eli)
 
 
-    HADOOP-7248. Update eclipse target to generate .classpath from ivy config.
-    (Thomas Graves and Tom White via cdouglas)
+    HDFS-2117. DiskChecker#mkdirsWithExistsAndPermissionCheck may
+    return true even when the dir is not created. (eli)
+    
+    MAPREDUCE-2489. Jobsplits with random hostnames can make the 
+    queue unusable. (Jeffrey Naisbitt via mahadev)
 
 
-    HADOOP-7277. Add generation of run configurations to eclipse target.
-    (Jeffrey Naisbitt and Philip Zeyliger via cdouglas)
+    HDFS-2190. NN fails to start if it encounters an empty or malformed fstime
+    file. (atm)
 
 
-    HADOOP-7274. Fix typos in IOUtils. (Jonathan Eagles via cdouglas)
+    HDFS-2259. DN web-UI doesn't work with paths that contain html. (eli)
 
 
-    MAPREDUCE-2479. Move distributed cache cleanup to a background task,
-    backporting MAPREDUCE-1568. (Robert Joseph Evans via cdouglas)
+    HDFS-561. Fix write pipeline READ_TIMEOUT.
+    (Todd Lipcon via dhruba)
 
 
-    HADOOP-7272. Remove unnecessary security related info logs. (suresh)
+    HDFS-606. Fix ConcurrentModificationException in invalidateCorruptReplicas.
+    (Todd Lipcon via dhruba)
 
 
-    HDFS-1906. Remove logging exception stack trace in client logs when one of
-    the datanode targets to read from is not reachable. (suresh)
+    HDFS-1118. Fix socketleak on DFSClient. 
+    (Zheng Shao via dhruba)
 
 
-    MAPREDUCE-2514. Fix typo in TaskTracker ReinitTrackerAction log message.
-    (Jonathan Eagles via cdouglas)
+    HDFS-988. Fix bug where savenameSpace can corrupt edits log.
+    (Nicolas Spiegelberg via dhruba)
 
 
-    MAPREDUCE-2490. Add logging to graylist and blacklist activity to aid
-    diagnosis of related issues. (Jonathan Eagles via cdouglas)
+    HDFS-1054. remove sleep before retry for allocating a block.
+    (Todd Lipcon via dhruba)
 
 
-    MAPREDUCE-2495. exit() the TaskTracker when the distributed cache cleanup
-    thread dies. (Robert Joseph Evans via cdouglas)
+    HDFS-1207. FSNamesystem.stallReplicationWork should be volatile.
+    (Todd Lipcon via dhruba)
 
 
-    MAPREDUCE-2555. Avoid sprious logging from completedtasks. (Thomas Graves
-    via cdouglas)
+    HDFS-1141. completeFile does not check lease ownership.
+    (Todd Lipcon via dhruba)
 
 
-    MAPREDUCE-2558. Add JobTracker metrics for scheduling queues. (Jeffrey
-    Naisbitt via cdouglas)
+    HDFS-1204. Lease expiration should recover single files, 
+    not entire lease holder (Sam Rash via dhruba)
 
 
-    MAPREDUCE-2535. Fix NPE in JobClient caused by retirement. (Robert Joseph
-    Evans via cdouglas)
+    HDFS-1346. DFSClient receives out of order packet ack. (hairong)
 
 
-    HADOOP-7144. Expose JMX metrics via JSON servlet. (Robert Joseph Evans via
-    cdouglas)
+    HDFS-1057.  Concurrent readers hit ChecksumExceptions if following 
+    a writer to very end of file (Sam Rash via dhruba)
 
 
-    MAPREDUCE-2524. Port reduce failure reporting semantics from trunk, to
-    fail faulty maps more aggressively. (Thomas Graves via cdouglas)
+    HDFS-724.  Use a bidirectional heartbeat to detect stuck
+    pipeline. (hairong)
 
 
-    MAPREDUCE-2529. Add support for regex-based shuffle metric counting
-    exceptions. (Thomas Graves via cdouglas)
+    HDFS-1555. Disallow pipelien recovery if a file is already being
+    lease recovered. (hairong)
 
 
-    HDFS-2044. TestQueueProcessingStatistics failing automatic test due to 
-    timing issues. (mattf)
+    HDFS-1554. New semantics for recoverLease. (hairong)
 
 
-    MAPREDUCE-2558. Add queue-level metrics 0.20-security branch - test fix
-    (jeffrey nasbit via mahadev)
-   
-    HADOOP-7364. TestMiniMRDFSCaching fails if test.build.dir is set to something other 
-    than build/test. (Thomas Graves via mahadev)
+    HADOOP-7596. Makes packaging of 64-bit jsvc possible. Has other
+    bug fixes to do with packaging. (Eric Yang via ddas) 
+
+    HDFS-2309. TestRenameWhileOpen fails. (jitendra)
+
+    HDFS-2300. TestFileAppend4 and TestMultiThreadedSync failure. (jitendra)
+
+    HDFS-1122. client block verification may result in blocks in 
+    DataBlockScanner prematurely. (Sam Rash via jitendra)
+
+    HADOOP-6722. NetUtils.connect should check that it hasn't connected a socket
+    to itself. (Todd Lipcon via suresh)
+
+    HDFS-1779. After NameNode restart , Clients can not read partial files even after 
+    client invokes Sync. (Uma Maheswara Rao G via jitendra)
+
+    HDFS-1197. Blocks are considered "complete" prematurely after 
+    commitBlockSynchronization or DN restart. (Todd Lipcon via jitendra)
+
+    HDFS-1218. Blocks recovered on startup should be treated with lower 
+    priority during block synchronization. (Todd Lipcon via suresh)
+
+    HDFS-1186. DNs should interrupt writers at start of recovery.
+    (Todd Lipcon via suresh)
+
+    HDFS-1252. Fix TestDFSConcurrentFileOperations.
+    (Todd Lipcon via suresh).
+
+    HDFS-1260. Block lost when multiple DNs trying to recover it to different
+    genstamps. (Todd Lipcon via jitendra)
+
+  IMPROVEMENTS
+
+    MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via
+    acmurthy) 
+
+    MAPREDUCE-2705. Implements launch of multiple tasks concurrently.
+    (Thomas Graves via ddas)
+
+    HADOOP-7343. Make the number of warnings accepted by test-patch
+    configurable to limit false positives. (Thomas Graves via cdouglas)
+
+    HDFS-1836. Thousand of CLOSE_WAIT socket. Contributed by Todd Lipcon,
+    ported to security branch by Bharath Mundlapudi. (via mattf)
+
+    HADOOP-7432. Back-port HADOOP-7110 to 0.20-security: Implement chmod
+    in NativeIO library. (Sherry Chen via mattf)
+
+    HADOOP-7314. Add support for throwing UnknownHostException when a host
+    doesn't resolve. Needed for MAPREDUCE-2489. (Jeffrey Naisbitt via mattf)
+
+    MAPREDUCE-2494. Make the distributed cache delete entires using LRU 
+    priority (Robert Joseph Evans via mahadev)
+
+    HADOOP-6889. Make RPC to have an option to timeout - backport to 
+    0.20-security. Unit tests updated to 17/Aug/2011 version.
+    (John George and Ravi Prakash via mattf)
+
+    MAPREDUCE-2780. Use a utility method to set service in token. 
+    (Daryn Sharp via jitendra)
+
+    HADOOP-7472. RPC client should deal with IP address change.
+    (Kihwal Lee via suresh)
+  
+    MAPREDUCE-2489. Jobsplits with random hostnames can make the queue unusable
+    (Jeffrey Naisbit via mahadev)
+
+    MAPREDUCE-2852. Jira for YDH bug 2854624. (Kihwal Lee via eli)
+
+    HDFS-1210. DFSClient should log exception when block recovery fails.
+    (Todd Lipcon via dhruba)
+ 
+    HDFS-1211. Block receiver should not log "rewind" packets at INFO level.
+    (Todd Lipcon)
+
+    HDFS-1164. TestHdfsProxy is failing. (Todd Lipcon)
+
+    HDFS-1202. DataBlockScanner throws NPE when updated before initialized. 
+    (Todd Lipcon)
+
+    HADOOP-7539. merge hadoop archive goodness from trunk to .20 (John George 
+    via mahadev)
+
+    HADOOP-7594. Support HTTP REST in HttpServer.  (szetszwo)
+
+    HDFS-1242. Add test for appendFile() race solved in HDFS-142.
+    (Todd Lipcon via jitendra)
+
+    HDFS-2320. Make 0.20-append protocol changes compatible with
+    0.20-secuirty. (suresh)
 
 
-Release 0.20.204.0 - unreleased
+    MAPREDUCE-2610. Make QueueAclsInfo public. (Joep Rottinghuis via acmurthy) 
+
+    MAPREDUCE-2915. Ensure LTC passes java.library.path. (Kihwal Lee via
+    acmurthy) 
+
+    HADOOP-7599. Script improvements to setup a secure Hadoop cluster 
+    (Eric Yang via ddas)
+
+    MAPREDUCE-2981. Backport FairScheduler from trunk. (Matei Zaharia via
+    acmurthy) 
+
+Release 0.20.204.0 - 2011-8-25
 
 
   NEW FEATURES
   NEW FEATURES
 
 
@@ -81,16 +214,88 @@ Release 0.20.204.0 - unreleased
     scripts for easy one node cluster configuration and user creation.
     scripts for easy one node cluster configuration and user creation.
     (Eric Yang via omalley)
     (Eric Yang via omalley)
 
 
+    HADOOP-7324. Ganglia plugins for metrics v2. (Priyo Mustafi via llu)
+
   BUG FIXES
   BUG FIXES
 
 
+    MAPREDUCE-2804. Fixed a race condition in setting up the log directories
+    for tasks that are starting at the same time. (omalley)
+
+    MAPREDUCE-2846. Fixed a race condition in writing the log index file that
+    caused tasks to fail. (omalley)
+
+    MAPREDUCE-2651. Fix race condition in Linux task controller for
+    job log directory creation. (Bharath Mundlapudi via llu)
+
+    MAPREDUCE-2621. TestCapacityScheduler fails with "Queue "q1" does not 
+    exist". (Sherry Chen via mahadev)
+
+    HADOOP-7475. Fix hadoop-setup-single-node.sh to reflect new layout. (eyang
+    via omalley)
+
+    HADOOP-7045. TestDU fails on systems with local file systems with 
+    extended attributes. (eli)
+
+    MAPREDUCE-2495. exit() the TaskTracker when the distributed cache cleanup
+    thread dies. (Robert Joseph Evans via cdouglas)
+
+    HDFS-1878. TestHDFSServerPorts unit test failure - race condition 
+    in FSNamesystem.close() causes NullPointerException without serious
+    consequence. (mattf)
+
+    MAPREDUCE-2452. Moves the cancellation of delegation tokens to a separate
+    thread. (ddas)
+
+    MAPREDUCE-2555. Avoid sprious logging from completedtasks. (Thomas Graves
+    via cdouglas)
+
+    MAPREDUCE-2451. Log the details from health check script at the
+    JobTracker. (Thomas Graves via cdouglas)
+
+    MAPREDUCE-2535. Fix NPE in JobClient caused by retirement. (Robert Joseph
+    Evans via cdouglas)
+
+    MAPREDUCE-2456. Log the reduce taskID and associated TaskTrackers with
+    failed fetch notifications in the JobTracker log.
+    (Jeffrey Naisbitt via cdouglas)
+
+    HDFS-2044. TestQueueProcessingStatistics failing automatic test due to 
+    timing issues. (mattf)
+
+    HADOOP-7248. Update eclipse target to generate .classpath from ivy config.
+    (Thomas Graves and Tom White via cdouglas)
+
+    MAPREDUCE-2558. Add queue-level metrics 0.20-security branch (test fixes)
+    (Jeffrey Naisbitt via mahadev)
+    
+    HADOOP-7364. TestMiniMRDFSCaching fails if test.build.dir is set to 
+    something other than build/test. (Thomas Graves via mahadev)
+
+    HADOOP-7277. Add generation of run configurations to eclipse target.
+    (Jeffrey Naisbitt and Philip Zeyliger via cdouglas)
+
     HADOOP-7373. Fix {start,stop}-{dfs,mapred} and hadoop-daemons.sh from
     HADOOP-7373. Fix {start,stop}-{dfs,mapred} and hadoop-daemons.sh from
     trying to use the wrong bin directory. (omalley)
     trying to use the wrong bin directory. (omalley)
 
 
+    HADOOP-7274. Fix typos in IOUtils. (Jonathan Eagles via cdouglas)
+
     HADOOP-7369. Fix permissions in tarball for sbin/* and libexec/* (omalley)
     HADOOP-7369. Fix permissions in tarball for sbin/* and libexec/* (omalley)
 
 
+    MAPREDUCE-2479. Move distributed cache cleanup to a background task,
+    backporting MAPREDUCE-1568. (Robert Joseph Evans via cdouglas)
+
     HADOOP-7356. Fix bin/hadoop scripts (eyang via omalley)
     HADOOP-7356. Fix bin/hadoop scripts (eyang via omalley)
 
 
-    MAPREDUCE-2316. Updated CapacityScheduler documentation. (acmurthy) 
+    HADOOP-7272. Remove unnecessary security related info logs. (suresh)
+
+    MAPREDUCE-2514. Fix typo in TaskTracker ReinitTrackerAction log message.
+    (Jonathan Eagles via cdouglas)
+
+    HDFS-1906. Remove logging exception stack trace in client logs when one of
+    the datanode targets to read from is not reachable. (suresh)
+
+    MAPREDUCE-2490. Add logging to graylist and blacklist activity to aid
+    diagnosis of related issues. (Jonathan Eagles via cdouglas)
 
 
     MAPREDUCE-2447. Fix Child.java to set Task.jvmContext sooner to avoid
     MAPREDUCE-2447. Fix Child.java to set Task.jvmContext sooner to avoid
     corner cases in error handling. (Siddharth Seth via acmurthy) 
     corner cases in error handling. (Siddharth Seth via acmurthy) 
@@ -128,9 +333,6 @@ Release 0.20.204.0 - unreleased
     HDFS-1258. Clearing namespace quota on "/" corrupts fs image.  
     HDFS-1258. Clearing namespace quota on "/" corrupts fs image.  
     (Aaron T. Myers via szetszwo)
     (Aaron T. Myers via szetszwo)
 
 
-    HADOOP-7215. RPC clients must use network interface corresponding to 
-    the host in the client's kerberos principal key. (suresh)
-
     HDFS-1189. Quota counts missed between clear quota and set quota.
     HDFS-1189. Quota counts missed between clear quota and set quota.
     (John George via szetszwo)
     (John George via szetszwo)
 
 
@@ -147,12 +349,29 @@ Release 0.20.204.0 - unreleased
     by throwing an error to indicate the editlog needs to be empty.
     by throwing an error to indicate the editlog needs to be empty.
     (suresh)
     (suresh)
 
 
+    HDFS-1377. Quota bug for partial blocks allows quotas to be violated. (eli)
+
     HDFS-2057. Wait time to terminate the threads causes unit tests to
     HDFS-2057. Wait time to terminate the threads causes unit tests to
     take longer time. (Bharath Mundlapudi via suresh)
     take longer time. (Bharath Mundlapudi via suresh)
 
 
+    HDFS-2218. Disable TestHdfsProxy.testHdfsProxyInterface in automated test 
+    suite for 0.20-security-204 release. (Matt Foley)
+    
+    HADOOP-7610. Fix for hadoop debian package (Eric Yang via gkesavan)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
-    HADOOP-7398. Suppress warnings about use of HADOOP_HOME. (omalley)
+    HADOOP-7144. Expose JMX metrics via JSON servlet. (Robert Joseph Evans via
+    cdouglas)
+
+    MAPREDUCE-2524. Port reduce failure reporting semantics from trunk, to
+    fail faulty maps more aggressively. (Thomas Graves via cdouglas)
+
+    MAPREDUCE-2529. Add support for regex-based shuffle metric counting
+    exceptions. (Thomas Graves via cdouglas)
+
+    HADOOP-7398. Add mechanism to suppress warnings about use of HADOOP_HOME.
+    (omalley)
 
 
     HDFS-2023. Backport of NPE for File.list and File.listFiles.
     HDFS-2023. Backport of NPE for File.list and File.listFiles.
     Merged ports of HADOOP-7322, HDFS-1934, HADOOP-7342, and HDFS-2019.
     Merged ports of HADOOP-7322, HDFS-1934, HADOOP-7342, and HDFS-2019.
@@ -172,7 +391,7 @@ Release 0.20.204.0 - unreleased
 
 
     MAPREDUCE-1251. c++ utils doesn't compile. (Eli Collins via shv)
     MAPREDUCE-1251. c++ utils doesn't compile. (Eli Collins via shv)
 
 
-Release 0.20.203.1 - Unreleased
+    HADOOP-7459. Remove jdk-1.6.0 dependency check from rpm. (omalley)
 
 
     HADOOP-7330. Fix MetricsSourceAdapter to use the value instead of the 
     HADOOP-7330. Fix MetricsSourceAdapter to use the value instead of the 
     object. (Luke Lu via omalley)
     object. (Luke Lu via omalley)
@@ -195,10 +414,10 @@ Release 0.20.203.0 - 2011-5-11
 
 
     HADOOP-7143. Restore HadoopArchives. (Joep Rottinghuis via omalley)
     HADOOP-7143. Restore HadoopArchives. (Joep Rottinghuis via omalley)
 
 
+    MAPREDUCE-2316. Updated CapacityScheduler documentation. (acmurthy) 
+
     HADOOP-7243. Fix contrib unit tests missing dependencies. (omalley)
     HADOOP-7243. Fix contrib unit tests missing dependencies. (omalley)
 
 
-    MAPREDUCE-2355. Add a dampner to out-of-band heartbeats. (acmurthy) 
- 
     HADOOP-7190. Add metrics v1 back for backwards compatibility. (omalley)
     HADOOP-7190. Add metrics v1 back for backwards compatibility. (omalley)
 
 
     MAPREDUCE-2360. Remove stripping of scheme, authority from submit dir in 
     MAPREDUCE-2360. Remove stripping of scheme, authority from submit dir in 
@@ -249,6 +468,9 @@ Release 0.20.203.0 - 2011-5-11
     HADOOP-6879. Provide SSH based (Jsch) remote execution API for system
     HADOOP-6879. Provide SSH based (Jsch) remote execution API for system
     tests. (cos)
     tests. (cos)
 
 
+    HADOOP-7215. RPC clients must use network interface corresponding to 
+    the host in the client's kerberos principal key. (suresh)
+
     HADOOP-7232. Fix Javadoc warnings. (omalley)
     HADOOP-7232. Fix Javadoc warnings. (omalley)
 
 
     HADOOP-7258. The Gzip codec should not return null decompressors. (omalley)
     HADOOP-7258. The Gzip codec should not return null decompressors. (omalley)

+ 80 - 41
bin/hadoop

@@ -50,7 +50,16 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ "$HADOOP_HOME_WARN_SUPPRESS" == "" ] && [ "$HADOOP_HOME" != "" ]; then
+  echo "Warning: \$HADOOP_HOME is deprecated." 1>&2
+  echo 1>&2
+fi
+
+if [ -e "$bin"/../libexec/hadoop-config.sh ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin"/hadoop-config.sh
+fi
 
 
 cygwin=false
 cygwin=false
 case "`uname`" in
 case "`uname`" in
@@ -152,34 +161,58 @@ fi
 IFS=
 IFS=
 
 
 # for releases, add core hadoop jar & webapps to CLASSPATH
 # for releases, add core hadoop jar & webapps to CLASSPATH
-if [ -d "$HADOOP_HOME/webapps" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME
-fi
-for f in $HADOOP_HOME/hadoop-core-*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-# add libs to CLASSPATH
-for f in $HADOOP_HOME/lib/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-if [ -d "$HADOOP_HOME/build/ivy/lib/Hadoop/common" ]; then
-for f in $HADOOP_HOME/build/ivy/lib/Hadoop/common/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-fi
+if [ -e $HADOOP_PREFIX/share/hadoop/hadoop-core-* ]; then
+  # binary layout
+  if [ -d "$HADOOP_PREFIX/share/hadoop/webapps" ]; then
+    CLASSPATH=${CLASSPATH}:$HADOOP_PREFIX/share/hadoop
+  fi
+  for f in $HADOOP_PREFIX/share/hadoop/hadoop-core-*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+
+  # add libs to CLASSPATH
+  for f in $HADOOP_PREFIX/share/hadoop/lib/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+
+  for f in $HADOOP_PREFIX/share/hadoop/lib/jsp-2.1/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+
+  for f in $HADOOP_PREFIX/share/hadoop/hadoop-tools-*.jar; do
+    TOOL_PATH=${TOOL_PATH}:$f;
+  done
+else
+  # tarball layout
+  if [ -d "$HADOOP_HOME/webapps" ]; then
+    CLASSPATH=${CLASSPATH}:$HADOOP_HOME
+  fi
+  for f in $HADOOP_HOME/hadoop-core-*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+
+  # add libs to CLASSPATH
+  for f in $HADOOP_HOME/lib/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+
+  if [ -d "$HADOOP_HOME/build/ivy/lib/Hadoop/common" ]; then
+    for f in $HADOOP_HOME/build/ivy/lib/Hadoop/common/*.jar; do
+      CLASSPATH=${CLASSPATH}:$f;
+    done
+  fi
 
 
-for f in $HADOOP_HOME/lib/jsp-2.1/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
+  for f in $HADOOP_HOME/lib/jsp-2.1/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
 
 
-for f in $HADOOP_HOME/hadoop-tools-*.jar; do
-  TOOL_PATH=${TOOL_PATH}:$f;
-done
-for f in $HADOOP_HOME/build/hadoop-tools-*.jar; do
-  TOOL_PATH=${TOOL_PATH}:$f;
-done
+  for f in $HADOOP_HOME/hadoop-tools-*.jar; do
+    TOOL_PATH=${TOOL_PATH}:$f;
+  done
+  for f in $HADOOP_HOME/build/hadoop-tools-*.jar; do
+    TOOL_PATH=${TOOL_PATH}:$f;
+  done
+fi
 
 
 # add user-specified CLASSPATH last
 # add user-specified CLASSPATH last
 if [ "$HADOOP_USER_CLASSPATH_FIRST" = "" ] && [ "$HADOOP_CLASSPATH" != "" ]; then
 if [ "$HADOOP_USER_CLASSPATH_FIRST" = "" ] && [ "$HADOOP_CLASSPATH" != "" ]; then
@@ -293,9 +326,15 @@ if $cygwin; then
 fi
 fi
 # setup 'java.library.path' for native-hadoop code if necessary
 # setup 'java.library.path' for native-hadoop code if necessary
 JAVA_LIBRARY_PATH=''
 JAVA_LIBRARY_PATH=''
-if [ -d "${HADOOP_HOME}/build/native" -o -d "${HADOOP_HOME}/lib/native" ]; then
+if [ -d "${HADOOP_HOME}/build/native" -o -d "${HADOOP_HOME}/lib/native" -o -e "${HADOOP_PREFIX}/lib/libhadoop.a" ]; then
   JAVA_PLATFORM=`CLASSPATH=${CLASSPATH} ${JAVA} -Xmx32m ${HADOOP_JAVA_PLATFORM_OPTS} org.apache.hadoop.util.PlatformName | sed -e "s/ /_/g"`
   JAVA_PLATFORM=`CLASSPATH=${CLASSPATH} ${JAVA} -Xmx32m ${HADOOP_JAVA_PLATFORM_OPTS} org.apache.hadoop.util.PlatformName | sed -e "s/ /_/g"`
   
   
+  if [ "$JAVA_PLATFORM" = "Linux-amd64-64" ]; then
+    JSVC_ARCH="amd64"
+  else
+    JSVC_ARCH="i386"
+  fi
+
   if [ -d "$HADOOP_HOME/build/native" ]; then
   if [ -d "$HADOOP_HOME/build/native" ]; then
     JAVA_LIBRARY_PATH=${HADOOP_HOME}/build/native/${JAVA_PLATFORM}/lib
     JAVA_LIBRARY_PATH=${HADOOP_HOME}/build/native/${JAVA_PLATFORM}/lib
   fi
   fi
@@ -307,11 +346,11 @@ if [ -d "${HADOOP_HOME}/build/native" -o -d "${HADOOP_HOME}/lib/native" ]; then
       JAVA_LIBRARY_PATH=${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
       JAVA_LIBRARY_PATH=${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
     fi
     fi
   fi
   fi
-fi
-if [ -e "${HADOOP_PREFIX}/lib/libhadoop.a" ]; then
-  JAVA_LIBRARY_PATH=${HADOOP_PREFIX}/lib
-fi
 
 
+  if [ -e "${HADOOP_PREFIX}/lib/libhadoop.a" ]; then
+    JAVA_LIBRARY_PATH=${HADOOP_PREFIX}/lib
+  fi
+fi
 
 
 # cygwin path translation
 # cygwin path translation
 if $cygwin; then
 if $cygwin; then
@@ -340,17 +379,17 @@ if [ "$starting_secure_dn" = "true" ]; then
   if [ "$HADOOP_PID_DIR" = "" ]; then
   if [ "$HADOOP_PID_DIR" = "" ]; then
     HADOOP_SECURE_DN_PID="/tmp/hadoop_secure_dn.pid"
     HADOOP_SECURE_DN_PID="/tmp/hadoop_secure_dn.pid"
   else
   else
-   HADOOP_SECURE_DN_PID="$HADOOP_PID_DIR/hadoop_secure_dn.pid"
+    HADOOP_SECURE_DN_PID="$HADOOP_PID_DIR/hadoop_secure_dn.pid"
   fi
   fi
 
 
-  exec "$HADOOP_HOME/libexec/jsvc" -Dproc_$COMMAND -outfile "$HADOOP_LOG_DIR/jsvc.out" \
-                                                   -errfile "$HADOOP_LOG_DIR/jsvc.err" \
-                                                   -pidfile "$HADOOP_SECURE_DN_PID" \
-                                                   -nodetach \
-                                                   -user "$HADOOP_SECURE_DN_USER" \
-                                                   -cp "$CLASSPATH" \
-                                                   $JAVA_HEAP_MAX $HADOOP_OPTS \
-                                                   org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter "$@"
+  exec "$HADOOP_HOME/libexec/jsvc.${JSVC_ARCH}" -Dproc_$COMMAND -outfile "$HADOOP_LOG_DIR/jsvc.out" \
+                                                -errfile "$HADOOP_LOG_DIR/jsvc.err" \
+                                                -pidfile "$HADOOP_SECURE_DN_PID" \
+                                                -nodetach \
+                                                -user "$HADOOP_SECURE_DN_USER" \
+                                                -cp "$CLASSPATH" \
+                                                $JAVA_HEAP_MAX $HADOOP_OPTS \
+                                                org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter "$@"
 else
 else
   # run it
   # run it
   exec "$JAVA" -Dproc_$COMMAND $JAVA_HEAP_MAX $HADOOP_OPTS -classpath "$CLASSPATH" $CLASS "$@"
   exec "$JAVA" -Dproc_$COMMAND $JAVA_HEAP_MAX $HADOOP_OPTS -classpath "$CLASSPATH" $CLASS "$@"

+ 1 - 5
bin/hadoop-config.sh

@@ -32,11 +32,7 @@ this="$config_bin/$script"
 
 
 # the root of the Hadoop installation
 # the root of the Hadoop installation
 export HADOOP_PREFIX=`dirname "$this"`/..
 export HADOOP_PREFIX=`dirname "$this"`/..
-if [ -d ${HADOOP_PREFIX}/share/hadoop/bin ]; then
-  export HADOOP_HOME=${HADOOP_PREFIX}/share/hadoop
-else
-  export HADOOP_HOME=${HADOOP_PREFIX}
-fi
+export HADOOP_HOME=${HADOOP_PREFIX}
 
 
 #check to see if the conf dir is given as an optional argument
 #check to see if the conf dir is given as an optional argument
 if [ $# -gt 1 ]
 if [ $# -gt 1 ]

+ 12 - 2
bin/hadoop-daemon.sh

@@ -39,7 +39,11 @@ fi
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # get arguments
 # get arguments
 startStop=$1
 startStop=$1
@@ -84,7 +88,13 @@ if [ "$HADOOP_LOG_DIR" = "" ]; then
   export HADOOP_LOG_DIR="$HADOOP_HOME/logs"
   export HADOOP_LOG_DIR="$HADOOP_HOME/logs"
 fi
 fi
 mkdir -p "$HADOOP_LOG_DIR"
 mkdir -p "$HADOOP_LOG_DIR"
-chown $HADOOP_IDENT_STRING $HADOOP_LOG_DIR 
+touch $HADOOP_LOG_DIR/.hadoop_test > /dev/null 2>&1
+TEST_LOG_DIR=$?
+if [ "${TEST_LOG_DIR}" = "0" ]; then
+  rm -f $HADOOP_LOG_DIR/.hadoop_test
+else
+  chown $HADOOP_IDENT_STRING $HADOOP_LOG_DIR 
+fi
 
 
 if [ "$HADOOP_PID_DIR" = "" ]; then
 if [ "$HADOOP_PID_DIR" = "" ]; then
   HADOOP_PID_DIR=/tmp
   HADOOP_PID_DIR=/tmp

+ 5 - 1
bin/hadoop-daemons.sh

@@ -29,6 +29,10 @@ fi
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. $bin/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 exec "$bin/slaves.sh" --config $HADOOP_CONF_DIR cd "$HADOOP_HOME" \; "$bin/hadoop-daemon.sh" --config $HADOOP_CONF_DIR "$@"
 exec "$bin/slaves.sh" --config $HADOOP_CONF_DIR cd "$HADOOP_HOME" \; "$bin/hadoop-daemon.sh" --config $HADOOP_CONF_DIR "$@"

+ 5 - 1
bin/rcc

@@ -30,7 +30,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
 if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
   . "${HADOOP_CONF_DIR}/hadoop-env.sh"
   . "${HADOOP_CONF_DIR}/hadoop-env.sh"

+ 5 - 1
bin/slaves.sh

@@ -38,7 +38,11 @@ fi
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # If the slaves file is specified in the command line,
 # If the slaves file is specified in the command line,
 # then it takes precedence over the definition in 
 # then it takes precedence over the definition in 

+ 5 - 1
bin/start-all.sh

@@ -21,7 +21,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # start dfs daemons
 # start dfs daemons
 "$bin"/start-dfs.sh --config $HADOOP_CONF_DIR
 "$bin"/start-dfs.sh --config $HADOOP_CONF_DIR

+ 5 - 1
bin/start-balancer.sh

@@ -18,7 +18,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # Start balancer daemon.
 # Start balancer daemon.
 
 

+ 5 - 1
bin/start-dfs.sh

@@ -25,7 +25,11 @@ usage="Usage: start-dfs.sh [-upgrade|-rollback]"
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # get arguments
 # get arguments
 if [ $# -ge 1 ]; then
 if [ $# -ge 1 ]; then

+ 5 - 1
bin/start-jobhistoryserver.sh

@@ -21,7 +21,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # start daemon
 # start daemon
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start historyserver
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start historyserver

+ 5 - 1
bin/start-mapred.sh

@@ -21,7 +21,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # start mapred daemons
 # start mapred daemons
 # start jobtracker first to minimize connection errors at startup
 # start jobtracker first to minimize connection errors at startup

+ 5 - 1
bin/stop-all.sh

@@ -21,7 +21,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 "$bin"/stop-mapred.sh --config $HADOOP_CONF_DIR
 "$bin"/stop-mapred.sh --config $HADOOP_CONF_DIR
 "$bin"/stop-dfs.sh --config $HADOOP_CONF_DIR
 "$bin"/stop-dfs.sh --config $HADOOP_CONF_DIR

+ 5 - 1
bin/stop-balancer.sh

@@ -18,7 +18,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 # Stop balancer daemon.
 # Stop balancer daemon.
 # Run this on the machine where the balancer is running
 # Run this on the machine where the balancer is running

+ 5 - 1
bin/stop-dfs.sh

@@ -21,7 +21,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop namenode
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop namenode
 "$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR stop datanode
 "$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR stop datanode

+ 5 - 1
bin/stop-jobhistoryserver.sh

@@ -21,7 +21,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop historyserver
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop historyserver
 
 

+ 5 - 1
bin/stop-mapred.sh

@@ -21,7 +21,11 @@
 bin=`dirname "$0"`
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 bin=`cd "$bin"; pwd`
 
 
-. "$bin"/../libexec/hadoop-config.sh
+if [ -e "$bin/../libexec/hadoop-config.sh" ]; then
+  . "$bin"/../libexec/hadoop-config.sh
+else
+  . "$bin/hadoop-config.sh"
+fi
 
 
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop jobtracker
 "$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop jobtracker
 "$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR stop tasktracker
 "$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR stop tasktracker

+ 33 - 6
build.xml

@@ -28,7 +28,7 @@
  
  
   <property name="Name" value="Hadoop"/>
   <property name="Name" value="Hadoop"/>
   <property name="name" value="hadoop"/>
   <property name="name" value="hadoop"/>
-  <property name="version" value="0.20.205.0-SNAPSHOT"/>
+  <property name="version" value="0.20.206.0-SNAPSHOT"/>
   <property name="final.name" value="${name}-${version}"/>
   <property name="final.name" value="${name}-${version}"/>
   <property name="test.final.name" value="${name}-test-${version}"/>
   <property name="test.final.name" value="${name}-test-${version}"/>
   <property name="year" value="2009"/>
   <property name="year" value="2009"/>
@@ -168,10 +168,23 @@
   <property name="patch.cmd" value="patch"/>
   <property name="patch.cmd" value="patch"/>
   <property name="make.cmd" value="make"/>
   <property name="make.cmd" value="make"/>
 
 
-  <property name="jsvc.build.dir" value="${build.dir}/jsvc" />
+  <property name="jsvc.build.dir" value="${build.dir}/jsvc.${os.arch}" />
   <property name="jsvc.install.dir" value="${dist.dir}/libexec" /> 
   <property name="jsvc.install.dir" value="${dist.dir}/libexec" /> 
-  <property name="jsvc.location" value="http://archive.apache.org/dist/commons/daemon/binaries/1.0.2/linux/commons-daemon-1.0.2-bin-linux-i386.tar.gz" />
-  <property name="jsvc.dest.name" value="jsvc.tar.gz" />
+  <condition property="os-arch" value="x86_64">
+    <and>
+      <os arch="amd64" />
+    </and>
+  </condition>
+  <condition property="os-arch" value="i386">
+    <or>
+      <os arch="i386" />
+      <os arch="i486" />
+      <os arch="i586" />
+      <os arch="i686" />
+    </or>
+  </condition>
+  <property name="jsvc.location" value="http://archive.apache.org/dist/commons/daemon/binaries/1.0.2/linux/commons-daemon-1.0.2-bin-linux-${os-arch}.tar.gz" />
+  <property name="jsvc.dest.name" value="jsvc.${os.arch}.tar.gz" />
 
 
   <!-- task-controller properties set here -->
   <!-- task-controller properties set here -->
   <!-- Source directory from where configure is run and files are copied
   <!-- Source directory from where configure is run and files are copied
@@ -691,6 +704,13 @@
           <attribute name="Implementation-Vendor" value="Apache"/>
           <attribute name="Implementation-Vendor" value="Apache"/>
         </section>
         </section>
       </manifest>
       </manifest>
+      <service type="org.apache.hadoop.security.token.TokenRenewer">
+        <provider classname="org.apache.hadoop.hdfs.DFSClient$Renewer"/>
+        <provider classname="org.apache.hadoop.mapred.JobClient$Renewer"/>
+        <provider classname="org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer"/>
+        <provider classname="org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier$Renewer"/>
+        <provider classname="org.apache.hadoop.hdfs.HftpFileSystem$TokenManager"/>
+      </service>
       <fileset dir="${conf.dir}" includes="${jar.properties.list}" />
       <fileset dir="${conf.dir}" includes="${jar.properties.list}" />
       <fileset file="${jar.extra.properties.list}" />
       <fileset file="${jar.extra.properties.list}" />
       <zipfileset dir="${build.webapps}" prefix="webapps"/>
       <zipfileset dir="${build.webapps}" prefix="webapps"/>
@@ -1413,6 +1433,13 @@
     <mkdir dir="${dist.dir}/docs/api"/>
     <mkdir dir="${dist.dir}/docs/api"/>
     <mkdir dir="${dist.dir}/docs/jdiff"/>
     <mkdir dir="${dist.dir}/docs/jdiff"/>
     <mkdir dir="${dist.dir}/sbin"/>
     <mkdir dir="${dist.dir}/sbin"/>
+    <mkdir dir="${dist.dir}/share/${name}/templates/conf"/>
+
+    <copy todir="${dist.dir}/share/${name}/templates/conf" includeEmptyDirs="false">
+      <fileset dir="${basedir}/src/packages/templates/conf">
+        <include name="*"/>
+      </fileset>
+    </copy>
 
 
     <copy todir="${dist.dir}/lib" includeEmptyDirs="false" flatten="true">
     <copy todir="${dist.dir}/lib" includeEmptyDirs="false" flatten="true">
       <fileset dir="${common.ivy.lib.dir}"/>
       <fileset dir="${common.ivy.lib.dir}"/>
@@ -2606,9 +2633,9 @@
 
 
     <untar compression="gzip" src="${jsvc.build.dir}/${jsvc.dest.name}" dest="${jsvc.build.dir}" />
     <untar compression="gzip" src="${jsvc.build.dir}/${jsvc.dest.name}" dest="${jsvc.build.dir}" />
 
 
-    <copy file="${jsvc.build.dir}/jsvc" todir="${jsvc.install.dir}" verbose="true" />
+    <copy file="${jsvc.build.dir}/jsvc" toFile="${jsvc.install.dir}/jsvc.${os.arch}" verbose="true" />
     <chmod perm="ugo+x" type="file">
     <chmod perm="ugo+x" type="file">
-      <fileset file="${jsvc.install.dir}/jsvc"/>
+      <fileset file="${jsvc.install.dir}/jsvc.${os.arch}"/>
     </chmod>
     </chmod>
  </target>
  </target>
 
 

+ 12 - 0
conf/fair-scheduler.xml.template

@@ -0,0 +1,12 @@
+<?xml version="1.0"?>
+
+<!--
+  This file contains pool and user allocations for the Fair Scheduler.
+  Its format is explained in the Fair Scheduler documentation at
+  http://hadoop.apache.org/common/docs/r0.20.205.0/fair_scheduler.html.
+  The documentation also includes a sample config file.
+-->
+
+<allocations>
+
+</allocations>

+ 30 - 0
conf/hadoop-metrics2.properties

@@ -14,3 +14,33 @@
 #maptask.sink.file.filename=maptask-metrics.out
 #maptask.sink.file.filename=maptask-metrics.out
 
 
 #reducetask.sink.file.filename=reducetask-metrics.out
 #reducetask.sink.file.filename=reducetask-metrics.out
+
+
+#
+# Below are for sending metrics to Ganglia
+#
+# for Ganglia 3.0 support
+# *.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink30
+#
+# for Ganglia 3.1 support
+# *.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink31
+
+# *.sink.ganglia.period=10
+
+# default for supportsparse is false
+# *.sink.ganglia.supportsparse=true
+
+#*.sink.ganglia.slope=jvm.metrics.gcCount=zero,jvm.metrics.memHeapUsedM=both
+#*.sink.ganglia.dmax=jvm.metrics.threadsBlocked=70,jvm.metrics.memHeapUsedM=40
+
+#namenode.sink.ganglia.servers=yourgangliahost_1:8649,yourgangliahost_2:8649
+
+#datanode.sink.ganglia.servers=yourgangliahost_1:8649,yourgangliahost_2:8649
+
+#jobtracker.sink.ganglia.servers=yourgangliahost_1:8649,yourgangliahost_2:8649
+
+#tasktracker.sink.ganglia.servers=yourgangliahost_1:8649,yourgangliahost_2:8649
+
+#maptask.sink.ganglia.servers=yourgangliahost_1:8649,yourgangliahost_2:8649
+
+#reducetask.sink.ganglia.servers=yourgangliahost_1:8649,yourgangliahost_2:8649

+ 17 - 0
ivy.xml

@@ -151,6 +151,23 @@
       rev="${jetty-util.version}"
       rev="${jetty-util.version}"
       conf="jetty->master"/>
       conf="jetty->master"/>
 
 
+    <dependency org="asm"
+      name="asm"
+      rev="${asm.version}"
+      conf="jetty->master"/>
+    <dependency org="com.sun.jersey"
+      name="jersey-core"
+      rev="${jersey-core.version}"
+      conf="jetty->master"/>
+    <dependency org="com.sun.jersey"
+      name="jersey-json"
+      rev="${jersey-json.version}"
+      conf="jetty->master"/>
+    <dependency org="com.sun.jersey"
+      name="jersey-server"
+      rev="${jersey-server.version}"
+      conf="jetty->master"/>
+
     <dependency org="tomcat"
     <dependency org="tomcat"
       name="jasper-runtime"
       name="jasper-runtime"
       rev="${jasper.version}"
       rev="${jasper.version}"

+ 5 - 1
ivy/libraries.properties

@@ -14,13 +14,14 @@
 #It drives ivy and the generation of a maven POM
 #It drives ivy and the generation of a maven POM
 
 
 # This is the version of hadoop we are generating
 # This is the version of hadoop we are generating
-hadoop.version=0.20.205.0
+hadoop.version=0.20.206.0
 hadoop-gpl-compression.version=0.1.0
 hadoop-gpl-compression.version=0.1.0
 
 
 #These are the versions of our dependencies (in alphabetical order)
 #These are the versions of our dependencies (in alphabetical order)
 apacheant.version=1.7.0
 apacheant.version=1.7.0
 ant-task.version=2.0.10
 ant-task.version=2.0.10
 
 
+asm.version=3.2
 aspectj.version=1.6.5
 aspectj.version=1.6.5
 
 
 checkstyle.version=4.2
 checkstyle.version=4.2
@@ -57,6 +58,9 @@ jsp-2.1.version=6.1.14
 jets3t.version=0.6.1
 jets3t.version=0.6.1
 jetty.version=6.1.26
 jetty.version=6.1.26
 jetty-util.version=6.1.26
 jetty-util.version=6.1.26
+jersey-core.version=1.8
+jersey-json.version=1.8
+jersey-server.version=1.8
 junit.version=4.5
 junit.version=4.5
 jdeb.version=0.8
 jdeb.version=0.8
 jdiff.version=1.0.9
 jdiff.version=1.0.9

+ 9 - 17
src/c++/task-controller/impl/task-controller.c

@@ -620,9 +620,16 @@ int create_directory_for_user(const char* path) {
   uid_t user = geteuid();
   uid_t user = geteuid();
   gid_t group = getegid();
   gid_t group = getegid();
   int ret = 0;
   int ret = 0;
-  ret = change_effective_user(tt_uid, tt_gid);
+  uid_t root = 0;
+
+  //This check is particularly required for c-based unit tests since 
+  //tests run as a regular user.
+  if (getuid() == root) {
+    ret = change_effective_user(root, tt_gid);
+  }
+
   if (ret == 0) {
   if (ret == 0) {
-    if (mkdir(path, permissions) == 0) {
+    if (mkdir(path, permissions) == 0 || errno == EEXIST) {
       // need to reassert the group sticky bit
       // need to reassert the group sticky bit
       if (chmod(path, permissions) != 0) {
       if (chmod(path, permissions) != 0) {
         fprintf(LOGFILE, "Can't chmod %s to add the sticky bit - %s\n",
         fprintf(LOGFILE, "Can't chmod %s to add the sticky bit - %s\n",
@@ -631,21 +638,6 @@ int create_directory_for_user(const char* path) {
       } else if (change_owner(path, user, tt_gid) != 0) {
       } else if (change_owner(path, user, tt_gid) != 0) {
         ret = -1;
         ret = -1;
       }
       }
-    } else if (errno == EEXIST) {
-      struct stat file_stat;
-      if (stat(path, &file_stat) != 0) {
-        fprintf(LOGFILE, "Can't stat directory %s - %s\n", path, 
-                strerror(errno));
-        ret = -1;
-      } else {
-        if (file_stat.st_uid != user ||
-            file_stat.st_gid != tt_gid) {
-          fprintf(LOGFILE, "Directory %s owned by wrong user or group. "
-                  "Expected %d:%d and found %d:%d.\n",
-                  path, user, tt_gid, file_stat.st_uid, file_stat.st_gid);
-          ret = -1;
-        }
-      }
     } else {
     } else {
       fprintf(LOGFILE, "Failed to create directory %s - %s\n", path,
       fprintf(LOGFILE, "Failed to create directory %s - %s\n", path,
               strerror(errno));
               strerror(errno));

+ 18 - 5
src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacitySchedulerQueue.java

@@ -197,13 +197,13 @@ class CapacitySchedulerQueue {
       }
       }
     }
     }
     
     
-    void updateSlotsUsage(String user, int pendingTasks, int numRunningTasks, int numSlotsOccupied) {
+    void updateSlotsUsage(String user, boolean pendingTasks, int numRunningTasks, int numSlotsOccupied) {
       this.numRunningTasks += numRunningTasks;
       this.numRunningTasks += numRunningTasks;
       this.numSlotsOccupied += numSlotsOccupied;
       this.numSlotsOccupied += numSlotsOccupied;
       Integer i = this.numSlotsOccupiedByUser.get(user);
       Integer i = this.numSlotsOccupiedByUser.get(user);
       int slots = numSlotsOccupied + ((i == null) ? 0 : i.intValue());
       int slots = numSlotsOccupied + ((i == null) ? 0 : i.intValue());
       this.numSlotsOccupiedByUser.put(user, slots);
       this.numSlotsOccupiedByUser.put(user, slots);
-      if (pendingTasks > 0) {
+      if (pendingTasks) {
         users.add(user);
         users.add(user);
       }
       }
     }
     }
@@ -581,12 +581,25 @@ class CapacitySchedulerQueue {
    */
    */
   void update(TaskType type, JobInProgress job, String user, 
   void update(TaskType type, JobInProgress job, String user, 
       int numRunningTasks, int numSlotsOccupied) {
       int numRunningTasks, int numSlotsOccupied) {
+    // pendingTasks keeps tracking of whether a user's job has tasks that
+    // still need to be scheduled. The number of users with pending tasks is
+    // used in the limit calculations.
+    boolean pendingTasks = false;
     if (type == TaskType.MAP) {
     if (type == TaskType.MAP) {
-      mapSlots.updateSlotsUsage(user, job.pendingMaps(), 
+      // A job has map tasks to be scheduled when job.pendingMaps > 0
+      if (job.pendingMaps() > 0) {
+        pendingTasks = true;
+      }
+      mapSlots.updateSlotsUsage(user, pendingTasks,
           numRunningTasks, numSlotsOccupied);
           numRunningTasks, numSlotsOccupied);
     } else if (type == TaskType.REDUCE) {
     } else if (type == TaskType.REDUCE) {
-      reduceSlots.updateSlotsUsage(user, job.pendingReduces(), 
-          numRunningTasks, numSlotsOccupied);
+      // A job has reduce tasks to be scheduled when job.pendingReduces() > 0 &&
+      // the minimum number of maps have been completed
+      if (job.scheduleReduces() && (job.pendingReduces() > 0)) {
+        pendingTasks = true;
+      }
+      reduceSlots.updateSlotsUsage(user, pendingTasks,
+        numRunningTasks, numSlotsOccupied);
     }
     }
   }
   }
   
   

+ 3 - 2
src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java

@@ -509,8 +509,9 @@ class CapacityTaskScheduler extends TaskScheduler {
       job.schedulingOpportunity();
       job.schedulingOpportunity();
       
       
       // First, try to get a 'local' task
       // First, try to get a 'local' task
-      Task t = 
-        job.obtainNewLocalMapTask(taskTracker, numTaskTrackers, numUniqueHosts);
+      Task t = job.obtainNewNodeOrRackLocalMapTask(taskTracker,
+                                                   numTaskTrackers,
+                                                   numUniqueHosts);
       
       
       if (t != null) {
       if (t != null) {
         return TaskLookupResult.getTaskFoundResult(t, job); 
         return TaskLookupResult.getTaskFoundResult(t, job); 

+ 166 - 3
src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java

@@ -195,8 +195,8 @@ public class TestCapacityScheduler extends TestCase {
     }
     }
 
 
     @Override
     @Override
-    public Task obtainNewLocalMapTask(final TaskTrackerStatus tts, int clusterSize,
-        int ignored) throws IOException {
+    public Task obtainNewNodeOrRackLocalMapTask(final TaskTrackerStatus tts,
+        int clusterSize, int ignored) throws IOException {
       return obtainNewMapTask(tts, clusterSize, ignored);
       return obtainNewMapTask(tts, clusterSize, ignored);
     }
     }
     
     
@@ -252,6 +252,9 @@ public class TestCapacityScheduler extends TestCase {
     @Override
     @Override
     public Task obtainNewReduceTask(final TaskTrackerStatus tts,
     public Task obtainNewReduceTask(final TaskTrackerStatus tts,
         int clusterSize, int ignored) throws IOException {
         int clusterSize, int ignored) throws IOException {
+      if (!scheduleReduces()) {
+        return null;
+      }
       boolean areAllReducesRunning = (redTaskCtr == numReduceTasks);
       boolean areAllReducesRunning = (redTaskCtr == numReduceTasks);
       if (areAllReducesRunning){
       if (areAllReducesRunning){
         if(!getJobConf().getReduceSpeculativeExecution() || 
         if(!getJobConf().getReduceSpeculativeExecution() || 
@@ -550,6 +553,12 @@ public class TestCapacityScheduler extends TestCase {
       return statuses;
       return statuses;
     }
     }
 
 
+    @Override
+    public boolean killTask(TaskAttemptID taskid, boolean shouldFail)
+      throws IOException {
+      return false;
+    }
+
 
 
     public void addJobInProgressListener(JobInProgressListener listener) {
     public void addJobInProgressListener(JobInProgressListener listener) {
       mylisteners.add(listener);
       mylisteners.add(listener);
@@ -794,7 +803,7 @@ public class TestCapacityScheduler extends TestCase {
     FakeJobInProgress job =
     FakeJobInProgress job =
         new FakeJobInProgress(new JobID("test", ++jobCounter),
         new FakeJobInProgress(new JobID("test", ++jobCounter),
             (jobConf == null ? new JobConf(conf) : jobConf), taskTrackerManager,
             (jobConf == null ? new JobConf(conf) : jobConf), taskTrackerManager,
-            jobConf.getUser(), UtilsForTests.getJobTracker());
+            jobConf.getUser(), UtilsForTests.getJobTracker(taskTrackerManager.getQueueManager()));
     job.getStatus().setRunState(state);
     job.getStatus().setRunState(state);
     taskTrackerManager.submitJob(job);
     taskTrackerManager.submitJob(job);
     return job;
     return job;
@@ -3424,6 +3433,160 @@ public class TestCapacityScheduler extends TestCase {
     assertEquals(scheduler.getLimitMaxMemForReduceSlot(),3);
     assertEquals(scheduler.getLimitMaxMemForReduceSlot(),3);
   }
   }
 
 
+  // test that 1st user gets reduce slots when 2nd user haven't finished 
+  // enough map tasks yet
+  public void testUserLimit() throws Exception {
+    // set up some queues
+    String[] qs = {"default", "q2"};
+    taskTrackerManager.addQueues(qs);
+    ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
+    queues.add(new FakeQueueInfo("default", 40.0f, true, 50));
+    queues.add(new FakeQueueInfo("q2", 60.0f, false, 50));
+    resConf.setFakeQueues(queues);
+    scheduler.setResourceManagerConf(resConf);
+    scheduler.start();
+
+    // add some more TTs 
+    taskTrackerManager.addTaskTracker("tt3");
+    taskTrackerManager.addTaskTracker("tt4");
+    taskTrackerManager.addTaskTracker("tt5");
+
+    // submit a job
+    FakeJobInProgress j1 = submitJobAndInit(JobStatus.PREP, 10, 10, "default", "u1");
+    // for queue 'default', the capacity for maps is 4. Since we're the only user,
+    // we should get 2 map tasks & 1 reduce
+    checkAssignments("tt1",
+        new String[] {"attempt_test_0001_m_000001_0 on tt1",
+        "attempt_test_0001_m_000002_0 on tt1",
+        "attempt_test_0001_r_000001_0 on tt1"});
+
+    // Submit another job, from a different user
+    FakeJobInProgress j2 = submitJobAndInit(JobStatus.PREP, 6, 1, "default", "u2");
+    // FakeJobInProgress uses override initTasks() to init tasks, which does
+    // not initialize completedMapsForReduceSlowstart, and there is no proper
+    // API to set completedMapsForReduceSlowstart neither.
+    // We manually set completedMapsForReduceSlowstart here to accommadate this
+    // test, and to avoid changing mapredcuce code for testing purpose only
+    j2.completedMapsForReduceSlowstart = 3;
+
+    // Now if I ask for a map task, it should come from the second job
+    // reduce task will be from j1 since j2 hasn't completed enough map task
+    checkAssignments("tt2",
+        new String[] {"attempt_test_0002_m_000001_0 on tt2",
+        "attempt_test_0002_m_000002_0 on tt2",
+        "attempt_test_0001_r_000002_0 on tt2"});
+
+    // Now if I ask for map tasks again, 1 from j2, 1 from job1
+    // no reduce task since j1 used up queue capacity & j2 hasn't completed
+    // enough map tasks yet
+    checkAssignments("tt3",
+        new String[] {"attempt_test_0001_m_000003_0 on tt3",
+        "attempt_test_0002_m_000003_0 on tt3"});
+
+    // Now if I ask for map tasks again, 1 from j2, 1 from j1,
+    // no reduce task still
+    checkAssignments("tt4",
+        new String[] {"attempt_test_0001_m_000004_0 on tt4",
+        "attempt_test_0002_m_000004_0 on tt4"});
+
+    // complete 3 tasks from j2
+    taskTrackerManager.finishTask("tt2", "attempt_test_0002_m_000001_0", j2);
+    taskTrackerManager.finishTask("tt2", "attempt_test_0002_m_000002_0", j2);
+    taskTrackerManager.finishTask("tt3", "attempt_test_0002_m_000003_0", j2);
+
+    // Now if I ask for map tasks again, 2 from j2,
+    // for reduce tasks, 1 from j2
+    checkAssignments("tt5",
+        new String[] {"attempt_test_0002_m_000005_0 on tt5",
+        "attempt_test_0002_m_000006_0 on tt5",
+        "attempt_test_0002_r_000001_0 on tt5"});
+
+    // complete 2 tasks from j1
+    taskTrackerManager.finishTask("tt1", "attempt_test_0001_m_000001_0", j1);
+    taskTrackerManager.finishTask("tt1", "attempt_test_0001_r_000001_0", j1);
+   
+    // Now if I ask for tasks again, 1 map & 1 reduce from j1
+    checkAssignments("tt1",
+        new String[] {"attempt_test_0001_m_000005_0 on tt1",
+        "attempt_test_0001_r_000003_0 on tt1"});
+  }
+
+  // test that 1st user gets reduce slots when 2nd user haven't finished 
+  // enough map tasks yet, witout exceeding MaxCapacity
+  public void testUserLimitWithMaxCapacity() throws Exception {
+    // set up some queues
+    String[] qs = {"default", "q2"};
+    taskTrackerManager.addQueues(qs);
+    ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
+    queues.add(new FakeQueueInfo("default", 40.0f, true, 50));
+    queues.add(new FakeQueueInfo("q2", 60.0f, false, 50));
+    resConf.setFakeQueues(queues);
+    resConf.setMaxCapacity("default", 60.0f);
+    scheduler.setResourceManagerConf(resConf);
+    scheduler.start();
+
+    // add some more TTs 
+    taskTrackerManager.addTaskTracker("tt3");
+    taskTrackerManager.addTaskTracker("tt4");
+    taskTrackerManager.addTaskTracker("tt5");
+
+    // submit a job
+    FakeJobInProgress j1 = submitJobAndInit(JobStatus.PREP, 10, 10, "default", "u1");
+    // for queue 'default', the capacity for maps is 4. Since we're the only user,
+    // we should get 2 map tasks & 1 reduce
+    checkAssignments("tt1",
+        new String[] {"attempt_test_0001_m_000001_0 on tt1",
+        "attempt_test_0001_m_000002_0 on tt1",
+        "attempt_test_0001_r_000001_0 on tt1"});
+
+    // Submit another job, from a different user
+    FakeJobInProgress j2 = submitJobAndInit(JobStatus.PREP, 6, 1, "default", "u2");
+    // FakeJobInProgress uses override initTasks() to init tasks, which does
+    // not initialize completedMapsForReduceSlowstart, and there is no proper
+    // API to set completedMapsForReduceSlowstart neither.
+    // We manually set completedMapsForReduceSlowstart here to accommadate this
+    // test, and to avoid changing mapredcuce code for testing purpose only
+    j2.completedMapsForReduceSlowstart = 3;
+
+    // Now if I ask for a map task, it should come from the second job
+    // reduce task will be from j1 since j2 hasn't completed enough map task
+    checkAssignments("tt2",
+        new String[] {"attempt_test_0002_m_000001_0 on tt2",
+        "attempt_test_0002_m_000002_0 on tt2",
+        "attempt_test_0001_r_000002_0 on tt2"});
+
+    // Now if I ask for map tasks again, 1 from j2, 1 from job1
+    // no reduce task since j1 used up queue capacity & j2 hasn't completed
+    // enough map tasks yet
+    checkAssignments("tt3",
+        new String[] {"attempt_test_0001_m_000003_0 on tt3",
+        "attempt_test_0002_m_000003_0 on tt3"});
+
+    // Now if I ask for map tasks again, no tasks since it reached maxcapacity
+    checkAssignments("tt4", new String[] {});
+
+    // complete 3 tasks from j2
+    taskTrackerManager.finishTask("tt2", "attempt_test_0002_m_000001_0", j2);
+    taskTrackerManager.finishTask("tt2", "attempt_test_0002_m_000002_0", j2);
+    taskTrackerManager.finishTask("tt3", "attempt_test_0002_m_000003_0", j2);
+
+    // Now if I ask for map tasks again, 2 from j2,
+    // for reduce tasks, 1 from j2
+    checkAssignments("tt5",
+        new String[] {"attempt_test_0002_m_000004_0 on tt5",
+        "attempt_test_0002_m_000005_0 on tt5",
+        "attempt_test_0002_r_000001_0 on tt5"});
+
+    // complete 2 tasks from j1
+    taskTrackerManager.finishTask("tt1", "attempt_test_0001_m_000001_0", j1);
+    taskTrackerManager.finishTask("tt1", "attempt_test_0001_r_000001_0", j1);
+
+    // Now if I ask for tasks again, 1 map & 1 reduce from j1
+    checkAssignments("tt1",
+        new String[] {"attempt_test_0001_m_000004_0 on tt1",
+        "attempt_test_0001_r_000003_0 on tt1"});
+  }
+
   /**
   /**
    * Checks for multiple assignment.
    * Checks for multiple assignment.
    *
    *

+ 253 - 0
src/contrib/fairscheduler/designdoc/fair_scheduler_design_doc.tex

@@ -0,0 +1,253 @@
+% 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.
+
+\documentclass[11pt]{article}
+\usepackage{geometry}
+\geometry{letterpaper}
+
+\begin{document}
+
+\title{Hadoop Fair Scheduler Design Document}
+\author{}
+\maketitle
+\tableofcontents
+
+\section{Introduction}
+
+The Hadoop Fair Scheduler started as a simple means to share MapReduce clusters. Over time, it has grown in functionality to support hierarchical scheduling, preemption, and multiple ways of organizing and weighing jobs. This document explains the goals and features of the Fair Scheduler and its internal design.
+
+\section{Fair Scheduler Goals}
+
+The Fair Scheduler was designed with four main goals:
+\begin{enumerate}
+  \item Run small jobs quickly even if they are sharing a cluster with large jobs. Unlike Hadoop's built-in FIFO scheduler, fair scheduling lets small jobs make progress even if a large job is running, without starving the large job.
+  \item Provide guaranteed service levels to ``production" jobs, to let them run alongside experimental jobs in a shared cluster.
+  \item Be simple to administer and configure. The scheduler should do something reasonable ``out of the box," and users should only need to configure it as they discover that they want to use more advanced features.
+  \item Support reconfiguration at runtime, without requiring a cluster restart.
+\end{enumerate}
+
+\section{Scheduler Features}
+
+This section provides a quick overview of the features of the Fair Scheduler. A detailed usage guide is available in the Hadoop documentation in {\tt build/docs/fair\_scheduler.html}.
+
+\subsection{Pools}
+
+The Fair Scheduler groups jobs into ``pools" and performs fair sharing between these pools. Each pool can use either FIFO or fair sharing to schedule jobs internal to the pool. The pool that a job is placed in is determined by a JobConf property, the ``pool name property". By default, this is {\tt user.name}, so that there is one pool per user. However, different properties can be used, e.g.~{\tt group.name} to have one pool per Unix group.
+
+A common trick is to set the pool name property to an unused property name such as {\tt pool.name} and make this default to {\tt user.name}, so that there is one pool per user but it is also possible to place jobs into ``special" pools by setting their {\tt pool.name} directly. The {\tt mapred-site.xml} snippet below shows how to do this:
+
+\begin{verbatim}
+<property>
+  <name>mapred.fairscheduler.poolnameproperty</name>
+  <value>pool.name</value>
+</property>
+
+<property>
+  <name>pool.name</name>
+  <value>${user.name}</value>
+</property>
+\end{verbatim}
+
+\subsection{Minimum Shares}
+
+Normally, active pools (those that contain jobs) will get equal shares of the map and reduce task slots in the cluster. However, it is also possible to set a \emph{minimum share} of map and reduce slots on a given pool, which is a number of slots that it will always get when it is active, even if its fair share would be below this number. This is useful for guaranteeing that production jobs get a certain desired level of service when sharing a cluster with non-production jobs. Minimum shares have three effects:
+\begin{enumerate}
+  \item The pool's fair share will always be at least as large as its minimum share. Slots are taken from the share of other pools to achieve this. The only exception is if the minimum shares of the active pools add up to more than the total number of slots in the cluster; in this case, each pool's share will be scaled down proportionally.
+  \item Pools whose running task count is below their minimum share get assigned slots first when slots are available.
+  \item It is possible to set a \emph{preemption timeout} on the pool after which, if it has not received enough task slots to meet its minimum share, it is allowed to kill tasks in other jobs to meet its share. Minimum shares with preemption timeouts thus act like SLAs.
+\end{enumerate}
+
+Note that when a pool is inactive (contains no jobs), its minimum share is not ``reserved" for it -- the slots are split up among the other pools.
+
+\subsection{Preemption}
+
+As explained above, the scheduler may kill tasks from a job in one pool in order to meet the minimum share of another pool. We call this preemption, although this usage of the word is somewhat strange given the normal definition of preemption as pausing; really it is the \emph{job} that gets preempted, while the task gets killed. The feature explained above is called \emph{min share preemption}. In addition, the scheduler supports \emph{fair share preemption}, to kill tasks when a pool's fair share is not being met. Fair share preemption is much more conservative than min share preemption, because pools without min shares are expected to be non-production jobs where some amount of unfairness is tolerable. In particular, fair share preemption activates if a pool has been below \emph{half} of its fair share for a configurable fair share preemption timeout, which is recommended to be set fairly high (e.g. 10 minutes).
+
+In both types of preemption, the scheduler kills the most recently launched tasks from over-scheduled pools, to minimize the amount of computation wasted by preemption.
+
+\subsection{Running Job Limits}
+
+The fair scheduler can limit the number of concurrently running jobs from each user and from each pool. This is useful for limiting the amount of intermediate data generated on the cluster. The jobs that will run are chosen in order of submit time and priority. Jobs submitted beyond the limit wait for one of the running jobs to finish.
+
+\subsection{Job Priorities}
+
+Within a pool, job priorities can be used to control the scheduling of jobs, whether the pool's internal scheduling mode is FIFO or fair sharing:
+\begin{itemize}
+  \item In FIFO pools, jobs are ordered first by priority and then by submit time, as in Hadoop's default scheduler.
+  \item In fair sharing pools, job priorities are used as weights to control how much share a job gets. The normal priority corresponds to a weight of 1.0, and each level gives 2x more weight. For example, a high-priority job gets a weight of 2.0, and will therefore get 2x the share of a normal-priority job. 
+\end{itemize}
+
+\subsection{Pool Weights}
+
+Pools can be given weights to achieve unequal sharing of the cluster. For example, a pool with weight 2.0 gets 2x the share of a pool with weight 1.0.
+
+\subsection{Delay Scheduling}
+
+The Fair Scheduler contains an algorithm called delay scheduling to improve data locality. Jobs that cannot launch a data-local map task wait for some period of time before they are allowed to launch non-data-local tasks, ensuring that they will run locally if some node in the cluster has the relevant data. Delay scheduling is described in detail in Section \ref{sec:delay-scheduling}.
+
+\subsection{Administration}
+
+The Fair Scheduler includes a web UI displaying the active pools and jobs and their fair shares, moving jobs between pools, and changing job priorities.
+In addition, the Fair Scheduler's allocation file (specifying min shares and preemption timeouts for the pools) is automatically reloaded if it is modified on disk, to allow runtime reconfiguration.
+
+\section{Implementation}
+
+\subsection{Hadoop Scheduling Background}
+
+Hadoop jobs consist of a number of map and reduce \emph{tasks}. These task run in \emph{slots} on the nodes on the cluster. Each node is configured with a number of map slots and reduce slots based on its computational resources (typically one slot per core). The role of the scheduler is to assign tasks to any slots that are free.
+
+All schedulers in Hadoop, including the Fair Scheduler, inherit from the {\tt TaskScheduler} abstract class. This class provides access to a {\tt TaskTrackerManager} -- an interface to the JobTracker -- as well as a {\tt Configuration} instance. It also ask the scheduler to implement three abstract methods: the lifecycle methods {\tt start} and {\tt terminate}, and a method called {\tt assignTasks} to launch tasks on a given TaskTracker.
+Task assignment in Hadoop is reactive. TaskTrackers periodically send heartbeats to the JobTracker with their {\tt TaskTrackerStatus}, which contains a list of running tasks, the number of slots on the node, and other information. The JobTracker then calls {\tt assignTasks} on the scheduler to obtain tasks to launch. These are returned with the heartbeat response.
+
+Apart from reacting to heartbeats through {\tt assignTasks}, schedulers can also be notified when jobs have been submitted to the cluster, killed, or removed by adding listeners to the {\tt TaskTrackerManager}. The Fair Scheduler sets up these listeners in its {\tt start} method. An important role of the listeners is to initialize jobs that are submitted -- until a job is initialized, it cannot launch tasks. The Fair Scheduler currently initializes all jobs right away, but it may also be desirable to hold off initializing jobs if too many are submitted to limit memory usage on the JobTracker.
+
+Selection of tasks \emph{within} a job is mostly done by the {\tt JobInProgress} class, and not by individual schedulers. {\tt JobInProgress} exposes two methods, {\tt obtainNewMapTask} and {\tt obtainNewReduceTask}, to launch a task of either type. Both methods may either return a {\tt Task} object or {\tt null} if the job does not wish to launch a task. Whether a job wishes to launch a task may change back and forth during its lifetime. Even after all tasks in the job have been started, the job may wish to run another task for speculative execution. In addition, if the node containing a map task failed, the job will wish to re-run it to rebuild its output for use in the reduce tasks. Schedulers may therefore need to poll multiple jobs until they find one with a task to run.
+
+Finally, for map tasks, an important scheduling criterion is data locality: running the task on a node or rack that contains its input data. Normally, {\tt JobInProgress.obtainNewMapTask} returns the ``closest" map task to a given node. However, to give schedulers slightly more control over data locality, there is also a version of {\tt obtainNewMapTask} that allow the scheduler to cap the level of non-locality allowed for the task (e.g.~request a task only on the same node, or {\tt null} if none is available). The Fair Scheduler uses this method with an algorithm called delay scheduling (Section \ref{sec:delay-scheduling}) to optimize data locality.
+
+\subsection{Fair Scheduler Basics}
+
+At a high level, the Fair Scheduler uses hierarchical scheduling to assign tasks. First it selects a pool to assign a task to according to the fair sharing algorithm in Section \ref{sec:fair-sharing-alg}. Then it asks the pool obtain a task. The pool chooses among its jobs according to its internal scheduling order (FIFO or fair sharing).
+
+In fact, because jobs might not have tasks to launch ({\tt obtainNew(Map|Reduce)Task} can return null), the scheduler actually establishes an ordering on jobs and asks them for tasks in turn. Within a pool, jobs are sorted either by priority and start time (for FIFO) or by distance below fair share. If the first job in the ordering does not have a task to launch, the pool will ask the second, third, etc jobs. Pools themselves are sorted by distance below min share and fair share, so if the first pool does not have any jobs that can launch tasks, the second pool is asked, etc. This makes it straightforward to implement features like delay scheduling (Section \ref{sec:delay-scheduling}) that may cause jobs to ``pass" on a slot.
+
+Apart from the assign tasks code path, the Fair Scheduler also has a periodic update thread that calls {\tt update} every few seconds. This thread is responsible for recomputing fair shares to display them on the UI (Section \ref{sec:fair-share-computation}), checking whether jobs need to be preempted (Section \ref{sec:preemption}), and checking whether the allocations file has changed to reload pool allocations (through {\tt PoolManager}).
+
+\subsection{The {\tt Schedulable} Class}
+
+To allow the same fair sharing algorithm to be used both between pools and within a pool, the Fair Scheduler uses an abstract class called {\tt Schedulable} to represent both pools and jobs. Its subclasses for these roles are {\tt PoolSchedulable} and {\tt JobSchedulable}. A {\tt Schedulable} is responsible for three roles:
+\begin{enumerate}
+  \item It can be asked to obtain a task through {\tt assignTask}. This may return {\tt null} if the {\tt Schedulable} has no tasks to launch.
+  \item It can be queried for information about the pool/job to use in scheduling, such as:
+  \begin{itemize}
+    \item Number of running tasks.
+    \item Demand (number of tasks the {\tt Schedulable} \emph{wants} to run; this is equal to number of running tasks + number of unlaunched tasks).
+    \item Min share assigned through config file.
+    \item Weight (for fair sharing).
+    \item Priority and start time (for FIFO scheduling).
+  \end{itemize}
+  \item It can be assigned a fair share through {\tt setFairShare}.
+\end{enumerate}
+
+There are separate {\tt Schedulable}s for map and reduce tasks, to make it possible to use the same algorithm on both types of tasks.
+
+\subsection{Fair Sharing Algorithm}
+\label{sec:fair-sharing-alg}
+
+A simple way to achieve fair sharing is the following: whenever a slot is available, assign it to the pool that has the fewest running tasks. This will ensure that all pool get an equal number of slots, unless a pool's demand is less than its fair share, in which case the extra slots are divided evenly among the other pools. Two features of the Fair Scheduler complicate this algorithm a little:
+\begin{itemize}
+  \item Pool weights mean that some pools should get more slots than others. For example, a pool with weight 2 should get 2x more slots than a pool with weight 1. This is accomplished by changing the scheduling rule to ``assign the slot to the pool whose value of $runningTasks/weight$ is smallest."
+  \item Minimum shares mean that pools below their min share should get slots first. When we sort pools to choose which ones to schedule next, we place pools below their min share ahead of pools above their min share. We order the pools below their min share by how far they are below it as a percentage of the share.
+\end{itemize}
+
+This fair sharing algorithm is implemented in {\tt FairShareComparator} in the {\tt SchedulingAlgorithms} class. The comparator orders jobs by distance below min share and then by $runningTasks/weight$.
+
+\subsection{Preemption}
+\label{sec:preemption}
+
+To determine when to preempt tasks, the Fair Schedulers maintains two values for each {\tt PoolSchedulable}: the last time when the pool was at its min share, and the last time when the pool was at half its fair share. These conditions are checked periodically by the update thread in {\tt FairScheduler.updatePreemptionVariables}, using the methods {\tt isStarvedForMinShare} and {\tt isStarvedForFairShare}. These methods also take into account the demand of the pool, so that a pool is not counted as starving if its demand is below its min/fair share but is otherwise met.
+
+When preempting tasks, the scheduler kills the most recently launched tasks from over-scheduled pools. This minimizes the amount of computation wasted by preemption and ensures that all jobs can eventually finish (it is as if the preempted jobs just never got their last few slots). The tasks are chosen and preempted in {\tt preemptTasks}.
+
+Note that for min share preemption, it is clear when a pool is below its min share because the min share is given as a number of slots, but for fair share preemption, we must be able to compute a pool's fair share to determine when it is being starved. This computation is trickier than dividing the number of slots by the number of pools due to weights, min shares and demands. Section \ref{sec:fair-share-computation} explains how fair shares are computed.
+
+\subsection{Fair Share Computation}
+\label{sec:fair-share-computation}
+
+The scheduling algorithm in Section \ref{sec:fair-sharing-alg} achieves fair shares without actually needing to compute pools' numerical shares beforehand. However, for preemption and for displaying shares in the Web UI, we want to know what a pool's fair share is even if the pool is not currently at its share. That is, we want to know how many slots the pool \emph{would} get if we started with all slots being empty and ran the algorithm in Section \ref{sec:fair-sharing-alg} until we filled them.
+One way to compute these shares would be to simulate starting out with empty slots and calling {\tt assignTasks} repeatedly until they filled, but this is expensive, because each scheduling decision takes $O(numJobs)$ time and we need to make $O(numSlots)$ decisions.
+
+To compute fair shares efficiently, the Fair Scheduler includes an algorithm based on binary search in {\tt SchedulingAlgorithms.computeFairShares}. This algorithm is based on the following observation. If all slots had been assigned according to weighted fair sharing respecting pools' demands and min shares, then there would exist a ratio $r$ such that:
+\begin{enumerate}
+  \item Pools whose demand $d_i$ is less than $r w_i$ (where $w_i$ is the weight of the pool) are assigned $d_i$ slots.
+  \item Pools whose min share $m_i$ is more than $r w_i$ are assigned $\min(m_i, d_i)$ slots.
+  \item All other pools are assigned $r w_i$ slots.
+  \item The pools' shares sum up to the total number of slots $t$.
+\end{enumerate}
+
+The Fair Scheduler uses binary search to compute the correct $r$. We define a function $f(r)$ as the number of slots that would be used for a given $r$ if conditions 1-3 above were met, and then find a value of $r$ that makes $f(r)=t$. More precisely, $f(r)$ is defined as:
+$$f(r) = \sum_i{\min(d_i, \max(r w_i, m_i)).}$$
+
+Note that $f(r)$ is increasing in $r$ because every term of the sum is increasing, so the equation $f(r) = t$ can be solved by binary search. We choose 0 as a lower bound of our binary search because with $r=0$, only min shares are assigned. (An earlier check in {\tt computeFairShares} checks whether the min shares add up to more than the total number of slots, and if so, computes fair shares by scaling down the min shares proportionally and returns.) To compute an upper bound for the binary search, we try $r=1,2,4,8,\dots$ until we find a value large enough that either more than $t$ slots are used or all pools' demands are met (in case the demands added up to less than $t$).
+
+The steps of the algorithm are explained in detail in {\tt SchedulingAlgorithms.java}.
+
+This algorithm runs in time $O(NP)$, where $N$ is the number of jobs/pools and $P$ is the desired number of bits of precision in the computed values (number of iterations of binary search), which we've set to 25. It thus scales linearly in the number of jobs and pools.
+
+\subsection{Running Job Limits}
+
+Running job limits are implemented by marking jobs as not runnable if there are too many jobs submitted by the same user or pool. This is done in {\tt FairScheduler.updateRunnability}. A job that is not runnable declares its demand as 0 and always returns {\tt null} from {\tt assignTasks}.
+
+\subsection{Delay Scheduling}
+\label{sec:delay-scheduling}
+
+In Hadoop, running map tasks on the nodes or racks that contain their input data is critical for performance, because it avoids shipping the data over the network. However, always assigning slots to the first job in order of pool shares and in-pool ordering (the ``head-of-line job") can sometimes lead to poor locality:
+\begin{itemize}
+  \item If the head-of-line job is small, the chance of it having data on the node that a heartbeat was received from is small. Therefore, locality would be poor in a small-job workload if we always assigned slots to the head-of-line job.
+  \item When fair sharing is used, there is a strong bias for a job to be reassigned into a slot that it just finished a task in, because when it finishes the task, the job falls below its fair share. This can mean that jobs have a difficult time running in slots that other jobs have taken and thus achieve poor locality.
+\end{itemize}
+
+To deal with both of these situations, the Fair Scheduler can sacrifice fairness temporarily to improve locality through an algorithm called delay scheduling. If the head-of-line job cannot launch a local task on the TaskTracker that sent a heartbeat, then it is skipped, and other running jobs are looked at in order of pool shares and in-pool scheduling rules to find a job with a local task. However, if the head-of-line job has been skipped for a sufficiently long time, it is allowed to launch rack-local tasks. Then, if it is skipped for a longer time, it is also allowed to launch off-rack tasks. These skip times are called locality delays. Delays of a few seconds are sufficient to drastically increase locality.
+
+The Fair Scheduler allows locality delays to be set through {\tt mapred-site.xml} or to be turned off by setting them to zero. However, by default, it computes the delay automatically based on the heartbeat interval of the cluster. The delay is set to 1.5x the heartbeat interval.
+
+When a job that has been allowed to launch non-local tasks ends up launching a local task again, its ``locality level" resets and it must wait again before launching non-local tasks. This is done so that a job that gets ``unlucky" early in its lifetime does not continue to launch non-local tasks throughout its life.
+
+Delay scheduling is implemented by keeping track of two variables on each job: the locality level of the last map it launched (0 for node-local, 1 for rack-local and 2 for off-rack) and the time it has spent being skipped for a task. These are kept in a {\tt JobInfo} structure associated with each job in {\tt FairScheduler.java}. Whenever a job is asked for tasks, it checks the locality level it is allowed to launch them at through {\tt FairScheduler.getAllowedLocalityLevel}. If it does not launch a task, it is marked as ``visited" on that heartbeat by appending itself to a {\tt visited} job list that is passed around between calls to {\tt assignTasks} on the same heartbeat. Jobs that are visited on a heartbeat but do not launch any tasks during it are considered as skipped for the time interval between this heartbeat and the next. Code at the beginning of {\tt FairScheduler.assignTasks} increments the wait time of each skipped job by the time elapsed since the last heartbeat. Once a job has been skipped for more than the locality delay, {\tt getAllowedLocalityLevel} starts returning higher locality so that it is allowed to launch less-local tasks. Whenever the job launches a task, its wait time is reset, but we remember the locality level of the launched task so that the job is allowed to launch more tasks at this level without further waiting.
+
+\subsection{Locking Order}
+
+Fair Scheduler data structures can be touched by several threads. Most commonly, the JobTracker invokes {\tt assignTasks}. This happens inside a block of code where the JobTracker has locked itself already. Therefore, to prevent deadlocks, we always ensure that \emph{if both the FairScheduler and the JobTracker must be locked, the JobTracker is locked first}. Other threads that can lock the FairScheduler include the update thread and the web UI.
+
+\subsection{Unit Tests}
+
+The Fair Scheduler contains extensive unit tests using mock {\tt TaskTrackerManager}, {\tt JobInProgress}, {\tt TaskInProgress}, and {\tt Schedulable} objects. Scheduler tests are in {\tt TestFairScheduler.java}. The {\tt computeFairShares} algorithm is tested separately in {\tt TestComputeFairShares.java}. All tests use accelerated time via a fake {\tt Clock} class.
+
+\pagebreak
+\section{Code Guide}
+
+The following table lists some key source files in the Fair Scheduler:
+
+\begin{center}
+\begin{tabular}{|l|p{0.7\columnwidth}|}
+  \hline
+  {\bf File} & {\bf Contents} 
+  \\ \hline
+  {\tt FairScheduler.java} & Scheduler entry point. Also contains update thread, and logic for preemption, delay scheduling, and running job limits.
+  \\ \hline
+  {\tt Schedulable.java} & Definition of the {\tt Schedulable} class. Extended by {\tt JobSchedulable} and {\tt PoolSchedulable}.
+  \\ \hline
+  {\tt SchedulingAlgorithms.java} & Contains FIFO and fair sharing comparators, as well as the {\tt computeFairShares} algorithm in Section \ref{sec:fair-share-computation}.
+  \\ \hline
+  {\tt PoolManager.java} & Reads pool properties from the allocation file and maintains a collection of {\tt Pool} objects. Pools are created on demand.
+  \\ \hline
+  {\tt Pool.java} & Represents a pool and stores its map and reduce {\tt Schedulables}.
+  \\ \hline
+  {\tt FairSchedulerServlet.java} & Implements the scheduler's web UI.
+  \\ \hline
+  {\tt FairSchedulerEventLog.java} & An easy-to-parse event log for debugging. Must be enabled through {\tt mapred.fairscheduler.eventlog.enabled}.
+  If enabled, logs are placed in {\tt \$HADOOP\_LOG\_DIR/fairscheduler}.
+  \\ \hline
+  {\tt TaskSelector.java} & A pluggable class responsible for picking tasks within a job. Currently, {\tt DefaultTaskSelector} delegates to {\tt JobInProgress}, but this would be a useful place to experiment with new algorithms for speculative execution and locality.
+  \\ \hline
+  {\tt LoadManager.java} & A pluggable class responsible for determining when to launch more tasks on a TaskTracker. Currently, {\tt CapBasedLoadManager} uses slot counts, but this would be a useful place to experiment with scheduling based on machine load.
+  \\ \hline
+  {\tt WeightAdjuster.java} & A pluggable class responsible for setting job weights. An example, {\tt NewJobWeightBooster}, is provided, which increases weight temporarily for new jobs.
+  \\ \hline
+\end{tabular}
+\end{center}
+
+\end{document}

+ 8 - 0
src/contrib/fairscheduler/ivy.xml

@@ -26,6 +26,14 @@
       name="commons-logging"
       name="commons-logging"
       rev="${commons-logging.version}"
       rev="${commons-logging.version}"
       conf="common->default"/>
       conf="common->default"/>
+    <dependency org="commons-collections"
+      name="commons-collections"
+      rev="${commons-collections.version}"
+      conf="common->default"/>
+    <dependency org="commons-cli"
+      name="commons-cli"
+      rev="${commons-cli.version}"
+      conf="common->default"/>
     <dependency org="log4j"
     <dependency org="log4j"
       name="log4j"
       name="log4j"
       rev="${log4j.version}"
       rev="${log4j.version}"

+ 18 - 1
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/CapBasedLoadManager.java

@@ -18,12 +18,23 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.conf.Configuration;
+
 /**
 /**
  * A {@link LoadManager} for use by the {@link FairScheduler} that allocates
  * A {@link LoadManager} for use by the {@link FairScheduler} that allocates
  * tasks evenly across nodes up to their per-node maximum, using the default
  * tasks evenly across nodes up to their per-node maximum, using the default
  * load management algorithm in Hadoop.
  * load management algorithm in Hadoop.
  */
  */
 public class CapBasedLoadManager extends LoadManager {
 public class CapBasedLoadManager extends LoadManager {
+  
+  float maxDiff = 0.0f;
+  
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    maxDiff = conf.getFloat("mapred.fairscheduler.load.max.diff", 0.0f);
+  }
+  
   /**
   /**
    * Determine how many tasks of a given type we want to run on a TaskTracker. 
    * Determine how many tasks of a given type we want to run on a TaskTracker. 
    * This cap is chosen based on how many tasks of that type are outstanding in
    * This cap is chosen based on how many tasks of that type are outstanding in
@@ -32,7 +43,7 @@ public class CapBasedLoadManager extends LoadManager {
    * machines sent out heartbeats earliest.
    * machines sent out heartbeats earliest.
    */
    */
   int getCap(int totalRunnableTasks, int localMaxTasks, int totalSlots) {
   int getCap(int totalRunnableTasks, int localMaxTasks, int totalSlots) {
-    double load = ((double)totalRunnableTasks) / totalSlots;
+    double load = maxDiff + ((double)totalRunnableTasks) / totalSlots;
     return (int) Math.ceil(localMaxTasks * Math.min(1.0, load));
     return (int) Math.ceil(localMaxTasks * Math.min(1.0, load));
   }
   }
 
 
@@ -49,4 +60,10 @@ public class CapBasedLoadManager extends LoadManager {
     return tracker.countReduceTasks() < getCap(totalRunnableReduces,
     return tracker.countReduceTasks() < getCap(totalRunnableReduces,
         tracker.getMaxReduceSlots(), totalReduceSlots);
         tracker.getMaxReduceSlots(), totalReduceSlots);
   }
   }
+
+  @Override
+  public boolean canLaunchTask(TaskTrackerStatus tracker,
+      JobInProgress job,  TaskType type) {
+    return true;
+  }
 }
 }

+ 13 - 4
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/DefaultTaskSelector.java

@@ -56,12 +56,21 @@ public class DefaultTaskSelector extends TaskSelector {
   }
   }
 
 
   @Override
   @Override
-  public Task obtainNewMapTask(TaskTrackerStatus taskTracker, JobInProgress job)
-      throws IOException {
+  public Task obtainNewMapTask(TaskTrackerStatus taskTracker, JobInProgress job,
+      int localityLevel) throws IOException {
     ClusterStatus clusterStatus = taskTrackerManager.getClusterStatus();
     ClusterStatus clusterStatus = taskTrackerManager.getClusterStatus();
     int numTaskTrackers = clusterStatus.getTaskTrackers();
     int numTaskTrackers = clusterStatus.getTaskTrackers();
-    return job.obtainNewMapTask(taskTracker, numTaskTrackers,
-        taskTrackerManager.getNumberOfUniqueHosts());
+    switch (localityLevel) {
+      case 1:
+        return job.obtainNewNodeLocalMapTask(taskTracker, numTaskTrackers,
+          taskTrackerManager.getNumberOfUniqueHosts());
+      case 2:
+        return job.obtainNewNodeOrRackLocalMapTask(taskTracker, numTaskTrackers,
+          taskTrackerManager.getNumberOfUniqueHosts());
+      default:
+        return job.obtainNewMapTask(taskTracker, numTaskTrackers,
+          taskTrackerManager.getNumberOfUniqueHosts());
+    }
   }
   }
 
 
   @Override
   @Override

Різницю між файлами не показано, бо вона завелика
+ 506 - 427
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java


+ 142 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerEventLog.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.mapred;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.DailyRollingFileAppender;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import org.apache.log4j.spi.LoggingEvent;
+
+/**
+ * Event log used by the fair scheduler for machine-readable debug info.
+ * This class uses a log4j rolling file appender to write the log, but uses
+ * a custom tab-separated event format of the form:
+ * <pre>
+ * DATE    EVENT_TYPE   PARAM_1   PARAM_2   ...
+ * </pre>
+ * Various event types are used by the fair scheduler. The purpose of logging
+ * in this format is to enable tools to parse the history log easily and read
+ * internal scheduler variables, rather than trying to make the log human
+ * readable. The fair scheduler also logs human readable messages in the
+ * JobTracker's main log.
+ * 
+ * Constructing this class creates a disabled log. It must be initialized
+ * using {@link FairSchedulerEventLog#init(Configuration, String)} to begin
+ * writing to the file.
+ */
+class FairSchedulerEventLog {
+  private static final Log LOG = LogFactory.getLog(
+    "org.apache.hadoop.mapred.FairSchedulerEventLog");
+  
+  /** Set to true if logging is disabled due to an error. */
+  private boolean logDisabled = true;
+  
+  /**
+   * Log directory, set by mapred.fairscheduler.eventlog.location in conf file;
+   * defaults to {hadoop.log.dir}/fairscheduler.
+   */
+  private String logDir;
+  
+  /** 
+   * Active log file, which is {LOG_DIR}/hadoop-{user}-fairscheduler.{host}.log.
+   * Older files are also stored as {LOG_FILE}.date (date format YYYY-MM-DD).
+   */ 
+  private String logFile;
+  
+  /** Log4j appender used to write to the log file */
+  private DailyRollingFileAppender appender;
+
+  boolean init(Configuration conf, String jobtrackerHostname) {
+    try {
+      logDir = conf.get("mapred.fairscheduler.eventlog.location",
+          new File(System.getProperty("hadoop.log.dir")).getAbsolutePath()
+          + File.separator + "fairscheduler");
+      Path logDirPath = new Path(logDir);
+      FileSystem fs = logDirPath.getFileSystem(conf);
+      if (!fs.exists(logDirPath)) {
+        if (!fs.mkdirs(logDirPath)) {
+          throw new IOException(
+              "Mkdirs failed to create " + logDirPath.toString());
+        }
+      }
+      String username = System.getProperty("user.name");
+      logFile = String.format("%s%shadoop-%s-fairscheduler-%s.log",
+          logDir, File.separator, username, jobtrackerHostname);
+      logDisabled = false;
+      PatternLayout layout = new PatternLayout("%d{ISO8601}\t%m%n");
+      appender = new DailyRollingFileAppender(layout, logFile, "'.'yyyy-MM-dd");
+      appender.activateOptions();
+      LOG.info("Initialized fair scheduler event log, logging to " + logFile);
+    } catch (IOException e) {
+      LOG.error(
+          "Failed to initialize fair scheduler event log. Disabling it.", e);
+      logDisabled = true;
+    }
+    return !(logDisabled);
+  }
+  
+  /**
+   * Log an event, writing a line in the log file of the form
+   * <pre>
+   * DATE    EVENT_TYPE   PARAM_1   PARAM_2   ...
+   * </pre>
+   */
+  synchronized void log(String eventType, Object... params) {
+    try {
+      if (logDisabled)
+        return;
+      StringBuffer buffer = new StringBuffer();
+      buffer.append(eventType);
+      for (Object param: params) {
+        buffer.append("\t");
+        buffer.append(param);
+      }
+      String message = buffer.toString();
+      Logger logger = Logger.getLogger(getClass());
+      appender.append(new LoggingEvent("", logger, Level.INFO, message, null));
+    } catch (Exception e) {
+      LOG.error("Failed to append to fair scheduler event log", e);
+      logDisabled = true;
+    }
+  }
+  
+  /**
+   * Flush and close the log.
+   */
+  void shutdown() {
+    try {
+      if (appender != null)
+        appender.close();
+    } catch (Exception e) {}
+    logDisabled = true;
+  }
+
+  boolean isEnabled() {
+    return !logDisabled;
+  }
+}

+ 113 - 86
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java

@@ -30,6 +30,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.Date;
+import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 
 
 import javax.servlet.ServletContext;
 import javax.servlet.ServletContext;
@@ -39,16 +40,15 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponse;
 
 
 import org.apache.hadoop.mapred.FairScheduler.JobInfo;
 import org.apache.hadoop.mapred.FairScheduler.JobInfo;
+import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
 /**
 /**
  * Servlet for displaying fair scheduler information, installed at
  * Servlet for displaying fair scheduler information, installed at
  * [job tracker URL]/scheduler when the {@link FairScheduler} is in use.
  * [job tracker URL]/scheduler when the {@link FairScheduler} is in use.
  * 
  * 
- * The main features are viewing each job's task count and fair share, ability
- * to change job priorities and pools from the UI, and ability to switch the
- * scheduler to FIFO mode without restarting the JobTracker if this is required
- * for any reason.
+ * The main features are viewing each job's task count and fair share,
+ * and admin controls to change job priorities and pools from the UI.
  * 
  * 
  * There is also an "advanced" view for debugging that can be turned on by
  * There is also an "advanced" view for debugging that can be turned on by
  * going to [job tracker URL]/scheduler?advanced.
  * going to [job tracker URL]/scheduler?advanced.
@@ -82,13 +82,9 @@ public class FairSchedulerServlet extends HttpServlet {
     // If the request has a set* param, handle that and redirect to the regular
     // If the request has a set* param, handle that and redirect to the regular
     // view page so that the user won't resubmit the data if they hit refresh.
     // view page so that the user won't resubmit the data if they hit refresh.
     boolean advancedView = request.getParameter("advanced") != null;
     boolean advancedView = request.getParameter("advanced") != null;
-    if (request.getParameter("setFifo") != null) {
-      scheduler.setUseFifo(request.getParameter("setFifo").equals("true"));
-      response.sendRedirect("/scheduler" + (advancedView ? "?advanced" : ""));
-      return;
-    }
-    if (request.getParameter("setPool") != null) {
-      Collection<JobInProgress> runningJobs = jobTracker.getRunningJobs();
+    if (JSPUtil.privateActionsAllowed(jobTracker.conf)
+        && request.getParameter("setPool") != null) {
+      Collection<JobInProgress> runningJobs = getInitedJobs();
       PoolManager poolMgr = null;
       PoolManager poolMgr = null;
       synchronized (scheduler) {
       synchronized (scheduler) {
         poolMgr = scheduler.getPoolManager();
         poolMgr = scheduler.getPoolManager();
@@ -107,8 +103,9 @@ public class FairSchedulerServlet extends HttpServlet {
       response.sendRedirect("/scheduler" + (advancedView ? "?advanced" : ""));
       response.sendRedirect("/scheduler" + (advancedView ? "?advanced" : ""));
       return;
       return;
     }
     }
-    if (request.getParameter("setPriority") != null) {
-      Collection<JobInProgress> runningJobs = jobTracker.getRunningJobs();      
+    if (JSPUtil.privateActionsAllowed(jobTracker.conf)
+        && request.getParameter("setPriority") != null) {
+      Collection<JobInProgress> runningJobs = getInitedJobs();
       JobPriority priority = JobPriority.valueOf(request.getParameter(
       JobPriority priority = JobPriority.valueOf(request.getParameter(
           "setPriority"));
           "setPriority"));
       String jobId = request.getParameter("jobid");
       String jobId = request.getParameter("jobid");
@@ -126,22 +123,21 @@ public class FairSchedulerServlet extends HttpServlet {
     response.setContentType("text/html");
     response.setContentType("text/html");
 
 
     // Because the client may read arbitrarily slow, and we hold locks while
     // Because the client may read arbitrarily slow, and we hold locks while
-    // the servlet output, we want to write to our own buffer which we know
+    // the servlet outputs, we want to write to our own buffer which we know
     // won't block.
     // won't block.
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PrintWriter out = new PrintWriter(baos);
     PrintWriter out = new PrintWriter(baos);
     String hostname = StringUtils.simpleHostname(
     String hostname = StringUtils.simpleHostname(
         jobTracker.getJobTrackerMachine());
         jobTracker.getJobTrackerMachine());
     out.print("<html><head>");
     out.print("<html><head>");
-    out.printf("<title>%s Job Scheduler Admininstration</title>\n", hostname);
+    out.printf("<title>%s Fair Scheduler Administration</title>\n", hostname);
     out.print("<link rel=\"stylesheet\" type=\"text/css\" " + 
     out.print("<link rel=\"stylesheet\" type=\"text/css\" " + 
         "href=\"/static/hadoop.css\">\n");
         "href=\"/static/hadoop.css\">\n");
     out.print("</head><body>\n");
     out.print("</head><body>\n");
     out.printf("<h1><a href=\"/jobtracker.jsp\">%s</a> " + 
     out.printf("<h1><a href=\"/jobtracker.jsp\">%s</a> " + 
-        "Job Scheduler Administration</h1>\n", hostname);
+        "Fair Scheduler Administration</h1>\n", hostname);
     showPools(out, advancedView);
     showPools(out, advancedView);
     showJobs(out, advancedView);
     showJobs(out, advancedView);
-    showAdminForm(out, advancedView);
     out.print("</body></html>\n");
     out.print("</body></html>\n");
     out.close();
     out.close();
 
 
@@ -156,12 +152,17 @@ public class FairSchedulerServlet extends HttpServlet {
    */
    */
   private void showPools(PrintWriter out, boolean advancedView) {
   private void showPools(PrintWriter out, boolean advancedView) {
     synchronized(scheduler) {
     synchronized(scheduler) {
+      boolean warnInverted = false;
       PoolManager poolManager = scheduler.getPoolManager();
       PoolManager poolManager = scheduler.getPoolManager();
       out.print("<h2>Pools</h2>\n");
       out.print("<h2>Pools</h2>\n");
       out.print("<table border=\"2\" cellpadding=\"5\" cellspacing=\"2\">\n");
       out.print("<table border=\"2\" cellpadding=\"5\" cellspacing=\"2\">\n");
-      out.print("<tr><th>Pool</th><th>Running Jobs</th>" + 
-          "<th>Min Maps</th><th>Min Reduces</th>" + 
-          "<th>Running Maps</th><th>Running Reduces</th></tr>\n");
+      out.print("<tr><th rowspan=2>Pool</th>" +
+          "<th rowspan=2>Running Jobs</th>" + 
+          "<th colspan=4>Map Tasks</th>" + 
+          "<th colspan=4>Reduce Tasks</th>" +
+          "<th rowspan=2>Scheduling Mode</th></tr>\n<tr>" + 
+          "<th>Min Share</th><th>Max Share</th><th>Running</th><th>Fair Share</th>" + 
+          "<th>Min Share</th><th>Max Share</th><th>Running</th><th>Fair Share</th></tr>\n");
       List<Pool> pools = new ArrayList<Pool>(poolManager.getPools());
       List<Pool> pools = new ArrayList<Pool>(poolManager.getPools());
       Collections.sort(pools, new Comparator<Pool>() {
       Collections.sort(pools, new Comparator<Pool>() {
         public int compare(Pool p1, Pool p2) {
         public int compare(Pool p1, Pool p2) {
@@ -172,27 +173,54 @@ public class FairSchedulerServlet extends HttpServlet {
           else return p1.getName().compareTo(p2.getName());
           else return p1.getName().compareTo(p2.getName());
         }});
         }});
       for (Pool pool: pools) {
       for (Pool pool: pools) {
-        int runningMaps = 0;
-        int runningReduces = 0;
-        for (JobInProgress job: pool.getJobs()) {
-          JobInfo info = scheduler.infos.get(job);
-          if (info != null) {
-            runningMaps += info.runningMaps;
-            runningReduces += info.runningReduces;
-          }
-        }
-        out.print("<tr>\n");
-        out.printf("<td>%s</td>\n", pool.getName());
-        out.printf("<td>%s</td>\n", pool.getJobs().size());
-        out.printf("<td>%s</td>\n", poolManager.getAllocation(pool.getName(),
+        String name = pool.getName();
+        int runningMaps = pool.getMapSchedulable().getRunningTasks();
+        int runningReduces = pool.getReduceSchedulable().getRunningTasks();
+        int maxMaps = poolManager.getMaxSlots(name, TaskType.MAP);
+        int maxReduces = poolManager.getMaxSlots(name, TaskType.REDUCE);
+        boolean invertedMaps = poolManager.invertedMinMax(TaskType.MAP, name);
+        boolean invertedReduces = poolManager.invertedMinMax(TaskType.REDUCE, name);
+        warnInverted = warnInverted || invertedMaps || invertedReduces;
+        out.print("<tr>");
+        out.printf("<td>%s</td>", name);
+        out.printf("<td>%d</td>", pool.getJobs().size());
+        // Map Tasks
+        out.printf("<td>%d</td>", poolManager.getAllocation(name,
             TaskType.MAP));
             TaskType.MAP));
-        out.printf("<td>%s</td>\n", poolManager.getAllocation(pool.getName(), 
+        out.print("<td>");
+        if(maxMaps == Integer.MAX_VALUE) {
+          out.print("-");
+        } else {
+          out.print(maxMaps);
+        }
+        if(invertedMaps) {
+          out.print("*");
+        }
+        out.print("</td>");
+        out.printf("<td>%d</td>", runningMaps);
+        out.printf("<td>%.1f</td>", pool.getMapSchedulable().getFairShare());
+        // Reduce Tasks
+        out.printf("<td>%d</td>", poolManager.getAllocation(name,
             TaskType.REDUCE));
             TaskType.REDUCE));
-        out.printf("<td>%s</td>\n", runningMaps);
-        out.printf("<td>%s</td>\n", runningReduces);
+        out.print("<td>");
+        if(maxReduces == Integer.MAX_VALUE) {
+          out.print("-");
+        } else {
+          out.print(maxReduces);
+        }
+        if(invertedReduces) {
+          out.print("*");
+        }
+        out.print("</td>");
+        out.printf("<td>%d</td>", runningReduces);
+        out.printf("<td>%.1f</td>", pool.getReduceSchedulable().getFairShare());
+        out.printf("<td>%s</td>", pool.getSchedulingMode());
         out.print("</tr>\n");
         out.print("</tr>\n");
       }
       }
       out.print("</table>\n");
       out.print("</table>\n");
+      if(warnInverted) {
+        out.print("<p>* One or more pools have max share set lower than min share. Max share will be used and minimum will be treated as if set equal to max.</p>");
+      }
     }
     }
   }
   }
 
 
@@ -202,66 +230,70 @@ public class FairSchedulerServlet extends HttpServlet {
   private void showJobs(PrintWriter out, boolean advancedView) {
   private void showJobs(PrintWriter out, boolean advancedView) {
     out.print("<h2>Running Jobs</h2>\n");
     out.print("<h2>Running Jobs</h2>\n");
     out.print("<table border=\"2\" cellpadding=\"5\" cellspacing=\"2\">\n");
     out.print("<table border=\"2\" cellpadding=\"5\" cellspacing=\"2\">\n");
-    int colsPerTaskType = advancedView ? 6 : 3;
+    int colsPerTaskType = advancedView ? 4 : 3;
     out.printf("<tr><th rowspan=2>Submitted</th>" + 
     out.printf("<tr><th rowspan=2>Submitted</th>" + 
         "<th rowspan=2>JobID</th>" +
         "<th rowspan=2>JobID</th>" +
         "<th rowspan=2>User</th>" +
         "<th rowspan=2>User</th>" +
         "<th rowspan=2>Name</th>" +
         "<th rowspan=2>Name</th>" +
         "<th rowspan=2>Pool</th>" +
         "<th rowspan=2>Pool</th>" +
         "<th rowspan=2>Priority</th>" +
         "<th rowspan=2>Priority</th>" +
-        "<th colspan=%d>Maps</th>" +
-        "<th colspan=%d>Reduces</th>",
+        "<th colspan=%d>Map Tasks</th>" +
+        "<th colspan=%d>Reduce Tasks</th>",
         colsPerTaskType, colsPerTaskType);
         colsPerTaskType, colsPerTaskType);
     out.print("</tr><tr>\n");
     out.print("</tr><tr>\n");
     out.print("<th>Finished</th><th>Running</th><th>Fair Share</th>" +
     out.print("<th>Finished</th><th>Running</th><th>Fair Share</th>" +
-        (advancedView ? "<th>Weight</th><th>Deficit</th><th>minMaps</th>" : ""));
+        (advancedView ? "<th>Weight</th>" : ""));
     out.print("<th>Finished</th><th>Running</th><th>Fair Share</th>" +
     out.print("<th>Finished</th><th>Running</th><th>Fair Share</th>" +
-        (advancedView ? "<th>Weight</th><th>Deficit</th><th>minReduces</th>" : ""));
+        (advancedView ? "<th>Weight</th>" : ""));
     out.print("</tr>\n");
     out.print("</tr>\n");
     synchronized (jobTracker) {
     synchronized (jobTracker) {
-      Collection<JobInProgress> runningJobs = jobTracker.getRunningJobs();
+      Collection<JobInProgress> runningJobs = getInitedJobs();
       synchronized (scheduler) {
       synchronized (scheduler) {
         for (JobInProgress job: runningJobs) {
         for (JobInProgress job: runningJobs) {
           JobProfile profile = job.getProfile();
           JobProfile profile = job.getProfile();
           JobInfo info = scheduler.infos.get(job);
           JobInfo info = scheduler.infos.get(job);
           if (info == null) { // Job finished, but let's show 0's for info
           if (info == null) { // Job finished, but let's show 0's for info
-            info = new JobInfo();
+            info = new JobInfo(null, null);
           }
           }
           out.print("<tr>\n");
           out.print("<tr>\n");
           out.printf("<td>%s</td>\n", DATE_FORMAT.format(
           out.printf("<td>%s</td>\n", DATE_FORMAT.format(
-                       new Date(job.getStartTime())));
+              new Date(job.getStartTime())));
           out.printf("<td><a href=\"jobdetails.jsp?jobid=%s\">%s</a></td>",
           out.printf("<td><a href=\"jobdetails.jsp?jobid=%s\">%s</a></td>",
-                     profile.getJobID(), profile.getJobID());
+              profile.getJobID(), profile.getJobID());
           out.printf("<td>%s</td>\n", profile.getUser());
           out.printf("<td>%s</td>\n", profile.getUser());
           out.printf("<td>%s</td>\n", profile.getJobName());
           out.printf("<td>%s</td>\n", profile.getJobName());
-          out.printf("<td>%s</td>\n", generateSelect(
-                       scheduler.getPoolManager().getPoolNames(),
-                       scheduler.getPoolManager().getPoolName(job),
-                       "/scheduler?setPool=<CHOICE>&jobid=" + profile.getJobID() +
-                       (advancedView ? "&advanced" : "")));
-          out.printf("<td>%s</td>\n", generateSelect(
-                       Arrays.asList(new String[]
-                         {"VERY_LOW", "LOW", "NORMAL", "HIGH", "VERY_HIGH"}),
-                       job.getPriority().toString(),
-                       "/scheduler?setPriority=<CHOICE>&jobid=" + profile.getJobID() +
-                       (advancedView ? "&advanced" : "")));
-          out.printf("<td>%d / %d</td><td>%d</td><td>%8.1f</td>\n",
-                     job.finishedMaps(), job.desiredMaps(), info.runningMaps,
-                     info.mapFairShare);
+          if (JSPUtil.privateActionsAllowed(jobTracker.conf)) {
+            out.printf("<td>%s</td>\n", generateSelect(scheduler
+                .getPoolManager().getPoolNames(), scheduler.getPoolManager()
+                .getPoolName(job), "/scheduler?setPool=<CHOICE>&jobid="
+                + profile.getJobID() + (advancedView ? "&advanced" : "")));
+            out.printf("<td>%s</td>\n", generateSelect(Arrays
+                .asList(new String[] { "VERY_LOW", "LOW", "NORMAL", "HIGH",
+                    "VERY_HIGH" }), job.getPriority().toString(),
+                "/scheduler?setPriority=<CHOICE>&jobid=" + profile.getJobID()
+                    + (advancedView ? "&advanced" : "")));
+          } else {
+            out.printf("<td>%s</td>\n", scheduler.getPoolManager().getPoolName(job));
+            out.printf("<td>%s</td>\n", job.getPriority().toString());
+          }
+          Pool pool = scheduler.getPoolManager().getPool(job);
+          String mapShare = (pool.getSchedulingMode() == SchedulingMode.FAIR) ?
+              String.format("%.1f", info.mapSchedulable.getFairShare()) : "NA";
+          out.printf("<td>%d / %d</td><td>%d</td><td>%s</td>\n",
+              job.finishedMaps(), job.desiredMaps(), 
+              info.mapSchedulable.getRunningTasks(),
+              mapShare);
           if (advancedView) {
           if (advancedView) {
-            out.printf("<td>%8.1f</td>\n", info.mapWeight);
-            out.printf("<td>%s</td>\n", info.neededMaps > 0 ?
-                       (info.mapDeficit / 1000) + "s" : "--");
-            out.printf("<td>%d</td>\n", info.minMaps);
+            out.printf("<td>%.1f</td>\n", info.mapSchedulable.getWeight());
           }
           }
-          out.printf("<td>%d / %d</td><td>%d</td><td>%8.1f</td>\n",
-                     job.finishedReduces(), job.desiredReduces(), info.runningReduces,
-                     info.reduceFairShare);
+          String reduceShare = (pool.getSchedulingMode() == SchedulingMode.FAIR) ?
+              String.format("%.1f", info.reduceSchedulable.getFairShare()) : "NA";
+          out.printf("<td>%d / %d</td><td>%d</td><td>%s</td>\n",
+              job.finishedReduces(), job.desiredReduces(), 
+              info.reduceSchedulable.getRunningTasks(),
+              reduceShare);
           if (advancedView) {
           if (advancedView) {
-            out.printf("<td>%8.1f</td>\n", info.reduceWeight);
-            out.printf("<td>%s</td>\n", info.neededReduces > 0 ?
-                       (info.reduceDeficit / 1000) + "s" : "--");
-            out.printf("<td>%d</td>\n", info.minReduces);
+            out.printf("<td>%.1f</td>\n", info.reduceSchedulable.getWeight());
           }
           }
           out.print("</tr>\n");
           out.print("</tr>\n");
         }
         }
@@ -294,22 +326,17 @@ public class FairSchedulerServlet extends HttpServlet {
   }
   }
 
 
   /**
   /**
-   * Print the administration form at the bottom of the page, which currently
-   * only includes the button for switching between FIFO and Fair Scheduling.
+   * Obtained all initialized jobs
    */
    */
-  private void showAdminForm(PrintWriter out, boolean advancedView) {
-    out.print("<h2>Scheduling Mode</h2>\n");
-    String curMode = scheduler.getUseFifo() ? "FIFO" : "Fair Sharing";
-    String otherMode = scheduler.getUseFifo() ? "Fair Sharing" : "FIFO";
-    String advParam = advancedView ? "?advanced" : "";
-    out.printf("<form method=\"post\" action=\"/scheduler%s\">\n", advParam);
-    out.printf("<p>The scheduler is currently using <b>%s mode</b>. " +
-        "<input type=\"submit\" value=\"Switch to %s mode.\" " + 
-        "onclick=\"return confirm('Are you sure you want to change " +
-        "scheduling mode to %s?')\" />\n",
-        curMode, otherMode, otherMode);
-    out.printf("<input type=\"hidden\" name=\"setFifo\" value=\"%s\" />",
-        !scheduler.getUseFifo());
-    out.print("</form>\n");
+  private Collection<JobInProgress> getInitedJobs() {
+    Collection<JobInProgress> runningJobs = jobTracker.getRunningJobs();
+    for (Iterator<JobInProgress> it = runningJobs.iterator(); it.hasNext();) {
+      JobInProgress job = it.next();
+      if (!job.inited()) {
+        it.remove();
+      }
+    }
+    return runningJobs;
   }
   }
+
 }
 }

+ 2 - 1
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FifoJobComparator.java

@@ -35,7 +35,8 @@ public class FifoJobComparator implements Comparator<JobInProgress> {
       }
       }
     }
     }
     if (res == 0) {
     if (res == 0) {
-      res = j1.hashCode() - j2.hashCode();
+      // If there is a tie, break it by job ID to get a deterministic order
+      res = j1.getJobID().compareTo(j2.getJobID());
     }
     }
     return res;
     return res;
   }
   }

+ 185 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/JobSchedulable.java

@@ -0,0 +1,185 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.mapred.FairScheduler.JobInfo;
+import org.apache.hadoop.mapreduce.TaskType;
+
+public class JobSchedulable extends Schedulable {
+  private FairScheduler scheduler;
+  private JobInProgress job;
+  private TaskType taskType;
+  private int demand = 0;
+
+  public JobSchedulable(FairScheduler scheduler, JobInProgress job, 
+      TaskType taskType) {
+    this.scheduler = scheduler;
+    this.job = job;
+    this.taskType = taskType;
+    
+    initMetrics();
+  }
+  
+  @Override
+  public TaskType getTaskType() {
+    return taskType;
+  }
+  
+  @Override
+  public String getName() {
+    return job.getJobID().toString();
+  }
+
+  public JobInProgress getJob() {
+    return job;
+  }
+  
+  @Override
+  public void updateDemand() {
+    demand = 0;
+    if (isRunnable()) {
+      // For reduces, make sure enough maps are done that reduces can launch
+      if (taskType == TaskType.REDUCE && !job.scheduleReduces())
+        return;
+      // Add up demand from each TaskInProgress; each TIP can either
+      // - have no attempts running, in which case it demands 1 slot
+      // - have N attempts running, in which case it demands N slots, and may
+      //   potentially demand one more slot if it needs to be speculated
+      TaskInProgress[] tips = (taskType == TaskType.MAP ? 
+          job.getTasks(TaskType.MAP) : job.getTasks(TaskType.REDUCE));
+      boolean speculationEnabled = (taskType == TaskType.MAP ?
+          job.getMapSpeculativeExecution() : job.getReduceSpeculativeExecution());
+      double avgProgress = (taskType == TaskType.MAP ?
+          job.getStatus().mapProgress() : job.getStatus().reduceProgress());
+      long time = scheduler.getClock().getTime();
+      for (TaskInProgress tip: tips) {
+        if (!tip.isComplete()) {
+          if (tip.isRunning()) {
+            // Count active tasks and any speculative task we want to launch
+            demand += tip.getActiveTasks().size();
+            if (speculationEnabled && tip.hasSpeculativeTask(time, avgProgress))
+              demand += 1;
+          } else {
+            // Need to launch 1 task
+            demand += 1;
+          }
+        }
+      }
+    }
+  }
+
+  private boolean isRunnable() {
+    JobInfo info = scheduler.getJobInfo(job);
+    int runState = job.getStatus().getRunState();
+    return (info != null && info.runnable && runState == JobStatus.RUNNING);
+  }
+
+  @Override
+  public int getDemand() {
+    return demand;
+  }
+  
+  @Override
+  public void redistributeShare() {}
+
+  @Override
+  public JobPriority getPriority() {
+    return job.getPriority();
+  }
+
+  @Override
+  public int getRunningTasks() {
+    if (!job.inited()) {
+      return 0;
+    }
+    return taskType == TaskType.MAP ? job.runningMaps() : job.runningReduces();
+  }
+
+  @Override
+  public long getStartTime() {
+    return job.startTime;
+  }
+  
+  @Override
+  public double getWeight() {
+    return scheduler.getJobWeight(job, taskType);
+  }
+  
+  @Override
+  public int getMinShare() {
+    return 0;
+  }
+
+  @Override
+  public Task assignTask(TaskTrackerStatus tts, long currentTime,
+      Collection<JobInProgress> visited) throws IOException {
+    if (isRunnable()) {
+      visited.add(job);
+      TaskTrackerManager ttm = scheduler.taskTrackerManager;
+      ClusterStatus clusterStatus = ttm.getClusterStatus();
+      int numTaskTrackers = clusterStatus.getTaskTrackers();
+
+      // check with the load manager whether it is safe to 
+      // launch this task on this taskTracker.
+      LoadManager loadMgr = scheduler.getLoadManager();
+      if (!loadMgr.canLaunchTask(tts, job, taskType)) {
+        return null;
+      }
+      if (taskType == TaskType.MAP) {
+        LocalityLevel localityLevel = scheduler.getAllowedLocalityLevel(
+            job, currentTime);
+        scheduler.getEventLog().log(
+            "ALLOWED_LOC_LEVEL", job.getJobID(), localityLevel);
+        switch (localityLevel) {
+          case NODE:
+            return job.obtainNewNodeLocalMapTask(tts, numTaskTrackers,
+                ttm.getNumberOfUniqueHosts());
+          case RACK:
+            return job.obtainNewNodeOrRackLocalMapTask(tts, numTaskTrackers,
+                ttm.getNumberOfUniqueHosts());
+          default:
+            return job.obtainNewMapTask(tts, numTaskTrackers,
+                ttm.getNumberOfUniqueHosts());
+        }
+      } else {
+        return job.obtainNewReduceTask(tts, numTaskTrackers,
+            ttm.getNumberOfUniqueHosts());
+      }
+    } else {
+      return null;
+    }
+  }
+
+  
+  @Override
+  protected String getMetricsContextName() {
+    return "jobs";
+  }
+  
+  @Override
+  void updateMetrics() {
+    assert metrics != null;
+    
+    super.setMetricValues(metrics);
+    metrics.update();
+  }
+}

+ 22 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/LoadManager.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskType;
 
 
 /**
 /**
  * A pluggable object that manages the load on each {@link TaskTracker}, telling
  * A pluggable object that manages the load on each {@link TaskTracker}, telling
@@ -30,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 public abstract class LoadManager implements Configurable {
 public abstract class LoadManager implements Configurable {
   protected Configuration conf;
   protected Configuration conf;
   protected TaskTrackerManager taskTrackerManager;
   protected TaskTrackerManager taskTrackerManager;
+  protected FairSchedulerEventLog schedulingLog;
   
   
   public Configuration getConf() {
   public Configuration getConf() {
     return conf;
     return conf;
@@ -43,6 +45,10 @@ public abstract class LoadManager implements Configurable {
       TaskTrackerManager taskTrackerManager) {
       TaskTrackerManager taskTrackerManager) {
     this.taskTrackerManager = taskTrackerManager;
     this.taskTrackerManager = taskTrackerManager;
   }
   }
+
+  public void setEventLog(FairSchedulerEventLog schedulingLog) {
+    this.schedulingLog = schedulingLog;
+  }
   
   
   /**
   /**
    * Lifecycle method to allow the LoadManager to start any work in separate
    * Lifecycle method to allow the LoadManager to start any work in separate
@@ -61,6 +67,8 @@ public abstract class LoadManager implements Configurable {
   
   
   /**
   /**
    * Can a given {@link TaskTracker} run another map task?
    * Can a given {@link TaskTracker} run another map task?
+   * This method may check whether the specified tracker has
+   * enough resources to run another map task.
    * @param tracker The machine we wish to run a new map on
    * @param tracker The machine we wish to run a new map on
    * @param totalRunnableMaps Set of running jobs in the cluster
    * @param totalRunnableMaps Set of running jobs in the cluster
    * @param totalMapSlots The total number of map slots in the cluster
    * @param totalMapSlots The total number of map slots in the cluster
@@ -71,6 +79,8 @@ public abstract class LoadManager implements Configurable {
 
 
   /**
   /**
    * Can a given {@link TaskTracker} run another reduce task?
    * Can a given {@link TaskTracker} run another reduce task?
+   * This method may check whether the specified tracker has
+   * enough resources to run another reduce task.
    * @param tracker The machine we wish to run a new map on
    * @param tracker The machine we wish to run a new map on
    * @param totalRunnableReduces Set of running jobs in the cluster
    * @param totalRunnableReduces Set of running jobs in the cluster
    * @param totalReduceSlots The total number of reduce slots in the cluster
    * @param totalReduceSlots The total number of reduce slots in the cluster
@@ -78,4 +88,16 @@ public abstract class LoadManager implements Configurable {
    */
    */
   public abstract boolean canAssignReduce(TaskTrackerStatus tracker,
   public abstract boolean canAssignReduce(TaskTrackerStatus tracker,
       int totalRunnableReduces, int totalReduceSlots);
       int totalRunnableReduces, int totalReduceSlots);
+
+  /**
+   * Can a given {@link TaskTracker} run another new task from a given job? 
+   * This method is provided for use by LoadManagers that take into 
+   * account jobs' individual resource needs when placing tasks.
+   * @param tracker The machine we wish to run a new map on
+   * @param job The job from which we want to run a task on this machine
+   * @param type The type of task that we want to run on
+   * @return true if this task can be launched on <code>tracker</code>
+   */
+  public abstract boolean canLaunchTask(TaskTrackerStatus tracker,
+      JobInProgress job,  TaskType type);
 }
 }

+ 65 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/LocalityLevel.java

@@ -0,0 +1,65 @@
+/**
+ * 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.mapred;
+
+/**
+ * Represents the level of data-locality at which a job in the fair scheduler
+ * is allowed to launch tasks. By default, jobs are not allowed to launch
+ * non-data-local tasks until they have waited a small number of seconds to
+ * find a slot on a node that they have data on. If a job has waited this
+ * long, it is allowed to launch rack-local tasks as well (on nodes that may
+ * not have the task's input data, but share a rack with a node that does).
+ * Finally, after a further wait, jobs are allowed to launch tasks anywhere
+ * in the cluster.
+ * 
+ * This enum defines three levels - NODE, RACK and ANY (for allowing tasks
+ * to be launched on any node). A map task's level can be obtained from
+ * its job through {@link #fromTask(JobInProgress, Task, TaskTrackerStatus)}. In
+ * addition, for any locality level, it is possible to get a "level cap" to pass
+ * to {@link JobInProgress#obtainNewMapTask(TaskTrackerStatus, int, int, int)}
+ * to ensure that only tasks at this level or lower are launched, through
+ * the {@link #toCacheLevelCap()} method.
+ */
+public enum LocalityLevel {
+  NODE, RACK, ANY;
+  
+  public static LocalityLevel fromTask(JobInProgress job, Task mapTask,
+      TaskTrackerStatus tracker) {
+    TaskID tipID = mapTask.getTaskID().getTaskID();
+    TaskInProgress tip = job.getTaskInProgress(tipID);
+    switch (job.getLocalityLevel(tip, tracker)) {
+    case 0: return LocalityLevel.NODE;
+    case 1: return LocalityLevel.RACK;
+    default: return LocalityLevel.ANY;
+    }
+  }
+  
+  /**
+   * Obtain a JobInProgress cache level cap to pass to
+   * {@link JobInProgress#obtainNewMapTask(TaskTrackerStatus, int, int, int)}
+   * to ensure that only tasks of this locality level and lower are launched.
+   */
+  public int toCacheLevelCap() {
+    switch(this) {
+    case NODE: return 1;
+    case RACK: return 2;
+    default: return Integer.MAX_VALUE;
+    }
+  }
+}

+ 1 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/NewJobWeightBooster.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.mapred;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.mapreduce.TaskType;
 
 
 /**
 /**
  * A {@link WeightAdjuster} implementation that gives a weight boost to new jobs
  * A {@link WeightAdjuster} implementation that gives a weight boost to new jobs

+ 41 - 1
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/Pool.java

@@ -21,6 +21,9 @@ package org.apache.hadoop.mapred;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 
 
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.metrics.MetricsContext;
+
 /**
 /**
  * A schedulable pool of jobs.
  * A schedulable pool of jobs.
  */
  */
@@ -33,9 +36,17 @@ public class Pool {
   
   
   /** Jobs in this specific pool; does not include children pools' jobs. */
   /** Jobs in this specific pool; does not include children pools' jobs. */
   private Collection<JobInProgress> jobs = new ArrayList<JobInProgress>();
   private Collection<JobInProgress> jobs = new ArrayList<JobInProgress>();
+  
+  /** Scheduling mode for jobs inside the pool (fair or FIFO) */
+  private SchedulingMode schedulingMode;
 
 
-  public Pool(String name) {
+  private PoolSchedulable mapSchedulable;
+  private PoolSchedulable reduceSchedulable;
+
+  public Pool(FairScheduler scheduler, String name) {
     this.name = name;
     this.name = name;
+    mapSchedulable = new PoolSchedulable(scheduler, this, TaskType.MAP);
+    reduceSchedulable = new PoolSchedulable(scheduler, this, TaskType.REDUCE);
   }
   }
   
   
   public Collection<JobInProgress> getJobs() {
   public Collection<JobInProgress> getJobs() {
@@ -44,17 +55,46 @@ public class Pool {
   
   
   public void addJob(JobInProgress job) {
   public void addJob(JobInProgress job) {
     jobs.add(job);
     jobs.add(job);
+    mapSchedulable.addJob(job);
+    reduceSchedulable.addJob(job);
   }
   }
   
   
   public void removeJob(JobInProgress job) {
   public void removeJob(JobInProgress job) {
     jobs.remove(job);
     jobs.remove(job);
+    mapSchedulable.removeJob(job);
+    reduceSchedulable.removeJob(job);
   }
   }
   
   
   public String getName() {
   public String getName() {
     return name;
     return name;
   }
   }
 
 
+  public SchedulingMode getSchedulingMode() {
+    return schedulingMode;
+  }
+  
+  public void setSchedulingMode(SchedulingMode schedulingMode) {
+    this.schedulingMode = schedulingMode;
+  }
+
   public boolean isDefaultPool() {
   public boolean isDefaultPool() {
     return Pool.DEFAULT_POOL_NAME.equals(name);
     return Pool.DEFAULT_POOL_NAME.equals(name);
   }
   }
+  
+  public PoolSchedulable getMapSchedulable() {
+    return mapSchedulable;
+  }
+  
+  public PoolSchedulable getReduceSchedulable() {
+    return reduceSchedulable;
+  }
+  
+  public PoolSchedulable getSchedulable(TaskType type) {
+    return type == TaskType.MAP ? mapSchedulable : reduceSchedulable;
+  }
+
+  public void updateMetrics() {
+    mapSchedulable.updateMetrics();
+    reduceSchedulable.updateMetrics();
+  }
 }
 }

+ 228 - 23
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.mapred;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.net.URL;
+import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
@@ -34,6 +36,8 @@ import javax.xml.parsers.ParserConfigurationException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.metrics.MetricsContext;
 import org.w3c.dom.Document;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.Node;
@@ -42,7 +46,8 @@ import org.w3c.dom.Text;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXException;
 
 
 /**
 /**
- * Maintains a hierarchy of pools.
+ * Maintains a list of pools as well as scheduling parameters for each pool,
+ * such as guaranteed share allocations, from the fair scheduler config file.
  */
  */
 public class PoolManager {
 public class PoolManager {
   public static final Log LOG = LogFactory.getLog(
   public static final Log LOG = LogFactory.getLog(
@@ -56,11 +61,19 @@ public class PoolManager {
    * (this is done to prevent loading a file that hasn't been fully written).
    * (this is done to prevent loading a file that hasn't been fully written).
    */
    */
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000; 
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000; 
+
+  public static final String EXPLICIT_POOL_PROPERTY = "mapred.fairscheduler.pool";
+
+  private final FairScheduler scheduler;
   
   
   // Map and reduce minimum allocations for each pool
   // Map and reduce minimum allocations for each pool
   private Map<String, Integer> mapAllocs = new HashMap<String, Integer>();
   private Map<String, Integer> mapAllocs = new HashMap<String, Integer>();
   private Map<String, Integer> reduceAllocs = new HashMap<String, Integer>();
   private Map<String, Integer> reduceAllocs = new HashMap<String, Integer>();
 
 
+  // If set, cap number of map and reduce tasks in a pool
+  private Map<String, Integer> poolMaxMaps = new HashMap<String, Integer>();
+  private Map<String, Integer> poolMaxReduces = new HashMap<String, Integer>();
+
   // Sharing weights for each pool
   // Sharing weights for each pool
   private Map<String, Double> poolWeights = new HashMap<String, Double>();
   private Map<String, Double> poolWeights = new HashMap<String, Double>();
   
   
@@ -69,10 +82,31 @@ public class PoolManager {
   private Map<String, Integer> poolMaxJobs = new HashMap<String, Integer>();
   private Map<String, Integer> poolMaxJobs = new HashMap<String, Integer>();
   private Map<String, Integer> userMaxJobs = new HashMap<String, Integer>();
   private Map<String, Integer> userMaxJobs = new HashMap<String, Integer>();
   private int userMaxJobsDefault = Integer.MAX_VALUE;
   private int userMaxJobsDefault = Integer.MAX_VALUE;
+  private int poolMaxJobsDefault = Integer.MAX_VALUE;
 
 
-  private String allocFile; // Path to XML file containing allocations
+  // Min share preemption timeout for each pool in seconds. If a job in the pool
+  // waits this long without receiving its guaranteed share, it is allowed to
+  // preempt other jobs' tasks.
+  private Map<String, Long> minSharePreemptionTimeouts =
+    new HashMap<String, Long>();
+  
+  // Default min share preemption timeout for pools where it is not set
+  // explicitly.
+  private long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
+  
+  // Preemption timeout for jobs below fair share in seconds. If a job remains
+  // below half its fair share for this long, it is allowed to preempt tasks.
+  private long fairSharePreemptionTimeout = Long.MAX_VALUE;
+  
+  SchedulingMode defaultSchedulingMode = SchedulingMode.FAIR;
+  
+  private Object allocFile; // Path to XML file containing allocations. This
+                            // is either a URL to specify a classpath resource
+                            // (if the fair-scheduler.xml on the classpath is
+                            // used) or a String to specify an absolute path (if
+                            // mapred.fairscheduler.allocation.file is used).
   private String poolNameProperty; // Jobconf property to use for determining a
   private String poolNameProperty; // Jobconf property to use for determining a
-                                   // job's pool name (default: mapred.job.queue.name)
+                                   // job's pool name (default: user.name)
   
   
   private Map<String, Pool> pools = new HashMap<String, Pool>();
   private Map<String, Pool> pools = new HashMap<String, Pool>();
   
   
@@ -80,14 +114,25 @@ public class PoolManager {
   private long lastSuccessfulReload; // Last time we successfully reloaded pools
   private long lastSuccessfulReload; // Last time we successfully reloaded pools
   private boolean lastReloadAttemptFailed = false;
   private boolean lastReloadAttemptFailed = false;
 
 
-  public PoolManager(Configuration conf) throws IOException, SAXException,
+  public PoolManager(FairScheduler scheduler) {
+    this.scheduler = scheduler;
+  }
+  
+  public void initialize() throws IOException, SAXException,
       AllocationConfigurationException, ParserConfigurationException {
       AllocationConfigurationException, ParserConfigurationException {
+    Configuration conf = scheduler.getConf();
     this.poolNameProperty = conf.get(
     this.poolNameProperty = conf.get(
         "mapred.fairscheduler.poolnameproperty", "user.name");
         "mapred.fairscheduler.poolnameproperty", "user.name");
     this.allocFile = conf.get("mapred.fairscheduler.allocation.file");
     this.allocFile = conf.get("mapred.fairscheduler.allocation.file");
     if (allocFile == null) {
     if (allocFile == null) {
-      LOG.warn("No mapred.fairscheduler.allocation.file given in jobconf - " +
-          "the fair scheduler will not use any queues.");
+      // No allocation file specified in jobconf. Use the default allocation
+      // file, fair-scheduler.xml, looking for it on the classpath.
+      allocFile = new Configuration().getResource("fair-scheduler.xml");
+      if (allocFile == null) {
+        LOG.error("The fair scheduler allocation file fair-scheduler.xml was "
+            + "not found on the classpath, and no other config file is given "
+            + "through mapred.fairscheduler.allocation.file.");
+      }
     }
     }
     reloadAllocs();
     reloadAllocs();
     lastSuccessfulReload = System.currentTimeMillis();
     lastSuccessfulReload = System.currentTimeMillis();
@@ -102,11 +147,19 @@ public class PoolManager {
   public synchronized Pool getPool(String name) {
   public synchronized Pool getPool(String name) {
     Pool pool = pools.get(name);
     Pool pool = pools.get(name);
     if (pool == null) {
     if (pool == null) {
-      pool = new Pool(name);
+      pool = new Pool(scheduler, name);
+      pool.setSchedulingMode(defaultSchedulingMode);
       pools.put(name, pool);
       pools.put(name, pool);
     }
     }
     return pool;
     return pool;
   }
   }
+  
+  /**
+   * Get the pool that a given job is in.
+   */
+  public Pool getPool(JobInProgress job) {
+    return getPool(getPoolName(job));
+  }
 
 
   /**
   /**
    * Reload allocations file if it hasn't been loaded in a while
    * Reload allocations file if it hasn't been loaded in a while
@@ -115,9 +168,20 @@ public class PoolManager {
     long time = System.currentTimeMillis();
     long time = System.currentTimeMillis();
     if (time > lastReloadAttempt + ALLOC_RELOAD_INTERVAL) {
     if (time > lastReloadAttempt + ALLOC_RELOAD_INTERVAL) {
       lastReloadAttempt = time;
       lastReloadAttempt = time;
+      if (null == allocFile) {
+        return;
+      }
       try {
       try {
-        File file = new File(allocFile);
-        long lastModified = file.lastModified();
+        // Get last modified time of alloc file depending whether it's a String
+        // (for a path name) or an URL (for a classloader resource)
+        long lastModified;
+        if (allocFile instanceof String) {
+          File file = new File((String) allocFile);
+          lastModified = file.lastModified();
+        } else { // allocFile is an URL
+          URLConnection conn = ((URL) allocFile).openConnection();
+          lastModified = conn.getLastModified();
+        }
         if (lastModified > lastSuccessfulReload &&
         if (lastModified > lastSuccessfulReload &&
             time > lastModified + ALLOC_RELOAD_WAIT) {
             time > lastModified + ALLOC_RELOAD_WAIT) {
           reloadAllocs();
           reloadAllocs();
@@ -131,7 +195,7 @@ public class PoolManager {
         // We log the error only on the first failure so we don't fill up the
         // We log the error only on the first failure so we don't fill up the
         // JobTracker's log with these messages.
         // JobTracker's log with these messages.
         if (!lastReloadAttemptFailed) {
         if (!lastReloadAttemptFailed) {
-          LOG.error("Failed to reload allocations file - " +
+          LOG.error("Failed to reload fair scheduler config file - " +
               "will use existing allocations.", e);
               "will use existing allocations.", e);
         }
         }
         lastReloadAttemptFailed = true;
         lastReloadAttemptFailed = true;
@@ -165,8 +229,16 @@ public class PoolManager {
     Map<String, Integer> reduceAllocs = new HashMap<String, Integer>();
     Map<String, Integer> reduceAllocs = new HashMap<String, Integer>();
     Map<String, Integer> poolMaxJobs = new HashMap<String, Integer>();
     Map<String, Integer> poolMaxJobs = new HashMap<String, Integer>();
     Map<String, Integer> userMaxJobs = new HashMap<String, Integer>();
     Map<String, Integer> userMaxJobs = new HashMap<String, Integer>();
+    Map<String, Integer> poolMaxMaps = new HashMap<String, Integer>();
+    Map<String, Integer> poolMaxReduces = new HashMap<String, Integer>();
     Map<String, Double> poolWeights = new HashMap<String, Double>();
     Map<String, Double> poolWeights = new HashMap<String, Double>();
+    Map<String, SchedulingMode> poolModes = new HashMap<String, SchedulingMode>();
+    Map<String, Long> minSharePreemptionTimeouts = new HashMap<String, Long>();
     int userMaxJobsDefault = Integer.MAX_VALUE;
     int userMaxJobsDefault = Integer.MAX_VALUE;
+    int poolMaxJobsDefault = Integer.MAX_VALUE;
+    long fairSharePreemptionTimeout = Long.MAX_VALUE;
+    long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
+    SchedulingMode defaultSchedulingMode = SchedulingMode.FAIR;
     
     
     // Remember all pool names so we can display them on web UI, etc.
     // Remember all pool names so we can display them on web UI, etc.
     List<String> poolNamesInAllocFile = new ArrayList<String>();
     List<String> poolNamesInAllocFile = new ArrayList<String>();
@@ -176,11 +248,16 @@ public class PoolManager {
       DocumentBuilderFactory.newInstance();
       DocumentBuilderFactory.newInstance();
     docBuilderFactory.setIgnoringComments(true);
     docBuilderFactory.setIgnoringComments(true);
     DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
     DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
-    Document doc = builder.parse(new File(allocFile));
+    Document doc;
+    if (allocFile instanceof String) {
+      doc = builder.parse(new File((String) allocFile));
+    } else {
+      doc = builder.parse(allocFile.toString());
+    }
     Element root = doc.getDocumentElement();
     Element root = doc.getDocumentElement();
     if (!"allocations".equals(root.getTagName()))
     if (!"allocations".equals(root.getTagName()))
-      throw new AllocationConfigurationException("Bad allocations file: " + 
-          "top-level element not <allocations>");
+      throw new AllocationConfigurationException("Bad fair scheduler config " + 
+          "file: top-level element not <allocations>");
     NodeList elements = root.getChildNodes();
     NodeList elements = root.getChildNodes();
     for (int i = 0; i < elements.getLength(); i++) {
     for (int i = 0; i < elements.getLength(); i++) {
       Node node = elements.item(i);
       Node node = elements.item(i);
@@ -204,6 +281,14 @@ public class PoolManager {
             String text = ((Text)field.getFirstChild()).getData().trim();
             String text = ((Text)field.getFirstChild()).getData().trim();
             int val = Integer.parseInt(text);
             int val = Integer.parseInt(text);
             reduceAllocs.put(poolName, val);
             reduceAllocs.put(poolName, val);
+          } else if ("maxMaps".equals(field.getTagName())) {
+            String text = ((Text)field.getFirstChild()).getData().trim();
+            int val = Integer.parseInt(text);
+            poolMaxMaps.put(poolName, val);
+          } else if ("maxReduces".equals(field.getTagName())) {
+            String text = ((Text)field.getFirstChild()).getData().trim();
+            int val = Integer.parseInt(text);
+            poolMaxReduces.put(poolName, val);
           } else if ("maxRunningJobs".equals(field.getTagName())) {
           } else if ("maxRunningJobs".equals(field.getTagName())) {
             String text = ((Text)field.getFirstChild()).getData().trim();
             String text = ((Text)field.getFirstChild()).getData().trim();
             int val = Integer.parseInt(text);
             int val = Integer.parseInt(text);
@@ -212,8 +297,25 @@ public class PoolManager {
             String text = ((Text)field.getFirstChild()).getData().trim();
             String text = ((Text)field.getFirstChild()).getData().trim();
             double val = Double.parseDouble(text);
             double val = Double.parseDouble(text);
             poolWeights.put(poolName, val);
             poolWeights.put(poolName, val);
+          } else if ("minSharePreemptionTimeout".equals(field.getTagName())) {
+            String text = ((Text)field.getFirstChild()).getData().trim();
+            long val = Long.parseLong(text) * 1000L;
+            minSharePreemptionTimeouts.put(poolName, val);
+          } else if ("schedulingMode".equals(field.getTagName())) {
+            String text = ((Text)field.getFirstChild()).getData().trim();
+            poolModes.put(poolName, parseSchedulingMode(text));
           }
           }
         }
         }
+        if (poolMaxMaps.containsKey(poolName) && mapAllocs.containsKey(poolName)
+            && poolMaxMaps.get(poolName) < mapAllocs.get(poolName)) {
+          LOG.warn(String.format("Pool %s has max maps %d less than min maps %d",
+              poolName, poolMaxMaps.get(poolName), mapAllocs.get(poolName)));        
+        }
+        if(poolMaxReduces.containsKey(poolName) && reduceAllocs.containsKey(poolName)
+            && poolMaxReduces.get(poolName) < reduceAllocs.get(poolName)) {
+          LOG.warn(String.format("Pool %s has max reduces %d less than min reduces %d",
+              poolName, poolMaxReduces.get(poolName), reduceAllocs.get(poolName)));        
+        }
       } else if ("user".equals(element.getTagName())) {
       } else if ("user".equals(element.getTagName())) {
         String userName = element.getAttribute("name");
         String userName = element.getAttribute("name");
         NodeList fields = element.getChildNodes();
         NodeList fields = element.getChildNodes();
@@ -232,6 +334,21 @@ public class PoolManager {
         String text = ((Text)element.getFirstChild()).getData().trim();
         String text = ((Text)element.getFirstChild()).getData().trim();
         int val = Integer.parseInt(text);
         int val = Integer.parseInt(text);
         userMaxJobsDefault = val;
         userMaxJobsDefault = val;
+      } else if ("poolMaxJobsDefault".equals(element.getTagName())) {
+        String text = ((Text)element.getFirstChild()).getData().trim();
+        int val = Integer.parseInt(text);
+        poolMaxJobsDefault = val;
+      } else if ("fairSharePreemptionTimeout".equals(element.getTagName())) {
+        String text = ((Text)element.getFirstChild()).getData().trim();
+        long val = Long.parseLong(text) * 1000L;
+        fairSharePreemptionTimeout = val;
+      } else if ("defaultMinSharePreemptionTimeout".equals(element.getTagName())) {
+        String text = ((Text)element.getFirstChild()).getData().trim();
+        long val = Long.parseLong(text) * 1000L;
+        defaultMinSharePreemptionTimeout = val;
+      } else if ("defaultPoolSchedulingMode".equals(element.getTagName())) {
+        String text = ((Text)element.getFirstChild()).getData().trim();
+        defaultSchedulingMode = parseSchedulingMode(text);
       } else {
       } else {
         LOG.warn("Bad element in allocations file: " + element.getTagName());
         LOG.warn("Bad element in allocations file: " + element.getTagName());
       }
       }
@@ -242,16 +359,60 @@ public class PoolManager {
     synchronized(this) {
     synchronized(this) {
       this.mapAllocs = mapAllocs;
       this.mapAllocs = mapAllocs;
       this.reduceAllocs = reduceAllocs;
       this.reduceAllocs = reduceAllocs;
+      this.poolMaxMaps = poolMaxMaps;
+      this.poolMaxReduces = poolMaxReduces;
       this.poolMaxJobs = poolMaxJobs;
       this.poolMaxJobs = poolMaxJobs;
       this.userMaxJobs = userMaxJobs;
       this.userMaxJobs = userMaxJobs;
-      this.userMaxJobsDefault = userMaxJobsDefault;
       this.poolWeights = poolWeights;
       this.poolWeights = poolWeights;
+      this.minSharePreemptionTimeouts = minSharePreemptionTimeouts;
+      this.userMaxJobsDefault = userMaxJobsDefault;
+      this.poolMaxJobsDefault = poolMaxJobsDefault;
+      this.fairSharePreemptionTimeout = fairSharePreemptionTimeout;
+      this.defaultMinSharePreemptionTimeout = defaultMinSharePreemptionTimeout;
+      this.defaultSchedulingMode = defaultSchedulingMode;
       for (String name: poolNamesInAllocFile) {
       for (String name: poolNamesInAllocFile) {
-        getPool(name);
+        Pool pool = getPool(name);
+        if (poolModes.containsKey(name)) {
+          pool.setSchedulingMode(poolModes.get(name));
+        } else {
+          pool.setSchedulingMode(defaultSchedulingMode);
+        }
       }
       }
     }
     }
   }
   }
 
 
+  /**
+   * Does the pool have incompatible max and min allocation set.
+   * 
+   * @param type
+   *          {@link TaskType#MAP} or {@link TaskType#REDUCE}
+   * @param pool
+   *          the pool name
+   * @return true if the max is less than the min
+   */
+  boolean invertedMinMax(TaskType type, String pool) {
+    Map<String, Integer> max = TaskType.MAP == type ? poolMaxMaps : poolMaxReduces;
+    Map<String, Integer> min = TaskType.MAP == type ? mapAllocs : reduceAllocs;
+    if (max.containsKey(pool) && min.containsKey(pool)
+        && max.get(pool) < min.get(pool)) {
+      return true;
+    }
+    return false;
+  }
+
+  private SchedulingMode parseSchedulingMode(String text)
+      throws AllocationConfigurationException {
+    text = text.toLowerCase();
+    if (text.equals("fair")) {
+      return SchedulingMode.FAIR;
+    } else if (text.equals("fifo")) {
+      return SchedulingMode.FIFO;
+    } else {
+      throw new AllocationConfigurationException(
+          "Unknown scheduling mode : " + text + "; expected 'fifo' or 'fair'");
+    }
+  }
+
   /**
   /**
    * Get the allocation for a particular pool
    * Get the allocation for a particular pool
    */
    */
@@ -261,7 +422,20 @@ public class PoolManager {
     Integer alloc = allocationMap.get(pool);
     Integer alloc = allocationMap.get(pool);
     return (alloc == null ? 0 : alloc);
     return (alloc == null ? 0 : alloc);
   }
   }
-  
+
+  /**
+   * Get the maximum map or reduce slots for the given pool.
+   * @return the cap set on this pool, or Integer.MAX_VALUE if not set.
+   */
+  int getMaxSlots(String poolName, TaskType taskType) {
+    Map<String, Integer> maxMap = (taskType == TaskType.MAP ? poolMaxMaps : poolMaxReduces);
+    if (maxMap.containsKey(poolName)) {
+      return maxMap.get(poolName);
+    } else {
+      return Integer.MAX_VALUE;
+    }
+  }
+ 
   /**
   /**
    * Add a job in the appropriate pool
    * Add a job in the appropriate pool
    */
    */
@@ -281,7 +455,7 @@ public class PoolManager {
    */
    */
   public synchronized void setPool(JobInProgress job, String pool) {
   public synchronized void setPool(JobInProgress job, String pool) {
     removeJob(job);
     removeJob(job);
-    job.getJobConf().set(poolNameProperty, pool);
+    job.getJobConf().set(EXPLICIT_POOL_PROPERTY, pool);
     addJob(job);
     addJob(job);
   }
   }
 
 
@@ -293,13 +467,16 @@ public class PoolManager {
   }
   }
   
   
   /**
   /**
-   * Get the pool name for a JobInProgress from its configuration. This uses
-   * the "project" property in the jobconf by default, or the property set with
-   * "mapred.fairscheduler.poolnameproperty".
+   * Get the pool name for a JobInProgress from its configuration.  This uses
+   * the value of mapred.fairscheduler.pool if specified, otherwise the value 
+   * of the property named in mapred.fairscheduler.poolnameproperty if that is
+   * specified.  Otherwise if neither is specified it uses the "user.name" property 
+   * in the jobconf by default.
    */
    */
   public String getPoolName(JobInProgress job) {
   public String getPoolName(JobInProgress job) {
-    JobConf conf = job.getJobConf();
-    return conf.get(poolNameProperty, Pool.DEFAULT_POOL_NAME).trim();
+    Configuration conf = job.getJobConf();
+    return conf.get(EXPLICIT_POOL_PROPERTY,
+      conf.get(poolNameProperty, Pool.DEFAULT_POOL_NAME)).trim();
   }
   }
 
 
   /**
   /**
@@ -327,7 +504,7 @@ public class PoolManager {
     if (poolMaxJobs.containsKey(pool)) {
     if (poolMaxJobs.containsKey(pool)) {
       return poolMaxJobs.get(pool);
       return poolMaxJobs.get(pool);
     } else {
     } else {
-      return Integer.MAX_VALUE;
+      return poolMaxJobsDefault;
     }
     }
   }
   }
 
 
@@ -338,4 +515,32 @@ public class PoolManager {
       return 1.0;
       return 1.0;
     }
     }
   }
   }
+
+  /**
+   * Get a pool's min share preemption timeout, in milliseconds. This is the
+   * time after which jobs in the pool may kill other pools' tasks if they
+   * are below their min share.
+   */
+  public long getMinSharePreemptionTimeout(String pool) {
+    if (minSharePreemptionTimeouts.containsKey(pool)) {
+      return minSharePreemptionTimeouts.get(pool);
+    } else {
+      return defaultMinSharePreemptionTimeout;
+    }
+  }
+  
+  /**
+   * Get the fair share preemption, in milliseconds. This is the time
+   * after which any job may kill other jobs' tasks if it is below half
+   * its fair share.
+   */
+  public long getFairSharePreemptionTimeout() {
+    return fairSharePreemptionTimeout;
+  }
+
+  synchronized void updateMetrics() {
+    for (Pool pool : pools.values()) {
+      pool.updateMetrics();
+    }
+  }
 }
 }

+ 221 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolSchedulable.java

@@ -0,0 +1,221 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.FairScheduler.JobInfo;
+import org.apache.hadoop.mapreduce.TaskType;
+
+public class PoolSchedulable extends Schedulable {
+  public static final Log LOG = LogFactory.getLog(
+      PoolSchedulable.class.getName());
+  
+  private FairScheduler scheduler;
+  private Pool pool;
+  private TaskType taskType;
+  private PoolManager poolMgr;
+  private List<JobSchedulable> jobScheds = new LinkedList<JobSchedulable>();
+  private int demand = 0;
+  
+  // Variables used for preemption
+  long lastTimeAtMinShare;
+  long lastTimeAtHalfFairShare;
+
+  public PoolSchedulable(FairScheduler scheduler, Pool pool, TaskType type) {
+    this.scheduler = scheduler;
+    this.pool = pool;
+    this.taskType = type;
+    this.poolMgr = scheduler.getPoolManager();
+    long currentTime = scheduler.getClock().getTime();
+    this.lastTimeAtMinShare = currentTime;
+    this.lastTimeAtHalfFairShare = currentTime;
+    
+    initMetrics();
+  }
+
+  public void addJob(JobInProgress job) {
+    JobInfo info = scheduler.getJobInfo(job);
+    jobScheds.add(taskType == TaskType.MAP ?
+        info.mapSchedulable : info.reduceSchedulable);
+  }
+  
+  public void removeJob(JobInProgress job) {
+    for (Iterator<JobSchedulable> it = jobScheds.iterator(); it.hasNext();) {
+      JobSchedulable jobSched = it.next();
+      if (jobSched.getJob() == job) {
+        it.remove();
+        break;
+      }
+    }
+  }
+
+  /**
+   * Update demand by asking jobs in the pool to update
+   */
+  @Override
+  public void updateDemand() {
+    demand = 0;
+    for (JobSchedulable sched: jobScheds) {
+      sched.updateDemand();
+      demand += sched.getDemand();
+    }
+    // if demand exceeds the cap for this pool, limit to the max
+    int maxTasks = poolMgr.getMaxSlots(pool.getName(), taskType);
+    if(demand > maxTasks) {
+      demand = maxTasks;
+    }
+  }
+  
+  /**
+   * Distribute the pool's fair share among its jobs
+   */
+  @Override
+  public void redistributeShare() {
+    if (pool.getSchedulingMode() == SchedulingMode.FAIR) {
+      SchedulingAlgorithms.computeFairShares(jobScheds, getFairShare());
+    } else {
+      for (JobSchedulable sched: jobScheds) {
+        sched.setFairShare(0);
+      }
+    } 
+  }
+
+  @Override
+  public int getDemand() {
+    return demand;
+  }
+
+  @Override
+  public int getMinShare() {
+    return poolMgr.getAllocation(pool.getName(), taskType);
+  }
+
+  @Override
+  public double getWeight() {
+    return poolMgr.getPoolWeight(pool.getName());
+  }
+
+  @Override
+  public JobPriority getPriority() {
+    return JobPriority.NORMAL;
+  }
+
+  @Override
+  public int getRunningTasks() {
+    int ans = 0;
+    for (JobSchedulable sched: jobScheds) {
+      ans += sched.getRunningTasks();
+    }
+    return ans;
+  }
+
+  @Override
+  public long getStartTime() {
+    return 0;
+  }
+
+  @Override
+  public Task assignTask(TaskTrackerStatus tts, long currentTime,
+      Collection<JobInProgress> visited) throws IOException {
+    int runningTasks = getRunningTasks();
+    if (runningTasks >= poolMgr.getMaxSlots(pool.getName(), taskType)) {
+      return null;
+    }
+    SchedulingMode mode = pool.getSchedulingMode();
+    Comparator<Schedulable> comparator;
+    if (mode == SchedulingMode.FIFO) {
+      comparator = new SchedulingAlgorithms.FifoComparator();
+    } else if (mode == SchedulingMode.FAIR) {
+      comparator = new SchedulingAlgorithms.FairShareComparator();
+    } else {
+      throw new RuntimeException("Unsupported pool scheduling mode " + mode);
+    }
+    Collections.sort(jobScheds, comparator);
+    for (JobSchedulable sched: jobScheds) {
+      Task task = sched.assignTask(tts, currentTime, visited);
+      if (task != null)
+        return task;
+    }
+    return null;
+  }
+  
+  @Override
+  public String getName() {
+    return pool.getName();
+  }
+
+  Pool getPool() {
+    return pool;
+  }
+
+  @Override
+  public TaskType getTaskType() {
+    return taskType;
+  }
+  
+  public Collection<JobSchedulable> getJobSchedulables() {
+    return jobScheds;
+  }
+  
+  public long getLastTimeAtMinShare() {
+    return lastTimeAtMinShare;
+  }
+  
+  public void setLastTimeAtMinShare(long lastTimeAtMinShare) {
+    this.lastTimeAtMinShare = lastTimeAtMinShare;
+  }
+  
+  public long getLastTimeAtHalfFairShare() {
+    return lastTimeAtHalfFairShare;
+  }
+  
+  public void setLastTimeAtHalfFairShare(long lastTimeAtHalfFairShare) {
+    this.lastTimeAtHalfFairShare = lastTimeAtHalfFairShare;
+  }
+
+  protected String getMetricsContextName() {
+    return "pools";
+  }
+  
+  @Override
+  public void updateMetrics() {
+    super.setMetricValues(metrics);
+    
+    if (scheduler.isPreemptionEnabled()) {
+      // These won't be set if preemption is off
+      long lastCheck = scheduler.getLastPreemptionUpdateTime();
+      metrics.setMetric("millisSinceAtMinShare", lastCheck - lastTimeAtMinShare);
+      metrics.setMetric("millisSinceAtHalfFairShare", lastCheck - lastTimeAtHalfFairShare);
+    }
+    metrics.update();
+
+    for (JobSchedulable job : jobScheds) {
+      job.updateMetrics();
+    }
+  }
+}

+ 171 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/Schedulable.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.mapred;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+
+/**
+ * A Schedulable represents an entity that can launch tasks, such as a job
+ * or a pool. It provides a common interface so that algorithms such as fair
+ * sharing can be applied both within a pool and across pools. There are 
+ * currently two types of Schedulables: JobSchedulables, which represent a
+ * single job, and PoolSchedulables, which allocate among jobs in their pool.
+ * 
+ * Separate sets of Schedulables are used for maps and reduces. Each pool has
+ * both a mapSchedulable and a reduceSchedulable, and so does each job.
+ * 
+ * A Schedulable is responsible for three roles:
+ * 1) It can launch tasks through assignTask().
+ * 2) It provides information about the job/pool to the scheduler, including:
+ *    - Demand (maximum number of tasks required)
+ *    - Number of currently running tasks
+ *    - Minimum share (for pools)
+ *    - Job/pool weight (for fair sharing)
+ *    - Start time and priority (for FIFO)
+ * 3) It can be assigned a fair share, for use with fair scheduling.
+ * 
+ * Schedulable also contains two methods for performing scheduling computations:
+ * - updateDemand() is called periodically to compute the demand of the various
+ *   jobs and pools, which may be expensive (e.g. jobs must iterate through all
+ *   their tasks to count failed tasks, tasks that can be speculated, etc).
+ * - redistributeShare() is called after demands are updated and a Schedulable's
+ *   fair share has been set by its parent to let it distribute its share among
+ *   the other Schedulables within it (e.g. for pools that want to perform fair
+ *   sharing among their jobs).
+ */
+abstract class Schedulable {
+  /** Fair share assigned to this Schedulable */
+  private double fairShare = 0;
+  protected MetricsRecord metrics;
+  
+  /**
+   * Name of job/pool, used for debugging as well as for breaking ties in
+   * scheduling order deterministically. 
+   */
+  public abstract String getName();
+  
+  /**
+   * @return the type of tasks that this pool schedules
+   */
+  public abstract TaskType getTaskType();
+  
+  /**
+   * Maximum number of tasks required by this Schedulable. This is defined as
+   * number of currently running tasks + number of unlaunched tasks (tasks that
+   * are either not yet launched or need to be speculated).
+   */
+  public abstract int getDemand();
+  
+  /** Number of tasks the schedulable is currently running. */
+  public abstract int getRunningTasks();
+  
+  /** Minimum share slots assigned to the schedulable. */
+  public abstract int getMinShare();
+  
+  /** Job/pool weight in fair sharing. */
+  public abstract double getWeight();
+  
+  /** Job priority for jobs in FIFO pools; meaningless for PoolSchedulables. */
+  public abstract JobPriority getPriority();
+  
+  /** Start time for jobs in FIFO pools; meaningless for PoolSchedulables. */
+  public abstract long getStartTime();
+  
+  /** Refresh the Schedulable's demand and those of its children if any. */
+  public abstract void updateDemand();
+  
+  /** 
+   * Distribute the fair share assigned to this Schedulable among its 
+   * children (used in pools where the internal scheduler is fair sharing). 
+   */
+  public abstract void redistributeShare();
+  
+  /**
+   * Obtain a task for a given TaskTracker, or null if the Schedulable has
+   * no tasks to launch at this moment or does not wish to launch a task on
+   * this TaskTracker (e.g. is waiting for a TaskTracker with local data). 
+   * In addition, if a job is skipped during this search because it is waiting
+   * for a TaskTracker with local data, this method is expected to add it to
+   * the <tt>visited</tt> collection passed in, so that the scheduler can
+   * properly mark it as skipped during this heartbeat. Please see
+   * {@link FairScheduler#getAllowedLocalityLevel(JobInProgress, long)}
+   * for details of delay scheduling (waiting for trackers with local data).
+   * 
+   * @param tts      TaskTracker that the task will be launched on
+   * @param currentTime Cached time (to prevent excessive calls to gettimeofday)
+   * @param visited  A Collection to which this method must add all jobs that
+   *                 were considered during the search for a job to assign.
+   * @return Task to launch, or null if Schedulable cannot currently launch one.
+   * @throws IOException Possible if obtainNew(Map|Reduce)Task throws exception.
+   */
+  public abstract Task assignTask(TaskTrackerStatus tts, long currentTime,
+      Collection<JobInProgress> visited) throws IOException;
+
+  /** Assign a fair share to this Schedulable. */
+  public void setFairShare(double fairShare) {
+    this.fairShare = fairShare;
+  }
+  
+  /** Get the fair share assigned to this Schedulable. */
+  public double getFairShare() {
+    return fairShare;
+  }
+  
+  /** Return the name of the metrics context for this schedulable */
+  protected abstract String getMetricsContextName();
+  
+  /**
+   * Set up metrics context
+   */
+  protected void initMetrics() {
+    MetricsContext metricsContext = MetricsUtil.getContext("fairscheduler");
+    this.metrics = MetricsUtil.createRecord(metricsContext,
+        getMetricsContextName());
+    metrics.setTag("name", getName());
+    metrics.setTag("taskType", getTaskType().toString());
+  }
+
+  void cleanupMetrics() {
+    metrics.remove();
+    metrics = null;
+  }
+
+  protected void setMetricValues(MetricsRecord metrics) {
+    metrics.setMetric("fairShare", (float)getFairShare());
+    metrics.setMetric("minShare", getMinShare());
+    metrics.setMetric("demand", getDemand());
+    metrics.setMetric("weight", (float)getWeight());
+    metrics.setMetric("runningTasks", getRunningTasks());
+  }
+  
+  abstract void updateMetrics();
+  
+  /** Convenient toString implementation for debugging. */
+  @Override
+  public String toString() {
+    return String.format("[%s, demand=%d, running=%d, share=%.1f, w=%.1f]",
+        getName(), getDemand(), getRunningTasks(), fairShare, getWeight());
+  }
+}

+ 209 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/SchedulingAlgorithms.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.mapred;
+
+import java.util.Collection;
+import java.util.Comparator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Utility class containing scheduling algorithms used in the fair scheduler.
+ */
+class SchedulingAlgorithms {
+  public static final Log LOG = LogFactory.getLog(
+      SchedulingAlgorithms.class.getName());
+  
+  /**
+   * Compare Schedulables in order of priority and then submission time, as in
+   * the default FIFO scheduler in Hadoop.
+   */
+  public static class FifoComparator implements Comparator<Schedulable> {
+    @Override
+    public int compare(Schedulable s1, Schedulable s2) {
+      int res = s1.getPriority().compareTo(s2.getPriority());
+      if (res == 0) {
+        res = (int) Math.signum(s1.getStartTime() - s2.getStartTime());
+      }
+      if (res == 0) {
+        // In the rare case where jobs were submitted at the exact same time,
+        // compare them by name (which will be the JobID) to get a deterministic
+        // ordering, so we don't alternately launch tasks from different jobs.
+        res = s1.getName().compareTo(s2.getName());
+      }
+      return res;
+    }
+  }
+
+  /**
+   * Compare Schedulables via weighted fair sharing. In addition, Schedulables
+   * below their min share get priority over those whose min share is met. 
+   * 
+   * Schedulables below their min share are compared by how far below it they
+   * are as a ratio. For example, if job A has 8 out of a min share of 10 tasks
+   * and job B has 50 out of a min share of 100, then job B is scheduled next, 
+   * because B is at 50% of its min share and A is at 80% of its min share.
+   * 
+   * Schedulables above their min share are compared by (runningTasks / weight).
+   * If all weights are equal, slots are given to the job with the fewest tasks;
+   * otherwise, jobs with more weight get proportionally more slots.
+   */
+  public static class FairShareComparator implements Comparator<Schedulable> {
+    @Override
+    public int compare(Schedulable s1, Schedulable s2) {
+      double minShareRatio1, minShareRatio2;
+      double tasksToWeightRatio1, tasksToWeightRatio2;
+      int minShare1 = Math.min(s1.getMinShare(), s1.getDemand());
+      int minShare2 = Math.min(s2.getMinShare(), s2.getDemand());
+      boolean s1Needy = s1.getRunningTasks() < minShare1;
+      boolean s2Needy = s2.getRunningTasks() < minShare2;
+      minShareRatio1 = s1.getRunningTasks() / Math.max(minShare1, 1.0);
+      minShareRatio2 = s2.getRunningTasks() / Math.max(minShare2, 1.0);
+      tasksToWeightRatio1 = s1.getRunningTasks() / s1.getWeight();
+      tasksToWeightRatio2 = s2.getRunningTasks() / s2.getWeight();
+      int res = 0;
+      if (s1Needy && !s2Needy)
+        res = -1;
+      else if (s2Needy && !s1Needy)
+        res = 1;
+      else if (s1Needy && s2Needy)
+        res = (int) Math.signum(minShareRatio1 - minShareRatio2);
+      else // Neither schedulable is needy
+        res = (int) Math.signum(tasksToWeightRatio1 - tasksToWeightRatio2);
+      if (res == 0) {
+        // Jobs are tied in fairness ratio. Break the tie by submit time and job 
+        // name to get a deterministic ordering, which is useful for unit tests.
+        res = (int) Math.signum(s1.getStartTime() - s2.getStartTime());
+        if (res == 0)
+          res = s1.getName().compareTo(s2.getName());
+      }
+      return res;
+    }
+  }
+
+  /** 
+   * Number of iterations for the binary search in computeFairShares. This is 
+   * equivalent to the number of bits of precision in the output. 25 iterations 
+   * gives precision better than 0.1 slots in clusters with one million slots.
+   */
+  private static final int COMPUTE_FAIR_SHARES_ITERATIONS = 25;
+  
+  /**
+   * Given a set of Schedulables and a number of slots, compute their weighted
+   * fair shares. The min shares and demands of the Schedulables are assumed to
+   * be set beforehand. We compute the fairest possible allocation of shares 
+   * to the Schedulables that respects their min shares and demands.
+   * 
+   * To understand what this method does, we must first define what weighted
+   * fair sharing means in the presence of minimum shares and demands. If there
+   * were no minimum shares and every Schedulable had an infinite demand (i.e.
+   * could launch infinitely many tasks), then weighted fair sharing would be
+   * achieved if the ratio of slotsAssigned / weight was equal for each
+   * Schedulable and all slots were assigned. Minimum shares and demands add
+   * two further twists:
+   * - Some Schedulables may not have enough tasks to fill all their share.
+   * - Some Schedulables may have a min share higher than their assigned share.
+   * 
+   * To deal with these possibilities, we define an assignment of slots as
+   * being fair if there exists a ratio R such that:
+   * - Schedulables S where S.demand < R * S.weight are assigned share S.demand
+   * - Schedulables S where S.minShare > R * S.weight are given share S.minShare
+   * - All other Schedulables S are assigned share R * S.weight
+   * - The sum of all the shares is totalSlots.
+   * 
+   * We call R the weight-to-slots ratio because it converts a Schedulable's
+   * weight to the number of slots it is assigned.
+   * 
+   * We compute a fair allocation by finding a suitable weight-to-slot ratio R.
+   * To do this, we use binary search. Given a ratio R, we compute the number
+   * of slots that would be used in total with this ratio (the sum of the shares
+   * computed using the conditions above). If this number of slots is less than
+   * totalSlots, then R is too small and more slots could be assigned. If the
+   * number of slots is more than totalSlots, then R is too large. 
+   * 
+   * We begin the binary search with a lower bound on R of 0 (which means that
+   * all Schedulables are only given their minShare) and an upper bound computed
+   * to be large enough that too many slots are given (by doubling R until we
+   * either use more than totalSlots slots or we fulfill all jobs' demands).
+   * The helper method slotsUsedWithWeightToSlotRatio computes the total number
+   * of slots used with a given value of R.
+   * 
+   * The running time of this algorithm is linear in the number of Schedulables,
+   * because slotsUsedWithWeightToSlotRatio is linear-time and the number of
+   * iterations of binary search is a constant (dependent on desired precision).
+   */
+  public static void computeFairShares(
+      Collection<? extends Schedulable> schedulables, double totalSlots) {
+    // Find an upper bound on R that we can use in our binary search. We start 
+    // at R = 1 and double it until we have either used totalSlots slots or we
+    // have met all Schedulables' demands (if total demand < totalSlots).
+    double totalDemand = 0;
+    for (Schedulable sched: schedulables) {
+      totalDemand += sched.getDemand();
+    }
+    double cap = Math.min(totalDemand, totalSlots);
+    double rMax = 1.0;
+    while (slotsUsedWithWeightToSlotRatio(rMax, schedulables) < cap) {
+      rMax *= 2.0;
+    }
+    // Perform the binary search for up to COMPUTE_FAIR_SHARES_ITERATIONS steps
+    double left = 0;
+    double right = rMax;
+    for (int i = 0; i < COMPUTE_FAIR_SHARES_ITERATIONS; i++) {
+      double mid = (left + right) / 2.0;
+      if (slotsUsedWithWeightToSlotRatio(mid, schedulables) < cap) {
+        left = mid;
+      } else {
+        right = mid;
+      }
+    }
+    // Set the fair shares based on the value of R we've converged to
+    for (Schedulable sched: schedulables) {
+      sched.setFairShare(computeShare(sched, right));
+    }
+  }
+  
+  /**
+   * Compute the number of slots that would be used given a weight-to-slot
+   * ratio w2sRatio, for use in the computeFairShares algorithm as described
+   * in #{@link SchedulingAlgorithms#computeFairShares(Collection, double)}.
+   */
+  private static double slotsUsedWithWeightToSlotRatio(double w2sRatio,
+      Collection<? extends Schedulable> schedulables) {
+    double slotsTaken = 0;
+    for (Schedulable sched: schedulables) {
+      double share = computeShare(sched, w2sRatio);
+      slotsTaken += share;
+    }
+    return slotsTaken;
+  }
+
+  /**
+   * Compute the number of slots assigned to a Schedulable given a particular
+   * weight-to-slot ratio w2sRatio, for use in computeFairShares as described
+   * in #{@link SchedulingAlgorithms#computeFairShares(Collection, double)}.
+   */
+  private static double computeShare(Schedulable sched, double w2sRatio) {
+    double share = sched.getWeight() * w2sRatio;
+    share = Math.max(share, sched.getMinShare());
+    share = Math.min(share, sched.getDemand());
+    return share;
+  }
+}

+ 26 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/SchedulingMode.java

@@ -0,0 +1,26 @@
+/**
+ * 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.mapred;
+
+/**
+ * Internal scheduling modes for pools.
+ */
+public enum SchedulingMode {
+  FAIR, FIFO
+}

+ 2 - 1
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/TaskSelector.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
 
 
 /**
 /**
  * A pluggable object for selecting tasks to run from a {@link JobInProgress} on
  * A pluggable object for selecting tasks to run from a {@link JobInProgress} on
@@ -86,7 +87,7 @@ public abstract class TaskSelector implements Configurable {
    * @throws IOException 
    * @throws IOException 
    */
    */
   public abstract Task obtainNewMapTask(TaskTrackerStatus taskTracker,
   public abstract Task obtainNewMapTask(TaskTrackerStatus taskTracker,
-      JobInProgress job) throws IOException;
+      JobInProgress job, int localityLevel) throws IOException;
 
 
   /**
   /**
    * Choose a reduce task to run from the given job on the given TaskTracker.
    * Choose a reduce task to run from the given job on the given TaskTracker.

+ 1 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/WeightAdjuster.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.mapreduce.TaskType;
 
 
 /**
 /**
  * A pluggable object for altering the weights of jobs in the fair scheduler,
  * A pluggable object for altering the weights of jobs in the fair scheduler,

+ 124 - 0
src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/FakeSchedulable.java

@@ -0,0 +1,124 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.mapreduce.TaskType;
+
+/**
+ * Dummy implementation of Schedulable for unit testing.
+ */
+public class FakeSchedulable extends Schedulable {
+  private int demand;
+  private int runningTasks;
+  private int minShare;
+  private double weight;
+  private JobPriority priority;
+  private long startTime;
+  
+  public FakeSchedulable() {
+    this(0, 0, 1, 0, 0, JobPriority.NORMAL, 0);
+  }
+  
+  public FakeSchedulable(int demand) {
+    this(demand, 0, 1, 0, 0, JobPriority.NORMAL, 0);
+  }
+  
+  public FakeSchedulable(int demand, int minShare) {
+    this(demand, minShare, 1, 0, 0, JobPriority.NORMAL, 0);
+  }
+  
+  public FakeSchedulable(int demand, int minShare, double weight) {
+    this(demand, minShare, weight, 0, 0, JobPriority.NORMAL, 0);
+  }
+  
+  public FakeSchedulable(int demand, int minShare, double weight, int fairShare,
+      int runningTasks, JobPriority priority, long startTime) {
+    this.demand = demand;
+    this.minShare = minShare;
+    this.weight = weight;
+    setFairShare(fairShare);
+    this.runningTasks = runningTasks;
+    this.priority = priority;
+    this.startTime = startTime;
+  }
+  
+  @Override
+  public Task assignTask(TaskTrackerStatus tts, long currentTime,
+      Collection<JobInProgress> visited) throws IOException {
+    return null;
+  }
+
+  @Override
+  public int getDemand() {
+    return demand;
+  }
+
+  @Override
+  public String getName() {
+    return "FakeSchedulable" + this.hashCode();
+  }
+
+  @Override
+  public JobPriority getPriority() {
+    return priority;
+  }
+
+  @Override
+  public int getRunningTasks() {
+    return runningTasks;
+  }
+
+  @Override
+  public long getStartTime() {
+    return startTime;
+  }
+  
+  @Override
+  public double getWeight() {
+    return weight;
+  }
+  
+  @Override
+  public int getMinShare() {
+    return minShare;
+  }
+
+  @Override
+  public void redistributeShare() {}
+
+  @Override
+  public void updateDemand() {}
+
+  @Override
+  public TaskType getTaskType() {
+    return TaskType.MAP;
+  }
+
+  @Override
+  protected String getMetricsContextName() {
+    return "fake";
+  }
+
+  @Override
+  void updateMetrics() {
+  }
+}

+ 150 - 0
src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestCapBasedLoadManager.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.mapred;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.TaskStatus.State;
+
+import junit.framework.TestCase;
+
+/**
+ * Exercise the canAssignMap and canAssignReduce methods in 
+ * CapBasedLoadManager.
+ */
+public class TestCapBasedLoadManager extends TestCase {
+  
+  /**
+   * Returns a running MapTaskStatus.
+   */
+  private TaskStatus getRunningMapTaskStatus() {
+    TaskStatus ts = new MapTaskStatus();
+    ts.setRunState(State.RUNNING);
+    return ts;
+  }
+
+  /**
+   * Returns a running ReduceTaskStatus.
+   */
+  private TaskStatus getRunningReduceTaskStatus() {
+    TaskStatus ts = new ReduceTaskStatus();
+    ts.setRunState(State.RUNNING);
+    return ts;
+  }
+  
+  /**
+   * Returns a TaskTrackerStatus with the specified statistics. 
+   * @param mapCap        The capacity of map tasks 
+   * @param reduceCap     The capacity of reduce tasks
+   * @param runningMap    The number of running map tasks
+   * @param runningReduce The number of running reduce tasks
+   */
+  private TaskTrackerStatus getTaskTrackerStatus(int mapCap, int reduceCap, 
+      int runningMap, int runningReduce) {
+    List<TaskStatus> ts = new ArrayList<TaskStatus>();
+    for (int i = 0; i < runningMap; i++) {
+      ts.add(getRunningMapTaskStatus());
+    }
+    for (int i = 0; i < runningReduce; i++) {
+      ts.add(getRunningReduceTaskStatus());
+    }
+    TaskTrackerStatus tracker = new TaskTrackerStatus("tracker", 
+        "tracker_host", 1234, ts, 0, mapCap, reduceCap);
+    return tracker;
+  }
+
+  /**
+   * A single test of canAssignMap.
+   */
+  private void oneTestCanAssignMap(float maxDiff, int mapCap, int runningMap,
+      int totalMapSlots, int totalRunnableMap, boolean expected) {
+    
+    CapBasedLoadManager manager = new CapBasedLoadManager();
+    Configuration conf = new Configuration();
+    conf.setFloat("mapred.fairscheduler.load.max.diff", maxDiff);
+    manager.setConf(conf);
+    
+    TaskTrackerStatus ts = getTaskTrackerStatus(mapCap, 1, runningMap, 1);
+    
+    assertEquals( "When maxDiff=" + maxDiff + ", with totalRunnableMap=" 
+        + totalRunnableMap + " and totalMapSlots=" + totalMapSlots
+        + ", a tracker with runningMap=" + runningMap + " and mapCap="
+        + mapCap + " should " + (expected ? "" : "not ")
+        + "be able to take more Maps.",
+        expected,
+        manager.canAssignMap(ts, totalRunnableMap, totalMapSlots)
+        );
+  }
+  
+  
+  /** 
+   * Test canAssignMap method.
+   */
+  public void testCanAssignMap() {
+    oneTestCanAssignMap(0.0f, 5, 0, 50, 1, true);
+    oneTestCanAssignMap(0.0f, 5, 1, 50, 10, false);
+    oneTestCanAssignMap(0.2f, 5, 1, 50, 10, true);
+    oneTestCanAssignMap(0.0f, 5, 1, 50, 11, true);
+    oneTestCanAssignMap(0.0f, 5, 2, 50, 11, false);
+    oneTestCanAssignMap(0.3f, 5, 2, 50, 6, true);
+    oneTestCanAssignMap(1.0f, 5, 5, 50, 50, false);
+  }
+  
+  
+  /**
+   * A single test of canAssignReduce.
+   */
+  private void oneTestCanAssignReduce(float maxDiff, int ReduceCap,
+      int runningReduce, int totalReduceSlots, int totalRunnableReduce,
+      boolean expected) {
+    
+    CapBasedLoadManager manager = new CapBasedLoadManager();
+    Configuration conf = new Configuration();
+    conf.setFloat("mapred.fairscheduler.load.max.diff", maxDiff);
+    manager.setConf(conf);
+    
+    TaskTrackerStatus ts = getTaskTrackerStatus(1, ReduceCap, 1,
+        runningReduce);
+    
+    assertEquals( "When maxDiff=" + maxDiff + ", with totalRunnableReduce=" 
+        + totalRunnableReduce + " and totalReduceSlots=" + totalReduceSlots
+        + ", a tracker with runningReduce=" + runningReduce
+        + " and ReduceCap=" + ReduceCap + " should "
+        + (expected ? "" : "not ") + "be able to take more Reduces.",
+        expected,
+        manager.canAssignReduce(ts, totalRunnableReduce, totalReduceSlots)
+        );
+  }
+    
+  /** 
+   * Test canAssignReduce method.
+   */
+  public void testCanAssignReduce() {
+    oneTestCanAssignReduce(0.0f, 5, 0, 50, 1, true);
+    oneTestCanAssignReduce(0.0f, 5, 1, 50, 10, false);
+    oneTestCanAssignReduce(0.2f, 5, 1, 50, 10, true);
+    oneTestCanAssignReduce(0.0f, 5, 1, 50, 11, true);
+    oneTestCanAssignReduce(0.0f, 5, 2, 50, 11, false);
+    oneTestCanAssignReduce(0.3f, 5, 2, 50, 6, true);
+    oneTestCanAssignReduce(1.0f, 5, 5, 50, 50, false);
+  }
+  
+}

+ 184 - 0
src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestComputeFairShares.java

@@ -0,0 +1,184 @@
+/**
+ * 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.mapred;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+/**
+ * Exercise the computeFairShares method in SchedulingAlgorithms.
+ */
+public class TestComputeFairShares extends TestCase {
+  private List<Schedulable> scheds;
+  
+  @Override
+  protected void setUp() throws Exception {
+    scheds = new ArrayList<Schedulable>();
+  }
+  
+  /** 
+   * Basic test - pools with different demands that are all higher than their
+   * fair share (of 10 slots) should each get their fair share.
+   */
+  public void testEqualSharing() {
+    scheds.add(new FakeSchedulable(100));
+    scheds.add(new FakeSchedulable(50));
+    scheds.add(new FakeSchedulable(30));
+    scheds.add(new FakeSchedulable(20));
+    SchedulingAlgorithms.computeFairShares(scheds, 40);
+    verifyShares(10, 10, 10, 10);
+  }
+  
+  /**
+   * In this test, pool 4 has a smaller demand than the 40 / 4 = 10 slots that
+   * it would be assigned with equal sharing. It should only get the 3 slots
+   * it demands. The other pools must then split the remaining 37 slots, but
+   * pool 3, with 11 slots demanded, is now below its share of 37/3 ~= 12.3,
+   * so it only gets 11 slots. Pools 1 and 2 split the rest and get 13 each. 
+   */
+  public void testLowDemands() {
+    scheds.add(new FakeSchedulable(100));
+    scheds.add(new FakeSchedulable(50));
+    scheds.add(new FakeSchedulable(11));
+    scheds.add(new FakeSchedulable(3));
+    SchedulingAlgorithms.computeFairShares(scheds, 40);
+    verifyShares(13, 13, 11, 3);
+  }
+  
+  /**
+   * In this test, some pools have minimum shares set. Pool 1 has a min share
+   * of 20 so it gets 20 slots. Pool 2 also has a min share of 20, but its
+   * demand is only 10 so it can only get 10 slots. The remaining pools have
+   * 10 slots to split between them. Pool 4 gets 3 slots because its demand is
+   * only 3, and pool 3 gets the remaining 7 slots. Pool 4 also had a min share
+   * of 2 slots but this should not affect the outcome.
+   */
+  public void testMinShares() {
+    scheds.add(new FakeSchedulable(100, 20));
+    scheds.add(new FakeSchedulable(10, 20));
+    scheds.add(new FakeSchedulable(10, 0));
+    scheds.add(new FakeSchedulable(3, 2));
+    SchedulingAlgorithms.computeFairShares(scheds, 40);
+    verifyShares(20, 10, 7, 3);
+  }
+  
+  /**
+   * Basic test for weighted shares with no minimum shares and no low demands.
+   * Each pool should get slots in proportion to its weight.
+   */
+  public void testWeightedSharing() {
+    scheds.add(new FakeSchedulable(100, 0, 2.0));
+    scheds.add(new FakeSchedulable(50,  0, 1.0));
+    scheds.add(new FakeSchedulable(30,  0, 1.0));
+    scheds.add(new FakeSchedulable(20,  0, 0.5));
+    SchedulingAlgorithms.computeFairShares(scheds, 45);
+    verifyShares(20, 10, 10, 5);
+  }
+
+  /**
+   * Weighted sharing test where pools 1 and 2 are now given lower demands than
+   * above. Pool 1 stops at 10 slots, leaving 35. If the remaining pools split
+   * this into a 1:1:0.5 ratio, they would get 14:14:7 slots respectively, but
+   * pool 2's demand is only 11, so it only gets 11. The remaining 2 pools split
+   * the 24 slots left into a 1:0.5 ratio, getting 16 and 8 slots respectively.
+   */
+  public void testWeightedSharingWithLowDemands() {
+    scheds.add(new FakeSchedulable(10, 0, 2.0));
+    scheds.add(new FakeSchedulable(11, 0, 1.0));
+    scheds.add(new FakeSchedulable(30, 0, 1.0));
+    scheds.add(new FakeSchedulable(20, 0, 0.5));
+    SchedulingAlgorithms.computeFairShares(scheds, 45);
+    verifyShares(10, 11, 16, 8);
+  }
+
+  /**
+   * Weighted fair sharing test with min shares. As in the min share test above,
+   * pool 1 has a min share greater than its demand so it only gets its demand.
+   * Pool 3 has a min share of 15 even though its weight is very small, so it
+   * gets 15 slots. The remaining pools share the remaining 20 slots equally,
+   * getting 10 each. Pool 3's min share of 5 slots doesn't affect this.
+   */
+  public void testWeightedSharingWithMinShares() {
+    scheds.add(new FakeSchedulable(10, 20, 2.0));
+    scheds.add(new FakeSchedulable(11, 0, 1.0));
+    scheds.add(new FakeSchedulable(30, 5, 1.0));
+    scheds.add(new FakeSchedulable(20, 15, 0.5));
+    SchedulingAlgorithms.computeFairShares(scheds, 45);
+    verifyShares(10, 10, 10, 15);
+  }
+
+  /**
+   * Test that shares are computed accurately even when there are many more
+   * frameworks than available slots.
+   */
+  public void testSmallShares() {
+    scheds.add(new FakeSchedulable(10));
+    scheds.add(new FakeSchedulable(5));
+    scheds.add(new FakeSchedulable(3));
+    scheds.add(new FakeSchedulable(2));
+    SchedulingAlgorithms.computeFairShares(scheds, 1);
+    verifyShares(0.25, 0.25, 0.25, 0.25);
+  }
+
+  /**
+   * Test that shares are computed accurately even when the number of slots is
+   * very large.
+   */  
+  public void testLargeShares() {
+    int million = 1000 * 1000;
+    scheds.add(new FakeSchedulable(100 * million));
+    scheds.add(new FakeSchedulable(50 * million));
+    scheds.add(new FakeSchedulable(30 * million));
+    scheds.add(new FakeSchedulable(20 * million));
+    SchedulingAlgorithms.computeFairShares(scheds, 40 * million);
+    verifyShares(10 * million, 10 * million, 10 * million, 10 * million);
+  }
+
+  /**
+   * Test that having a pool with 0 demand doesn't confuse the algorithm.
+   */
+  public void testZeroDemand() {
+    scheds.add(new FakeSchedulable(100));
+    scheds.add(new FakeSchedulable(50));
+    scheds.add(new FakeSchedulable(30));
+    scheds.add(new FakeSchedulable(0));
+    SchedulingAlgorithms.computeFairShares(scheds, 30);
+    verifyShares(10, 10, 10, 0);
+  }
+  
+  /**
+   * Test that being called on an empty list doesn't confuse the algorithm.
+   */
+  public void testEmptyList() {
+    SchedulingAlgorithms.computeFairShares(scheds, 40);
+    verifyShares();
+  }
+  
+  /**
+   * Check that a given list of shares have been assigned to this.scheds.
+   */
+  private void verifyShares(double... shares) {
+    assertEquals(scheds.size(), shares.length);
+    for (int i = 0; i < shares.length; i++) {
+      assertEquals(shares[i], scheds.get(i).getFairShare(), 0.01);
+    }
+  }
+}

Різницю між файлами не показано, бо вона завелика
+ 719 - 224
src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java


+ 199 - 0
src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerSystem.java

@@ -0,0 +1,199 @@
+/**
+ * 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.mapred;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.mapreduce.TestSleepJob;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.conf.Configuration;
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.net.URL;
+import java.net.HttpURLConnection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+import static org.junit.Assert.*;
+
+/**
+ * System tests for the fair scheduler. These run slower than the
+ * mock-based tests in TestFairScheduler but have a better chance
+ * of catching synchronization bugs with the real JT.
+ *
+ * This test suite will often be run inside JCarder in order to catch
+ * deadlock bugs which have plagued the scheduler in the past - hence
+ * it is a bit of a "grab-bag" of system tests, since it's important
+ * that they all run as part of the same JVM instantiation.
+ */
+public class TestFairSchedulerSystem {
+  static final int NUM_THREADS=2;
+
+  static MiniMRCluster mr;
+  static JobConf conf;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new JobConf();
+    final int taskTrackers = 1;
+
+    // Bump up the frequency of preemption updates to test against
+    // deadlocks, etc.
+    conf.set("mapred.jobtracker.taskScheduler", FairScheduler.class.getCanonicalName());
+    conf.set("mapred.fairscheduler.update.interval", "1");
+    conf.set("mapred.fairscheduler.preemption.interval", "1");
+    conf.set("mapred.fairscheduler.preemption", "true");
+    conf.set("mapred.fairscheduler.eventlog.enabled", "true");
+    conf.set("mapred.fairscheduler.poolnameproperty", "group.name");
+    conf.set("mapred.job.tracker.persist.jobstatus.active", "false");
+    mr = new MiniMRCluster(taskTrackers, "file:///", 1, null, null, conf);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (mr != null) {
+      mr.shutdown();
+    }
+  }
+
+  private void runSleepJob(JobConf conf) throws Exception {
+    String[] args = { "-m", "1", "-r", "1", "-mt", "1", "-rt", "1" };
+    ToolRunner.run(conf, new TestSleepJob(), args);
+  }
+
+  /**
+   * Submit some concurrent sleep jobs, and visit the scheduler servlet
+   * while they're running.
+   */
+  @Test
+  public void testFairSchedulerSystem() throws Exception {
+    ExecutorService exec = Executors.newFixedThreadPool(NUM_THREADS);
+    List<Future<Void>> futures = new ArrayList<Future<Void>>(NUM_THREADS);
+    for (int i = 0; i < NUM_THREADS; i++) {
+      futures.add(exec.submit(new Callable<Void>() {
+            public Void call() throws Exception {
+              JobConf jobConf = mr.createJobConf();
+              runSleepJob(jobConf);
+              return null;
+            }
+          }));
+    }
+
+    JobClient jc = new JobClient(mr.createJobConf(null));
+
+    // Wait for the tasks to finish, and visit the scheduler servlet
+    // every few seconds while waiting.
+    for (Future<Void> future : futures) {
+      while (true) {
+        try {
+          future.get(3, TimeUnit.SECONDS);
+          break;
+        } catch (TimeoutException te) {
+          // It's OK
+        }
+        checkServlet(true);
+        checkServlet(false);
+
+        JobStatus jobs[] = jc.getAllJobs();
+        if (jobs == null) {
+          System.err.println("No jobs running, not checking tasklog servlet");
+          continue;
+        }
+        for (JobStatus j : jobs) {
+          System.err.println("Checking task graph for " + j.getJobID());
+          try {
+            checkTaskGraphServlet(j.getJobID());
+          } catch (AssertionError err) {
+            // The task graph servlet will be empty if the job has retired.
+            // This is OK.
+            RunningJob rj = jc.getJob(j.getJobID());
+            if (!rj.isComplete()) {
+              throw err;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Check the fair scheduler servlet for good status code and smoke test
+   * for contents.
+   */
+  private void checkServlet(boolean advanced) throws Exception {
+    String jtURL = "http://localhost:" +
+      mr.getJobTrackerRunner().getJobTrackerInfoPort();
+    URL url = new URL(jtURL + "/scheduler" +
+                      (advanced ? "?advanced" : ""));
+    HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+    connection.setRequestMethod("GET");
+    connection.connect();
+    assertEquals(200, connection.getResponseCode());
+
+    // Just to be sure, slurp the content and make sure it looks like the scheduler
+    BufferedReader reader = new BufferedReader(
+      new InputStreamReader(connection.getInputStream()));
+    StringBuilder sb = new StringBuilder();
+
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      sb.append(line).append('\n');
+    }
+
+    String contents = sb.toString();
+    assertTrue("Bad contents for fair scheduler servlet: " + contents,
+      contents.contains("Fair Scheduler Administration"));
+  }
+
+  private void checkTaskGraphServlet(JobID job) throws Exception {
+    String jtURL = "http://localhost:" +
+      mr.getJobTrackerRunner().getJobTrackerInfoPort();
+    URL url = new URL(jtURL + "/taskgraph?jobid=" + job.toString() + "&type=map");
+    HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+    connection.setRequestMethod("GET");
+    connection.connect();
+    assertEquals(200, connection.getResponseCode());
+
+    // Just to be sure, slurp the content and make sure it looks like the scheduler
+    String contents = slurpContents(connection);
+    assertTrue("Bad contents for job " + job + ":\n" + contents,
+      contents.contains("</svg>"));
+  }
+
+  private String slurpContents(HttpURLConnection connection) throws Exception {
+    BufferedReader reader = new BufferedReader(
+      new InputStreamReader(connection.getInputStream()));
+    StringBuilder sb = new StringBuilder();
+
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      sb.append(line).append('\n');
+    }
+
+    return sb.toString();
+  }
+}

+ 3 - 0
src/contrib/hdfsproxy/build.xml

@@ -156,6 +156,8 @@
         <include name="slf4j-api-${slf4j-api.version}.jar"/>
         <include name="slf4j-api-${slf4j-api.version}.jar"/>
         <include name="slf4j-log4j12-${slf4j-log4j12.version}.jar"/>
         <include name="slf4j-log4j12-${slf4j-log4j12.version}.jar"/>
         <include name="xmlenc-${xmlenc.version}.jar"/>
         <include name="xmlenc-${xmlenc.version}.jar"/>
+	<include name="jetty-${jetty.version}.jar"/>
+	<include name="jetty-util-${jetty-util.version}.jar"/>
       </lib>
       </lib>
       <lib dir="${hadoop.root}/lib">
       <lib dir="${hadoop.root}/lib">
         <include name="hadoop-core-*.jar"/>
         <include name="hadoop-core-*.jar"/>
@@ -495,6 +497,7 @@
   <path id="test.classpath">
   <path id="test.classpath">
     <pathelement location="${proxy.conf.test}"/>
     <pathelement location="${proxy.conf.test}"/>
     <pathelement location="${test.build.dir}"/>
     <pathelement location="${test.build.dir}"/>
+    <pathelement location="${build.test}/classes"/> 
     <pathelement location="${hadoop.root}/build/test/classes"/>
     <pathelement location="${hadoop.root}/build/test/classes"/>
     <!--<pathelement location="${hadoop.root}/src/contrib/test"/>-->
     <!--<pathelement location="${hadoop.root}/src/contrib/test"/>-->
     <pathelement location="${hadoop.root}/conf"/>
     <pathelement location="${hadoop.root}/conf"/>

+ 9 - 5
src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/TestHdfsProxy.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 
 
 /**
 /**
@@ -199,13 +200,16 @@ public class TestHdfsProxy extends TestCase {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
     HdfsProxy proxy = null;
     HdfsProxy proxy = null;
     try {
     try {
+      final UserGroupInformation CLIENT_UGI = UserGroupInformation.getCurrentUser();
+      final String testUser = CLIENT_UGI.getShortUserName();
+      final String testGroup = CLIENT_UGI.getGroupNames()[0];
 
 
       final Configuration dfsConf = new Configuration();
       final Configuration dfsConf = new Configuration();
-      dfsConf.set("hadoop.proxyuser." + System.getProperty("user.name") +
-          ".groups", "users");
-      dfsConf.set("hadoop.proxyuser.users.hosts", "127.0.0.1,localhost");
-      dfsConf.set("hadoop.proxyuser." + System.getProperty("user.name") +
-          ".hosts", "127.0.0.1,localhost");
+      dfsConf.set("hadoop.proxyuser." + testUser + ".groups", testGroup);
+      dfsConf.set("hadoop.proxyuser." + testGroup + ".hosts",
+          "127.0.0.1,localhost");
+      dfsConf.set("hadoop.proxyuser." + testUser + ".hosts",
+          "127.0.0.1,localhost");
       dfsConf.set("hadoop.security.authentication", "simple");
       dfsConf.set("hadoop.security.authentication", "simple");
       
       
       //make sure server will look at the right config
       //make sure server will look at the right config

+ 5 - 0
src/core/core-default.xml

@@ -174,6 +174,11 @@
   <value>org.apache.hadoop.hdfs.HsftpFileSystem</value>
   <value>org.apache.hadoop.hdfs.HsftpFileSystem</value>
 </property>
 </property>
 
 
+<property>
+  <name>fs.webhdfs.impl</name>
+  <value>org.apache.hadoop.hdfs.web.WebHdfsFileSystem</value>
+</property>
+
 <property>
 <property>
   <name>fs.ftp.impl</name>
   <name>fs.ftp.impl</name>
   <value>org.apache.hadoop.fs.ftp.FTPFileSystem</value>
   <value>org.apache.hadoop.fs.ftp.FTPFileSystem</value>

+ 13 - 0
src/core/org/apache/hadoop/fs/FileSystem.java

@@ -89,6 +89,19 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
    */
   private Set<Path> deleteOnExit = new TreeSet<Path>();
   private Set<Path> deleteOnExit = new TreeSet<Path>();
 
 
+  /**
+   * This method adds a file system for testing so that we can find it later.
+   * It is only for testing.
+   * @param uri the uri to store it under
+   * @param conf the configuration to store it under
+   * @param fs the file system to store
+   * @throws IOException
+   */
+  public static void addFileSystemForTesting(URI uri, Configuration conf, 
+                                      FileSystem fs) throws IOException {
+    CACHE.map.put(new Cache.Key(uri, conf), fs);
+  }
+
   public static FileSystem get(final URI uri, final Configuration conf, 
   public static FileSystem get(final URI uri, final Configuration conf, 
       final String user)
       final String user)
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {

+ 95 - 0
src/core/org/apache/hadoop/fs/FileUtil.java

@@ -25,7 +25,10 @@ import java.util.zip.ZipFile;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
@@ -561,9 +564,25 @@ public class FileUtil {
     } catch(InterruptedException e){
     } catch(InterruptedException e){
       //do nothing as of yet
       //do nothing as of yet
     }
     }
+    if (returnVal != 0) {
+      LOG.warn("Command '" + cmd + "' failed " + returnVal + 
+               " with: " + copyStderr(p));
+    }
     return returnVal;
     return returnVal;
   }
   }
   
   
+  private static String copyStderr(Process p) throws IOException {
+    InputStream err = p.getErrorStream();
+    StringBuilder result = new StringBuilder();
+    byte[] buff = new byte[4096];
+    int len = err.read(buff);
+    while (len > 0) {
+      result.append(new String(buff, 0 , len));
+      len = err.read(buff);
+    }
+    return result.toString();
+  }
+
   /**
   /**
    * Change the permissions on a filename.
    * Change the permissions on a filename.
    * @param filename the name of the file to change
    * @param filename the name of the file to change
@@ -608,6 +627,82 @@ public class FileUtil {
     }
     }
     return shExec.getExitCode();
     return shExec.getExitCode();
   }
   }
+
+  /**
+   * Set permissions to the required value. Uses the java primitives instead
+   * of forking if group == other.
+   * @param f the file to change
+   * @param permission the new permissions
+   * @throws IOException
+   */
+  public static void setPermission(File f, FsPermission permission
+                                   ) throws IOException {
+    FsAction user = permission.getUserAction();
+    FsAction group = permission.getGroupAction();
+    FsAction other = permission.getOtherAction();
+
+    // use the native/fork if the group/other permissions are different
+    // or if the native is available    
+    if (group != other || NativeIO.isAvailable()) {
+      execSetPermission(f, permission);
+      return;
+    }
+    
+    boolean rv = true;
+    
+    // read perms
+    rv = f.setReadable(group.implies(FsAction.READ), false);
+    checkReturnValue(rv, f, permission);
+    if (group.implies(FsAction.READ) != user.implies(FsAction.READ)) {
+      f.setReadable(user.implies(FsAction.READ), true);
+      checkReturnValue(rv, f, permission);
+    }
+
+    // write perms
+    rv = f.setWritable(group.implies(FsAction.WRITE), false);
+    checkReturnValue(rv, f, permission);
+    if (group.implies(FsAction.WRITE) != user.implies(FsAction.WRITE)) {
+      f.setWritable(user.implies(FsAction.WRITE), true);
+      checkReturnValue(rv, f, permission);
+    }
+
+    // exec perms
+    rv = f.setExecutable(group.implies(FsAction.EXECUTE), false);
+    checkReturnValue(rv, f, permission);
+    if (group.implies(FsAction.EXECUTE) != user.implies(FsAction.EXECUTE)) {
+      f.setExecutable(user.implies(FsAction.EXECUTE), true);
+      checkReturnValue(rv, f, permission);
+    }
+  }
+
+  private static void checkReturnValue(boolean rv, File p, 
+                                       FsPermission permission
+                                       ) throws IOException {
+    if (!rv) {
+      throw new IOException("Failed to set permissions of path: " + p + 
+                            " to " + 
+                            String.format("%04o", permission.toShort()));
+    }
+  }
+  
+  private static void execSetPermission(File f, 
+                                        FsPermission permission
+                                       )  throws IOException {
+    if (NativeIO.isAvailable()) {
+      NativeIO.chmod(f.getCanonicalPath(), permission.toShort());
+    } else {
+      execCommand(f, Shell.SET_PERMISSION_COMMAND,
+                  String.format("%04o", permission.toShort()));
+    }
+  }
+  
+  static String execCommand(File f, String... cmd) throws IOException {
+    String[] args = new String[cmd.length + 1];
+    System.arraycopy(cmd, 0, args, 0, cmd.length);
+    args[cmd.length] = f.getCanonicalPath();
+    String output = Shell.execCommand(args);
+    return output;
+  }
   
   
   /**
   /**
    * Create a tmp file for a base file.
    * Create a tmp file for a base file.

+ 384 - 196
src/core/org/apache/hadoop/fs/HarFileSystem.java

@@ -19,10 +19,16 @@ package org.apache.hadoop.fs;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.UnsupportedEncodingException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.net.URLDecoder;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.HashMap;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -44,23 +50,21 @@ import org.apache.hadoop.util.Progressable;
  */
  */
 
 
 public class HarFileSystem extends FilterFileSystem {
 public class HarFileSystem extends FilterFileSystem {
-  public static final int VERSION = 1;
+  public static final int VERSION = 3;
+
+  private static final Map<URI, HarMetaData> harMetaCache = new HashMap<URI, HarMetaData>();
+
   // uri representation of this Har filesystem
   // uri representation of this Har filesystem
   private URI uri;
   private URI uri;
-  // the version of this har filesystem
-  private int version;
-  // underlying uri 
-  private URI underLyingURI;
   // the top level path of the archive
   // the top level path of the archive
   // in the underlying file system
   // in the underlying file system
   private Path archivePath;
   private Path archivePath;
-  // the masterIndex of the archive
-  private Path masterIndex;
-  // the index file 
-  private Path archiveIndex;
   // the har auth
   // the har auth
   private String harAuth;
   private String harAuth;
-  
+
+  // pointer into the static metadata cache
+  private HarMetaData metadata;
+
   /**
   /**
    * public construction of harfilesystem
    * public construction of harfilesystem
    *
    *
@@ -91,11 +95,12 @@ public class HarFileSystem extends FilterFileSystem {
    * to be used in case not specified.
    * to be used in case not specified.
    */
    */
   public void initialize(URI name, Configuration conf) throws IOException {
   public void initialize(URI name, Configuration conf) throws IOException {
-    //decode the name
-    underLyingURI = decodeHarURI(name, conf);
-    //  we got the right har Path- now check if this is 
-    //truly a har filesystem
-    Path harPath = archivePath(new Path(name.toString()));
+    // decode the name
+    URI underLyingURI = decodeHarURI(name, conf);
+    // we got the right har Path- now check if this is 
+    // truly a har filesystem
+    Path harPath = archivePath(
+      new Path(name.getScheme(), name.getAuthority(), name.getPath()));
     if (harPath == null) { 
     if (harPath == null) { 
       throw new IOException("Invalid path for the Har Filesystem. " + 
       throw new IOException("Invalid path for the Har Filesystem. " + 
                            name.toString());
                            name.toString());
@@ -103,49 +108,49 @@ public class HarFileSystem extends FilterFileSystem {
     if (fs == null) {
     if (fs == null) {
       fs = FileSystem.get(underLyingURI, conf);
       fs = FileSystem.get(underLyingURI, conf);
     }
     }
-    this.uri = harPath.toUri();
-    this.archivePath = new Path(this.uri.getPath());
-    this.harAuth = getHarAuth(this.underLyingURI);
+    uri = harPath.toUri();
+    archivePath = new Path(uri.getPath());
+    harAuth = getHarAuth(underLyingURI);
     //check for the underlying fs containing
     //check for the underlying fs containing
     // the index file
     // the index file
-    this.masterIndex = new Path(archivePath, "_masterindex");
-    this.archiveIndex = new Path(archivePath, "_index");
-    if (!fs.exists(masterIndex) || !fs.exists(archiveIndex)) {
+    Path masterIndexPath = new Path(archivePath, "_masterindex");
+    Path archiveIndexPath = new Path(archivePath, "_index");
+    if (!fs.exists(masterIndexPath) || !fs.exists(archiveIndexPath)) {
       throw new IOException("Invalid path for the Har Filesystem. " +
       throw new IOException("Invalid path for the Har Filesystem. " +
           "No index file in " + harPath);
           "No index file in " + harPath);
     }
     }
-    try{ 
-      this.version = getHarVersion();
-    } catch(IOException io) {
-      throw new IOException("Unable to " +
-          "read the version of the Har file system: " + this.archivePath);
+
+    metadata = harMetaCache.get(uri);
+    if (metadata != null) {
+      FileStatus mStat = fs.getFileStatus(masterIndexPath);
+      FileStatus aStat = fs.getFileStatus(archiveIndexPath);
+      if (mStat.getModificationTime() != metadata.getMasterIndexTimestamp() ||
+          aStat.getModificationTime() != metadata.getArchiveIndexTimestamp()) {
+        // the archive has been overwritten since we last read it
+        // remove the entry from the meta data cache
+        metadata = null;
+        harMetaCache.remove(uri);
+      }
     }
     }
-    if (this.version != HarFileSystem.VERSION) {
-      throw new IOException("Invalid version " + 
-          this.version + " expected " + HarFileSystem.VERSION);
+    if (metadata == null) {
+      metadata = new HarMetaData(fs, masterIndexPath, archiveIndexPath);
+      metadata.parseMetaData();
+      harMetaCache.put(uri, metadata);
     }
     }
   }
   }
-  
+
   // get the version of the filesystem from the masterindex file
   // get the version of the filesystem from the masterindex file
-  // the version is currently not useful since its the first version 
+  // the version is currently not useful since its the first version
   // of archives
   // of archives
-  public int getHarVersion() throws IOException { 
-    FSDataInputStream masterIn = fs.open(masterIndex);
-    LineReader lmaster = new LineReader(masterIn, getConf());
-    Text line = new Text();
-    lmaster.readLine(line);
-    try {
-      masterIn.close();
-    } catch(IOException e){
-      //disregard it.
-      // its a read.
+  public int getHarVersion() throws IOException {
+    if (metadata != null) {
+      return metadata.getVersion();
+    }
+    else {
+      throw new IOException("Invalid meta data for the Har Filesystem");
     }
     }
-    String versionLine = line.toString();
-    String[] arr = versionLine.split(" ");
-    int version = Integer.parseInt(arr[0]);
-    return version;
   }
   }
-  
+
   /*
   /*
    * find the parent path that is the 
    * find the parent path that is the 
    * archive path in the path. The last
    * archive path in the path. The last
@@ -181,12 +186,20 @@ public class HarFileSystem extends FilterFileSystem {
       return FileSystem.getDefaultUri(conf);
       return FileSystem.getDefaultUri(conf);
     }
     }
     String host = rawURI.getHost();
     String host = rawURI.getHost();
-    String[] str = host.split("-", 2);
-    if (str[0] == null) {
-      throw new IOException("URI: " + rawURI + " is an invalid Har URI.");
+    if (host == null) {
+      throw new IOException("URI: " + rawURI
+          + " is an invalid Har URI since host==null."
+          + "  Expecting har://<scheme>-<host>/<path>.");
     }
     }
-    String underLyingScheme = str[0];
-    String underLyingHost = (str.length > 1)? str[1]:null;
+    int i = host.indexOf('-');
+    if (i < 0) {
+      throw new IOException("URI: " + rawURI
+          + " is an invalid Har URI since '-' not found."
+          + "  Expecting har://<scheme>-<host>/<path>.");
+    }
+    final String underLyingScheme = host.substring(0, i);
+    i++;
+    final String underLyingHost = i == host.length()? null: host.substring(i);
     int underLyingPort = rawURI.getPort();
     int underLyingPort = rawURI.getPort();
     String auth = (underLyingHost == null && underLyingPort == -1)?
     String auth = (underLyingHost == null && underLyingPort == -1)?
                   null:(underLyingHost+":"+underLyingPort);
                   null:(underLyingHost+":"+underLyingPort);
@@ -203,7 +216,21 @@ public class HarFileSystem extends FilterFileSystem {
     }
     }
     return tmp;
     return tmp;
   }
   }
-  
+
+  private static String decodeString(String str)
+    throws UnsupportedEncodingException {
+    return URLDecoder.decode(str, "UTF-8");
+  }
+
+  private String decodeFileName(String fname) 
+    throws UnsupportedEncodingException {
+    int version = metadata.getVersion();
+    if (version == 2 || version == 3){
+      return decodeString(fname);
+    }
+    return fname;
+  }
+
   /**
   /**
    * return the top level archive.
    * return the top level archive.
    */
    */
@@ -274,16 +301,19 @@ public class HarFileSystem extends FilterFileSystem {
   // string manipulation is not good - so
   // string manipulation is not good - so
   // just use the path api to do it.
   // just use the path api to do it.
   private Path makeRelative(String initial, Path p) {
   private Path makeRelative(String initial, Path p) {
+    String scheme = this.uri.getScheme();
+    String authority = this.uri.getAuthority();
     Path root = new Path(Path.SEPARATOR);
     Path root = new Path(Path.SEPARATOR);
     if (root.compareTo(p) == 0)
     if (root.compareTo(p) == 0)
-      return new Path(initial);
+      return new Path(scheme, authority, initial);
     Path retPath = new Path(p.getName());
     Path retPath = new Path(p.getName());
     Path parent = p.getParent();
     Path parent = p.getParent();
     for (int i=0; i < p.depth()-1; i++) {
     for (int i=0; i < p.depth()-1; i++) {
       retPath = new Path(parent.getName(), retPath);
       retPath = new Path(parent.getName(), retPath);
       parent = parent.getParent();
       parent = parent.getParent();
     }
     }
-    return new Path(initial, retPath.toString());
+    return new Path(new Path(scheme, authority, initial),
+      retPath.toString());
   }
   }
   
   
   /* this makes a path qualified in the har filesystem
   /* this makes a path qualified in the har filesystem
@@ -304,52 +334,74 @@ public class HarFileSystem extends FilterFileSystem {
     //change this to Har uri 
     //change this to Har uri 
     return new Path(uri.getScheme(), harAuth, tmpURI.getPath());
     return new Path(uri.getScheme(), harAuth, tmpURI.getPath());
   }
   }
+
+  /**
+   * Fix offset and length of block locations.
+   * Note that this method modifies the original array.
+   * @param locations block locations of har part file
+   * @param start the start of the desired range in the contained file
+   * @param len the length of the desired range
+   * @param fileOffsetInHar the offset of the desired file in the har part file
+   * @return block locations with fixed offset and length
+   */  
+  static BlockLocation[] fixBlockLocations(BlockLocation[] locations,
+                                          long start,
+                                          long len,
+                                          long fileOffsetInHar) {
+    // offset 1 past last byte of desired range
+    long end = start + len;
+
+    for (BlockLocation location : locations) {
+      // offset of part block relative to beginning of desired file
+      // (may be negative if file starts in this part block)
+      long harBlockStart = location.getOffset() - fileOffsetInHar;
+      // offset 1 past last byte of har block relative to beginning of
+      // desired file
+      long harBlockEnd = harBlockStart + location.getLength();
+      
+      if (start > harBlockStart) {
+        // desired range starts after beginning of this har block
+        // fix offset to beginning of relevant range (relative to desired file)
+        location.setOffset(start);
+        // fix length to relevant portion of har block
+        location.setLength(location.getLength() - (start - harBlockStart));
+      } else {
+        // desired range includes beginning of this har block
+        location.setOffset(harBlockStart);
+      }
+      
+      if (harBlockEnd > end) {
+        // range ends before end of this har block
+        // fix length to remove irrelevant portion at the end
+        location.setLength(location.getLength() - (harBlockEnd - end));
+      }
+    }
+    
+    return locations;
+  }
   
   
   /**
   /**
-   * get block locations from the underlying fs
+   * Get block locations from the underlying fs and fix their
+   * offsets and lengths.
    * @param file the input filestatus to get block locations
    * @param file the input filestatus to get block locations
-   * @param start the start in the file
-   * @param len the length in the file
+   * @param start the start of the desired range in the contained file
+   * @param len the length of the desired range
    * @return block locations for this segment of file
    * @return block locations for this segment of file
    * @throws IOException
    * @throws IOException
    */
    */
   @Override
   @Override
   public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
   public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
-      long len) throws IOException {
-    // need to look up the file in the underlying fs
-    // look up the index 
-    
-    // make sure this is a prt of this har filesystem
-    Path p = makeQualified(file.getPath());
-    Path harPath = getPathInHar(p);
-    String line = fileStatusInIndex(harPath);
-    if (line == null)  {
-      throw new FileNotFoundException("File " + file.getPath() + " not found");
-    }
-    HarStatus harStatus = new HarStatus(line);
-    if (harStatus.isDir()) {
-      return new BlockLocation[0];
-    }
-    FileStatus fsFile = fs.getFileStatus(new Path(archivePath,
-        harStatus.getPartName()));
-    BlockLocation[] rawBlocks = fs.getFileBlockLocations(fsFile, 
-        harStatus.getStartIndex() + start, len);
-    return fakeBlockLocations(rawBlocks, harStatus.getStartIndex());
-  }
-  
-  /**
-   * fake the rawblocks since map reduce uses the block offsets to 
-   * fo some computations regarding the blocks
-   * @param rawBlocks the raw blocks returned by the filesystem
-   * @return faked blocks with changed offsets.
-   */
-  private BlockLocation[] fakeBlockLocations(BlockLocation[] rawBlocks, 
-		  long startIndex) {
-	for (BlockLocation block : rawBlocks) {
-		long rawOffset = block.getOffset();
-		block.setOffset(rawOffset - startIndex);
-	}
-	return rawBlocks;
+                                               long len) throws IOException {
+    HarStatus hstatus = getFileHarStatus(file.getPath());
+    Path partPath = new Path(archivePath, hstatus.getPartName());
+    FileStatus partStatus = metadata.getPartFileStatus(partPath);
+
+    // get all part blocks that overlap with the desired file blocks
+    BlockLocation[] locations = 
+      fs.getFileBlockLocations(partStatus,
+                               hstatus.getStartIndex() + start, len);
+
+    return fixBlockLocations(locations, start, len, hstatus.getStartIndex());
   }
   }
   
   
   /**
   /**
@@ -378,97 +430,136 @@ public class HarFileSystem extends FilterFileSystem {
     public int endHash;
     public int endHash;
   }
   }
   
   
-  // make sure that this harPath is relative to the har filesystem
-  // this only works for relative paths. This returns the line matching
-  // the file in the index. Returns a null if there is not matching 
-  // filename in the index file.
-  private String fileStatusInIndex(Path harPath) throws IOException {
-    // read the index file 
-    int hashCode = getHarHash(harPath);
-    // get the master index to find the pos 
-    // in the index file
-    FSDataInputStream in = fs.open(masterIndex);
-    FileStatus masterStat = fs.getFileStatus(masterIndex);
-    LineReader lin = new LineReader(in, getConf());
-    Text line = new Text();
-    long read = lin.readLine(line);
-   //ignore the first line. this is the header of the index files
-    String[] readStr = null;
-    List<Store> stores = new ArrayList<Store>();
-    while(read < masterStat.getLen()) {
-      int b = lin.readLine(line);
-      read += b;
-      readStr = line.toString().split(" ");
-      int startHash = Integer.parseInt(readStr[0]);
-      int endHash  = Integer.parseInt(readStr[1]);
-      if (startHash <= hashCode && hashCode <= endHash) {
-        stores.add(new Store(Long.parseLong(readStr[2]), 
-            Long.parseLong(readStr[3]), startHash,
-            endHash));
-      }
-      line.clear();
+  /**
+   * Get filestatuses of all the children of a given directory. This just reads
+   * through index file and reads line by line to get all statuses for children
+   * of a directory. Its a brute force way of getting all such filestatuses
+   * 
+   * @param parent
+   *          the parent path directory
+   * @param statuses
+   *          the list to add the children filestatuses to
+   * @param children
+   *          the string list of children for this parent
+   * @param archiveIndexStat
+   *          the archive index filestatus
+   */
+  private void fileStatusesInIndex(HarStatus parent, List<FileStatus> statuses,
+      List<String> children) throws IOException {
+    String parentString = parent.getName();
+    if (!parentString.endsWith(Path.SEPARATOR)){
+        parentString += Path.SEPARATOR;
     }
     }
-    try {
-      lin.close();
-    } catch(IOException io){
-      // do nothing just a read.
-    }
-    FSDataInputStream aIn = fs.open(archiveIndex);
-    LineReader aLin;
-    String retStr = null;
-    // now start reading the real index file
-    for (Store s: stores) {
-      read = 0;
-      aIn.seek(s.begin);
-      aLin = new LineReader(aIn, getConf());
-      while (read + s.begin < s.end) {
-        int tmp = aLin.readLine(line);
-        read += tmp;
-        String lineFeed = line.toString();
-        String[] parsed = lineFeed.split(" ");
-        if (harPath.compareTo(new Path(parsed[0])) == 0) {
-          // bingo!
-          retStr = lineFeed;
-          break;
+    Path harPath = new Path(parentString);
+    int harlen = harPath.depth();
+    final Map<String, FileStatus> cache = new TreeMap<String, FileStatus>();
+
+    for (HarStatus hstatus : metadata.archive.values()) {
+      String child = hstatus.getName();
+      if ((child.startsWith(parentString))) {
+        Path thisPath = new Path(child);
+        if (thisPath.depth() == harlen + 1) {
+          statuses.add(toFileStatus(hstatus, cache));
         }
         }
-        line.clear();
       }
       }
-      if (retStr != null)
-        break;
     }
     }
-    try {
-      aIn.close();
-    } catch(IOException io) {
-      //do nothing
+  }
+
+  /**
+   * Combine the status stored in the index and the underlying status. 
+   * @param h status stored in the index
+   * @param cache caching the underlying file statuses
+   * @return the combined file status
+   * @throws IOException
+   */
+  private FileStatus toFileStatus(HarStatus h,
+      Map<String, FileStatus> cache) throws IOException {
+    FileStatus underlying = null;
+    if (cache != null) {
+      underlying = cache.get(h.partName);
+    }
+    if (underlying == null) {
+      final Path p = h.isDir? archivePath: new Path(archivePath, h.partName);
+      underlying = fs.getFileStatus(p);
+      if (cache != null) {
+        cache.put(h.partName, underlying);
+      }
     }
     }
-    return retStr;
+
+    long modTime = 0;
+    int version = metadata.getVersion();
+    if (version < 3) {
+      modTime = underlying.getModificationTime();
+    } else if (version == 3) {
+      modTime = h.getModificationTime();
+    }
+
+    return new FileStatus(
+        h.isDir()? 0L: h.getLength(),
+        h.isDir(),
+        underlying.getReplication(),
+        underlying.getBlockSize(),
+        modTime,
+        underlying.getAccessTime(),
+        underlying.getPermission(),
+        underlying.getOwner(),
+        underlying.getGroup(),
+        makeRelative(this.uri.getPath(), new Path(h.name)));
   }
   }
-  
+
   // a single line parser for hadoop archives status 
   // a single line parser for hadoop archives status 
   // stored in a single line in the index files 
   // stored in a single line in the index files 
   // the format is of the form 
   // the format is of the form 
   // filename "dir"/"file" partFileName startIndex length 
   // filename "dir"/"file" partFileName startIndex length 
   // <space seperated children>
   // <space seperated children>
-  private static class HarStatus {
+  private class HarStatus {
     boolean isDir;
     boolean isDir;
     String name;
     String name;
     List<String> children;
     List<String> children;
     String partName;
     String partName;
     long startIndex;
     long startIndex;
     long length;
     long length;
-    public HarStatus(String harString) {
+    long modificationTime = 0;
+
+    public HarStatus(String harString) throws UnsupportedEncodingException {
       String[] splits = harString.split(" ");
       String[] splits = harString.split(" ");
-      this.name = splits[0];
+      this.name = decodeFileName(splits[0]);
       this.isDir = "dir".equals(splits[1]) ? true: false;
       this.isDir = "dir".equals(splits[1]) ? true: false;
       // this is equal to "none" if its a directory
       // this is equal to "none" if its a directory
       this.partName = splits[2];
       this.partName = splits[2];
       this.startIndex = Long.parseLong(splits[3]);
       this.startIndex = Long.parseLong(splits[3]);
       this.length = Long.parseLong(splits[4]);
       this.length = Long.parseLong(splits[4]);
+
+      int version = metadata.getVersion();
+      String[] propSplits = null;
+      // propSplits is used to retrieve the metainformation that Har versions
+      // 1 & 2 missed (modification time, permission, owner group).
+      // These fields are stored in an encoded string placed in different
+      // locations depending on whether it's a file or directory entry.
+      // If it's a directory, the string will be placed at the partName
+      // location (directories have no partName because they don't have data
+      // to be stored). This is done because the number of fields in a
+      // directory entry is unbounded (all children are listed at the end)
+      // If it's a file, the string will be the last field.
       if (isDir) {
       if (isDir) {
+        if (version == 3){
+          propSplits = decodeString(this.partName).split(" ");
+        }
         children = new ArrayList<String>();
         children = new ArrayList<String>();
         for (int i = 5; i < splits.length; i++) {
         for (int i = 5; i < splits.length; i++) {
-          children.add(splits[i]);
+          children.add(decodeFileName(splits[i]));
         }
         }
+      } else if (version == 3) {
+        propSplits = decodeString(splits[5]).split(" ");
+      }
+
+      if (propSplits != null && propSplits.length >= 4) {
+        modificationTime = Long.parseLong(propSplits[0]);
+        // the fields below are stored in the file but are currently not used
+        // by HarFileSystem
+        // permission = new FsPermission(Short.parseShort(propSplits[1]));
+        // owner = decodeString(propSplits[2]);
+        // group = decodeString(propSplits[3]);
       }
       }
     }
     }
     public boolean isDir() {
     public boolean isDir() {
@@ -494,6 +585,9 @@ public class HarFileSystem extends FilterFileSystem {
     public long getLength() {
     public long getLength() {
       return length;
       return length;
     }
     }
+    public long getModificationTime() {
+      return modificationTime;
+    }
   }
   }
   
   
   /**
   /**
@@ -507,7 +601,11 @@ public class HarFileSystem extends FilterFileSystem {
    */
    */
   @Override
   @Override
   public FileStatus getFileStatus(Path f) throws IOException {
   public FileStatus getFileStatus(Path f) throws IOException {
-    FileStatus archiveStatus = fs.getFileStatus(archiveIndex);
+    HarStatus hstatus = getFileHarStatus(f);
+    return toFileStatus(hstatus, null);
+  }
+
+  private HarStatus getFileHarStatus(Path f) throws IOException {
     // get the fs DataInputStream for the underlying file
     // get the fs DataInputStream for the underlying file
     // look up the index.
     // look up the index.
     Path p = makeQualified(f);
     Path p = makeQualified(f);
@@ -515,19 +613,18 @@ public class HarFileSystem extends FilterFileSystem {
     if (harPath == null) {
     if (harPath == null) {
       throw new IOException("Invalid file name: " + f + " in " + uri);
       throw new IOException("Invalid file name: " + f + " in " + uri);
     }
     }
-    String readStr = fileStatusInIndex(harPath);
-    if (readStr == null) {
+    HarStatus hstatus = metadata.archive.get(harPath);
+    if (hstatus == null) {
       throw new FileNotFoundException("File: " +  f + " does not exist in " + uri);
       throw new FileNotFoundException("File: " +  f + " does not exist in " + uri);
     }
     }
-    HarStatus hstatus = null;
-    hstatus = new HarStatus(readStr);
-    return new FileStatus(hstatus.isDir()?0:hstatus.getLength(), hstatus.isDir(),
-        (int)archiveStatus.getReplication(), archiveStatus.getBlockSize(),
-        archiveStatus.getModificationTime(), archiveStatus.getAccessTime(),
-        new FsPermission(
-        archiveStatus.getPermission()), archiveStatus.getOwner(), 
-        archiveStatus.getGroup(), 
-            makeRelative(this.uri.toString(), new Path(hstatus.name)));
+    return hstatus;
+  }
+
+  /**
+   * @return null since no checksum algorithm is implemented.
+   */
+  public FileChecksum getFileChecksum(Path f) {
+    return null;
   }
   }
 
 
   /**
   /**
@@ -538,17 +635,7 @@ public class HarFileSystem extends FilterFileSystem {
   @Override
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
     // get the fs DataInputStream for the underlying file
     // get the fs DataInputStream for the underlying file
-    // look up the index.
-    Path p = makeQualified(f);
-    Path harPath = getPathInHar(p);
-    if (harPath == null) {
-      throw new IOException("Invalid file name: " + f + " in " + uri);
-    }
-    String readStr = fileStatusInIndex(harPath);
-    if (readStr == null) {
-      throw new FileNotFoundException(f + ": not found in " + archivePath);
-    }
-    HarStatus hstatus = new HarStatus(readStr); 
+    HarStatus hstatus = getFileHarStatus(f);
     // we got it.. woo hooo!!! 
     // we got it.. woo hooo!!! 
     if (hstatus.isDir()) {
     if (hstatus.isDir()) {
       throw new FileNotFoundException(f + " : not a file in " +
       throw new FileNotFoundException(f + " : not a file in " +
@@ -617,27 +704,18 @@ public class HarFileSystem extends FilterFileSystem {
     // we will create fake filestatuses to return
     // we will create fake filestatuses to return
     // to the client
     // to the client
     List<FileStatus> statuses = new ArrayList<FileStatus>();
     List<FileStatus> statuses = new ArrayList<FileStatus>();
-    FileStatus archiveStatus = fs.getFileStatus(archiveIndex);
     Path tmpPath = makeQualified(f);
     Path tmpPath = makeQualified(f);
     Path harPath = getPathInHar(tmpPath);
     Path harPath = getPathInHar(tmpPath);
-    String readStr = fileStatusInIndex(harPath);
-    if (readStr == null) {
+    HarStatus hstatus = metadata.archive.get(harPath);
+    if (hstatus == null) {
       throw new FileNotFoundException("File " + f + " not found in " + archivePath);
       throw new FileNotFoundException("File " + f + " not found in " + archivePath);
     }
     }
-    HarStatus hstatus = new HarStatus(readStr);
-    if (!hstatus.isDir()) 
-        statuses.add(new FileStatus(hstatus.getLength(), 
-            hstatus.isDir(),
-            archiveStatus.getReplication(), archiveStatus.getBlockSize(),
-            archiveStatus.getModificationTime(), archiveStatus.getAccessTime(),
-            new FsPermission(archiveStatus.getPermission()),
-            archiveStatus.getOwner(), archiveStatus.getGroup(), 
-            makeRelative(this.uri.toString(), new Path(hstatus.name))));
-    else 
-      for (String child: hstatus.children) {
-        FileStatus tmp = getFileStatus(new Path(tmpPath, child));
-        statuses.add(tmp);
-      }
+    if (hstatus.isDir()) {
+      fileStatusesInIndex(hstatus, statuses, hstatus.children);
+    } else {
+      statuses.add(toFileStatus(hstatus, null));
+    }
+    
     return statuses.toArray(new FileStatus[statuses.size()]);
     return statuses.toArray(new FileStatus[statuses.size()]);
   }
   }
   
   
@@ -879,4 +957,114 @@ public class HarFileSystem extends FilterFileSystem {
         super(new HarFsInputStream(fs, p, start, length, 0));
         super(new HarFsInputStream(fs, p, start, length, 0));
     }
     }
   }
   }
+
+  private class HarMetaData {
+    private FileSystem fs;
+    private int version;
+    // the masterIndex of the archive
+    private Path masterIndexPath;
+    // the index file 
+    private Path archiveIndexPath;
+
+    private long masterIndexTimestamp;
+    private long archiveIndexTimestamp;
+
+    List<Store> stores = new ArrayList<Store>();
+    Map<Path, HarStatus> archive = new HashMap<Path, HarStatus>();
+    private Map<Path, FileStatus> partFileStatuses = new HashMap<Path, FileStatus>();
+
+    public HarMetaData(FileSystem fs, Path masterIndexPath, Path archiveIndexPath) {
+      this.fs = fs;
+      this.masterIndexPath = masterIndexPath;
+      this.archiveIndexPath = archiveIndexPath;
+    }
+
+    public FileStatus getPartFileStatus(Path partPath) throws IOException {
+      FileStatus status;
+      status = partFileStatuses.get(partPath);
+      if (status == null) {
+        status = fs.getFileStatus(partPath);
+        partFileStatuses.put(partPath, status);
+      }
+      return status;
+    }
+
+    public long getMasterIndexTimestamp() {
+      return masterIndexTimestamp;
+    }
+
+    public long getArchiveIndexTimestamp() {
+      return archiveIndexTimestamp;
+    }
+
+    private int getVersion() {
+      return version;
+    }
+
+    private void parseMetaData() throws IOException {
+      FSDataInputStream in = fs.open(masterIndexPath);
+      FileStatus masterStat = fs.getFileStatus(masterIndexPath);
+      masterIndexTimestamp = masterStat.getModificationTime();
+      LineReader lin = new LineReader(in, getConf());
+      Text line = new Text();
+      long read = lin.readLine(line);
+
+     // the first line contains the version of the index file
+      String versionLine = line.toString();
+      String[] arr = versionLine.split(" ");
+      version = Integer.parseInt(arr[0]);
+      // make it always backwards-compatible
+      if (this.version > HarFileSystem.VERSION) {
+        throw new IOException("Invalid version " + 
+            this.version + " expected " + HarFileSystem.VERSION);
+      }
+
+      // each line contains a hashcode range and the index file name
+      String[] readStr = null;
+      while(read < masterStat.getLen()) {
+        int b = lin.readLine(line);
+        read += b;
+        readStr = line.toString().split(" ");
+        int startHash = Integer.parseInt(readStr[0]);
+        int endHash  = Integer.parseInt(readStr[1]);
+        stores.add(new Store(Long.parseLong(readStr[2]), 
+            Long.parseLong(readStr[3]), startHash,
+            endHash));
+        line.clear();
+      }
+      try {
+        // close the master index
+        lin.close();
+      } catch(IOException io){
+        // do nothing just a read.
+      }
+
+      FSDataInputStream aIn = fs.open(archiveIndexPath);
+      FileStatus archiveStat = fs.getFileStatus(archiveIndexPath);
+      archiveIndexTimestamp = archiveStat.getModificationTime();
+      LineReader aLin;
+      String retStr = null;
+      // now start reading the real index file
+      for (Store s: stores) {
+        read = 0;
+        aIn.seek(s.begin);
+        aLin = new LineReader(aIn, getConf());
+        while (read + s.begin < s.end) {
+          int tmp = aLin.readLine(line);
+          read += tmp;
+          String lineFeed = line.toString();
+          String[] parsed = lineFeed.split(" ");
+          parsed[0] = decodeFileName(parsed[0]);
+          archive.put(new Path(parsed[0]), new HarStatus(lineFeed));
+          line.clear();
+        }
+      }
+      try {
+        // close the archive index
+        aIn.close();
+      } catch(IOException io) {
+        // do nothing just a read.
+      }
+    }
+  }
 }
 }

+ 8 - 65
src/core/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -21,11 +21,11 @@ package org.apache.hadoop.fs;
 import java.io.*;
 import java.io.*;
 import java.net.URI;
 import java.net.URI;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileLock;
 import java.util.*;
 import java.util.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
@@ -258,6 +258,7 @@ public class RawLocalFileSystem extends FileSystem {
     if (pathToFile(src).renameTo(pathToFile(dst))) {
     if (pathToFile(src).renameTo(pathToFile(dst))) {
       return true;
       return true;
     }
     }
+    LOG.debug("Falling through to a copy of " + src + " to " + dst);
     return FileUtil.copy(this, src, this, dst, true, getConf());
     return FileUtil.copy(this, src, this, dst, true, getConf());
   }
   }
   
   
@@ -415,8 +416,8 @@ public class RawLocalFileSystem extends FileSystem {
       IOException e = null;
       IOException e = null;
       try {
       try {
         StringTokenizer t = new StringTokenizer(
         StringTokenizer t = new StringTokenizer(
-            execCommand(new File(getPath().toUri()), 
-                        Shell.getGET_PERMISSION_COMMAND()));
+            FileUtil.execCommand(new File(getPath().toUri()), 
+                                 Shell.getGET_PERMISSION_COMMAND()));
         //expected format
         //expected format
         //-rw-------    1 username groupname ...
         //-rw-------    1 username groupname ...
         String permission = t.nextToken();
         String permission = t.nextToken();
@@ -466,11 +467,11 @@ public class RawLocalFileSystem extends FileSystem {
     }
     }
 
 
     if (username == null) {
     if (username == null) {
-      execCommand(pathToFile(p), Shell.SET_GROUP_COMMAND, groupname); 
+      FileUtil.execCommand(pathToFile(p), Shell.SET_GROUP_COMMAND, groupname); 
     } else {
     } else {
       //OWNER[:[GROUP]]
       //OWNER[:[GROUP]]
       String s = username + (groupname == null? "": ":" + groupname);
       String s = username + (groupname == null? "": ":" + groupname);
-      execCommand(pathToFile(p), Shell.SET_OWNER_COMMAND, s);
+      FileUtil.execCommand(pathToFile(p), Shell.SET_OWNER_COMMAND, s);
     }
     }
   }
   }
 
 
@@ -479,65 +480,7 @@ public class RawLocalFileSystem extends FileSystem {
    */
    */
   @Override
   @Override
   public void setPermission(Path p, FsPermission permission
   public void setPermission(Path p, FsPermission permission
-      ) throws IOException {
-    FsAction user = permission.getUserAction();
-    FsAction group = permission.getGroupAction();
-    FsAction other = permission.getOtherAction();
-    
-    File f = pathToFile(p);
-    
-    // Fork chmod if group and other permissions are different...
-    if (group != other) {
-      execSetPermission(f, permission);
-      return;
-    }
-    
-    boolean rv = true;
-    
-    // read perms
-    rv = f.setReadable(group.implies(FsAction.READ), false);
-    checkReturnValue(rv, p, permission);
-    if (group.implies(FsAction.READ) != user.implies(FsAction.READ)) {
-      f.setReadable(user.implies(FsAction.READ), true);
-      checkReturnValue(rv, p, permission);
-    }
-
-    // write perms
-    rv = f.setWritable(group.implies(FsAction.WRITE), false);
-    checkReturnValue(rv, p, permission);
-    if (group.implies(FsAction.WRITE) != user.implies(FsAction.WRITE)) {
-      f.setWritable(user.implies(FsAction.WRITE), true);
-      checkReturnValue(rv, p, permission);
-    }
-
-    // exec perms
-    rv = f.setExecutable(group.implies(FsAction.EXECUTE), false);
-    checkReturnValue(rv, p, permission);
-    if (group.implies(FsAction.EXECUTE) != user.implies(FsAction.EXECUTE)) {
-      f.setExecutable(user.implies(FsAction.EXECUTE), true);
-      checkReturnValue(rv, p, permission);
-    }
-  }
-
-  private void checkReturnValue(boolean rv, Path p, FsPermission permission) 
-  throws IOException {
-    if (!rv) {
-      throw new IOException("Failed to set permissions of path: " + p + " to " + 
-                            String.format("%04o", permission.toShort()));
-    }
-  }
-  
-  private void execSetPermission(File f, FsPermission permission) 
-  throws IOException {
-    execCommand(f, Shell.SET_PERMISSION_COMMAND,
-        String.format("%04o", permission.toShort()));
-  }
-  
-  private static String execCommand(File f, String... cmd) throws IOException {
-    String[] args = new String[cmd.length + 1];
-    System.arraycopy(cmd, 0, args, 0, cmd.length);
-    args[cmd.length] = f.getCanonicalPath();
-    String output = Shell.execCommand(args);
-    return output;
+                            ) throws IOException {
+    FileUtil.setPermission(pathToFile(p), permission);
   }
   }
 }
 }

+ 20 - 3
src/core/org/apache/hadoop/http/HttpServer.java

@@ -43,15 +43,14 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.jmx.JMXJsonServlet;
 import org.apache.hadoop.jmx.JMXJsonServlet;
 import org.apache.hadoop.log.LogLevel;
 import org.apache.hadoop.log.LogLevel;
 import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector;
 import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector.MODE;
 import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector.MODE;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-
 import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Handler;
 import org.mortbay.jetty.Handler;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.Server;
@@ -68,6 +67,8 @@ import org.mortbay.jetty.webapp.WebAppContext;
 import org.mortbay.thread.QueuedThreadPool;
 import org.mortbay.thread.QueuedThreadPool;
 import org.mortbay.util.MultiException;
 import org.mortbay.util.MultiException;
 
 
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+
 /**
 /**
  * Create a Jetty embedded server to answer http requests. The primary goal
  * Create a Jetty embedded server to answer http requests. The primary goal
  * is to serve up status information for the server.
  * is to serve up status information for the server.
@@ -307,6 +308,22 @@ public class HttpServer implements FilterContainer {
     context.setAttribute(name, value);
     context.setAttribute(name, value);
   }
   }
 
 
+  /** 
+   * Add a Jersey resource package.
+   * @param packageName The Java package name containing the Jersey resource.
+   * @param pathSpec The path spec for the servlet
+   */
+  public void addJerseyResourcePackage(final String packageName,
+      final String pathSpec) {
+    LOG.info("addJerseyResourcePackage: packageName=" + packageName
+        + ", pathSpec=" + pathSpec);
+    final ServletHolder sh = new ServletHolder(ServletContainer.class);
+    sh.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+        "com.sun.jersey.api.core.PackagesResourceConfig");
+    sh.setInitParameter("com.sun.jersey.config.property.packages", packageName);
+    webAppContext.addServlet(sh, pathSpec);
+  }
+
   /**
   /**
    * Add a servlet in the server.
    * Add a servlet in the server.
    * @param name The name of the servlet (can be passed as null)
    * @param name The name of the servlet (can be passed as null)

+ 24 - 0
src/core/org/apache/hadoop/io/IOUtils.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.io;
 
 
 import java.io.*;
 import java.io.*;
 import java.net.Socket;
 import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 
 
@@ -118,6 +120,28 @@ public class IOUtils {
       off += ret;
       off += ret;
     }
     }
   }
   }
+
+  /** Reads len bytes in a loop using the channel of the stream
+   * @param fileChannel a FileChannel to read len bytes into buf
+   * @param buf The buffer to fill
+   * @param off offset from the buffer
+   * @param len the length of bytes to read
+   * @throws IOException if it could not read requested number of bytes 
+   * for any reason (including EOF)
+   */
+  public static void readFileChannelFully( FileChannel fileChannel, byte buf[],
+      int off, int len ) throws IOException {
+    int toRead = len;
+    ByteBuffer byteBuffer = ByteBuffer.wrap(buf, off, len);
+    while ( toRead > 0 ) {
+      int ret = fileChannel.read(byteBuffer);
+      if ( ret < 0 ) {
+        throw new IOException( "Premeture EOF from inputStream");
+      }
+      toRead -= ret;
+      off += ret;
+    }
+  }
   
   
   /** Similar to readFully(). Skips bytes in a loop.
   /** Similar to readFully(). Skips bytes in a loop.
    * @param in The InputStream to skip bytes from
    * @param in The InputStream to skip bytes from

+ 7 - 0
src/core/org/apache/hadoop/io/SequenceFile.java

@@ -938,6 +938,13 @@ public class SequenceFile {
       }
       }
     }
     }
 
 
+    /** flush all currently written data to the file system */
+    public void syncFs() throws IOException {
+      if (out != null) {
+        out.sync();                               // flush contents to file system
+      }
+    }
+
     /** Returns the configuration of this file. */
     /** Returns the configuration of this file. */
     Configuration getConf() { return conf; }
     Configuration getConf() { return conf; }
     
     

+ 3 - 1
src/core/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -79,11 +79,13 @@ public class NativeIO {
   //TODO: fstat is an old implementation. Doesn't use the cache. This should be 
   //TODO: fstat is an old implementation. Doesn't use the cache. This should be 
   //changed to use the cache.
   //changed to use the cache.
   public static native Stat fstat(FileDescriptor fd) throws IOException;
   public static native Stat fstat(FileDescriptor fd) throws IOException;
+
   private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
   private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
   private static native String getUserName(long uid) throws IOException;
   private static native String getUserName(long uid) throws IOException;
   /** Initialize the JNI method ID and class ID cache */
   /** Initialize the JNI method ID and class ID cache */
   private static native void initNative();
   private static native void initNative();
-  
+  /** Wrapper around chmod(2) */
+  public static native void chmod(String path, int mode) throws IOException;
   private static class CachedUid {
   private static class CachedUid {
     final long timestamp;
     final long timestamp;
     final String username;
     final String username;

+ 75 - 22
src/core/org/apache/hadoop/ipc/Client.java

@@ -197,6 +197,7 @@ public class Client {
     private Socket socket = null;                 // connected socket
     private Socket socket = null;                 // connected socket
     private DataInputStream in;
     private DataInputStream in;
     private DataOutputStream out;
     private DataOutputStream out;
+    private int rpcTimeout;
     private int maxIdleTime; //connections will be culled if it was idle for
     private int maxIdleTime; //connections will be culled if it was idle for
          //maxIdleTime msecs
          //maxIdleTime msecs
     private int maxRetries; //the max. no. of retries for socket connections
     private int maxRetries; //the max. no. of retries for socket connections
@@ -223,7 +224,7 @@ public class Client {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("The ping interval is" + this.pingInterval + "ms.");
         LOG.debug("The ping interval is" + this.pingInterval + "ms.");
       }
       }
-      
+      this.rpcTimeout = remoteId.getRpcTimeout();
       UserGroupInformation ticket = remoteId.getTicket();
       UserGroupInformation ticket = remoteId.getTicket();
       Class<?> protocol = remoteId.getProtocol();
       Class<?> protocol = remoteId.getProtocol();
       this.useSasl = UserGroupInformation.isSecurityEnabled();
       this.useSasl = UserGroupInformation.isSecurityEnabled();
@@ -239,9 +240,8 @@ public class Client {
             throw new IOException(e.toString());
             throw new IOException(e.toString());
           }
           }
           InetSocketAddress addr = remoteId.getAddress();
           InetSocketAddress addr = remoteId.getAddress();
-          token = tokenSelector.selectToken(new Text(addr.getAddress()
-              .getHostAddress() + ":" + addr.getPort()), 
-              ticket.getTokens());
+          token = tokenSelector.selectToken(
+              SecurityUtil.buildTokenService(addr), ticket.getTokens());
         }
         }
         KerberosInfo krbInfo = protocol.getAnnotation(KerberosInfo.class);
         KerberosInfo krbInfo = protocol.getAnnotation(KerberosInfo.class);
         if (krbInfo != null) {
         if (krbInfo != null) {
@@ -305,11 +305,13 @@ public class Client {
       }
       }
 
 
       /* Process timeout exception
       /* Process timeout exception
-       * if the connection is not going to be closed, send a ping.
+       * if the connection is not going to be closed or 
+       * is not configured to have a RPC timeout, send a ping.
+       * (if rpcTimeout is not set to be 0, then RPC should timeout.
        * otherwise, throw the timeout exception.
        * otherwise, throw the timeout exception.
        */
        */
       private void handleTimeout(SocketTimeoutException e) throws IOException {
       private void handleTimeout(SocketTimeoutException e) throws IOException {
-        if (shouldCloseConnection.get() || !running.get()) {
+        if (shouldCloseConnection.get() || !running.get() || rpcTimeout > 0) {
           throw e;
           throw e;
         } else {
         } else {
           sendPing();
           sendPing();
@@ -381,6 +383,27 @@ public class Client {
       saslRpcClient = new SaslRpcClient(authMethod, token, serverPrincipal);
       saslRpcClient = new SaslRpcClient(authMethod, token, serverPrincipal);
       return saslRpcClient.saslConnect(in2, out2);
       return saslRpcClient.saslConnect(in2, out2);
     }
     }
+
+    /**
+     * Update the server address if the address corresponding to the host
+     * name has changed.
+     *
+     * @return true if an addr change was detected.
+     * @throws IOException when the hostname cannot be resolved.
+     */
+    private synchronized boolean updateAddress() throws IOException {
+      // Do a fresh lookup with the old host name.
+      InetSocketAddress currentAddr =  new InetSocketAddress(
+                               server.getHostName(), server.getPort());
+
+      if (!server.equals(currentAddr)) {
+        LOG.warn("Address change detected. Old: " + server.toString() +
+                                 " New: " + currentAddr.toString());
+        server = currentAddr;
+        return true;
+      }
+      return false;
+    }
     
     
     private synchronized void setupConnection() throws IOException {
     private synchronized void setupConnection() throws IOException {
       short ioFailures = 0;
       short ioFailures = 0;
@@ -411,15 +434,28 @@ public class Client {
           }
           }
           
           
           // connection time out is 20s
           // connection time out is 20s
-          NetUtils.connect(this.socket, remoteId.getAddress(), 20000);
+          NetUtils.connect(this.socket, server, 20000);
+          if (rpcTimeout > 0) {
+            pingInterval = rpcTimeout;  // rpcTimeout overwrites pingInterval
+          }
+
           this.socket.setSoTimeout(pingInterval);
           this.socket.setSoTimeout(pingInterval);
           return;
           return;
         } catch (SocketTimeoutException toe) {
         } catch (SocketTimeoutException toe) {
+          /* Check for an address change and update the local reference.
+           * Reset the failure counter if the address was changed
+           */
+          if (updateAddress()) {
+            timeoutFailures = ioFailures = 0;
+          }
           /* The max number of retries is 45,
           /* The max number of retries is 45,
            * which amounts to 20s*45 = 15 minutes retries.
            * which amounts to 20s*45 = 15 minutes retries.
            */
            */
           handleConnectionFailure(timeoutFailures++, 45, toe);
           handleConnectionFailure(timeoutFailures++, 45, toe);
         } catch (IOException ie) {
         } catch (IOException ie) {
+          if (updateAddress()) {
+            timeoutFailures = ioFailures = 0;
+          }
           handleConnectionFailure(ioFailures++, maxRetries, ie);
           handleConnectionFailure(ioFailures++, maxRetries, ie);
         }
         }
       }
       }
@@ -780,6 +816,7 @@ public class Client {
         } else if (state == Status.ERROR.state) {
         } else if (state == Status.ERROR.state) {
           call.setException(new RemoteException(WritableUtils.readString(in),
           call.setException(new RemoteException(WritableUtils.readString(in),
                                                 WritableUtils.readString(in)));
                                                 WritableUtils.readString(in)));
+          calls.remove(id);
         } else if (state == Status.FATAL.state) {
         } else if (state == Status.FATAL.state) {
           // Close the connection
           // Close the connection
           markClosed(new RemoteException(WritableUtils.readString(in), 
           markClosed(new RemoteException(WritableUtils.readString(in), 
@@ -964,39 +1001,41 @@ public class Client {
   public Writable call(Writable param, InetSocketAddress addr, 
   public Writable call(Writable param, InetSocketAddress addr, 
       UserGroupInformation ticket)  
       UserGroupInformation ticket)  
       throws InterruptedException, IOException {
       throws InterruptedException, IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(addr, null, ticket,
+    ConnectionId remoteId = ConnectionId.getConnectionId(addr, null, ticket, 0,
         conf);
         conf);
     return call(param, remoteId);
     return call(param, remoteId);
   }
   }
   
   
   /** Make a call, passing <code>param</code>, to the IPC server running at
   /** Make a call, passing <code>param</code>, to the IPC server running at
-   * <code>address</code> which is servicing the <code>protocol</code> protocol, 
-   * with the <code>ticket</code> credentials, returning the value.  
+   * <code>address</code> which is servicing the <code>protocol</code> protocol,
+   * with the <code>ticket</code> credentials and <code>rpcTimeout</code> as
+   * timeout, returning the value.
    * Throws exceptions if there are network problems or if the remote code 
    * Throws exceptions if there are network problems or if the remote code 
    * threw an exception. 
    * threw an exception. 
    * @deprecated Use {@link #call(Writable, ConnectionId)} instead 
    * @deprecated Use {@link #call(Writable, ConnectionId)} instead 
    */
    */
   @Deprecated
   @Deprecated
   public Writable call(Writable param, InetSocketAddress addr, 
   public Writable call(Writable param, InetSocketAddress addr, 
-                       Class<?> protocol, UserGroupInformation ticket)  
+                       Class<?> protocol, UserGroupInformation ticket,
+                       int rpcTimeout)
                        throws InterruptedException, IOException {
                        throws InterruptedException, IOException {
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, conf);
+        ticket, rpcTimeout, conf);
     return call(param, remoteId);
     return call(param, remoteId);
   }
   }
   
   
   /** Make a call, passing <code>param</code>, to the IPC server running at
   /** Make a call, passing <code>param</code>, to the IPC server running at
    * <code>address</code> which is servicing the <code>protocol</code> protocol, 
    * <code>address</code> which is servicing the <code>protocol</code> protocol, 
-   * with the <code>ticket</code> credentials and <code>conf</code> as 
-   * configuration for this connection, returning the value.  
-   * Throws exceptions if there are network problems or if the remote code 
+   * with the <code>ticket</code> credentials, <code>rpcTimeout</code> as timeout 
+   * and <code>conf</code> as configuration for this connection, returning the 
+   * value. Throws exceptions if there are network problems or if the remote code 
    * threw an exception. */
    * threw an exception. */
-  public Writable call(Writable param, InetSocketAddress addr, 
+  public Writable call(Writable param, InetSocketAddress addr,
                        Class<?> protocol, UserGroupInformation ticket,
                        Class<?> protocol, UserGroupInformation ticket,
-                       Configuration conf)  
+                       int rpcTimeout, Configuration conf)
                        throws InterruptedException, IOException {
                        throws InterruptedException, IOException {
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, conf);
+        ticket, rpcTimeout, conf);
     return call(param, remoteId);
     return call(param, remoteId);
   }
   }
   
   
@@ -1106,7 +1145,7 @@ public class Client {
         ParallelCall call = new ParallelCall(params[i], results, i);
         ParallelCall call = new ParallelCall(params[i], results, i);
         try {
         try {
           ConnectionId remoteId = ConnectionId.getConnectionId(addresses[i],
           ConnectionId remoteId = ConnectionId.getConnectionId(addresses[i],
-              protocol, ticket, conf);
+              protocol, ticket, 0, conf);
           Connection connection = getConnection(remoteId, call);
           Connection connection = getConnection(remoteId, call);
           connection.sendParam(call);             // send each parameter
           connection.sendParam(call);             // send each parameter
         } catch (IOException e) {
         } catch (IOException e) {
@@ -1174,6 +1213,7 @@ public class Client {
      UserGroupInformation ticket;
      UserGroupInformation ticket;
      Class<?> protocol;
      Class<?> protocol;
      private static final int PRIME = 16777619;
      private static final int PRIME = 16777619;
+     private int rpcTimeout;
      private String serverPrincipal;
      private String serverPrincipal;
      private int maxIdleTime; //connections will be culled if it was idle for 
      private int maxIdleTime; //connections will be culled if it was idle for 
      //maxIdleTime msecs
      //maxIdleTime msecs
@@ -1182,13 +1222,14 @@ public class Client {
      private int pingInterval; // how often sends ping to the server in msecs
      private int pingInterval; // how often sends ping to the server in msecs
      
      
      ConnectionId(InetSocketAddress address, Class<?> protocol, 
      ConnectionId(InetSocketAddress address, Class<?> protocol, 
-                  UserGroupInformation ticket,
+                  UserGroupInformation ticket, int rpcTimeout,
                   String serverPrincipal, int maxIdleTime, 
                   String serverPrincipal, int maxIdleTime, 
                   int maxRetries, boolean tcpNoDelay,
                   int maxRetries, boolean tcpNoDelay,
                   int pingInterval) {
                   int pingInterval) {
        this.protocol = protocol;
        this.protocol = protocol;
        this.address = address;
        this.address = address;
        this.ticket = ticket;
        this.ticket = ticket;
+       this.rpcTimeout = rpcTimeout;
        this.serverPrincipal = serverPrincipal;
        this.serverPrincipal = serverPrincipal;
        this.maxIdleTime = maxIdleTime;
        this.maxIdleTime = maxIdleTime;
        this.maxRetries = maxRetries;
        this.maxRetries = maxRetries;
@@ -1207,7 +1248,11 @@ public class Client {
      UserGroupInformation getTicket() {
      UserGroupInformation getTicket() {
        return ticket;
        return ticket;
      }
      }
-     
+    
+     private int getRpcTimeout() {
+       return rpcTimeout;
+     }
+ 
      String getServerPrincipal() {
      String getServerPrincipal() {
        return serverPrincipal;
        return serverPrincipal;
      }
      }
@@ -1231,9 +1276,15 @@ public class Client {
      static ConnectionId getConnectionId(InetSocketAddress addr,
      static ConnectionId getConnectionId(InetSocketAddress addr,
          Class<?> protocol, UserGroupInformation ticket,
          Class<?> protocol, UserGroupInformation ticket,
          Configuration conf) throws IOException {
          Configuration conf) throws IOException {
+       return getConnectionId(addr, protocol, ticket, 0, conf);
+     }
+
+     static ConnectionId getConnectionId(InetSocketAddress addr,
+         Class<?> protocol, UserGroupInformation ticket, int rpcTimeout,
+         Configuration conf) throws IOException {
        String remotePrincipal = getRemotePrincipal(conf, addr, protocol);
        String remotePrincipal = getRemotePrincipal(conf, addr, protocol);
        return new ConnectionId(addr, protocol, ticket,
        return new ConnectionId(addr, protocol, ticket,
-           remotePrincipal,
+           rpcTimeout, remotePrincipal,
            conf.getInt("ipc.client.connection.maxidletime", 10000), // 10s
            conf.getInt("ipc.client.connection.maxidletime", 10000), // 10s
            conf.getInt("ipc.client.connect.max.retries", 10),
            conf.getInt("ipc.client.connect.max.retries", 10),
            conf.getBoolean("ipc.client.tcpnodelay", false),
            conf.getBoolean("ipc.client.tcpnodelay", false),
@@ -1275,6 +1326,7 @@ public class Client {
              && this.maxRetries == that.maxRetries
              && this.maxRetries == that.maxRetries
              && this.pingInterval == that.pingInterval
              && this.pingInterval == that.pingInterval
              && isEqual(this.protocol, that.protocol)
              && isEqual(this.protocol, that.protocol)
+             && this.rpcTimeout == that.rpcTimeout
              && isEqual(this.serverPrincipal, that.serverPrincipal)
              && isEqual(this.serverPrincipal, that.serverPrincipal)
              && this.tcpNoDelay == that.tcpNoDelay
              && this.tcpNoDelay == that.tcpNoDelay
              && isEqual(this.ticket, that.ticket);
              && isEqual(this.ticket, that.ticket);
@@ -1290,6 +1342,7 @@ public class Client {
        result = PRIME * result + maxRetries;
        result = PRIME * result + maxRetries;
        result = PRIME * result + pingInterval;
        result = PRIME * result + pingInterval;
        result = PRIME * result + ((protocol == null) ? 0 : protocol.hashCode());
        result = PRIME * result + ((protocol == null) ? 0 : protocol.hashCode());
+       result = PRIME * rpcTimeout;
        result = PRIME * result
        result = PRIME * result
            + ((serverPrincipal == null) ? 0 : serverPrincipal.hashCode());
            + ((serverPrincipal == null) ? 0 : serverPrincipal.hashCode());
        result = PRIME * result + (tcpNoDelay ? 1231 : 1237);
        result = PRIME * result + (tcpNoDelay ? 1231 : 1237);

+ 54 - 14
src/core/org/apache/hadoop/ipc/RPC.java

@@ -207,9 +207,10 @@ public class RPC {
 
 
     public Invoker(Class<? extends VersionedProtocol> protocol,
     public Invoker(Class<? extends VersionedProtocol> protocol,
         InetSocketAddress address, UserGroupInformation ticket,
         InetSocketAddress address, UserGroupInformation ticket,
-        Configuration conf, SocketFactory factory) throws IOException {
+        Configuration conf, SocketFactory factory,
+        int rpcTimeout) throws IOException {
       this.remoteId = Client.ConnectionId.getConnectionId(address, protocol,
       this.remoteId = Client.ConnectionId.getConnectionId(address, protocol,
-          ticket, conf);
+          ticket, rpcTimeout, conf);
       this.client = CLIENTS.getClient(conf, factory);
       this.client = CLIENTS.getClient(conf, factory);
     }
     }
 
 
@@ -292,7 +293,7 @@ public class RPC {
       InetSocketAddress addr,
       InetSocketAddress addr,
       Configuration conf
       Configuration conf
       ) throws IOException {
       ) throws IOException {
-    return waitForProxy(protocol, clientVersion, addr, conf, Long.MAX_VALUE);
+    return waitForProxy(protocol, clientVersion, addr, conf, 0, Long.MAX_VALUE);
   }
   }
 
 
   /**
   /**
@@ -301,7 +302,7 @@ public class RPC {
    * @param clientVersion client version
    * @param clientVersion client version
    * @param addr remote address
    * @param addr remote address
    * @param conf configuration to use
    * @param conf configuration to use
-   * @param timeout time in milliseconds before giving up
+   * @param connTimeout time in milliseconds before giving up
    * @return the proxy
    * @return the proxy
    * @throws IOException if the far end through a RemoteException
    * @throws IOException if the far end through a RemoteException
    */
    */
@@ -310,13 +311,24 @@ public class RPC {
                                                long clientVersion,
                                                long clientVersion,
                                                InetSocketAddress addr,
                                                InetSocketAddress addr,
                                                Configuration conf,
                                                Configuration conf,
-                                               long timeout
-                                               ) throws IOException { 
+                                               long connTimeout)
+                                               throws IOException { 
+    return waitForProxy(protocol, clientVersion, addr, conf, 0, connTimeout);
+  }
+
+  static VersionedProtocol waitForProxy(
+                      Class<? extends VersionedProtocol> protocol,
+                                               long clientVersion,
+                                               InetSocketAddress addr,
+                                               Configuration conf,
+                                               int rpcTimeout,
+                                               long connTimeout)
+                                               throws IOException { 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     IOException ioe;
     IOException ioe;
     while (true) {
     while (true) {
       try {
       try {
-        return getProxy(protocol, clientVersion, addr, conf);
+        return getProxy(protocol, clientVersion, addr, conf, rpcTimeout);
       } catch(ConnectException se) {  // namenode has not been started
       } catch(ConnectException se) {  // namenode has not been started
         LOG.info("Server at " + addr + " not available yet, Zzzzz...");
         LOG.info("Server at " + addr + " not available yet, Zzzzz...");
         ioe = se;
         ioe = se;
@@ -325,7 +337,7 @@ public class RPC {
         ioe = te;
         ioe = te;
       }
       }
       // check if timed out
       // check if timed out
-      if (System.currentTimeMillis()-timeout >= startTime) {
+      if (System.currentTimeMillis()-connTimeout >= startTime) {
         throw ioe;
         throw ioe;
       }
       }
 
 
@@ -337,6 +349,7 @@ public class RPC {
       }
       }
     }
     }
   }
   }
+
   /** Construct a client-side proxy object that implements the named protocol,
   /** Construct a client-side proxy object that implements the named protocol,
    * talking to a server at the named address. */
    * talking to a server at the named address. */
   public static VersionedProtocol getProxy(
   public static VersionedProtocol getProxy(
@@ -344,15 +357,34 @@ public class RPC {
       long clientVersion, InetSocketAddress addr, Configuration conf,
       long clientVersion, InetSocketAddress addr, Configuration conf,
       SocketFactory factory) throws IOException {
       SocketFactory factory) throws IOException {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    return getProxy(protocol, clientVersion, addr, ugi, conf, factory);
+    return getProxy(protocol, clientVersion, addr, ugi, conf, factory, 0);
   }
   }
-  
+ 
+  /** Construct a client-side proxy object that implements the named protocol,
+   * talking to a server at the named address. */
+  public static VersionedProtocol getProxy(
+      Class<? extends VersionedProtocol> protocol,
+      long clientVersion, InetSocketAddress addr, Configuration conf,
+      SocketFactory factory, int rpcTimeout) throws IOException {
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    return getProxy(protocol, clientVersion, addr, ugi, conf, factory, rpcTimeout);
+  }
+    
+  /** Construct a client-side proxy object that implements the named protocol,
+   * talking to a server at the named address. */
+  public static VersionedProtocol getProxy(
+      Class<? extends VersionedProtocol> protocol,
+      long clientVersion, InetSocketAddress addr, UserGroupInformation ticket,
+      Configuration conf, SocketFactory factory) throws IOException {
+    return getProxy(protocol, clientVersion, addr, ticket, conf, factory, 0);
+  }
+
   /** Construct a client-side proxy object that implements the named protocol,
   /** Construct a client-side proxy object that implements the named protocol,
    * talking to a server at the named address. */
    * talking to a server at the named address. */
   public static VersionedProtocol getProxy(
   public static VersionedProtocol getProxy(
       Class<? extends VersionedProtocol> protocol,
       Class<? extends VersionedProtocol> protocol,
       long clientVersion, InetSocketAddress addr, UserGroupInformation ticket,
       long clientVersion, InetSocketAddress addr, UserGroupInformation ticket,
-      Configuration conf, SocketFactory factory) throws IOException {    
+      Configuration conf, SocketFactory factory, int rpcTimeout) throws IOException {
 
 
     if (UserGroupInformation.isSecurityEnabled()) {
     if (UserGroupInformation.isSecurityEnabled()) {
       SaslRpcServer.init(conf);
       SaslRpcServer.init(conf);
@@ -360,7 +392,7 @@ public class RPC {
     VersionedProtocol proxy =
     VersionedProtocol proxy =
         (VersionedProtocol) Proxy.newProxyInstance(
         (VersionedProtocol) Proxy.newProxyInstance(
             protocol.getClassLoader(), new Class[] { protocol },
             protocol.getClassLoader(), new Class[] { protocol },
-            new Invoker(protocol, addr, ticket, conf, factory));
+            new Invoker(protocol, addr, ticket, conf, factory, rpcTimeout));
     long serverVersion = proxy.getProtocolVersion(protocol.getName(), 
     long serverVersion = proxy.getProtocolVersion(protocol.getName(), 
                                                   clientVersion);
                                                   clientVersion);
     if (serverVersion == clientVersion) {
     if (serverVersion == clientVersion) {
@@ -385,9 +417,17 @@ public class RPC {
       Class<? extends VersionedProtocol> protocol,
       Class<? extends VersionedProtocol> protocol,
       long clientVersion, InetSocketAddress addr, Configuration conf)
       long clientVersion, InetSocketAddress addr, Configuration conf)
       throws IOException {
       throws IOException {
+    return getProxy(protocol, clientVersion, addr, conf,
+        NetUtils.getDefaultSocketFactory(conf), 0);
+  }
+
+  public static VersionedProtocol getProxy(
+      Class<? extends VersionedProtocol> protocol,
+      long clientVersion, InetSocketAddress addr, Configuration conf, int rpcTimeout)
+      throws IOException {
 
 
-    return getProxy(protocol, clientVersion, addr, conf, 
-        NetUtils.getDefaultSocketFactory(conf));
+    return getProxy(protocol, clientVersion, addr, conf,
+        NetUtils.getDefaultSocketFactory(conf), rpcTimeout);
   }
   }
 
 
   /**
   /**

+ 2 - 2
src/core/org/apache/hadoop/ipc/Server.java

@@ -1032,14 +1032,14 @@ public abstract class Server {
         }
         }
         if (saslServer.isComplete()) {
         if (saslServer.isComplete()) {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
-            LOG.info("SASL server context established. Negotiated QoP is "
+            LOG.debug("SASL server context established. Negotiated QoP is "
                 + saslServer.getNegotiatedProperty(Sasl.QOP));
                 + saslServer.getNegotiatedProperty(Sasl.QOP));
           }
           }
           String qop = (String) saslServer.getNegotiatedProperty(Sasl.QOP);
           String qop = (String) saslServer.getNegotiatedProperty(Sasl.QOP);
           useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
           useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
           user = getAuthorizedUgi(saslServer.getAuthorizationID());
           user = getAuthorizedUgi(saslServer.getAuthorizationID());
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
-            LOG.info("SASL server successfully authenticated client: " + user);
+            LOG.debug("SASL server successfully authenticated client: " + user);
           }
           }
           rpcMetrics.incrAuthenticationSuccesses();
           rpcMetrics.incrAuthenticationSuccesses();
           AUDITLOG.info(AUTH_SUCCESSFULL_FOR + user);
           AUDITLOG.info(AUTH_SUCCESSFULL_FOR + user);

+ 287 - 0
src/core/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java

@@ -0,0 +1,287 @@
+/**
+ * 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.metrics2.sink.ganglia;
+
+import java.io.IOException;
+import java.net.DatagramPacket;
+import java.net.DatagramSocket;
+import java.net.SocketAddress;
+import java.net.SocketException;
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.configuration.SubsetConfiguration;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.util.Servers;
+import org.apache.hadoop.net.DNS;
+
+/**
+ * This the base class for Ganglia sink classes using metrics2. Lot of the code
+ * has been derived from org.apache.hadoop.metrics.ganglia.GangliaContext.
+ * As per the documentation, sink implementations doesn't have to worry about
+ * thread safety. Hence the code wasn't written for thread safety and should
+ * be modified in case the above assumption changes in the future.
+ */
+public abstract class AbstractGangliaSink implements MetricsSink {
+
+  public final Log LOG = LogFactory.getLog(this.getClass());
+
+  /*
+   * Output of "gmetric --help" showing allowable values
+   * -t, --type=STRING
+   *     Either string|int8|uint8|int16|uint16|int32|uint32|float|double
+   * -u, --units=STRING Unit of measure for the value e.g. Kilobytes, Celcius
+   *     (default='')
+   * -s, --slope=STRING Either zero|positive|negative|both
+   *     (default='both')
+   * -x, --tmax=INT The maximum time in seconds between gmetric calls
+   *     (default='60')
+   */
+  public static final String DEFAULT_UNITS = "";
+  public static final int DEFAULT_TMAX = 60;
+  public static final int DEFAULT_DMAX = 0;
+  public static final GangliaSlope DEFAULT_SLOPE = GangliaSlope.both;
+  public static final int DEFAULT_PORT = 8649;
+  public static final String SERVERS_PROPERTY = "servers";
+  public static final int BUFFER_SIZE = 1500; // as per libgmond.c
+  public static final String SUPPORT_SPARSE_METRICS_PROPERTY = "supportsparse";
+  public static final boolean SUPPORT_SPARSE_METRICS_DEFAULT = false;
+  public static final String EQUAL = "=";
+
+  private String hostName = "UNKNOWN.example.com";
+  private DatagramSocket datagramSocket;
+  private List<? extends SocketAddress> metricsServers;
+  private byte[] buffer = new byte[BUFFER_SIZE];
+  private int offset;
+  private boolean supportSparseMetrics = SUPPORT_SPARSE_METRICS_DEFAULT;
+
+  /**
+   * Used for visiting Metrics
+   */
+  protected final GangliaMetricVisitor gangliaMetricVisitor =
+    new GangliaMetricVisitor();
+
+  private SubsetConfiguration conf;
+  private Map<String, GangliaConf> gangliaConfMap;
+  private GangliaConf DEFAULT_GANGLIA_CONF = new GangliaConf();
+
+  /**
+   * ganglia slope values which equal the ordinal
+   */
+  public enum GangliaSlope {
+    zero,       // 0
+    positive,   // 1
+    negative,   // 2
+    both        // 3
+  };
+
+  /**
+   * define enum for various type of conf
+   */
+  public enum GangliaConfType {
+    slope, units, dmax, tmax
+  };
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.metrics2.MetricsPlugin#init(org.apache.commons.configuration
+   * .SubsetConfiguration)
+   */
+  public void init(SubsetConfiguration conf) {
+    LOG.debug("Initializing the GangliaSink for Ganglia metrics.");
+
+    this.conf = conf;
+
+    // Take the hostname from the DNS class.
+    if (conf.getString("slave.host.name") != null) {
+      hostName = conf.getString("slave.host.name");
+    } else {
+      try {
+        hostName = DNS.getDefaultHost(
+            conf.getString("dfs.datanode.dns.interface", "default"),
+            conf.getString("dfs.datanode.dns.nameserver", "default"));
+      } catch (UnknownHostException uhe) {
+        LOG.error(uhe);
+        hostName = "UNKNOWN.example.com";
+      }
+    }
+
+    // load the gannglia servers from properties
+    metricsServers = Servers.parse(conf.getString(SERVERS_PROPERTY),
+        DEFAULT_PORT);
+
+    // extract the Ganglia conf per metrics
+    gangliaConfMap = new HashMap<String, GangliaConf>();
+    loadGangliaConf(GangliaConfType.units);
+    loadGangliaConf(GangliaConfType.tmax);
+    loadGangliaConf(GangliaConfType.dmax);
+    loadGangliaConf(GangliaConfType.slope);
+
+    try {
+      datagramSocket = new DatagramSocket();
+    } catch (SocketException se) {
+      LOG.error(se);
+    }
+
+    // see if sparseMetrics is supported. Default is false
+    supportSparseMetrics = conf.getBoolean(SUPPORT_SPARSE_METRICS_PROPERTY,
+        SUPPORT_SPARSE_METRICS_DEFAULT);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.hadoop.metrics2.MetricsSink#flush()
+   */
+  public void flush() {
+    // nothing to do as we are not buffering data
+  }
+
+  /**
+   * Load the configurations for a conf type
+   *
+   * @param gtype Only load metrics for given type
+   */
+  private void loadGangliaConf(GangliaConfType gtype) {
+    String propertyarr[] = conf.getStringArray(gtype.name());
+    if (propertyarr != null && propertyarr.length > 0) {
+      for (String metricNValue : propertyarr) {
+        String metricNValueArr[] = metricNValue.split(EQUAL);
+        if (metricNValueArr.length != 2 || metricNValueArr[0].length() == 0) {
+          LOG.error("Invalid propertylist for " + gtype.name());
+        }
+
+        String metricName = metricNValueArr[0].trim();
+        String metricValue = metricNValueArr[1].trim();
+        GangliaConf gconf = gangliaConfMap.get(metricName);
+        if (gconf == null) {
+          gconf = new GangliaConf();
+          gangliaConfMap.put(metricName, gconf);
+        }
+
+        switch (gtype) {
+        case units:
+          gconf.setUnits(metricValue);
+          break;
+        case dmax:
+          gconf.setDmax(Integer.parseInt(metricValue));
+          break;
+        case tmax:
+          gconf.setTmax(Integer.parseInt(metricValue));
+          break;
+        case slope:
+          gconf.setSlope(GangliaSlope.valueOf(metricValue));
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Lookup GangliaConf from cache. If not found, return default values
+   *
+   * @param metricName
+   * @return looked up GangliaConf
+   */
+  protected GangliaConf getGangliaConfForMetric(String metricName) {
+    GangliaConf gconf = gangliaConfMap.get(metricName);
+
+    return gconf != null ? gconf : DEFAULT_GANGLIA_CONF;
+  }
+
+  /**
+   * @return the hostName
+   */
+  protected String getHostName() {
+    return hostName;
+  }
+
+  /**
+   * Puts a string into the buffer by first writing the size of the string as an
+   * int, followed by the bytes of the string, padded if necessary to a multiple
+   * of 4.
+   * @param s the string to be written to buffer at offset location
+   */
+  protected void xdr_string(String s) {
+    byte[] bytes = s.getBytes();
+    int len = bytes.length;
+    xdr_int(len);
+    System.arraycopy(bytes, 0, buffer, offset, len);
+    offset += len;
+    pad();
+  }
+
+  /**
+   * Pads the buffer with zero bytes up to the nearest multiple of 4.
+   */
+  private void pad() {
+    int newOffset = ((offset + 3) / 4) * 4;
+    while (offset < newOffset) {
+      buffer[offset++] = 0;
+    }
+  }
+
+  /**
+   * Puts an integer into the buffer as 4 bytes, big-endian.
+   */
+  protected void xdr_int(int i) {
+    buffer[offset++] = (byte) ((i >> 24) & 0xff);
+    buffer[offset++] = (byte) ((i >> 16) & 0xff);
+    buffer[offset++] = (byte) ((i >> 8) & 0xff);
+    buffer[offset++] = (byte) (i & 0xff);
+  }
+
+  /**
+   * Sends Ganglia Metrics to the configured hosts
+   * @throws IOException
+   */
+  protected void emitToGangliaHosts() throws IOException {
+    try {
+      for (SocketAddress socketAddress : metricsServers) {
+        DatagramPacket packet =
+          new DatagramPacket(buffer, offset, socketAddress);
+        datagramSocket.send(packet);
+      }
+    } finally {
+      // reset the buffer for the next metric to be built
+      offset = 0;
+    }
+  }
+
+  /**
+   * @return whether sparse metrics are supported
+   */
+  protected boolean isSupportSparseMetrics() {
+    return supportSparseMetrics;
+  }
+
+  /**
+   * Used only by unit test
+   * @param datagramSocket the datagramSocket to set.
+   */
+  void setDatagramSocket(DatagramSocket datagramSocket) {
+    this.datagramSocket = datagramSocket;
+  }
+}

+ 95 - 0
src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaConf.java

@@ -0,0 +1,95 @@
+/**
+ * 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.metrics2.sink.ganglia;
+
+import org.apache.hadoop.metrics2.sink.ganglia.AbstractGangliaSink.GangliaSlope;
+
+/**
+ * class which is used to store ganglia properties
+ */
+class GangliaConf {
+  private String units = AbstractGangliaSink.DEFAULT_UNITS;
+  private GangliaSlope slope;
+  private int dmax = AbstractGangliaSink.DEFAULT_DMAX;
+  private int tmax = AbstractGangliaSink.DEFAULT_TMAX;
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("unit=").append(units).append(", slope=").append(slope)
+        .append(", dmax=").append(dmax).append(", tmax=").append(tmax);
+    return buf.toString();
+  }
+
+  /**
+   * @return the units
+   */
+  String getUnits() {
+    return units;
+  }
+
+  /**
+   * @param units the units to set
+   */
+  void setUnits(String units) {
+    this.units = units;
+  }
+
+  /**
+   * @return the slope
+   */
+  GangliaSlope getSlope() {
+    return slope;
+  }
+
+  /**
+   * @param slope the slope to set
+   */
+  void setSlope(GangliaSlope slope) {
+    this.slope = slope;
+  }
+
+  /**
+   * @return the dmax
+   */
+  int getDmax() {
+    return dmax;
+  }
+
+  /**
+   * @param dmax the dmax to set
+   */
+  void setDmax(int dmax) {
+    this.dmax = dmax;
+  }
+
+  /**
+   * @return the tmax
+   */
+  int getTmax() {
+    return tmax;
+  }
+
+  /**
+   * @param tmax the tmax to set
+   */
+  void setTmax(int tmax) {
+    this.tmax = tmax;
+  }
+}

+ 99 - 0
src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaMetricVisitor.java

@@ -0,0 +1,99 @@
+/**
+ * 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.metrics2.sink.ganglia;
+
+import org.apache.hadoop.metrics2.MetricCounter;
+import org.apache.hadoop.metrics2.MetricGauge;
+import org.apache.hadoop.metrics2.MetricsVisitor;
+import org.apache.hadoop.metrics2.sink.ganglia.AbstractGangliaSink.GangliaSlope;
+
+/**
+ * Since implementations of Metric are not public, hence use a visitor to
+ * figure out the type and slope of the metric. Counters have "positive"
+ * slope.
+ */
+class GangliaMetricVisitor implements MetricsVisitor {
+  private static final String INT32 = "int32";
+  private static final String FLOAT = "float";
+  private static final String DOUBLE = "double";
+
+  private String type;
+  private GangliaSlope slope;
+
+  /**
+   * @return the type of a visited metric
+   */
+  String getType() {
+    return type;
+  }
+
+  /**
+   * @return the slope of a visited metric. Slope is positive for counters
+   * and null for others
+   */
+  GangliaSlope getSlope() {
+    return slope;
+  }
+
+  @Override
+  public void gauge(MetricGauge<Integer> metric, int value) {
+    // MetricGaugeInt.class ==> "int32"
+    type = INT32;
+    slope = null; // set to null as cannot figure out from Metric
+  }
+
+  @Override
+  public void counter(MetricCounter<Integer> metric, int value) {
+    // MetricCounterInt.class ==> "int32"
+    type = INT32;
+
+    // counters have positive slope
+    slope = GangliaSlope.positive;
+  }
+
+  @Override
+  public void gauge(MetricGauge<Long> metric, long value) {
+    // MetricGaugeLong.class ==> "float"
+    type = FLOAT;
+    slope = null; // set to null as cannot figure out from Metric
+  }
+
+  @Override
+  public void counter(MetricCounter<Long> metric, long value) {
+    // MetricCounterLong.class ==> "float"
+    type = FLOAT;
+
+    // counters have positive slope
+    slope = GangliaSlope.positive;
+  }
+
+  @Override
+  public void gauge(MetricGauge<Float> metric, float value) {
+    // MetricGaugeFloat.class ==> "float"
+    type = FLOAT;
+    slope = null; // set to null as cannot figure out from Metric
+  }
+
+  @Override
+  public void gauge(MetricGauge<Double> metric, double value) {
+    // MetricGaugeDouble.class ==> "double"
+    type = DOUBLE;
+    slope = null; // set to null as cannot figure out from Metric
+  }
+}

+ 198 - 0
src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink30.java

@@ -0,0 +1,198 @@
+/**
+ * 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.metrics2.sink.ganglia;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.Metric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.util.MetricsCache;
+import org.apache.hadoop.metrics2.util.MetricsCache.Record;
+
+/**
+ * This code supports Ganglia 3.0
+ *
+ */
+public class GangliaSink30 extends AbstractGangliaSink {
+
+  public final Log LOG = LogFactory.getLog(this.getClass());
+
+  protected MetricsCache metricsCache = new MetricsCache();
+
+  /*
+   *
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.metrics2.MetricsSink#putMetrics(org.apache.hadoop.metrics2
+   * .MetricsRecord)
+   */
+  @Override
+  public void putMetrics(MetricsRecord record) {
+    // The method handles both cases whether Ganglia support dense publish of
+    // metrics of sparse (only on change) publish of metrics
+    try {
+      String recordName = record.name();
+      String contextName = record.context();
+
+      StringBuilder sb = new StringBuilder();
+      sb.append(contextName);
+      sb.append('.');
+      sb.append(recordName);
+
+      String groupName = sb.toString();
+      sb.append('.');
+      int sbBaseLen = sb.length();
+
+      String type = null;
+      GangliaSlope slopeFromMetric = null;
+      GangliaSlope calculatedSlope = null;
+      Record cachedMetrics = null;
+      if (!isSupportSparseMetrics()) {
+        // for sending dense metrics, update metrics cache
+        // and get the updated data
+        cachedMetrics = metricsCache.update(record);
+
+        if (cachedMetrics != null && cachedMetrics.metricsEntrySet() != null) {
+          for (Map.Entry<String, Metric> entry : cachedMetrics.metricsEntrySet()) {
+            Metric metric = entry.getValue();
+            sb.append(metric.name());
+            String name = sb.toString();
+
+            // visit the metric to identify the Ganglia type and slope
+            metric.visit(gangliaMetricVisitor);
+            type = gangliaMetricVisitor.getType();
+            slopeFromMetric = gangliaMetricVisitor.getSlope();
+
+
+            GangliaConf gConf = getGangliaConfForMetric(name);
+            calculatedSlope = calculateSlope(gConf, slopeFromMetric);
+
+            // send metric to Ganglia
+            emitMetric(groupName, name, type, metric.value().toString(),
+                gConf, calculatedSlope);
+
+            // reset the length of the buffer for next iteration
+            sb.setLength(sbBaseLen);
+          }
+        }
+      } else {
+        // we support sparse updates
+
+        Collection<Metric> metrics = (Collection<Metric>) record.metrics();
+        if (metrics.size() > 0) {
+          // we got metrics. so send the latest
+          for (Metric metric : record.metrics()) {
+            sb.append(metric.name());
+            String name = sb.toString();
+
+            // visit the metric to identify the Ganglia type and slope
+            metric.visit(gangliaMetricVisitor);
+            type = gangliaMetricVisitor.getType();
+            slopeFromMetric = gangliaMetricVisitor.getSlope();
+
+
+            GangliaConf gConf = getGangliaConfForMetric(name);
+            calculatedSlope = calculateSlope(gConf, slopeFromMetric);
+
+            // send metric to Ganglia
+            emitMetric(groupName, name, type, metric.value().toString(),
+                gConf, calculatedSlope);
+
+            // reset the length of the buffer for next iteration
+            sb.setLength(sbBaseLen);
+          }
+        }
+      }
+    } catch (IOException io) {
+      throw new MetricsException("Failed to putMetrics", io);
+    }
+  }
+
+
+  /**
+   * Calculate the slope from properties and metric
+   *
+   * @param gConf Pass
+   * @param slopeFromMetric
+   * @return
+   */
+  private GangliaSlope calculateSlope(GangliaConf gConf, GangliaSlope slopeFromMetric) {
+    if (gConf.getSlope() != null) {
+      // if slope has been specified in properties, use that
+      return gConf.getSlope();
+    } else if (slopeFromMetric != null) {
+      // slope not specified in properties, use derived from Metric
+      return slopeFromMetric;
+    } else {
+      return DEFAULT_SLOPE;
+    }
+  }
+
+  /**
+   * The method sends metrics to Ganglia servers. The method has been taken from
+   * org.apache.hadoop.metrics.ganglia.GangliaContext30 with minimal changes in
+   * order to keep it in sync.
+
+   * @param groupName The group name of the metric
+   * @param name The metric name
+   * @param type The type of the metric
+   * @param value The value of the metric
+   * @param gConf The GangliaConf for this metric
+   * @param gSlope The slope for this metric
+   * @throws IOException
+   */
+  protected void emitMetric(String groupName, String name, String type,
+      String value, GangliaConf gConf, GangliaSlope gSlope)
+    throws IOException {
+
+    if (name == null) {
+      LOG.warn("Metric was emitted with no name.");
+      return;
+    } else if (value == null) {
+      LOG.warn("Metric name " + name + " was emitted with a null value.");
+      return;
+    } else if (type == null) {
+      LOG.warn("Metric name " + name + ", value " + value + " has no type.");
+      return;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Emitting metric " + name + ", type " + type + ", value " + value
+          + ", slope " + gSlope.name()+ " from hostname " + getHostName());
+    }
+
+    xdr_int(0); // metric_user_defined
+    xdr_string(type);
+    xdr_string(name);
+    xdr_string(value);
+    xdr_string(gConf.getUnits());
+    xdr_int(gSlope.ordinal());
+    xdr_int(gConf.getTmax());
+    xdr_int(gConf.getDmax());
+
+    // send the metric to Ganglia hosts
+    emitToGangliaHosts();
+  }
+}

+ 104 - 0
src/core/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink31.java

@@ -0,0 +1,104 @@
+/**
+ * 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.metrics2.sink.ganglia;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * This code supports Ganglia 3.1
+ *
+ */
+public class GangliaSink31 extends GangliaSink30 {
+
+  public final Log LOG = LogFactory.getLog(this.getClass());
+
+  /**
+   * The method sends metrics to Ganglia servers. The method has been taken from
+   * org.apache.hadoop.metrics.ganglia.GangliaContext31 with minimal changes in
+   * order to keep it in sync.
+
+  * @param groupName The group name of the metric
+   * @param name The metric name
+   * @param type The type of the metric
+   * @param value The value of the metric
+   * @param gConf The GangliaConf for this metric
+   * @param gSlope The slope for this metric
+   * @throws IOException
+   */
+  protected void emitMetric(String groupName, String name, String type,
+      String value, GangliaConf gConf, GangliaSlope gSlope)
+    throws IOException {
+
+    if (name == null) {
+      LOG.warn("Metric was emitted with no name.");
+      return;
+    } else if (value == null) {
+      LOG.warn("Metric name " + name +" was emitted with a null value.");
+      return;
+    } else if (type == null) {
+      LOG.warn("Metric name " + name + ", value " + value + " has no type.");
+      return;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Emitting metric " + name + ", type " + type + ", value " + value
+          + ", slope " + gSlope.name()+ " from hostname " + getHostName());
+    }
+
+    // The following XDR recipe was done through a careful reading of
+    // gm_protocol.x in Ganglia 3.1 and carefully examining the output of
+    // the gmetric utility with strace.
+
+    // First we send out a metadata message
+    xdr_int(128);               // metric_id = metadata_msg
+    xdr_string(getHostName());       // hostname
+    xdr_string(name);           // metric name
+    xdr_int(0);                 // spoof = False
+    xdr_string(type);           // metric type
+    xdr_string(name);           // metric name
+    xdr_string(gConf.getUnits());    // units
+    xdr_int(gSlope.ordinal());  // slope
+    xdr_int(gConf.getTmax());        // tmax, the maximum time between metrics
+    xdr_int(gConf.getDmax());        // dmax, the maximum data value
+    xdr_int(1);                 /*Num of the entries in extra_value field for
+                                  Ganglia 3.1.x*/
+    xdr_string("GROUP");        /*Group attribute*/
+    xdr_string(groupName);      /*Group value*/
+
+    // send the metric to Ganglia hosts
+    emitToGangliaHosts();
+
+    // Now we send out a message with the actual value.
+    // Technically, we only need to send out the metadata message once for
+    // each metric, but I don't want to have to record which metrics we did and
+    // did not send.
+    xdr_int(133);         // we are sending a string value
+    xdr_string(getHostName()); // hostName
+    xdr_string(name);     // metric name
+    xdr_int(0);           // spoof = False
+    xdr_string("%s");     // format field
+    xdr_string(value);    // metric value
+
+    // send the metric to Ganglia hosts
+    emitToGangliaHosts();
+  }
+}

+ 24 - 3
src/core/org/apache/hadoop/metrics2/util/MetricsCache.java

@@ -45,7 +45,7 @@ public class MetricsCache {
    */
    */
   public static class Record {
   public static class Record {
     final Map<String, String> tags = new LinkedHashMap<String, String>();
     final Map<String, String> tags = new LinkedHashMap<String, String>();
-    final Map<String, Number> metrics = new LinkedHashMap<String, Number>();
+    final Map<String, Metric> metrics = new LinkedHashMap<String, Metric>();
 
 
     /**
     /**
      * Get the tag value
      * Get the tag value
@@ -62,6 +62,16 @@ public class MetricsCache {
      * @return the metric value
      * @return the metric value
      */
      */
     public Number getMetric(String key) {
     public Number getMetric(String key) {
+      Metric metric = metrics.get(key);
+      return metric != null ? metric.value() : null;
+    }
+
+    /**
+     * Get the metric value
+     * @param key name of the metric
+     * @return the metric value
+     */
+    public Metric getMetricInstance(String key) {
       return metrics.get(key);
       return metrics.get(key);
     }
     }
 
 
@@ -69,6 +79,18 @@ public class MetricsCache {
      * @return entry set of metrics
      * @return entry set of metrics
      */
      */
     public Set<Map.Entry<String, Number>> metrics() {
     public Set<Map.Entry<String, Number>> metrics() {
+      Map<String, Number> map =
+        new LinkedHashMap<String,Number>(metrics.size());
+      for (Map.Entry<String, Metric> mapEntry : metrics.entrySet()) {
+        map.put(mapEntry.getKey(), mapEntry.getValue().value());
+      }
+      return map.entrySet();
+    }
+
+    /**
+     * @return entry set of metrics
+     */
+    public Set<Map.Entry<String, Metric>> metricsEntrySet() {
       return metrics.entrySet();
       return metrics.entrySet();
     }
     }
   }
   }
@@ -93,7 +115,7 @@ public class MetricsCache {
       recMap.put(tags, rec);
       recMap.put(tags, rec);
     }
     }
     for (Metric m : mr.metrics()) {
     for (Metric m : mr.metrics()) {
-      rec.metrics.put(m.name(), m.value());
+      rec.metrics.put(m.name(), m);
     }
     }
     if (includingTags) {
     if (includingTags) {
       // mostly for some sinks that include tags as part of a dense schema
       // mostly for some sinks that include tags as part of a dense schema
@@ -119,5 +141,4 @@ public class MetricsCache {
     if (tmap == null) return null;
     if (tmap == null) return null;
     return tmap.get(tags);
     return tmap.get(tags);
   }
   }
-
 }
 }

+ 43 - 20
src/core/org/apache/hadoop/net/CachedDNSToSwitchMapping.java

@@ -37,16 +37,11 @@ public class CachedDNSToSwitchMapping implements DNSToSwitchMapping {
     this.rawMapping = rawMapping;
     this.rawMapping = rawMapping;
   }
   }
   
   
-  public List<String> resolve(List<String> names) {
-    // normalize all input names to be in the form of IP addresses
-    names = NetUtils.normalizeHostNames(names);
-    
-    List <String> result = new ArrayList<String>(names.size());
-    if (names.isEmpty()) {
-      return result;
-    }
-
-
+  
+  /**
+   * Returns the hosts from 'names' that have not been cached previously
+   */
+  private List<String> getUncachedHosts(List<String> names) {
     // find out all names without cached resolved location
     // find out all names without cached resolved location
     List<String> unCachedHosts = new ArrayList<String>(names.size());
     List<String> unCachedHosts = new ArrayList<String>(names.size());
     for (String name : names) {
     for (String name : names) {
@@ -54,27 +49,55 @@ public class CachedDNSToSwitchMapping implements DNSToSwitchMapping {
         unCachedHosts.add(name);
         unCachedHosts.add(name);
       } 
       } 
     }
     }
-    
-    // Resolve those names
-    List<String> rNames = rawMapping.resolve(unCachedHosts);
-    
+    return unCachedHosts;
+  }
+  
+  /**
+   * Caches the resolved hosts
+   */
+  private void cacheResolvedHosts(List<String> uncachedHosts, 
+      List<String> resolvedHosts) {
     // Cache the result
     // Cache the result
-    if (rNames != null) {
-      for (int i=0; i<unCachedHosts.size(); i++) {
-        cache.put(unCachedHosts.get(i), rNames.get(i));
+    if (resolvedHosts != null) {
+      for (int i=0; i<uncachedHosts.size(); i++) {
+        cache.put(uncachedHosts.get(i), resolvedHosts.get(i));
       }
       }
     }
     }
-    
+  }
+  
+  /**
+   * Returns the cached resolution of the list of hostnames/addresses.
+   * Returns null if any of the names are not currently in the cache
+   */
+  private List<String> getCachedHosts(List<String> names) {
+    List<String> result = new ArrayList<String>(names.size());
     // Construct the result
     // Construct the result
     for (String name : names) {
     for (String name : names) {
-      //now everything is in the cache
       String networkLocation = cache.get(name);
       String networkLocation = cache.get(name);
       if (networkLocation != null) {
       if (networkLocation != null) {
         result.add(networkLocation);
         result.add(networkLocation);
-      } else { //resolve all or nothing
+      } else {
         return null;
         return null;
       }
       }
     }
     }
     return result;
     return result;
   }
   }
+
+  public List<String> resolve(List<String> names) {
+    // normalize all input names to be in the form of IP addresses
+    names = NetUtils.normalizeHostNames(names);
+
+    List <String> result = new ArrayList<String>(names.size());
+    if (names.isEmpty()) {
+      return result;
+    }
+
+    List<String> uncachedHosts = this.getUncachedHosts(names);
+
+    // Resolve the uncached hosts
+    List<String> resolvedHosts = rawMapping.resolve(uncachedHosts);
+    this.cacheResolvedHosts(uncachedHosts, resolvedHosts);
+    return this.getCachedHosts(names);
+
+  }
 }
 }

+ 7 - 0
src/core/org/apache/hadoop/net/DNSToSwitchMapping.java

@@ -18,12 +18,18 @@
 package org.apache.hadoop.net;
 package org.apache.hadoop.net;
 
 
 import java.util.List;
 import java.util.List;
+import java.net.UnknownHostException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 
 
 /**
 /**
  * An interface that should be implemented to allow pluggable 
  * An interface that should be implemented to allow pluggable 
  * DNS-name/IP-address to RackID resolvers.
  * DNS-name/IP-address to RackID resolvers.
  *
  *
  */
  */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 public interface DNSToSwitchMapping {
 public interface DNSToSwitchMapping {
   /**
   /**
    * Resolves a list of DNS-names/IP-addresses and returns back a list of
    * Resolves a list of DNS-names/IP-addresses and returns back a list of
@@ -39,4 +45,5 @@ public interface DNSToSwitchMapping {
    * @return list of resolved network paths
    * @return list of resolved network paths
    */
    */
   public List<String> resolve(List<String> names);
   public List<String> resolve(List<String> names);
+
 }
 }

+ 47 - 1
src/core/org/apache/hadoop/net/NetUtils.java

@@ -27,7 +27,9 @@ import java.net.Socket;
 import java.net.SocketAddress;
 import java.net.SocketAddress;
 import java.net.SocketException;
 import java.net.SocketException;
 import java.net.URI;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
+import java.net.ConnectException;
 import java.nio.channels.SocketChannel;
 import java.nio.channels.SocketChannel;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.util.*;
 import java.util.*;
@@ -405,6 +407,21 @@ public class NetUtils {
     } else {
     } else {
       SocketIOWithTimeout.connect(ch, endpoint, timeout);
       SocketIOWithTimeout.connect(ch, endpoint, timeout);
     }
     }
+
+    // There is a very rare case allowed by the TCP specification, such that
+    // if we are trying to connect to an endpoint on the local machine,
+    // and we end up choosing an ephemeral port equal to the destination port,
+    // we will actually end up getting connected to ourself (ie any data we
+    // send just comes right back). This is only possible if the target
+    // daemon is down, so we'll treat it like connection refused.
+    if (socket.getLocalPort() == socket.getPort() &&
+        socket.getLocalAddress().equals(socket.getInetAddress())) {
+      LOG.info("Detected a loopback TCP socket, disconnecting it");
+      socket.close();
+      throw new ConnectException(
+        "Localhost targeted connection resulted in a loopback. " +
+        "No daemon is listening on the target port.");
+    }
   }
   }
   
   
   /** 
   /** 
@@ -443,7 +460,36 @@ public class NetUtils {
     }
     }
     return hostNames;
     return hostNames;
   }
   }
-  
+
+  /**
+   * Performs a sanity check on the list of hostnames/IPs to verify they at least
+   * appear to be valid.
+   * @param names - List of hostnames/IPs
+   * @throws UnknownHostException
+   */
+  public static void verifyHostnames(String[] names) throws UnknownHostException {
+    for (String name: names) {
+      if (name == null) {
+        throw new UnknownHostException("null hostname found");
+      }
+      // The first check supports URL formats (e.g. hdfs://, etc.). 
+      // java.net.URI requires a schema, so we add a dummy one if it doesn't
+      // have one already.
+      URI uri = null;
+      try {
+        uri = new URI(name);
+        if (uri.getHost() == null) {
+          uri = new URI("http://" + name);
+        }
+      } catch (URISyntaxException e) {
+        uri = null;
+      }
+      if (uri == null || uri.getHost() == null) {
+        throw new UnknownHostException(name + " is not a valid Inet address");
+      }
+    }
+  }
+
   /**
   /**
    * Checks if {@code host} is a local host name and return {@link InetAddress}
    * Checks if {@code host} is a local host name and return {@link InetAddress}
    * corresponding to that address.
    * corresponding to that address.

+ 1 - 1
src/core/org/apache/hadoop/net/ScriptBasedMapping.java

@@ -114,7 +114,7 @@ implements Configurable
     
     
     return m;
     return m;
   }
   }
-  
+
   private String runResolveCommand(List<String> args) {
   private String runResolveCommand(List<String> args) {
     int loopCount = 0;
     int loopCount = 0;
     if (args.size() == 0) {
     if (args.size() == 0) {

+ 75 - 0
src/core/org/apache/hadoop/security/AuthenticationFilterInitializer.java

@@ -0,0 +1,75 @@
+/**
+ * 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.security;
+
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.http.FilterInitializer;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Initializes Alfredo AuthenticationFilter which provides support for
+ * Kerberos HTTP SPENGO authentication.
+ * <p/>
+ * It enables anonymous access, simple/speudo and Kerberos HTTP SPNEGO
+ * authentication  for Hadoop JobTracker, NameNode, DataNodes and
+ * TaskTrackers.
+ * <p/>
+ * Refer to the <code>core-default.xml</code> file, after the comment
+ * 'HTTP Authentication' for details on the configuration options.
+ * All related configuration properties have 'hadoop.http.authentication.'
+ * as prefix.
+ */
+public class AuthenticationFilterInitializer extends FilterInitializer {
+
+  private static final String PREFIX = "hadoop.http.authentication.";
+
+  /**
+   * Initializes Alfredo AuthenticationFilter.
+   * <p/>
+   * Propagates to Alfredo AuthenticationFilter configuration all Hadoop
+   * configuration properties prefixed with "hadoop.http.authentication."
+   *
+   * @param container The filter container
+   * @param conf Configuration for run-time parameters
+   */
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    Map<String, String> filterConfig = new HashMap<String, String>();
+
+    //setting the cookie path to root '/' so it is used for all resources.
+    filterConfig.put(AuthenticationFilter.COOKIE_PATH, "/");
+
+    for (Map.Entry<String, String> entry : conf) {
+      String name = entry.getKey();
+      if (name.startsWith(PREFIX)) {
+        String value = conf.get(name);
+        name = name.substring(PREFIX.length());
+        filterConfig.put(name, value);
+      }
+    }
+
+    container.addFilter("authentication",
+                        AuthenticationFilter.class.getName(),
+                        filterConfig);
+  }
+
+}

+ 47 - 16
src/core/org/apache/hadoop/security/SecurityUtil.java

@@ -18,6 +18,7 @@ package org.apache.hadoop.security;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URL;
 import java.net.URL;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
@@ -30,8 +31,10 @@ import javax.security.auth.kerberos.KerberosTicket;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.security.token.Token;
 
 
 import sun.security.jgss.krb5.Krb5Util;
 import sun.security.jgss.krb5.Krb5Util;
 import sun.security.krb5.Credentials;
 import sun.security.krb5.Credentials;
@@ -231,28 +234,56 @@ public class SecurityUtil {
     UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename);
     UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename);
   }
   }
   
   
+  /**
+   * Set the given token's service to the format expected by the RPC client 
+   * @param token a delegation token
+   * @param addr the socket for the rpc connection
+   */
+  public static void setTokenService(Token<?> token, InetSocketAddress addr) {
+    token.setService(buildTokenService(addr));
+  }
+  
+  /**
+   * Construct the service key for a token
+   * @param addr the socket for the rpc connection
+   * @return Text formatted for the service field in a token 
+   */
+  public static Text buildTokenService(InetSocketAddress addr) {
+    return new Text(buildDTAuthority(addr));
+  }
+  
   /**
   /**
    * create service name for Delegation token ip:port
    * create service name for Delegation token ip:port
    * @param uri
    * @param uri
    * @return "ip:port"
    * @return "ip:port"
    */
    */
   public static String buildDTServiceName(URI uri, int defPort) {
   public static String buildDTServiceName(URI uri, int defPort) {
-    int port = uri.getPort();
-    if(port == -1) 
-      port = defPort;
-    
-    // build the service name string "/ip:port"
-    // for whatever reason using NetUtils.createSocketAddr(target).toString()
-    // returns "localhost/ip:port"
-    StringBuffer sb = new StringBuffer();
-    String host = uri.getHost();
-    if (host != null) {
-      host = NetUtils.normalizeHostName(host);
-    } else {
-      host = "";
-    }
-    sb.append(host).append(":").append(port);
-    return sb.toString();
+    InetSocketAddress addr = NetUtils.createSocketAddr(uri.getAuthority(),
+                                                       defPort);
+    return buildDTAuthority(addr);
+   }
+  
+  /**
+   * create an authority name for looking up a Delegation token based
+   * on a socket
+   * @param addr InetSocketAddress of remote connection with a token
+   * @return "ip:port"
+   */
+  static String buildDTAuthority(InetSocketAddress addr) {
+    String host= addr.getAddress().getHostAddress();
+    return buildDTAuthority(host, addr.getPort());
+  }
+  
+  /**
+   * create an authority name for looking up a Delegation token based
+   * on a host/ip pair
+   * @param host the remote host
+   * @param port the remote port
+   * @return "ip:port"
+   */
+  static String buildDTAuthority(String host, int port) {
+    host = (host != null) ? NetUtils.normalizeHostName(host) : "";
+    return host + ":" + port;
   }
   }
   
   
   /**
   /**

+ 15 - 0
src/core/org/apache/hadoop/security/authentication/README.txt

@@ -0,0 +1,15 @@
+Hadoop Auth, Java HTTP SPNEGO
+
+Hadoop Auth consists of a client and a server
+components to enable Kerberos SPNEGO authentication for HTTP.
+
+The client component is the AuthenticatedURL class.
+
+The server component is the AuthenticationFilter servlet filter class.
+
+Authentication mechanisms support is pluggable in both the client and
+the server components via interfaces.
+
+In addition to Kerberos SPNEGO, Hadoop Auth also supports Pseudo/Simple
+authentication (trusting the value of the query string parameter
+'user.name').

+ 274 - 0
src/core/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java

@@ -0,0 +1,274 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.client;
+
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The {@link AuthenticatedURL} class enables the use of the JDK {@link URL} class
+ * against HTTP endpoints protected with the {@link AuthenticationFilter}.
+ * <p/>
+ * The authentication mechanisms supported by default are Hadoop Simple  authentication
+ * (also known as pseudo authentication) and Kerberos SPNEGO authentication.
+ * <p/>
+ * Additional authentication mechanisms can be supported via {@link Authenticator} implementations.
+ * <p/>
+ * The default {@link Authenticator} is the {@link KerberosAuthenticator} class which supports
+ * automatic fallback from Kerberos SPNEGO to Hadoop Simple authentication.
+ * <p/>
+ * <code>AuthenticatedURL</code> instances are not thread-safe.
+ * <p/>
+ * The usage pattern of the {@link AuthenticatedURL} is:
+ * <p/>
+ * <pre>
+ *
+ * // establishing an initial connection
+ *
+ * URL url = new URL("http://foo:8080/bar");
+ * AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+ * AuthenticatedURL aUrl = new AuthenticatedURL();
+ * HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+ * ....
+ * // use the 'conn' instance
+ * ....
+ *
+ * // establishing a follow up connection using a token from the previous connection
+ *
+ * HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+ * ....
+ * // use the 'conn' instance
+ * ....
+ *
+ * </pre>
+ */
+public class AuthenticatedURL {
+
+  /**
+   * Name of the HTTP cookie used for the authentication token between the client and the server.
+   */
+  public static final String AUTH_COOKIE = "hadoop.auth";
+
+  private static final String AUTH_COOKIE_EQ = AUTH_COOKIE + "=";
+
+  /**
+   * Client side authentication token.
+   */
+  public static class Token {
+
+    private String token;
+
+    /**
+     * Creates a token.
+     */
+    public Token() {
+    }
+
+    /**
+     * Creates a token using an existing string representation of the token.
+     *
+     * @param tokenStr string representation of the tokenStr.
+     */
+    public Token(String tokenStr) {
+      if (tokenStr == null) {
+        throw new IllegalArgumentException("tokenStr cannot be null");
+      }
+      set(tokenStr);
+    }
+
+    /**
+     * Returns if a token from the server has been set.
+     *
+     * @return if a token from the server has been set.
+     */
+    public boolean isSet() {
+      return token != null;
+    }
+
+    /**
+     * Sets a token.
+     *
+     * @param tokenStr string representation of the tokenStr.
+     */
+    void set(String tokenStr) {
+      token = tokenStr;
+    }
+
+    /**
+     * Returns the string representation of the token.
+     *
+     * @return the string representation of the token.
+     */
+    @Override
+    public String toString() {
+      return token;
+    }
+
+    /**
+     * Return the hashcode for the token.
+     *
+     * @return the hashcode for the token.
+     */
+    @Override
+    public int hashCode() {
+      return (token != null) ? token.hashCode() : 0;
+    }
+
+    /**
+     * Return if two token instances are equal.
+     *
+     * @param o the other token instance.
+     *
+     * @return if this instance and the other instance are equal.
+     */
+    @Override
+    public boolean equals(Object o) {
+      boolean eq = false;
+      if (o instanceof Token) {
+        Token other = (Token) o;
+        eq = (token == null && other.token == null) || (token != null && this.token.equals(other.token));
+      }
+      return eq;
+    }
+  }
+
+  private static Class<? extends Authenticator> DEFAULT_AUTHENTICATOR = KerberosAuthenticator.class;
+
+  /**
+   * Sets the default {@link Authenticator} class to use when an {@link AuthenticatedURL} instance
+   * is created without specifying an authenticator.
+   *
+   * @param authenticator the authenticator class to use as default.
+   */
+  public static void setDefaultAuthenticator(Class<? extends Authenticator> authenticator) {
+    DEFAULT_AUTHENTICATOR = authenticator;
+  }
+
+  /**
+   * Returns the default {@link Authenticator} class to use when an {@link AuthenticatedURL} instance
+   * is created without specifying an authenticator.
+   *
+   * @return the authenticator class to use as default.
+   */
+  public static Class<? extends Authenticator> getDefaultAuthenticator() {
+    return DEFAULT_AUTHENTICATOR;
+  }
+
+  private Authenticator authenticator;
+
+  /**
+   * Creates an {@link AuthenticatedURL}.
+   */
+  public AuthenticatedURL() {
+    this(null);
+  }
+
+  /**
+   * Creates an <code>AuthenticatedURL</code>.
+   *
+   * @param authenticator the {@link Authenticator} instance to use, if <code>null</code> a {@link
+   * KerberosAuthenticator} is used.
+   */
+  public AuthenticatedURL(Authenticator authenticator) {
+    try {
+      this.authenticator = (authenticator != null) ? authenticator : DEFAULT_AUTHENTICATOR.newInstance();
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  /**
+   * Returns an authenticated {@link HttpURLConnection}.
+   *
+   * @param url the URL to connect to. Only HTTP/S URLs are supported.
+   * @param token the authentication token being used for the user.
+   *
+   * @return an authenticated {@link HttpURLConnection}.
+   *
+   * @throws IOException if an IO error occurred.
+   * @throws AuthenticationException if an authentication exception occurred.
+   */
+  public HttpURLConnection openConnection(URL url, Token token) throws IOException, AuthenticationException {
+    if (url == null) {
+      throw new IllegalArgumentException("url cannot be NULL");
+    }
+    if (!url.getProtocol().equalsIgnoreCase("http") && !url.getProtocol().equalsIgnoreCase("https")) {
+      throw new IllegalArgumentException("url must be for a HTTP or HTTPS resource");
+    }
+    if (token == null) {
+      throw new IllegalArgumentException("token cannot be NULL");
+    }
+    authenticator.authenticate(url, token);
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    injectToken(conn, token);
+    return conn;
+  }
+
+  /**
+   * Helper method that injects an authentication token to send with a connection.
+   *
+   * @param conn connection to inject the authentication token into.
+   * @param token authentication token to inject.
+   */
+  public static void injectToken(HttpURLConnection conn, Token token) {
+    String t = token.token;
+    if (t != null) {
+      if (!t.startsWith("\"")) {
+        t = "\"" + t + "\"";
+      }
+      conn.addRequestProperty("Cookie", AUTH_COOKIE_EQ + t);
+    }
+  }
+
+  /**
+   * Helper method that extracts an authentication token received from a connection.
+   * <p/>
+   * This method is used by {@link Authenticator} implementations.
+   *
+   * @param conn connection to extract the authentication token from.
+   * @param token the authentication token.
+   *
+   * @throws IOException if an IO error occurred.
+   * @throws AuthenticationException if an authentication exception occurred.
+   */
+  public static void extractToken(HttpURLConnection conn, Token token) throws IOException, AuthenticationException {
+    if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
+      Map<String, List<String>> headers = conn.getHeaderFields();
+      List<String> cookies = headers.get("Set-Cookie");
+      if (cookies != null) {
+        for (String cookie : cookies) {
+          if (cookie.startsWith(AUTH_COOKIE_EQ)) {
+            String value = cookie.substring(AUTH_COOKIE_EQ.length());
+            int separator = value.indexOf(";");
+            if (separator > -1) {
+              value = value.substring(0, separator);
+            }
+            if (value.length() > 0) {
+              token.set(value);
+            }
+          }
+        }
+      }
+    } else {
+      throw new AuthenticationException("Authentication failed, status: " + conn.getResponseCode() +
+                                        ", message: " + conn.getResponseMessage());
+    }
+  }
+
+}

+ 50 - 0
src/core/org/apache/hadoop/security/authentication/client/AuthenticationException.java

@@ -0,0 +1,50 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.client;
+
+/**
+ * Exception thrown when an authentication error occurrs.
+ */
+public class AuthenticationException extends Exception {
+  
+  static final long serialVersionUID = 0;
+
+  /**
+   * Creates an {@link AuthenticationException}.
+   *
+   * @param cause original exception.
+   */
+  public AuthenticationException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * Creates an {@link AuthenticationException}.
+   *
+   * @param msg exception message.
+   */
+  public AuthenticationException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * Creates an {@link AuthenticationException}.
+   *
+   * @param msg exception message.
+   * @param cause original exception.
+   */
+  public AuthenticationException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+}

+ 39 - 0
src/core/org/apache/hadoop/security/authentication/client/Authenticator.java

@@ -0,0 +1,39 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.client;
+
+
+import java.io.IOException;
+import java.net.URL;
+
+/**
+ * Interface for client authentication mechanisms.
+ * <p/>
+ * Implementations are use-once instances, they don't need to be thread safe.
+ */
+public interface Authenticator {
+
+  /**
+   * Authenticates against a URL and returns a {@link AuthenticatedURL.Token} to be
+   * used by subsequent requests.
+   *
+   * @param url the URl to authenticate against.
+   * @param token the authentication token being used for the user.
+   *
+   * @throws IOException if an IO error occurred.
+   * @throws AuthenticationException if an authentication error occurred.
+   */
+  public void authenticate(URL url, AuthenticatedURL.Token token) throws IOException, AuthenticationException;
+
+}

+ 270 - 0
src/core/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java

@@ -0,0 +1,270 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.client;
+
+import com.sun.security.auth.module.Krb5LoginModule;
+import org.apache.commons.codec.binary.Base64;
+import org.ietf.jgss.GSSContext;
+import org.ietf.jgss.GSSManager;
+import org.ietf.jgss.GSSName;
+import sun.security.jgss.GSSUtil;
+
+import javax.security.auth.Subject;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.security.AccessControlContext;
+import java.security.AccessController;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The {@link KerberosAuthenticator} implements the Kerberos SPNEGO authentication sequence.
+ * <p/>
+ * It uses the default principal for the Kerberos cache (normally set via kinit).
+ * <p/>
+ * It falls back to the {@link PseudoAuthenticator} if the HTTP endpoint does not trigger an SPNEGO authentication
+ * sequence.
+ */
+public class KerberosAuthenticator implements Authenticator {
+
+  /**
+   * HTTP header used by the SPNEGO server endpoint during an authentication sequence.
+   */
+  public static final String WWW_AUTHENTICATE = "WWW-Authenticate";
+
+  /**
+   * HTTP header used by the SPNEGO client endpoint during an authentication sequence.
+   */
+  public static final String AUTHORIZATION = "Authorization";
+
+  /**
+   * HTTP header prefix used by the SPNEGO client/server endpoints during an authentication sequence.
+   */
+  public static final String NEGOTIATE = "Negotiate";
+
+  private static final String AUTH_HTTP_METHOD = "OPTIONS";
+
+  /*
+  * Defines the Kerberos configuration that will be used to obtain the Kerberos principal from the
+  * Kerberos cache.
+  */
+  private static class KerberosConfiguration extends Configuration {
+
+    private static final String OS_LOGIN_MODULE_NAME;
+    private static final boolean windows = System.getProperty("os.name").startsWith("Windows");
+
+    static {
+      if (windows) {
+        OS_LOGIN_MODULE_NAME = "com.sun.security.auth.module.NTLoginModule";
+      } else {
+        OS_LOGIN_MODULE_NAME = "com.sun.security.auth.module.UnixLoginModule";
+      }
+    }
+
+    private static final AppConfigurationEntry OS_SPECIFIC_LOGIN =
+      new AppConfigurationEntry(OS_LOGIN_MODULE_NAME,
+                                AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                                new HashMap<String, String>());
+
+    private static final Map<String, String> USER_KERBEROS_OPTIONS = new HashMap<String, String>();
+
+    static {
+      USER_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+      USER_KERBEROS_OPTIONS.put("useTicketCache", "true");
+      USER_KERBEROS_OPTIONS.put("renewTGT", "true");
+      String ticketCache = System.getenv("KRB5CCNAME");
+      if (ticketCache != null) {
+        USER_KERBEROS_OPTIONS.put("ticketCache", ticketCache);
+      }
+    }
+
+    private static final AppConfigurationEntry USER_KERBEROS_LOGIN =
+      new AppConfigurationEntry(Krb5LoginModule.class.getName(),
+                                AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL,
+                                USER_KERBEROS_OPTIONS);
+
+    private static final AppConfigurationEntry[] USER_KERBEROS_CONF =
+      new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN, USER_KERBEROS_LOGIN};
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
+      return USER_KERBEROS_CONF;
+    }
+  }
+
+  static {
+    javax.security.auth.login.Configuration.setConfiguration(new KerberosConfiguration());
+  }
+
+  private URL url;
+  private HttpURLConnection conn;
+  private Base64 base64;
+
+  /**
+   * Performs SPNEGO authentication against the specified URL.
+   * <p/>
+   * If a token is given it does a NOP and returns the given token.
+   * <p/>
+   * If no token is given, it will perform the SPNEGO authentication sequence using an
+   * HTTP <code>OPTIONS</code> request.
+   *
+   * @param url the URl to authenticate against.
+   * @param token the authentication token being used for the user.
+   *
+   * @throws IOException if an IO error occurred.
+   * @throws AuthenticationException if an authentication error occurred.
+   */
+  @Override
+  public void authenticate(URL url, AuthenticatedURL.Token token)
+    throws IOException, AuthenticationException {
+    if (!token.isSet()) {
+      this.url = url;
+      base64 = new Base64(0);
+      conn = (HttpURLConnection) url.openConnection();
+      conn.setRequestMethod(AUTH_HTTP_METHOD);
+      conn.connect();
+      if (isNegotiate()) {
+        doSpnegoSequence(token);
+      } else {
+        getFallBackAuthenticator().authenticate(url, token);
+      }
+    }
+  }
+
+  /**
+   * If the specified URL does not support SPNEGO authentication, a fallback {@link Authenticator} will be used.
+   * <p/>
+   * This implementation returns a {@link PseudoAuthenticator}.
+   *
+   * @return the fallback {@link Authenticator}.
+   */
+  protected Authenticator getFallBackAuthenticator() {
+    return new PseudoAuthenticator();
+  }
+
+  /*
+  * Indicates if the response is starting a SPNEGO negotiation.
+  */
+  private boolean isNegotiate() throws IOException {
+    boolean negotiate = false;
+    if (conn.getResponseCode() == HttpURLConnection.HTTP_UNAUTHORIZED) {
+      String authHeader = conn.getHeaderField(WWW_AUTHENTICATE);
+      negotiate = authHeader != null && authHeader.trim().startsWith(NEGOTIATE);
+    }
+    return negotiate;
+  }
+
+  /**
+   * Implements the SPNEGO authentication sequence interaction using the current default principal
+   * in the Kerberos cache (normally set via kinit).
+   *
+   * @param token the authentication token being used for the user.
+   *
+   * @throws IOException if an IO error occurred.
+   * @throws AuthenticationException if an authentication error occurred.
+   */
+  private void doSpnegoSequence(AuthenticatedURL.Token token) throws IOException, AuthenticationException {
+    try {
+      AccessControlContext context = AccessController.getContext();
+      Subject subject = Subject.getSubject(context);
+      if (subject == null) {
+        subject = new Subject();
+        LoginContext login = new LoginContext("", subject);
+        login.login();
+      }
+      Subject.doAs(subject, new PrivilegedExceptionAction<Void>() {
+
+        @Override
+        public Void run() throws Exception {
+          GSSContext gssContext = null;
+          try {
+            GSSManager gssManager = GSSManager.getInstance();
+            String servicePrincipal = "HTTP/" + KerberosAuthenticator.this.url.getHost();
+            GSSName serviceName = gssManager.createName(servicePrincipal,
+                                                        GSSUtil.NT_GSS_KRB5_PRINCIPAL);
+            gssContext = gssManager.createContext(serviceName, GSSUtil.GSS_KRB5_MECH_OID, null,
+                                                  GSSContext.DEFAULT_LIFETIME);
+            gssContext.requestCredDeleg(true);
+            gssContext.requestMutualAuth(true);
+
+            byte[] inToken = new byte[0];
+            byte[] outToken;
+            boolean established = false;
+
+            // Loop while the context is still not established
+            while (!established) {
+              outToken = gssContext.initSecContext(inToken, 0, inToken.length);
+              if (outToken != null) {
+                sendToken(outToken);
+              }
+
+              if (!gssContext.isEstablished()) {
+                inToken = readToken();
+              } else {
+                established = true;
+              }
+            }
+          } finally {
+            if (gssContext != null) {
+              gssContext.dispose();
+              gssContext = null;
+            }
+          }
+          return null;
+        }
+      });
+    } catch (PrivilegedActionException ex) {
+      throw new AuthenticationException(ex.getException());
+    } catch (LoginException ex) {
+      throw new AuthenticationException(ex);
+    }
+    AuthenticatedURL.extractToken(conn, token);
+  }
+
+  /*
+  * Sends the Kerberos token to the server.
+  */
+  private void sendToken(byte[] outToken) throws IOException, AuthenticationException {
+    String token = base64.encodeToString(outToken);
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod(AUTH_HTTP_METHOD);
+    conn.setRequestProperty(AUTHORIZATION, NEGOTIATE + " " + token);
+    conn.connect();
+  }
+
+  /*
+  * Retrieves the Kerberos token returned by the server.
+  */
+  private byte[] readToken() throws IOException, AuthenticationException {
+    int status = conn.getResponseCode();
+    if (status == HttpURLConnection.HTTP_OK || status == HttpURLConnection.HTTP_UNAUTHORIZED) {
+      String authHeader = conn.getHeaderField(WWW_AUTHENTICATE);
+      if (authHeader == null || !authHeader.trim().startsWith(NEGOTIATE)) {
+        throw new AuthenticationException("Invalid SPNEGO sequence, '" + WWW_AUTHENTICATE +
+                                          "' header incorrect: " + authHeader);
+      }
+      String negotiation = authHeader.trim().substring((NEGOTIATE + " ").length()).trim();
+      return base64.decode(negotiation);
+    }
+    throw new AuthenticationException("Invalid SPNEGO sequence, status code: " + status);
+  }
+
+}

+ 74 - 0
src/core/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java

@@ -0,0 +1,74 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.client;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+/**
+ * The {@link PseudoAuthenticator} implementation provides an authentication equivalent to Hadoop's
+ * Simple authentication, it trusts the value of the 'user.name' Java System property.
+ * <p/>
+ * The 'user.name' value is propagated using an additional query string parameter {@link #USER_NAME} ('user.name').
+ */
+public class PseudoAuthenticator implements Authenticator {
+
+  /**
+   * Name of the additional parameter that carries the 'user.name' value.
+   */
+  public static final String USER_NAME = "user.name";
+
+  private static final String USER_NAME_EQ = USER_NAME + "=";
+
+  /**
+   * Performs simple authentication against the specified URL.
+   * <p/>
+   * If a token is given it does a NOP and returns the given token.
+   * <p/>
+   * If no token is given, it will perform an HTTP <code>OPTIONS</code> request injecting an additional
+   * parameter {@link #USER_NAME} in the query string with the value returned by the {@link #getUserName()}
+   * method.
+   * <p/>
+   * If the response is successful it will update the authentication token.
+   *
+   * @param url the URl to authenticate against.
+   * @param token the authencation token being used for the user.
+   *
+   * @throws IOException if an IO error occurred.
+   * @throws AuthenticationException if an authentication error occurred.
+   */
+  @Override
+  public void authenticate(URL url, AuthenticatedURL.Token token) throws IOException, AuthenticationException {
+    String strUrl = url.toString();
+    String paramSeparator = (strUrl.contains("?")) ? "&" : "?";
+    strUrl += paramSeparator + USER_NAME_EQ + getUserName();
+    url = new URL(strUrl);
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("OPTIONS");
+    conn.connect();
+    AuthenticatedURL.extractToken(conn, token);
+  }
+
+  /**
+   * Returns the current user name.
+   * <p/>
+   * This implementation returns the value of the Java system property 'user.name'
+   *
+   * @return the current user name.
+   */
+  protected String getUserName() {
+    return System.getProperty("user.name");
+  }
+}

+ 404 - 0
src/core/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java

@@ -0,0 +1,404 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.util.Signer;
+import org.apache.hadoop.security.authentication.util.SignerException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.Cookie;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.security.Principal;
+import java.util.Enumeration;
+import java.util.Properties;
+import java.util.Random;
+
+/**
+ * The {@link AuthenticationFilter} enables protecting web application resources with different (pluggable)
+ * authentication mechanisms.
+ * <p/>
+ * Out of the box it provides 2 authentication mechanisms: Pseudo and Kerberos SPNEGO.
+ * <p/>
+ * Additional authentication mechanisms are supported via the {@link AuthenticationHandler} interface.
+ * <p/>
+ * This filter delegates to the configured authentication handler for authentication and once it obtains an
+ * {@link AuthenticationToken} from it, sets a signed HTTP cookie with the token. For client requests
+ * that provide the signed HTTP cookie, it verifies the validity of the cookie, extracts the user information
+ * and lets the request proceed to the target resource.
+ * <p/>
+ * The supported configuration properties are:
+ * <ul>
+ * <li>config.prefix: indicates the prefix to be used by all other configuration properties, the default value
+ * is no prefix. See below for details on how/why this prefix is used.</li>
+ * <li>[#PREFIX#.]type: simple|kerberos|#CLASS#, 'simple' is short for the
+ * {@link PseudoAuthenticationHandler}, 'kerberos' is short for {@link KerberosAuthenticationHandler}, otherwise
+ * the full class name of the {@link AuthenticationHandler} must be specified.</li>
+ * <li>[#PREFIX#.]signature.secret: the secret used to sign the HTTP cookie value. The default value is a random
+ * value. Unless multiple webapp instances need to share the secret the random value is adequate.</li>
+ * <li>[#PREFIX#.]token.validity: time -in seconds- that the generated token is valid before a
+ * new authentication is triggered, default value is <code>3600</code> seconds.</li>
+ * <li>[#PREFIX#.]cookie.domain: domain to use for the HTTP cookie that stores the authentication token.</li>
+ * <li>[#PREFIX#.]cookie.path: path to use for the HTTP cookie that stores the authentication token.</li>
+ * </ul>
+ * <p/>
+ * The rest of the configuration properties are specific to the {@link AuthenticationHandler} implementation and the
+ * {@link AuthenticationFilter} will take all the properties that start with the prefix #PREFIX#, it will remove
+ * the prefix from it and it will pass them to the the authentication handler for initialization. Properties that do
+ * not start with the prefix will not be passed to the authentication handler initialization.
+ */
+public class AuthenticationFilter implements Filter {
+
+  private static Logger LOG = LoggerFactory.getLogger(AuthenticationFilter.class);
+
+  /**
+   * Constant for the property that specifies the configuration prefix.
+   */
+  public static final String CONFIG_PREFIX = "config.prefix";
+
+  /**
+   * Constant for the property that specifies the authentication handler to use.
+   */
+  public static final String AUTH_TYPE = "type";
+
+  /**
+   * Constant for the property that specifies the secret to use for signing the HTTP Cookies.
+   */
+  public static final String SIGNATURE_SECRET = "signature.secret";
+
+  /**
+   * Constant for the configuration property that indicates the validity of the generated token.
+   */
+  public static final String AUTH_TOKEN_VALIDITY = "token.validity";
+
+  /**
+   * Constant for the configuration property that indicates the domain to use in the HTTP cookie.
+   */
+  public static final String COOKIE_DOMAIN = "cookie.domain";
+
+  /**
+   * Constant for the configuration property that indicates the path to use in the HTTP cookie.
+   */
+  public static final String COOKIE_PATH = "cookie.path";
+
+  private static final Random RAN = new Random();
+
+  private Signer signer;
+  private AuthenticationHandler authHandler;
+  private boolean randomSecret;
+  private long validity;
+  private String cookieDomain;
+  private String cookiePath;
+
+  /**
+   * Initializes the authentication filter.
+   * <p/>
+   * It instantiates and initializes the specified {@link AuthenticationHandler}.
+   * <p/>
+   *
+   * @param filterConfig filter configuration.
+   *
+   * @throws ServletException thrown if the filter or the authentication handler could not be initialized properly.
+   */
+  @Override
+  public void init(FilterConfig filterConfig) throws ServletException {
+    String configPrefix = filterConfig.getInitParameter(CONFIG_PREFIX);
+    configPrefix = (configPrefix != null) ? configPrefix + "." : "";
+    Properties config = getConfiguration(configPrefix, filterConfig);
+    String authHandlerName = config.getProperty(AUTH_TYPE, null);
+    String authHandlerClassName;
+    if (authHandlerName == null) {
+      throw new ServletException("Authentication type must be specified: simple|kerberos|<class>");
+    }
+    if (authHandlerName.equals("simple")) {
+      authHandlerClassName = PseudoAuthenticationHandler.class.getName();
+    } else if (authHandlerName.equals("kerberos")) {
+      authHandlerClassName = KerberosAuthenticationHandler.class.getName();
+    } else {
+      authHandlerClassName = authHandlerName;
+    }
+
+    try {
+      Class<?> klass = Thread.currentThread().getContextClassLoader().loadClass(authHandlerClassName);
+      authHandler = (AuthenticationHandler) klass.newInstance();
+      authHandler.init(config);
+    } catch (ClassNotFoundException ex) {
+      throw new ServletException(ex);
+    } catch (InstantiationException ex) {
+      throw new ServletException(ex);
+    } catch (IllegalAccessException ex) {
+      throw new ServletException(ex);
+    }
+    String signatureSecret = config.getProperty(configPrefix + SIGNATURE_SECRET);
+    if (signatureSecret == null) {
+      signatureSecret = Long.toString(RAN.nextLong());
+      randomSecret = true;
+      LOG.warn("'signature.secret' configuration not set, using a random value as secret");
+    }
+    signer = new Signer(signatureSecret.getBytes());
+    validity = Long.parseLong(config.getProperty(AUTH_TOKEN_VALIDITY, "36000")) * 1000; //10 hours
+
+    cookieDomain = config.getProperty(COOKIE_DOMAIN, null);
+    cookiePath = config.getProperty(COOKIE_PATH, null);
+  }
+
+  /**
+   * Returns the authentication handler being used.
+   *
+   * @return the authentication handler being used.
+   */
+  protected AuthenticationHandler getAuthenticationHandler() {
+    return authHandler;
+  }
+
+  /**
+   * Returns if a random secret is being used.
+   *
+   * @return if a random secret is being used.
+   */
+  protected boolean isRandomSecret() {
+    return randomSecret;
+  }
+
+  /**
+   * Returns the validity time of the generated tokens.
+   *
+   * @return the validity time of the generated tokens, in seconds.
+   */
+  protected long getValidity() {
+    return validity / 1000;
+  }
+
+  /**
+   * Returns the cookie domain to use for the HTTP cookie.
+   *
+   * @return the cookie domain to use for the HTTP cookie.
+   */
+  protected String getCookieDomain() {
+    return cookieDomain;
+  }
+
+  /**
+   * Returns the cookie path to use for the HTTP cookie.
+   *
+   * @return the cookie path to use for the HTTP cookie.
+   */
+  protected String getCookiePath() {
+    return cookiePath;
+  }
+
+  /**
+   * Destroys the filter.
+   * <p/>
+   * It invokes the {@link AuthenticationHandler#destroy()} method to release any resources it may hold.
+   */
+  @Override
+  public void destroy() {
+    if (authHandler != null) {
+      authHandler.destroy();
+      authHandler = null;
+    }
+  }
+
+  /**
+   * Returns the filtered configuration (only properties starting with the specified prefix). The property keys
+   * are also trimmed from the prefix. The returned {@link Properties} object is used to initialized the
+   * {@link AuthenticationHandler}.
+   * <p/>
+   * This method can be overriden by subclasses to obtain the configuration from other configuration source than
+   * the web.xml file.
+   *
+   * @param configPrefix configuration prefix to use for extracting configuration properties.
+   * @param filterConfig filter configuration object
+   *
+   * @return the configuration to be used with the {@link AuthenticationHandler} instance.
+   *
+   * @throws ServletException thrown if the configuration could not be created.
+   */
+  protected Properties getConfiguration(String configPrefix, FilterConfig filterConfig) throws ServletException {
+    Properties props = new Properties();
+    Enumeration<?> names = filterConfig.getInitParameterNames();
+    while (names.hasMoreElements()) {
+      String name = (String) names.nextElement();
+      if (name.startsWith(configPrefix)) {
+        String value = filterConfig.getInitParameter(name);
+        props.put(name.substring(configPrefix.length()), value);
+      }
+    }
+    return props;
+  }
+
+  /**
+   * Returns the full URL of the request including the query string.
+   * <p/>
+   * Used as a convenience method for logging purposes.
+   *
+   * @param request the request object.
+   *
+   * @return the full URL of the request including the query string.
+   */
+  protected String getRequestURL(HttpServletRequest request) {
+    StringBuffer sb = request.getRequestURL();
+    if (request.getQueryString() != null) {
+      sb.append("?").append(request.getQueryString());
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Returns the {@link AuthenticationToken} for the request.
+   * <p/>
+   * It looks at the received HTTP cookies and extracts the value of the {@link AuthenticatedURL#AUTH_COOKIE}
+   * if present. It verifies the signature and if correct it creates the {@link AuthenticationToken} and returns
+   * it.
+   * <p/>
+   * If this method returns <code>null</code> the filter will invoke the configured {@link AuthenticationHandler}
+   * to perform user authentication.
+   *
+   * @param request request object.
+   *
+   * @return the Authentication token if the request is authenticated, <code>null</code> otherwise.
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if the token is invalid or if it has expired.
+   */
+  protected AuthenticationToken getToken(HttpServletRequest request) throws IOException, AuthenticationException {
+    AuthenticationToken token = null;
+    String tokenStr = null;
+    Cookie[] cookies = request.getCookies();
+    if (cookies != null) {
+      for (Cookie cookie : cookies) {
+        if (cookie.getName().equals(AuthenticatedURL.AUTH_COOKIE)) {
+          tokenStr = cookie.getValue();
+          try {
+            tokenStr = signer.verifyAndExtract(tokenStr);
+          } catch (SignerException ex) {
+            throw new AuthenticationException(ex);
+          }
+          break;
+        }
+      }
+    }
+    if (tokenStr != null) {
+      token = AuthenticationToken.parse(tokenStr);
+      if (!token.getType().equals(authHandler.getType())) {
+        throw new AuthenticationException("Invalid AuthenticationToken type");
+      }
+      if (token.isExpired()) {
+        throw new AuthenticationException("AuthenticationToken expired");
+      }
+    }
+    return token;
+  }
+
+  /**
+   * If the request has a valid authentication token it allows the request to continue to the target resource,
+   * otherwise it triggers an authentication sequence using the configured {@link AuthenticationHandler}.
+   *
+   * @param request the request object.
+   * @param response the response object.
+   * @param filterChain the filter chain object.
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws ServletException thrown if a processing error occurred.
+   */
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response, FilterChain filterChain)
+      throws IOException, ServletException {
+    HttpServletRequest httpRequest = (HttpServletRequest) request;
+    HttpServletResponse httpResponse = (HttpServletResponse) response;
+    try {
+      boolean newToken = false;
+      AuthenticationToken token = getToken(httpRequest);
+      if (token == null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Request [{}] triggering authentication", getRequestURL(httpRequest));
+        }
+        token = authHandler.authenticate(httpRequest, httpResponse);
+        if (token != null && token != AuthenticationToken.ANONYMOUS) {
+          token.setExpires(System.currentTimeMillis() + getValidity() * 1000);
+        }
+        newToken = true;
+      }
+      if (token != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Request [{}] user [{}] authenticated", getRequestURL(httpRequest), token.getUserName());
+        }
+        final AuthenticationToken authToken = token;
+        httpRequest = new HttpServletRequestWrapper(httpRequest) {
+
+          @Override
+          public String getAuthType() {
+            return authToken.getType();
+          }
+
+          @Override
+          public String getRemoteUser() {
+            return authToken.getUserName();
+          }
+
+          @Override
+          public Principal getUserPrincipal() {
+            return (authToken != AuthenticationToken.ANONYMOUS) ? authToken : null;
+          }
+        };
+        if (newToken && token != AuthenticationToken.ANONYMOUS) {
+          String signedToken = signer.sign(token.toString());
+          Cookie cookie = createCookie(signedToken);
+          httpResponse.addCookie(cookie);
+        }
+        filterChain.doFilter(httpRequest, httpResponse);
+      }
+    } catch (AuthenticationException ex) {
+      if (!httpResponse.isCommitted()) {
+        Cookie cookie = createCookie("");
+        cookie.setMaxAge(0);
+        httpResponse.addCookie(cookie);
+        httpResponse.sendError(HttpServletResponse.SC_UNAUTHORIZED, ex.getMessage());
+      }
+      LOG.warn("Authentication exception: " + ex.getMessage(), ex);
+    }
+  }
+
+  /**
+   * Creates the Hadoop authentiation HTTP cookie.
+   * <p/>
+   * It sets the domain and path specified in the configuration.
+   *
+   * @param token authentication token for the cookie.
+   *
+   * @return the HTTP cookie.
+   */
+  protected Cookie createCookie(String token) {
+    Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, token);
+    if (getCookieDomain() != null) {
+      cookie.setDomain(getCookieDomain());
+    }
+    if (getCookiePath() != null) {
+      cookie.setPath(getCookiePath());
+    }
+    return cookie;
+  }
+}

+ 89 - 0
src/core/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java

@@ -0,0 +1,89 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Interface for server authentication mechanisms.
+ * <p/>
+ * The {@link AuthenticationFilter} manages the lifecycle of the authentication handler.
+ * <p/>
+ * Implementations must be thread-safe as one instance is initialized and used for all requests.
+ */
+public interface AuthenticationHandler {
+
+  /**
+   * Returns the authentication type of the authentication handler.
+   * <p/>
+   * This should be a name that uniquely identifies the authentication type.
+   * For example 'simple' or 'kerberos'.
+   *
+   * @return the authentication type of the authentication handler.
+   */
+  public String getType();
+
+  /**
+   * Initializes the authentication handler instance.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#init} method.
+   *
+   * @param config configuration properties to initialize the handler.
+   *
+   * @throws ServletException thrown if the handler could not be initialized.
+   */
+  public void init(Properties config) throws ServletException;
+
+  /**
+   * Destroys the authentication handler instance.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#destroy} method.
+   */
+  public void destroy();
+
+  /**
+   * Performs an authentication step for the given HTTP client request.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter} only if the HTTP client request is
+   * not yet authenticated.
+   * <p/>
+   * Depending upon the authentication mechanism being implemented, a particular HTTP client may
+   * end up making a sequence of invocations before authentication is successfully established (this is
+   * the case of Kerberos SPNEGO).
+   * <p/>
+   * This method must return an {@link AuthenticationToken} only if the the HTTP client request has
+   * been successfully and fully authenticated.
+   * <p/>
+   * If the HTTP client request has not been completely authenticated, this method must take over
+   * the corresponding HTTP response and it must return <code>null</code>.
+   *
+   * @param request the HTTP client request.
+   * @param response the HTTP client response.
+   *
+   * @return an {@link AuthenticationToken} if the HTTP client request has been authenticated,
+   *         <code>null</code> otherwise (in this case it must take care of the response).
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if an Authentication error occurred.
+   */
+  public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
+    throws IOException, AuthenticationException;
+
+}

+ 226 - 0
src/core/org/apache/hadoop/security/authentication/server/AuthenticationToken.java

@@ -0,0 +1,226 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+
+import java.security.Principal;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringTokenizer;
+
+/**
+ * The {@link AuthenticationToken} contains information about an authenticated HTTP client and doubles
+ * as the {@link Principal} to be returned by authenticated {@link HttpServletRequest}s
+ * <p/>
+ * The token can be serialized/deserialized to and from a string as it is sent and received in HTTP client
+ * responses and requests as a HTTP cookie (this is done by the {@link AuthenticationFilter}).
+ */
+public class AuthenticationToken implements Principal {
+
+  /**
+   * Constant that identifies an anonymous request.
+   */
+  public static final AuthenticationToken ANONYMOUS = new AuthenticationToken();
+
+  private static final String ATTR_SEPARATOR = "&";
+  private static final String USER_NAME = "u";
+  private static final String PRINCIPAL = "p";
+  private static final String EXPIRES = "e";
+  private static final String TYPE = "t";
+
+  private final static Set<String> ATTRIBUTES =
+    new HashSet<String>(Arrays.asList(USER_NAME, PRINCIPAL, EXPIRES, TYPE));
+
+  private String userName;
+  private String principal;
+  private String type;
+  private long expires;
+  private String token;
+
+  private AuthenticationToken() {
+    userName = null;
+    principal = null;
+    type = null;
+    expires = -1;
+    token = "ANONYMOUS";
+    generateToken();
+  }
+
+  private static final String ILLEGAL_ARG_MSG = " is NULL, empty or contains a '" + ATTR_SEPARATOR + "'";
+
+  /**
+   * Creates an authentication token.
+   *
+   * @param userName user name.
+   * @param principal principal (commonly matches the user name, with Kerberos is the full/long principal
+   * name while the userName is the short name).
+   * @param type the authentication mechanism name.
+   * (<code>System.currentTimeMillis() + validityPeriod</code>).
+   */
+  public AuthenticationToken(String userName, String principal, String type) {
+    checkForIllegalArgument(userName, "userName");
+    checkForIllegalArgument(principal, "principal");
+    checkForIllegalArgument(type, "type");
+    this.userName = userName;
+    this.principal = principal;
+    this.type = type;
+    this.expires = -1;
+  }
+  
+  /**
+   * Check if the provided value is invalid. Throw an error if it is invalid, NOP otherwise.
+   * 
+   * @param value the value to check.
+   * @param name the parameter name to use in an error message if the value is invalid.
+   */
+  private static void checkForIllegalArgument(String value, String name) {
+    if (value == null || value.length() == 0 || value.contains(ATTR_SEPARATOR)) {
+      throw new IllegalArgumentException(name + ILLEGAL_ARG_MSG);
+    }
+  }
+
+  /**
+   * Sets the expiration of the token.
+   *
+   * @param expires expiration time of the token in milliseconds since the epoch.
+   */
+  public void setExpires(long expires) {
+    if (this != AuthenticationToken.ANONYMOUS) {
+      this.expires = expires;
+      generateToken();
+    }
+  }
+
+  /**
+   * Generates the token.
+   */
+  private void generateToken() {
+    StringBuffer sb = new StringBuffer();
+    sb.append(USER_NAME).append("=").append(userName).append(ATTR_SEPARATOR);
+    sb.append(PRINCIPAL).append("=").append(principal).append(ATTR_SEPARATOR);
+    sb.append(TYPE).append("=").append(type).append(ATTR_SEPARATOR);
+    sb.append(EXPIRES).append("=").append(expires);
+    token = sb.toString();
+  }
+
+  /**
+   * Returns the user name.
+   *
+   * @return the user name.
+   */
+  public String getUserName() {
+    return userName;
+  }
+
+  /**
+   * Returns the principal name (this method name comes from the JDK {@link Principal} interface).
+   *
+   * @return the principal name.
+   */
+  @Override
+  public String getName() {
+    return principal;
+  }
+
+  /**
+   * Returns the authentication mechanism of the token.
+   *
+   * @return the authentication mechanism of the token.
+   */
+  public String getType() {
+    return type;
+  }
+
+  /**
+   * Returns the expiration time of the token.
+   *
+   * @return the expiration time of the token, in milliseconds since Epoc.
+   */
+  public long getExpires() {
+    return expires;
+  }
+
+  /**
+   * Returns if the token has expired.
+   *
+   * @return if the token has expired.
+   */
+  public boolean isExpired() {
+    return expires != -1 && System.currentTimeMillis() > expires;
+  }
+
+  /**
+   * Returns the string representation of the token.
+   * <p/>
+   * This string representation is parseable by the {@link #parse} method.
+   *
+   * @return the string representation of the token.
+   */
+  @Override
+  public String toString() {
+    return token;
+  }
+
+  /**
+   * Parses a string into an authentication token.
+   *
+   * @param tokenStr string representation of a token.
+   *
+   * @return the parsed authentication token.
+   *
+   * @throws AuthenticationException thrown if the string representation could not be parsed into
+   * an authentication token.
+   */
+  public static AuthenticationToken parse(String tokenStr) throws AuthenticationException {
+    Map<String, String> map = split(tokenStr);
+    if (!map.keySet().equals(ATTRIBUTES)) {
+      throw new AuthenticationException("Invalid token string, missing attributes");
+    }
+    long expires = Long.parseLong(map.get(EXPIRES));
+    AuthenticationToken token = new AuthenticationToken(map.get(USER_NAME), map.get(PRINCIPAL), map.get(TYPE));
+    token.setExpires(expires);
+    return token;
+  }
+
+  /**
+   * Splits the string representation of a token into attributes pairs.
+   *
+   * @param tokenStr string representation of a token.
+   *
+   * @return a map with the attribute pairs of the token.
+   *
+   * @throws AuthenticationException thrown if the string representation of the token could not be broken into
+   * attribute pairs.
+   */
+  private static Map<String, String> split(String tokenStr) throws AuthenticationException {
+    Map<String, String> map = new HashMap<String, String>();
+    StringTokenizer st = new StringTokenizer(tokenStr, ATTR_SEPARATOR);
+    while (st.hasMoreTokens()) {
+      String part = st.nextToken();
+      int separator = part.indexOf('=');
+      if (separator == -1) {
+        throw new AuthenticationException("Invalid authentication token");
+      }
+      String key = part.substring(0, separator);
+      String value = part.substring(separator + 1);
+      map.put(key, value);
+    }
+    return map;
+  }
+
+}

+ 310 - 0
src/core/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java

@@ -0,0 +1,310 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
+import com.sun.security.auth.module.Krb5LoginModule;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.ietf.jgss.GSSContext;
+import org.ietf.jgss.GSSCredential;
+import org.ietf.jgss.GSSManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * The {@link KerberosAuthenticationHandler} implements the Kerberos SPNEGO authentication mechanism for HTTP.
+ * <p/>
+ * The supported configuration properties are:
+ * <ul>
+ * <li>kerberos.principal: the Kerberos principal to used by the server. As stated by the Kerberos SPNEGO
+ * specification, it should be <code>HTTP/${HOSTNAME}@{REALM}</code>. The realm can be omitted from the
+ * principal as the JDK GSS libraries will use the realm name of the configured default realm.
+ * It does not have a default value.</li>
+ * <li>kerberos.keytab: the keytab file containing the credentials for the Kerberos principal.
+ * It does not have a default value.</li>
+ * </ul>
+ */
+public class KerberosAuthenticationHandler implements AuthenticationHandler {
+  private static Logger LOG = LoggerFactory.getLogger(KerberosAuthenticationHandler.class);
+
+  /**
+   * Kerberos context configuration for the JDK GSS library.
+   */
+  private static class KerberosConfiguration extends Configuration {
+    private String keytab;
+    private String principal;
+
+    public KerberosConfiguration(String keytab, String principal) {
+      this.keytab = keytab;
+      this.principal = principal;
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      Map<String, String> options = new HashMap<String, String>();
+      options.put("keyTab", keytab);
+      options.put("principal", principal);
+      options.put("useKeyTab", "true");
+      options.put("storeKey", "true");
+      options.put("doNotPrompt", "true");
+      options.put("useTicketCache", "true");
+      options.put("renewTGT", "true");
+      options.put("refreshKrb5Config", "true");
+      options.put("isInitiator", "false");
+      String ticketCache = System.getenv("KRB5CCNAME");
+      if (ticketCache != null) {
+        options.put("ticketCache", ticketCache);
+      }
+      if (LOG.isDebugEnabled()) {
+        options.put("debug", "true");
+      }
+
+      return new AppConfigurationEntry[]{
+        new AppConfigurationEntry(Krb5LoginModule.class.getName(),
+                                  AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                                  options),};
+    }
+  }
+
+  /**
+   * Constant that identifies the authentication mechanism.
+   */
+  public static final String TYPE = "kerberos";
+
+  /**
+   * Constant for the configuration property that indicates the kerberos principal.
+   */
+  public static final String PRINCIPAL = TYPE + ".principal";
+
+  /**
+   * Constant for the configuration property that indicates the keytab file path.
+   */
+  public static final String KEYTAB = TYPE + ".keytab";
+
+  /**
+   * Constant for the configuration property that indicates the Kerberos name
+   * rules for the Kerberos principals.
+   */
+  public static final String NAME_RULES = TYPE + ".name.rules";
+
+  private String principal;
+  private String keytab;
+  private GSSManager gssManager;
+  private LoginContext loginContext;
+
+  /**
+   * Initializes the authentication handler instance.
+   * <p/>
+   * It creates a Kerberos context using the principal and keytab specified in the configuration.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#init} method.
+   *
+   * @param config configuration properties to initialize the handler.
+   *
+   * @throws ServletException thrown if the handler could not be initialized.
+   */
+  @Override
+  public void init(Properties config) throws ServletException {
+    try {
+      principal = config.getProperty(PRINCIPAL, principal);
+      if (principal == null || principal.trim().length() == 0) {
+        throw new ServletException("Principal not defined in configuration");
+      }
+      keytab = config.getProperty(KEYTAB, keytab);
+      if (keytab == null || keytab.trim().length() == 0) {
+        throw new ServletException("Keytab not defined in configuration");
+      }
+      if (!new File(keytab).exists()) {
+        throw new ServletException("Keytab does not exist: " + keytab);
+      }
+
+      String nameRules = config.getProperty(NAME_RULES, "DEFAULT");
+      KerberosName.setRules(nameRules);
+
+      Set<Principal> principals = new HashSet<Principal>();
+      principals.add(new KerberosPrincipal(principal));
+      Subject subject = new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
+
+      KerberosConfiguration kerberosConfiguration = new KerberosConfiguration(keytab, principal);
+
+      loginContext = new LoginContext("", subject, null, kerberosConfiguration);
+      loginContext.login();
+
+      Subject serverSubject = loginContext.getSubject();
+      try {
+        gssManager = Subject.doAs(serverSubject, new PrivilegedExceptionAction<GSSManager>() {
+
+          @Override
+          public GSSManager run() throws Exception {
+            return GSSManager.getInstance();
+          }
+        });
+      } catch (PrivilegedActionException ex) {
+        throw ex.getException();
+      }
+      LOG.info("Initialized, principal [{}] from keytab [{}]", principal, keytab);
+    } catch (Exception ex) {
+      throw new ServletException(ex);
+    }
+  }
+
+  /**
+   * Releases any resources initialized by the authentication handler.
+   * <p/>
+   * It destroys the Kerberos context.
+   */
+  @Override
+  public void destroy() {
+    try {
+      if (loginContext != null) {
+        loginContext.logout();
+        loginContext = null;
+      }
+    } catch (LoginException ex) {
+      LOG.warn(ex.getMessage(), ex);
+    }
+  }
+
+  /**
+   * Returns the authentication type of the authentication handler, 'kerberos'.
+   * <p/>
+   *
+   * @return the authentication type of the authentication handler, 'kerberos'.
+   */
+  @Override
+  public String getType() {
+    return TYPE;
+  }
+
+  /**
+   * Returns the Kerberos principal used by the authentication handler.
+   *
+   * @return the Kerberos principal used by the authentication handler.
+   */
+  protected String getPrincipal() {
+    return principal;
+  }
+
+  /**
+   * Returns the keytab used by the authentication handler.
+   *
+   * @return the keytab used by the authentication handler.
+   */
+  protected String getKeytab() {
+    return keytab;
+  }
+
+  /**
+   * It enforces the the Kerberos SPNEGO authentication sequence returning an {@link AuthenticationToken} only
+   * after the Kerberos SPNEGO sequence has completed successfully.
+   * <p/>
+   *
+   * @param request the HTTP client request.
+   * @param response the HTTP client response.
+   *
+   * @return an authentication token if the Kerberos SPNEGO sequence is complete and valid,
+   *         <code>null</code> if it is in progress (in this case the handler handles the response to the client).
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if Kerberos SPNEGO sequence failed.
+   */
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request, final HttpServletResponse response)
+    throws IOException, AuthenticationException {
+    AuthenticationToken token = null;
+    String authorization = request.getHeader(KerberosAuthenticator.AUTHORIZATION);
+
+    if (authorization == null || !authorization.startsWith(KerberosAuthenticator.NEGOTIATE)) {
+      response.setHeader(KerberosAuthenticator.WWW_AUTHENTICATE, KerberosAuthenticator.NEGOTIATE);
+      response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+      if (authorization == null) {
+        LOG.trace("SPNEGO starting");
+      } else {
+        LOG.warn("'" + KerberosAuthenticator.AUTHORIZATION + "' does not start with '" +
+            KerberosAuthenticator.NEGOTIATE + "' :  {}", authorization);
+      }
+    } else {
+      authorization = authorization.substring(KerberosAuthenticator.NEGOTIATE.length()).trim();
+      final Base64 base64 = new Base64(0);
+      final byte[] clientToken = base64.decode(authorization);
+      Subject serverSubject = loginContext.getSubject();
+      try {
+        token = Subject.doAs(serverSubject, new PrivilegedExceptionAction<AuthenticationToken>() {
+
+          @Override
+          public AuthenticationToken run() throws Exception {
+            AuthenticationToken token = null;
+            GSSContext gssContext = null;
+            try {
+              gssContext = gssManager.createContext((GSSCredential) null);
+              byte[] serverToken = gssContext.acceptSecContext(clientToken, 0, clientToken.length);
+              if (serverToken != null && serverToken.length > 0) {
+                String authenticate = base64.encodeToString(serverToken);
+                response.setHeader(KerberosAuthenticator.WWW_AUTHENTICATE,
+                                   KerberosAuthenticator.NEGOTIATE + " " + authenticate);
+              }
+              if (!gssContext.isEstablished()) {
+                response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+                LOG.trace("SPNEGO in progress");
+              } else {
+                String clientPrincipal = gssContext.getSrcName().toString();
+                KerberosName kerberosName = new KerberosName(clientPrincipal);
+                String userName = kerberosName.getShortName();
+                token = new AuthenticationToken(userName, clientPrincipal, TYPE);
+                response.setStatus(HttpServletResponse.SC_OK);
+                LOG.trace("SPNEGO completed for principal [{}]", clientPrincipal);
+              }
+            } finally {
+              if (gssContext != null) {
+                gssContext.dispose();
+              }
+            }
+            return token;
+          }
+        });
+      } catch (PrivilegedActionException ex) {
+        if (ex.getException() instanceof IOException) {
+          throw (IOException) ex.getException();
+        }
+        else {
+          throw new AuthenticationException(ex.getException());
+        }
+      }
+    }
+    return token;
+  }
+
+}

+ 134 - 0
src/core/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java

@@ -0,0 +1,134 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * The <code>PseudoAuthenticationHandler</code> provides a pseudo authentication mechanism that accepts
+ * the user name specified as a query string parameter.
+ * <p/>
+ * This mimics the model of Hadoop Simple authentication which trust the 'user.name' property provided in
+ * the configuration object.
+ * <p/>
+ * This handler can be configured to support anonymous users.
+ * <p/>
+ * The only supported configuration property is:
+ * <ul>
+ * <li>simple.anonymous.allowed: <code>true|false</code>, default value is <code>false</code></li>
+ * </ul>
+ */
+public class PseudoAuthenticationHandler implements AuthenticationHandler {
+
+  /**
+   * Constant that identifies the authentication mechanism.
+   */
+  public static final String TYPE = "simple";
+
+  /**
+   * Constant for the configuration property that indicates if anonymous users are allowed.
+   */
+  public static final String ANONYMOUS_ALLOWED = TYPE + ".anonymous.allowed";
+
+  private boolean acceptAnonymous;
+
+  /**
+   * Initializes the authentication handler instance.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#init} method.
+   *
+   * @param config configuration properties to initialize the handler.
+   *
+   * @throws ServletException thrown if the handler could not be initialized.
+   */
+  @Override
+  public void init(Properties config) throws ServletException {
+    acceptAnonymous = Boolean.parseBoolean(config.getProperty(ANONYMOUS_ALLOWED, "false"));
+  }
+
+  /**
+   * Returns if the handler is configured to support anonymous users.
+   *
+   * @return if the handler is configured to support anonymous users.
+   */
+  protected boolean getAcceptAnonymous() {
+    return acceptAnonymous;
+  }
+
+  /**
+   * Releases any resources initialized by the authentication handler.
+   * <p/>
+   * This implementation does a NOP.
+   */
+  @Override
+  public void destroy() {
+  }
+
+  /**
+   * Returns the authentication type of the authentication handler, 'simple'.
+   * <p/>
+   *
+   * @return the authentication type of the authentication handler, 'simple'.
+   */
+  @Override
+  public String getType() {
+    return TYPE;
+  }
+
+  /**
+   * Authenticates an HTTP client request.
+   * <p/>
+   * It extracts the {@link PseudoAuthenticator#USER_NAME} parameter from the query string and creates
+   * an {@link AuthenticationToken} with it.
+   * <p/>
+   * If the HTTP client request does not contain the {@link PseudoAuthenticator#USER_NAME} parameter and
+   * the handler is configured to allow anonymous users it returns the {@link AuthenticationToken#ANONYMOUS}
+   * token.
+   * <p/>
+   * If the HTTP client request does not contain the {@link PseudoAuthenticator#USER_NAME} parameter and
+   * the handler is configured to disallow anonymous users it throws an {@link AuthenticationException}.
+   *
+   * @param request the HTTP client request.
+   * @param response the HTTP client response.
+   *
+   * @return an authentication token if the HTTP client request is accepted and credentials are valid.
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if HTTP client request was not accepted as an authentication request.
+   */
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
+    throws IOException, AuthenticationException {
+    AuthenticationToken token;
+    String userName = request.getParameter(PseudoAuthenticator.USER_NAME);
+    if (userName == null) {
+      if (getAcceptAnonymous()) {
+        token = AuthenticationToken.ANONYMOUS;
+      } else {
+        throw new AuthenticationException("Anonymous requests are disallowed");
+      }
+    } else {
+      token = new AuthenticationToken(userName, userName, TYPE);
+    }
+    return token;
+  }
+
+}

+ 395 - 0
src/core/org/apache/hadoop/security/authentication/util/KerberosName.java

@@ -0,0 +1,395 @@
+package org.apache.hadoop.security.authentication.util;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import sun.security.krb5.Config;
+import sun.security.krb5.KrbException;
+
+/**
+ * This class implements parsing and handling of Kerberos principal names. In
+ * particular, it splits them apart and translates them down into local
+ * operating system names.
+ */
+@SuppressWarnings("all")
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class KerberosName {
+  /** The first component of the name */
+  private final String serviceName;
+  /** The second component of the name. It may be null. */
+  private final String hostName;
+  /** The realm of the name. */
+  private final String realm;
+
+  /**
+   * A pattern that matches a Kerberos name with at most 2 components.
+   */
+  private static final Pattern nameParser =
+    Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
+
+  /**
+   * A pattern that matches a string with out '$' and then a single
+   * parameter with $n.
+   */
+  private static Pattern parameterPattern =
+    Pattern.compile("([^$]*)(\\$(\\d*))?");
+
+  /**
+   * A pattern for parsing a auth_to_local rule.
+   */
+  private static final Pattern ruleParser =
+    Pattern.compile("\\s*((DEFAULT)|(RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?"+
+                    "(s/([^/]*)/([^/]*)/(g)?)?))");
+
+  /**
+   * A pattern that recognizes simple/non-simple names.
+   */
+  private static final Pattern nonSimplePattern = Pattern.compile("[/@]");
+
+  /**
+   * The list of translation rules.
+   */
+  private static List<Rule> rules;
+
+  private static String defaultRealm;
+  private static Config kerbConf;
+
+  static {
+    try {
+      kerbConf = Config.getInstance();
+      defaultRealm = kerbConf.getDefaultRealm();
+    } catch (KrbException ke) {
+        defaultRealm="";
+    }
+  }
+
+  /**
+   * Create a name from the full Kerberos principal name.
+   * @param name
+   */
+  public KerberosName(String name) {
+    Matcher match = nameParser.matcher(name);
+    if (!match.matches()) {
+      if (name.contains("@")) {
+        throw new IllegalArgumentException("Malformed Kerberos name: " + name);
+      } else {
+        serviceName = name;
+        hostName = null;
+        realm = null;
+      }
+    } else {
+      serviceName = match.group(1);
+      hostName = match.group(3);
+      realm = match.group(4);
+    }
+  }
+
+  /**
+   * Get the configured default realm.
+   * @return the default realm from the krb5.conf
+   */
+  public String getDefaultRealm() {
+    return defaultRealm;
+  }
+
+  /**
+   * Put the name back together from the parts.
+   */
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(serviceName);
+    if (hostName != null) {
+      result.append('/');
+      result.append(hostName);
+    }
+    if (realm != null) {
+      result.append('@');
+      result.append(realm);
+    }
+    return result.toString();
+  }
+
+  /**
+   * Get the first component of the name.
+   * @return the first section of the Kerberos principal name
+   */
+  public String getServiceName() {
+    return serviceName;
+  }
+
+  /**
+   * Get the second component of the name.
+   * @return the second section of the Kerberos principal name, and may be null
+   */
+  public String getHostName() {
+    return hostName;
+  }
+
+  /**
+   * Get the realm of the name.
+   * @return the realm of the name, may be null
+   */
+  public String getRealm() {
+    return realm;
+  }
+
+  /**
+   * An encoding of a rule for translating kerberos names.
+   */
+  private static class Rule {
+    private final boolean isDefault;
+    private final int numOfComponents;
+    private final String format;
+    private final Pattern match;
+    private final Pattern fromPattern;
+    private final String toPattern;
+    private final boolean repeat;
+
+    Rule() {
+      isDefault = true;
+      numOfComponents = 0;
+      format = null;
+      match = null;
+      fromPattern = null;
+      toPattern = null;
+      repeat = false;
+    }
+
+    Rule(int numOfComponents, String format, String match, String fromPattern,
+         String toPattern, boolean repeat) {
+      isDefault = false;
+      this.numOfComponents = numOfComponents;
+      this.format = format;
+      this.match = match == null ? null : Pattern.compile(match);
+      this.fromPattern =
+        fromPattern == null ? null : Pattern.compile(fromPattern);
+      this.toPattern = toPattern;
+      this.repeat = repeat;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buf = new StringBuilder();
+      if (isDefault) {
+        buf.append("DEFAULT");
+      } else {
+        buf.append("RULE:[");
+        buf.append(numOfComponents);
+        buf.append(':');
+        buf.append(format);
+        buf.append(']');
+        if (match != null) {
+          buf.append('(');
+          buf.append(match);
+          buf.append(')');
+        }
+        if (fromPattern != null) {
+          buf.append("s/");
+          buf.append(fromPattern);
+          buf.append('/');
+          buf.append(toPattern);
+          buf.append('/');
+          if (repeat) {
+            buf.append('g');
+          }
+        }
+      }
+      return buf.toString();
+    }
+
+    /**
+     * Replace the numbered parameters of the form $n where n is from 1 to
+     * the length of params. Normal text is copied directly and $n is replaced
+     * by the corresponding parameter.
+     * @param format the string to replace parameters again
+     * @param params the list of parameters
+     * @return the generated string with the parameter references replaced.
+     * @throws BadFormatString
+     */
+    static String replaceParameters(String format,
+                                    String[] params) throws BadFormatString {
+      Matcher match = parameterPattern.matcher(format);
+      int start = 0;
+      StringBuilder result = new StringBuilder();
+      while (start < format.length() && match.find(start)) {
+        result.append(match.group(1));
+        String paramNum = match.group(3);
+        if (paramNum != null) {
+          try {
+            int num = Integer.parseInt(paramNum);
+            if (num < 0 || num > params.length) {
+              throw new BadFormatString("index " + num + " from " + format +
+                                        " is outside of the valid range 0 to " +
+                                        (params.length - 1));
+            }
+            result.append(params[num]);
+          } catch (NumberFormatException nfe) {
+            throw new BadFormatString("bad format in username mapping in " +
+                                      paramNum, nfe);
+          }
+
+        }
+        start = match.end();
+      }
+      return result.toString();
+    }
+
+    /**
+     * Replace the matches of the from pattern in the base string with the value
+     * of the to string.
+     * @param base the string to transform
+     * @param from the pattern to look for in the base string
+     * @param to the string to replace matches of the pattern with
+     * @param repeat whether the substitution should be repeated
+     * @return
+     */
+    static String replaceSubstitution(String base, Pattern from, String to,
+                                      boolean repeat) {
+      Matcher match = from.matcher(base);
+      if (repeat) {
+        return match.replaceAll(to);
+      } else {
+        return match.replaceFirst(to);
+      }
+    }
+
+    /**
+     * Try to apply this rule to the given name represented as a parameter
+     * array.
+     * @param params first element is the realm, second and later elements are
+     *        are the components of the name "a/b@FOO" -> {"FOO", "a", "b"}
+     * @return the short name if this rule applies or null
+     * @throws IOException throws if something is wrong with the rules
+     */
+    String apply(String[] params) throws IOException {
+      String result = null;
+      if (isDefault) {
+        if (defaultRealm.equals(params[0])) {
+          result = params[1];
+        }
+      } else if (params.length - 1 == numOfComponents) {
+        String base = replaceParameters(format, params);
+        if (match == null || match.matcher(base).matches()) {
+          if (fromPattern == null) {
+            result = base;
+          } else {
+            result = replaceSubstitution(base, fromPattern, toPattern,  repeat);
+          }
+        }
+      }
+      if (result != null && nonSimplePattern.matcher(result).find()) {
+        throw new NoMatchingRule("Non-simple name " + result +
+                                 " after auth_to_local rule " + this);
+      }
+      return result;
+    }
+  }
+
+  static List<Rule> parseRules(String rules) {
+    List<Rule> result = new ArrayList<Rule>();
+    String remaining = rules.trim();
+    while (remaining.length() > 0) {
+      Matcher matcher = ruleParser.matcher(remaining);
+      if (!matcher.lookingAt()) {
+        throw new IllegalArgumentException("Invalid rule: " + remaining);
+      }
+      if (matcher.group(2) != null) {
+        result.add(new Rule());
+      } else {
+        result.add(new Rule(Integer.parseInt(matcher.group(4)),
+                            matcher.group(5),
+                            matcher.group(7),
+                            matcher.group(9),
+                            matcher.group(10),
+                            "g".equals(matcher.group(11))));
+      }
+      remaining = remaining.substring(matcher.end());
+    }
+    return result;
+  }
+
+  @SuppressWarnings("serial")
+  public static class BadFormatString extends IOException {
+    BadFormatString(String msg) {
+      super(msg);
+    }
+    BadFormatString(String msg, Throwable err) {
+      super(msg, err);
+    }
+  }
+
+  @SuppressWarnings("serial")
+  public static class NoMatchingRule extends IOException {
+    NoMatchingRule(String msg) {
+      super(msg);
+    }
+  }
+
+  /**
+   * Get the translation of the principal name into an operating system
+   * user name.
+   * @return the short name
+   * @throws IOException
+   */
+  public String getShortName() throws IOException {
+    String[] params;
+    if (hostName == null) {
+      // if it is already simple, just return it
+      if (realm == null) {
+        return serviceName;
+      }
+      params = new String[]{realm, serviceName};
+    } else {
+      params = new String[]{realm, serviceName, hostName};
+    }
+    for(Rule r: rules) {
+      String result = r.apply(params);
+      if (result != null) {
+        return result;
+      }
+    }
+    throw new NoMatchingRule("No rules applied to " + toString());
+  }
+
+  /**
+   * Set the rules.
+   * @param ruleString the rules string.
+   */
+  public static void setRules(String ruleString) {
+    rules = parseRules(ruleString);
+  }
+
+  static void printRules() throws IOException {
+    int i = 0;
+    for(Rule r: rules) {
+      System.out.println(++i + " " + r);
+    }
+  }
+
+}

+ 100 - 0
src/core/org/apache/hadoop/security/authentication/util/Signer.java

@@ -0,0 +1,100 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import org.apache.commons.codec.binary.Base64;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
+/**
+ * Signs strings and verifies signed strings using a SHA digest.
+ */
+public class Signer {
+  private static final String SIGNATURE = "&s=";
+
+  private byte[] secret;
+
+  /**
+   * Creates a Signer instance using the specified secret.
+   *
+   * @param secret secret to use for creating the digest.
+   */
+  public Signer(byte[] secret) {
+    if (secret == null) {
+      throw new IllegalArgumentException("secret cannot be NULL");
+    }
+    this.secret = secret.clone();
+  }
+
+  /**
+   * Returns a signed string.
+   * <p/>
+   * The signature '&s=SIGNATURE' is appended at the end of the string.
+   *
+   * @param str string to sign.
+   *
+   * @return the signed string.
+   */
+  public String sign(String str) {
+    if (str == null || str.length() == 0) {
+      throw new IllegalArgumentException("NULL or empty string to sign");
+    }
+    String signature = computeSignature(str);
+    return str + SIGNATURE + signature;
+  }
+
+  /**
+   * Verifies a signed string and extracts the original string.
+   *
+   * @param signedStr the signed string to verify and extract.
+   *
+   * @return the extracted original string.
+   *
+   * @throws SignerException thrown if the given string is not a signed string or if the signature is invalid.
+   */
+  public String verifyAndExtract(String signedStr) throws SignerException {
+    int index = signedStr.lastIndexOf(SIGNATURE);
+    if (index == -1) {
+      throw new SignerException("Invalid signed text: " + signedStr);
+    }
+    String originalSignature = signedStr.substring(index + SIGNATURE.length());
+    String rawValue = signedStr.substring(0, index);
+    String currentSignature = computeSignature(rawValue);
+    if (!originalSignature.equals(currentSignature)) {
+      throw new SignerException("Invalid signature");
+    }
+    return rawValue;
+  }
+
+  /**
+   * Returns then signature of a string.
+   *
+   * @param str string to sign.
+   *
+   * @return the signature for the string.
+   */
+  protected String computeSignature(String str) {
+    try {
+      MessageDigest md = MessageDigest.getInstance("SHA");
+      md.update(str.getBytes());
+      md.update(secret);
+      byte[] digest = md.digest();
+      return new Base64(0).encodeToString(digest);
+    } catch (NoSuchAlgorithmException ex) {
+      throw new RuntimeException("It should not happen, " + ex.getMessage(), ex);
+    }
+  }
+
+}

+ 31 - 0
src/core/org/apache/hadoop/security/authentication/util/SignerException.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+/**
+ * Exception thrown by {@link Signer} when a string signature is invalid.
+ */
+public class SignerException extends Exception {
+  
+  static final long serialVersionUID = 0;
+
+  /**
+   * Creates an exception instance.
+   *
+   * @param msg message for the exception.
+   */
+  public SignerException(String msg) {
+    super(msg);
+  }
+}

+ 112 - 0
src/core/org/apache/hadoop/security/token/Token.java

@@ -22,9 +22,14 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.ServiceLoader;
 
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -36,10 +41,12 @@ import org.apache.hadoop.io.WritableUtils;
  * The client-side form of the token.
  * The client-side form of the token.
  */
  */
 public class Token<T extends TokenIdentifier> implements Writable {
 public class Token<T extends TokenIdentifier> implements Writable {
+  public static final Log LOG = LogFactory.getLog(Token.class);
   private byte[] identifier;
   private byte[] identifier;
   private byte[] password;
   private byte[] password;
   private Text kind;
   private Text kind;
   private Text service;
   private Text service;
+  private TokenRenewer renewer;
   
   
   /**
   /**
    * Construct a token given a token identifier and a secret manager for the
    * Construct a token given a token identifier and a secret manager for the
@@ -78,6 +85,17 @@ public class Token<T extends TokenIdentifier> implements Writable {
     service = new Text();
     service = new Text();
   }
   }
 
 
+  /**
+   * Clone a token.
+   * @param other the token to clone
+   */
+  public Token(Token<T> other) {
+    this.identifier = other.identifier;
+    this.password = other.password;
+    this.kind = other.kind;
+    this.service = other.service;
+  }
+
   /**
   /**
    * Get the token identifier
    * Get the token identifier
    * @return the token identifier
    * @return the token identifier
@@ -102,6 +120,16 @@ public class Token<T extends TokenIdentifier> implements Writable {
     return kind;
     return kind;
   }
   }
 
 
+  /**
+   * Set the token kind. This is only intended to be used by services that
+   * wrap another service's token, such as HFTP wrapping HDFS.
+   * @param newKind
+   */
+  @InterfaceAudience.Private
+  public void setKind(Text newKind) {
+    kind = newKind;
+  }
+
   /**
   /**
    * Get the service on which the token is supposed to be used
    * Get the service on which the token is supposed to be used
    * @return the service name
    * @return the service name
@@ -242,4 +270,88 @@ public class Token<T extends TokenIdentifier> implements Writable {
     buffer.append(service.toString());
     buffer.append(service.toString());
     return buffer.toString();
     return buffer.toString();
   }
   }
+  
+  private static ServiceLoader<TokenRenewer> renewers =
+      ServiceLoader.load(TokenRenewer.class);
+
+  private synchronized TokenRenewer getRenewer() throws IOException {
+    if (renewer != null) {
+      return renewer;
+    }
+    renewer = TRIVIAL_RENEWER;
+    for (TokenRenewer canidate: renewers) {
+      if (canidate.handleKind(this.kind)) {
+        renewer = canidate;
+        return renewer;
+      }
+    }
+    LOG.warn("No TokenRenewer defined for token kind " + this.kind);
+    return renewer;
+  }
+
+  /**
+   * Is this token managed so that it can be renewed or cancelled?
+   * @return true, if it can be renewed and cancelled.
+   */
+  public boolean isManaged() throws IOException {
+    return getRenewer().isManaged(this);
+  }
+
+  /**
+   * Renew this delegation token
+   * @return the new expiration time
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public long renew(Configuration conf
+                    ) throws IOException, InterruptedException {
+    return getRenewer().renew(this, conf);
+  }
+  
+  /**
+   * Cancel this delegation token
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void cancel(Configuration conf
+                     ) throws IOException, InterruptedException {
+    getRenewer().cancel(this, conf);
+  }
+  
+  /**
+   * A trivial renewer for token kinds that aren't managed. Sub-classes need
+   * to implement getKind for their token kind.
+   */
+  public static class TrivialRenewer extends TokenRenewer {
+    
+    // define the kind for this renewer
+    protected Text getKind() {
+      return null;
+    }
+
+    @Override
+    public boolean handleKind(Text kind) {
+      return kind.equals(getKind());
+    }
+
+    @Override
+    public boolean isManaged(Token<?> token) {
+      return false;
+    }
+
+    @Override
+    public long renew(Token<?> token, Configuration conf) {
+      throw new UnsupportedOperationException("Token renewal is not supported "+
+                                              " for " + token.kind + " tokens");
+    }
+
+    @Override
+    public void cancel(Token<?> token, Configuration conf) throws IOException,
+        InterruptedException {
+      throw new UnsupportedOperationException("Token cancel is not supported " +
+          " for " + token.kind + " tokens");
+    }
+
+  }
+  private static final TokenRenewer TRIVIAL_RENEWER = new TrivialRenewer();
 }
 }

+ 65 - 0
src/core/org/apache/hadoop/security/token/TokenRenewer.java

@@ -0,0 +1,65 @@
+/**
+ * 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.security.token;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This is the interface for plugins that handle tokens.
+ */
+public abstract class TokenRenewer {
+
+  /**
+   * Does this renewer handle this kind of token?
+   * @param kind the kind of the token
+   * @return true if this renewer can renew it
+   */
+  public abstract boolean handleKind(Text kind);
+
+  /**
+   * Is the given token managed? Only managed tokens may be renewed or
+   * cancelled.
+   * @param token the token being checked
+   * @return true if the token may be renewed or cancelled
+   * @throws IOException
+   */
+  public abstract boolean isManaged(Token<?> token) throws IOException;
+  
+  /**
+   * Renew the given token.
+   * @return the new expiration time
+   * @throws IOException
+   * @throws InterruptedException 
+   */
+  public abstract long renew(Token<?> token,
+                             Configuration conf
+                             ) throws IOException, InterruptedException;
+  
+  /**
+   * Cancel the given token
+   * @throws IOException
+   * @throws InterruptedException 
+   */
+  public abstract void cancel(Token<?> token,
+                              Configuration conf
+                              ) throws IOException, InterruptedException;
+}

+ 58 - 0
src/core/org/apache/hadoop/util/ChecksumUtil.java

@@ -0,0 +1,58 @@
+/**
+ * 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.util;
+
+import java.io.IOException;
+
+public class ChecksumUtil {
+  /**
+   * updates the checksum for a buffer
+   * 
+   * @param buf - buffer to update the checksum in
+   * @param chunkOff - offset in the buffer where the checksum is to update
+   * @param dataOff - offset in the buffer of the data
+   * @param dataLen - length of data to compute checksum on
+   */
+  public static void updateChunkChecksum(
+    byte[] buf,
+    int checksumOff,
+    int dataOff, 
+    int dataLen,
+    DataChecksum checksum
+  ) throws IOException {
+    int bytesPerChecksum = checksum.getBytesPerChecksum();
+    int checksumSize = checksum.getChecksumSize();
+    int curChecksumOff = checksumOff;
+    int curDataOff = dataOff;
+    int numChunks = (dataLen + bytesPerChecksum - 1) / bytesPerChecksum;
+    int dataLeft = dataLen;
+    
+    for (int i = 0; i < numChunks; i++) {
+      int len = Math.min(dataLeft, bytesPerChecksum);
+      
+      checksum.reset();
+      checksum.update(buf, curDataOff, len);
+      checksum.writeValue(buf, curChecksumOff, false);
+      
+      curDataOff += len;
+      curChecksumOff += checksumSize;
+      dataLeft -= len;
+    }
+  }
+}

+ 2 - 0
src/core/org/apache/hadoop/util/DiskChecker.java

@@ -138,6 +138,8 @@ public class DiskChecker {
       if (created) {
       if (created) {
         localFS.setPermission(dir, expected);
         localFS.setPermission(dir, expected);
         return true;
         return true;
+      } else {
+        return false;
       }
       }
     }
     }
 
 

Різницю між файлами не показано, бо вона завелика
+ 200 - 0
src/docs/releasenotes.html


+ 88 - 0
src/docs/relnotes.py

@@ -0,0 +1,88 @@
+#!/usr/bin/python
+
+# Run this command as:
+#
+# jira.sh -s https://issues.apache.org/jira -u $user -p $pw \
+#   -a getIssueList --search \
+#   "project in (HADOOP,HDFS,MAPREDUCE) and fixVersion = '$vers' and resolution = Fixed" \
+#   | ./relnotes.py > $vers.html
+
+import csv
+import re
+import subprocess
+import sys
+
+namePattern = re.compile(r' \([0-9]+\)')
+htmlSpecialPattern = re.compile(r'[&<>\'"\n]')
+quotes = {'<' : '&lt;', '>': '&gt;', '"': '&quot;', "'": '&apos;',
+          '&': '&amp;', '\n': '<br>'}
+
+def clean(str):
+  return re.sub(namePattern, "", str)
+
+def formatComponents(str):
+  str = re.sub(namePattern, '', str).replace("'", "")
+  if str != "":
+    return "(" + str + ")"
+  else:
+    return ""
+    
+def quoteHtmlChar(m):
+  return quotes[m.group(0)]
+
+def quoteHtml(str):
+  return re.sub(htmlSpecialPattern, quoteHtmlChar, str)
+
+def readReleaseNote(id, default):
+  cmd = ['jira.sh', '-s', 'https://issues.apache.org/jira', '-u', user, 
+         '-p', password, '-a', 'getFieldValue', '--issue', id, '--field',
+         'Release Note']
+  proc = subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=sys.stderr)
+  lines = proc.stdout.readlines()
+  # throw away first line
+  if len(lines) < 2 or len(lines[1]) < 2:
+    return default
+  else:
+    return "\n".join(lines[1:])[1:-2]
+
+user = sys.argv[1]
+password = sys.argv[2]
+vers = sys.argv[3]
+
+cmd = ['jira.sh', '-s', 'https://issues.apache.org/jira', '-u', user, '-p',
+       password, '-a', 'getIssueList', '--search',
+       "project in (HADOOP,HDFS,MAPREDUCE) and fixVersion = '" + vers + 
+        "' and resolution = Fixed"]
+proc = subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=sys.stderr)
+
+reader = csv.reader(proc.stdout, skipinitialspace=True)
+
+# throw away number of issues
+reader.next()
+
+# read the columns
+columns = reader.next()
+
+key = columns.index('Key')
+type = columns.index('Type')
+priority = columns.index('Priority')
+assignee = columns.index('Assignee')
+reporter = columns.index('Reporter')
+summary = columns.index('Summary')
+description = columns.index('Description')
+components = columns.index('Components')
+
+print "<html><body><ul>"
+
+for row in reader:
+  row_descr = readReleaseNote(row[key], row[description])
+  print \
+    '<li> <a href="https://issues.apache.org/jira/browse/%s">%s</a>.\n' \
+    '     %s %s reported by %s and fixed by %s %s<br>\n' \
+    '     <b>%s</b><br>\n' \
+    '     <blockquote>%s</blockquote></li>\n' \
+    % (row[key], row[key], clean(row[priority]), clean(row[type]).lower(), 
+       row[reporter], row[assignee], formatComponents(row[components]),
+       quoteHtml(row[summary]), quoteHtml(row_descr))
+
+print "</ul>\n</body></html>"

+ 124 - 0
src/docs/src/documentation/content/xdocs/HttpAuthentication.xml

@@ -0,0 +1,124 @@
+<?xml version="1.0"?>
+<!--
+  Copyright 2002-2004 The Apache Software Foundation
+
+  Licensed 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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+
+<document>
+
+  <header>
+    <title> 
+      Authentication for Hadoop HTTP web-consoles
+    </title>
+  </header>
+
+  <body>
+    <section>
+      <title> Introduction </title>
+      <p>
+        This document describes how to configure Hadoop HTTP web-consoles to require user
+        authentication. 
+      </p>
+      <p>
+        By default Hadoop HTTP web-consoles (JobTracker, NameNode, TaskTrackers and DataNodes) allow 
+        access without any form of authentication. 
+      </p>
+      <p>
+        Similarly to Hadoop RPC, Hadoop HTTP web-consoles can be configured to require Kerberos 
+        authentication using HTTP SPNEGO protocol (supported by browsers like Firefox and Internet
+        Explorer).        
+      </p>
+      <p>
+        In addition, Hadoop HTTP web-consoles support the equivalent of Hadoop's Pseudo/Simple
+        authentication. If this option is enabled, user must specify their user name in the first
+        browser interaction using the <code>user.name</code> query string parameter. For example:
+        <code>http://localhost:50030/jobtracker.jsp?user.name=babu</code>.
+      </p>
+      <p>
+        If a custom authentication mechanism is required for the HTTP web-consoles, it is possible 
+        to implement a plugin to support the alternate authentication mechanism (refer to 
+        Hadoop Alfredo for details on writing an <code>AuthenticatorHandler</code>).
+      </p>
+      <p>       
+        The next section describes how to configure Hadoop HTTP web-consoles to require user 
+        authentication.
+      </p>
+    </section>
+
+    <section> 
+      <title> Configuration </title>
+
+      <p>
+        The following properties should be in the <code>core-site.xml</code> of all the nodes
+        in the cluster.
+      </p>
+
+      <p><code>hadoop.http.filter.initializers</code>: add to this property the 
+      <code>org.apache.hadoop.security.AuthenticationFilterInitializer</code> initializer class.
+      </p>
+      
+      <p><code>hadoop.http.authentication.type</code>: Defines authentication used for the HTTP 
+      web-consoles. The supported values are: <code>simple | kerberos | 
+      #AUTHENTICATION_HANDLER_CLASSNAME#</code>. The dfeault value is <code>simple</code>.
+      </p>
+
+      <p><code>hadoop.http.authentication.token.validity</code>: Indicates how long (in seconds) 
+      an authentication token is valid before it has to be renewed. The default value is 
+      <code>36000</code>.
+      </p>
+
+      <p><code>hadoop.http.authentication.signature.secret</code>: The signature secret for  
+      signing the authentication tokens. If not set a random secret is generated at 
+      startup time. The same secret should be used for all nodes in the cluster, JobTracker, 
+      NameNode, DataNode and TastTracker. The default value is a <code>hadoop</code> value.
+      </p>
+        
+      <p><code>hadoop.http.authentication.cookie.domain</code>: The domain to use for the HTTP 
+      cookie that stores the authentication token. In order to authentiation to work 
+      correctly across all nodes in the cluster the domain must be correctly set.
+      There is no default value, the HTTP cookie will not have a domain working only
+      with the hostname issuing the HTTP cookie.
+      </p>
+
+      <p>
+      IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings.
+      For this setting to work properly all nodes in the cluster must be configured
+      to generate URLs with hostname.domain names on it.
+      </p>
+
+      <p><code>hadoop.http.authentication.simple.anonymous.allowed</code>: Indicates if anonymous 
+      requests are allowed when using 'simple' authentication. The default value is 
+      <code>true</code>
+      </p>
+
+      <p><code>hadoop.http.authentication.kerberos.principal</code>: Indicates the Kerberos 
+      principal to be used for HTTP endpoint when using 'kerberos' authentication.
+      The principal short name must be <code>HTTP</code> per Kerberos HTTP SPENGO specification.
+      The default value is <code>HTTP/localhost@$LOCALHOST</code>.
+      </p>
+
+      <p><code>hadoop.http.authentication.kerberos.keytab</code>: Location of the keytab file 
+      with the credentials for the Kerberos principal used for the HTTP endpoint. 
+      The default value is <code>${user.home}/hadoop.keytab</code>.i
+      </p>
+
+    </section>
+
+  </body>
+</document>
+

+ 434 - 211
src/docs/src/documentation/content/xdocs/fair_scheduler.xml

@@ -18,16 +18,15 @@
 <!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
 <!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
 <document>
 <document>
   <header>
   <header>
-    <title>Fair Scheduler Guide</title>
+    <title>Fair Scheduler</title>
   </header>
   </header>
   <body>
   <body>
 
 
     <section>
     <section>
       <title>Purpose</title>
       <title>Purpose</title>
 
 
-      <p>This document describes the Fair Scheduler, a pluggable
-        MapReduce scheduler for Hadoop which provides a way to share
-        large clusters.</p>
+      <p>This document describes the Fair Scheduler, a pluggable MapReduce
+        scheduler that provides a way to share large clusters.</p>
     </section>
     </section>
 
 
     <section>
     <section>
@@ -39,52 +38,62 @@
         free up are assigned to the new jobs, so that each job gets
         free up are assigned to the new jobs, so that each job gets
         roughly the same amount of CPU time. Unlike the default Hadoop
         roughly the same amount of CPU time. Unlike the default Hadoop
         scheduler, which forms a queue of jobs, this lets short jobs finish
         scheduler, which forms a queue of jobs, this lets short jobs finish
-        in reasonable time while not starving long jobs. It is also a 
-        reasonable way to share a cluster between a number of users. Finally, 
-        fair sharing can also work with job priorities - the priorities are
+        in reasonable time while not starving long jobs. It is also an easy
+        way to share a cluster between multiple of users.
+        Fair sharing can also work with job priorities - the priorities are
         used as weights to determine the fraction of total compute time that
         used as weights to determine the fraction of total compute time that
-        each job should get.
+        each job gets.
       </p>
       </p>
       <p>
       <p>
-        The scheduler actually organizes jobs further into "pools", and 
-        shares resources fairly between these pools. By default, there is a 
-        separate pool for each user, so that each user gets the same share 
-        of the cluster no matter how many jobs they submit. However, it is 
-        also possible to set a job's pool based on the user's Unix group or
-        any other jobconf property, such as the queue name property used by 
-        <a href="capacity_scheduler.html">Capacity Scheduler</a>. 
-        Within each pool, fair sharing is used to share capacity between 
-        the running jobs. Pools can also be given weights to share the 
-        cluster non-proportionally in the config file.
+        The fair scheduler organizes jobs into <em>pools</em>, and 
+        divides resources fairly between these pools. By default, there is a 
+        separate pool for each user, so that each user gets an equal share 
+        of the cluster. It is also possible to set a job's pool based on the
+        user's Unix group or any jobconf property. 
+        Within each pool, jobs can be scheduled using either fair sharing or 
+        first-in-first-out (FIFO) scheduling.
       </p>
       </p>
       <p>
       <p>
         In addition to providing fair sharing, the Fair Scheduler allows
         In addition to providing fair sharing, the Fair Scheduler allows
-        assigning guaranteed minimum shares to pools, which is useful for
-        ensuring that certain users, groups or production applications
+        assigning guaranteed <em>minimum shares</em> to pools, which is useful
+        for ensuring that certain users, groups or production applications
         always get sufficient resources. When a pool contains jobs, it gets
         always get sufficient resources. When a pool contains jobs, it gets
         at least its minimum share, but when the pool does not need its full
         at least its minimum share, but when the pool does not need its full
-        guaranteed share, the excess is split between other running jobs.
-        This lets the scheduler guarantee capacity for pools while utilizing
-        resources efficiently when these pools don't contain jobs.       
+        guaranteed share, the excess is split between other pools.
       </p>
       </p>
       <p>
       <p>
-        The Fair Scheduler lets all jobs run by default, but it is also
-        possible to limit the number of running jobs per user and per pool
-        through the config file. This can be useful when a user must submit
-        hundreds of jobs at once, or in general to improve performance if
-        running too many jobs at once would cause too much intermediate data
-        to be created or too much context-switching. Limiting the jobs does
-        not cause any subsequently submitted jobs to fail, only to wait in the
-        sheduler's queue until some of the user's earlier jobs finish. Jobs to
-        run from each user/pool are chosen in order of priority and then
-        submit time, as in the default FIFO scheduler in Hadoop.
+        If a pool's minimum share is not met for some period of time, the
+        scheduler optionally supports <em>preemption</em> of jobs in other
+        pools. The pool will be allowed to kill tasks from other pools to make
+        room to run. Preemption can be used to guarantee
+        that "production" jobs are not starved while also allowing
+        the Hadoop cluster to also be used for experimental and research jobs.
+        In addition, a pool can also be allowed to preempt tasks if it is
+        below half of its fair share for a configurable timeout (generally
+        set larger than the minimum share preemption timeout).
+        When choosing tasks to kill, the fair scheduler picks the
+        most-recently-launched tasks from over-allocated jobs, 
+        to minimize wasted computation.
+        Preemption does not cause the preempted jobs to fail, because Hadoop
+        jobs tolerate losing tasks; it only makes them take longer to finish.
       </p>
       </p>
       <p>
       <p>
-        Finally, the fair scheduler provides several extension points where
-        the basic functionality can be extended. For example, the weight
-        calculation can be modified to give a priority boost to new jobs,
-        implementing a "shortest job first" policy which reduces response
-        times for interactive jobs even further.
+        The Fair Scheduler can limit the number of concurrent
+        running jobs per user and per pool. This can be useful when a 
+        user must submit hundreds of jobs at once, or for ensuring that
+        intermediate data does not fill up disk space on a cluster when too many
+        concurrent jobs are running.
+        Setting job limits causes jobs submitted beyond the limit to wait
+        until some of the user/pool's earlier jobs finish.
+        Jobs to run from each user/pool are chosen in order of priority and then
+        submit time.
+      </p>
+      <p>
+        Finally, the Fair Scheduler can limit the number of concurrent
+        running tasks per pool. This can be useful when jobs have a
+        dependency on an external service like a database or web
+        service that could be overloaded if too many map or reduce
+        tasks are run at once.
       </p>
       </p>
     </section>
     </section>
 
 
@@ -93,184 +102,367 @@
       <p>
       <p>
         To run the fair scheduler in your Hadoop installation, you need to put
         To run the fair scheduler in your Hadoop installation, you need to put
         it on the CLASSPATH. The easiest way is to copy the 
         it on the CLASSPATH. The easiest way is to copy the 
-        <em>hadoop-fairscheduler-*.jar</em> from
-        <em>HADOOP_HOME/contrib/fairscheduler</em> to <em>HADOOP_HOME/lib</em>.
+        <em>hadoop-*-fairscheduler.jar</em> from
+        <em>HADOOP_HOME/build/contrib/fairscheduler</em> to <em>HADOOP_HOME/lib</em>.
         Alternatively you can modify <em>HADOOP_CLASSPATH</em> to include this jar, in
         Alternatively you can modify <em>HADOOP_CLASSPATH</em> to include this jar, in
         <em>HADOOP_CONF_DIR/hadoop-env.sh</em>
         <em>HADOOP_CONF_DIR/hadoop-env.sh</em>
       </p>
       </p>
-      <p>
-        In order to compile fair scheduler, from sources execute <em> ant 
-        package</em> in source folder and copy the 
-        <em>build/contrib/fair-scheduler/hadoop-fairscheduler-*.jar</em> 
-        to <em>HADOOP_HOME/lib</em>
-      </p>
       <p>
       <p>
        You will also need to set the following property in the Hadoop config 
        You will also need to set the following property in the Hadoop config 
        file  <em>HADOOP_CONF_DIR/mapred-site.xml</em> to have Hadoop use 
        file  <em>HADOOP_CONF_DIR/mapred-site.xml</em> to have Hadoop use 
-       the fair scheduler: <br/>
-       <code>&lt;property&gt;</code><br/> 
-       <code>&nbsp;&nbsp;&lt;name&gt;mapred.jobtracker.taskScheduler&lt;/name&gt;</code><br/>
-       <code>&nbsp;&nbsp;&lt;value&gt;org.apache.hadoop.mapred.FairScheduler&lt;/value&gt;</code><br/>
-       <code>&lt;/property&gt;</code>
+       the fair scheduler:
       </p>
       </p>
+<source>
+&lt;property&gt;
+  &lt;name&gt;mapred.jobtracker.taskScheduler&lt;/name&gt;
+  &lt;value&gt;org.apache.hadoop.mapred.FairScheduler&lt;/value&gt;
+&lt;/property&gt;
+</source>
       <p>
       <p>
         Once you restart the cluster, you can check that the fair scheduler 
         Once you restart the cluster, you can check that the fair scheduler 
-        is running by going to http://&lt;jobtracker URL&gt;/scheduler 
+        is running by going to <em>http://&lt;jobtracker URL&gt;/scheduler</em> 
         on the JobTracker's web UI. A &quot;job scheduler administration&quot; page should 
         on the JobTracker's web UI. A &quot;job scheduler administration&quot; page should 
         be visible there. This page is described in the Administration section.
         be visible there. This page is described in the Administration section.
       </p>
       </p>
+      <p>
+        If you wish to compile the fair scheduler from source, run <em> ant 
+        package</em> in your HADOOP_HOME directory. This will build
+        <em>build/contrib/fair-scheduler/hadoop-*-fairscheduler.jar</em>.
+      </p>
     </section>
     </section>
     
     
     <section>
     <section>
-      <title>Configuring the Fair scheduler</title>
+      <title>Configuration</title>
       <p>
       <p>
-      The following properties can be set in mapred-site.xml to configure 
-      the fair scheduler:
+        The Fair Scheduler contains configuration in two places -- algorithm
+        parameters are set in <em>HADOOP_CONF_DIR/mapred-site.xml</em>, while 
+        a separate XML file called the <em>allocation file</em>, 
+        located by default in
+        <em>HADOOP_CONF_DIR/fair-scheduler.xml</em>, is used to configure
+        pools, minimum shares, running job limits and preemption timeouts.
+        The allocation file is reloaded periodically at runtime, 
+        allowing you to change pool settings without restarting 
+        your Hadoop cluster.
       </p>
       </p>
-      <table>
-        <tr>
-        <th>Name</th><th>Description</th>
-        </tr>
-        <tr>
-        <td>
-          mapred.fairscheduler.allocation.file
-        </td>
-        <td>
-          Specifies an absolute path to an XML file which contains the 
-          allocations for each pool, as well as the per-pool and per-user 
-          limits on number of running jobs. If this property is not 
-          provided, allocations are not used.<br/>
-          This file must be in XML format, and can contain three types of 
-          elements:
+      <p>
+        For a minimal installation, to just get equal sharing between users,
+        you will not need to edit the allocation file.
+      </p>
+      <section>
+      <title>Scheduler Parameters in mapred-site.xml</title>
+        <p>
+          The following parameters can be set in <em>mapred-site.xml</em>
+          to affect the behavior of the fair scheduler:
+        </p>
+        <p><strong>Basic Parameters</strong></p>
+        <table>
+          <tr>
+          <th>Name</th><th>Description</th>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.preemption
+          </td>
+          <td>
+            Boolean property for enabling preemption. Default: false.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.pool
+          </td>
+          <td>
+            Specify the pool that a job belongs in.  
+            If this is specified then mapred.fairscheduler.poolnameproperty is ignored.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.poolnameproperty
+          </td>
+          <td>
+            Specify which jobconf property is used to determine the pool that a
+            job belongs in. String, default: <em>user.name</em>
+            (i.e. one pool for each user). 
+            Another useful value is <em>mapred.job.queue.name</em> to use MapReduce's "queue"
+            system for access control lists (see below).
+            mapred.fairscheduler.poolnameproperty is used only for jobs in which 
+            mapred.fairscheduler.pool is not explicitly set.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.allocation.file
+          </td>
+          <td>
+            Can be used to have the scheduler use a different allocation file
+            than the default one (<em>HADOOP_CONF_DIR/fair-scheduler.xml</em>).
+            Must be an absolute path to the allocation file.
+          </td>
+          </tr>
+        </table>
+        <p> <br></br></p>
+        <p><strong>Advanced Parameters</strong> </p>
+        <table>
+          <tr>
+          <th>Name</th><th>Description</th>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.sizebasedweight
+          </td>
+          <td>
+            Take into account job sizes in calculating their weights for fair 
+            sharing. By default, weights are only based on job priorities. 
+            Setting this flag to true will make them based on the size of the 
+            job (number of tasks needed) as well,though not linearly 
+            (the weight will be proportional to the log of the number of tasks 
+            needed). This lets larger jobs get larger fair shares while still 
+            providing enough of a share to small jobs to let them finish fast. 
+            Boolean value, default: false.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.preemption.only.log
+          </td>
+          <td>
+            This flag will cause the scheduler to run through the preemption
+            calculations but simply log when it wishes to preempt a task,
+            without actually preempting the task. 
+            Boolean property, default: false.
+            This property can be useful for
+            doing a "dry run" of preemption before enabling it to make sure
+            that you have not set timeouts too aggressively.
+            You will see preemption log messages in your JobTracker's output
+            log (<em>HADOOP_LOG_DIR/hadoop-jobtracker-*.log</em>).
+            The messages look as follows:<br/>
+            <code>Should preempt 2 tasks for job_20090101337_0001: tasksDueToMinShare = 2, tasksDueToFairShare = 0</code>
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.update.interval
+          </td>
+          <td>
+            Interval at which to update fair share calculations. The default
+            of 500ms works well for clusters with fewer than 500 nodes, 
+            but larger values reduce load on the JobTracker for larger clusters.
+            Integer value in milliseconds, default: 500.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.preemption.interval
+          </td>
+          <td>
+            Interval at which to check for tasks to preempt. The default
+            of 15s works well for timeouts on the order of minutes.
+            It is not recommended to set timeouts much smaller than this
+            amount, but you can use this value to make preemption computations
+            run more often if you do set such timeouts. A value of less than
+            5s will probably be too small, however, as it becomes less than
+            the inter-heartbeat interval.
+            Integer value in milliseconds, default: 15000.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.weightadjuster
+          </td>
+          <td>
+          An extension point that lets you specify a class to adjust the 
+          weights of running jobs. This class should implement the 
+          <em>WeightAdjuster</em> interface. There is currently one example 
+          implementation - <em>NewJobWeightBooster</em>, which increases the 
+          weight of jobs for the first 5 minutes of their lifetime to let 
+          short jobs finish faster. To use it, set the weightadjuster 
+          property to the full class name, 
+          <code>org.apache.hadoop.mapred.NewJobWeightBooster</code>.
+          NewJobWeightBooster itself provides two parameters for setting the 
+          duration and boost factor.
           <ul>
           <ul>
-          <li>pool elements, which may contain elements for minMaps, 
-          minReduces, maxRunningJobs (limit the number of jobs from the 
-          pool to run at once),and weight (to share the cluster 
-          non-proportionally with other pools).
-          </li>
-          <li>user elements, which may contain a maxRunningJobs to limit 
-          jobs. Note that by default, there is a separate pool for each 
-          user, so these may not be necessary; they are useful, however, 
-          if you create a pool per user group or manually assign jobs 
-          to pools.</li>
-          <li>A userMaxJobsDefault element, which sets the default running 
-          job limit for any users whose limit is not specified.</li>
+          <li><em>mapred.newjobweightbooster.factor</em>
+            Factor by which new jobs weight should be boosted. 
+            Default is 3.</li>
+          <li><em>mapred.newjobweightbooster.duration</em>
+            Boost duration in milliseconds. Default is 300000 for 5 minutes.</li>
           </ul>
           </ul>
-          <br/>
-          Example Allocation file is listed below :<br/>
-          <code>&lt;?xml version="1.0"?&gt; </code> <br/>
-          <code>&lt;allocations&gt;</code> <br/> 
-          <code>&nbsp;&nbsp;&lt;pool name="sample_pool"&gt;</code><br/>
-          <code>&nbsp;&nbsp;&nbsp;&nbsp;&lt;minMaps&gt;5&lt;/minMaps&gt;</code><br/>
-          <code>&nbsp;&nbsp;&nbsp;&nbsp;&lt;minReduces&gt;5&lt;/minReduces&gt;</code><br/>
-          <code>&nbsp;&nbsp;&nbsp;&nbsp;&lt;weight&gt;2.0&lt;/weight&gt;</code><br/>
-          <code>&nbsp;&nbsp;&lt;/pool&gt;</code><br/>
-          <code>&nbsp;&nbsp;&lt;user name="sample_user"&gt;</code><br/>
-          <code>&nbsp;&nbsp;&nbsp;&nbsp;&lt;maxRunningJobs&gt;6&lt;/maxRunningJobs&gt;</code><br/>
-          <code>&nbsp;&nbsp;&lt;/user&gt;</code><br/>
-          <code>&nbsp;&nbsp;&lt;userMaxJobsDefault&gt;3&lt;/userMaxJobsDefault&gt;</code><br/>
-          <code>&lt;/allocations&gt;</code>
-          <br/>
-          This example creates a pool sample_pool with a guarantee of 5 map 
-          slots and 5 reduce slots. The pool also has a weight of 2.0, meaning 
-          it has a 2x higher share of the cluster than other pools (the default 
-          weight is 1). Finally, the example limits the number of running jobs 
-          per user to 3, except for sample_user, who can run 6 jobs concurrently. 
-          Any pool not defined in the allocations file will have no guaranteed 
-          capacity and a weight of 1.0. Also, any pool or user with no max 
-          running jobs set in the file will be allowed to run an unlimited 
-          number of jobs.
-        </td>
-        </tr>
-        <tr>
-        <td>
-          mapred.fairscheduler.assignmultiple
-        </td>
-        <td>
-          Allows the scheduler to assign both a map task and a reduce task 
-          on each heartbeat, which improves cluster throughput when there 
-          are many small tasks to run. Boolean value, default: false.
-        </td>
-        </tr>
-        <tr>
-        <td>
-          mapred.fairscheduler.sizebasedweight
-        </td>
-        <td>
-          Take into account job sizes in calculating their weights for fair 
-          sharing.By default, weights are only based on job priorities. 
-          Setting this flag to true will make them based on the size of the 
-          job (number of tasks needed) as well,though not linearly 
-          (the weight will be proportional to the log of the number of tasks 
-          needed). This lets larger jobs get larger fair shares while still 
-          providing enough of a share to small jobs to let them finish fast. 
-          Boolean value, default: false.
-        </td>
-        </tr>
-        <tr>
-        <td>
-          mapred.fairscheduler.poolnameproperty
-        </td>
-        <td>
-          Specify which jobconf property is used to determine the pool that a
-          job belongs in. String, default: user.name (i.e. one pool for each 
-          user). Some other useful values to set this to are: <br/>
-          <ul> 
-            <li> group.name (to create a pool per Unix group).</li>
-            <li>mapred.job.queue.name (the same property as the queue name in 
-            <a href="capacity_scheduler.html">Capacity Scheduler</a>).</li>
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.loadmanager
+          </td>
+          <td>
+            An extension point that lets you specify a class that determines 
+            how many maps and reduces can run on a given TaskTracker. This class 
+            should implement the LoadManager interface. By default the task caps 
+            in the Hadoop config file are used, but this option could be used to 
+            make the load based on available memory and CPU utilization for example.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.taskselector
+          </td>
+          <td>
+          An extension point that lets you specify a class that determines 
+          which task from within a job to launch on a given tracker. This can be 
+          used to change either the locality policy (e.g. keep some jobs within 
+          a particular rack) or the speculative execution algorithm (select 
+          when to launch speculative tasks). The default implementation uses 
+          Hadoop's default algorithms from JobInProgress.
+          </td>
+          </tr>
+          <!--
+          <tr>
+          <td>
+            mapred.fairscheduler.eventlog.enabled
+          </td>
+          <td>
+            Enable a detailed log of fair scheduler events, useful for
+            debugging.
+            This log is stored in <em>HADOOP_LOG_DIR/fairscheduler</em>.
+            Boolean value, default: false.
+          </td>
+          </tr>
+          <tr>
+          <td>
+            mapred.fairscheduler.dump.interval
+          </td>
+          <td>
+            If using the event log, this is the interval at which to dump
+            complete scheduler state (list of pools and jobs) to the log.
+            Integer value in milliseconds, default: 10000.
+          </td>
+          </tr>
+          -->
+        </table>
+      </section>  
+      <section>
+        <title>Allocation File (fair-scheduler.xml)</title>
+        <p>
+        The allocation file configures minimum shares, running job
+        limits, weights and preemption timeouts for each pool.
+        Only users/pools whose values differ from the defaults need to be
+        explicitly configured in this file.
+        The allocation file is located in
+        <em>HADOOP_HOME/conf/fair-scheduler.xml</em>.
+        It can contain the following types of elements:
+        </p>
+        <ul>
+        <li><em>pool</em> elements, which configure each pool.
+        These may contain the following sub-elements:
+          <ul>
+          <li><em>minMaps</em> and <em>minReduces</em>,
+            to set the pool's minimum share of task slots.</li>
+          <li><em>maxMaps</em> and <em>maxReduces</em>, to set the
+            pool's maximum concurrent task slots.</li>
+          <li><em>schedulingMode</em>, the pool's internal scheduling mode,
+          which can be <em>fair</em> for fair sharing or <em>fifo</em> for
+          first-in-first-out.</li>
+          <li><em>maxRunningJobs</em>, 
+          to limit the number of jobs from the 
+          pool to run at once (defaults to infinite).</li>
+          <li><em>weight</em>, to share the cluster 
+          non-proportionally with other pools. For example, a pool with weight 2.0 will get a 2x higher share than other pools. The default weight is 1.0.</li>
+          <li><em>minSharePreemptionTimeout</em>, the
+            number of seconds the pool will wait before
+            killing other pools' tasks if it is below its minimum share
+            (defaults to infinite).</li>
           </ul>
           </ul>
-        </td>
-        </tr>
-        <tr>
-        <td>
-          mapred.fairscheduler.weightadjuster
-        </td>
-        <td>
-        An extensibility point that lets you specify a class to adjust the 
-        weights of running jobs. This class should implement the 
-        <em>WeightAdjuster</em> interface. There is currently one example 
-        implementation - <em>NewJobWeightBooster</em>, which increases the 
-        weight of jobs for the first 5 minutes of their lifetime to let 
-        short jobs finish faster. To use it, set the weightadjuster 
-        property to the full class name, 
-        <code>org.apache.hadoop.mapred.NewJobWeightBooster</code> 
-        NewJobWeightBooster itself provides two parameters for setting the 
-        duration and boost factor. <br/>
-        <ol>
-        <li> <em>mapred.newjobweightbooster.factor</em>
-          Factor by which new jobs weight should be boosted. Default is 3</li>
-        <li><em>mapred.newjobweightbooster.duration</em>
-          Duration in milliseconds, default 300000 for 5 minutes</li>
-        </ol>
-        </td>
-        </tr>
-        <tr>
-        <td>
-          mapred.fairscheduler.loadmanager
-        </td>
-        <td>
-          An extensibility point that lets you specify a class that determines 
-          how many maps and reduces can run on a given TaskTracker. This class 
-          should implement the LoadManager interface. By default the task caps 
-          in the Hadoop config file are used, but this option could be used to 
-          make the load based on available memory and CPU utilization for example.
-        </td>
-        </tr>
-        <tr>
-        <td>
-          mapred.fairscheduler.taskselector:
-        </td>
-        <td>
-        An extensibility point that lets you specify a class that determines 
-        which task from within a job to launch on a given tracker. This can be 
-        used to change either the locality policy (e.g. keep some jobs within 
-        a particular rack) or the speculative execution algorithm (select 
-        when to launch speculative tasks). The default implementation uses 
-        Hadoop's default algorithms from JobInProgress.
-        </td>
-        </tr>
-      </table>      
+        </li>
+        <li><em>user</em> elements, which may contain a 
+        <em>maxRunningJobs</em> element to limit 
+        jobs. Note that by default, there is a pool for each 
+        user, so per-user limits are not necessary.</li>
+        <li><em>poolMaxJobsDefault</em>, which sets the default running 
+        job limit for any pools whose limit is not specified.</li>
+        <li><em>userMaxJobsDefault</em>, which sets the default running 
+        job limit for any users whose limit is not specified.</li>
+        <li><em>defaultMinSharePreemptionTimeout</em>, 
+        which sets the default minimum share preemption timeout 
+        for any pools where it is not specified.</li>
+        <li><em>fairSharePreemptionTimeout</em>, 
+        which sets the preemption timeout used when jobs are below half
+        their fair share.</li>
+        <li><em>defaultPoolSchedulingMode</em>, which sets the default scheduling 
+        mode (<em>fair</em> or <em>fifo</em>) for pools whose mode is
+        not specified.</li>
+        </ul>
+        <p>
+        Pool and user elements only required if you are setting
+        non-default values for the pool/user. That is, you do not need to
+        declare all users and all pools in your config file before running
+        the fair scheduler. If a user or pool is not listed in the config file,
+        the default values for limits, preemption timeouts, etc will be used.
+        </p>
+        <p>
+        An example allocation file is given below : </p>
+<source>
+&lt;?xml version="1.0"?&gt;  
+&lt;allocations&gt;  
+  &lt;pool name="sample_pool"&gt;
+    &lt;minMaps&gt;5&lt;/minMaps&gt;
+    &lt;minReduces&gt;5&lt;/minReduces&gt;
+    &lt;maxMaps&gt;25&lt;/maxMaps&gt;
+    &lt;maxReduces&gt;25&lt;/maxReduces&gt;
+    &lt;minSharePreemptionTimeout&gt;300&lt;/minSharePreemptionTimeout&gt;
+  &lt;/pool&gt;
+  &lt;user name="sample_user"&gt;
+    &lt;maxRunningJobs&gt;6&lt;/maxRunningJobs&gt;
+  &lt;/user&gt;
+  &lt;userMaxJobsDefault&gt;3&lt;/userMaxJobsDefault&gt;
+  &lt;fairSharePreemptionTimeout&gt;600&lt;/fairSharePreemptionTimeout&gt;
+&lt;/allocations&gt;
+</source>
+        <p>
+        This example creates a pool sample_pool with a guarantee of 5 map 
+        slots and 5 reduce slots. The pool also has a minimum share preemption
+        timeout of 300 seconds (5 minutes), meaning that if it does not get its
+        guaranteed share within this time, it is allowed to kill tasks from
+        other pools to achieve its share. The pool has a cap of 25 map and 25
+        reduce slots, which means that once 25 tasks are running, no more will
+        be scheduled even if the pool's fair share is higher.
+        The example also limits the number of running jobs 
+        per user to 3, except for sample_user, who can run 6 jobs concurrently. 
+        Finally, the example sets a fair share preemption timeout of 600 seconds
+        (10 minutes). If a job is below half its fair share for 10 minutes, it
+        will be allowed to kill tasks from other jobs to achieve its share.
+        Note that the preemption settings require preemption to be
+        enabled in <em>mapred-site.xml</em> as described earlier.
+        </p>
+        <p>
+        Any pool not defined in the allocation file will have no guaranteed 
+        capacity and no preemption timeout. Also, any pool or user with no max 
+        running jobs set in the file will be allowed to run an unlimited 
+        number of jobs.
+        </p>
+      </section>
+      <section>
+        <title>Access Control Lists (ACLs)</title>
+        <p>
+          The fair scheduler can be used in tandem with the "queue" based access
+          control system in MapReduce to restrict which pools each user can access.
+          To do this, first enable ACLs and set up some queues as described in the
+          <a href="mapred_tutorial.html#Job+Authorization">MapReduce usage guide</a>,
+          then set the fair scheduler to use one pool per queue by adding
+          the following property in <em>HADOOP_CONF_DIR/mapred-site.xml</em>:
+        </p>
+<source>
+&lt;property&gt;
+  &lt;name&gt;mapred.fairscheduler.poolnameproperty&lt;/name&gt;
+  &lt;value&gt;mapred.job.queue.name&lt;/value&gt;
+&lt;/property&gt;
+</source>
+        <p>
+          You can then set the minimum share, weight, and internal scheduling mode
+          for each pool as described earlier.
+          In addition, make sure that users submit jobs to the right queue by setting
+          the <em>mapred.job.queue.name</em> property in their jobs.
+        </p>
+      </section>
     </section>
     </section>
     <section>
     <section>
     <title> Administration</title>
     <title> Administration</title>
@@ -280,14 +472,15 @@
     </p> 
     </p> 
     <ol>
     <ol>
     <li>
     <li>
-      It is possible to modify pools' allocations 
-      and user and pool running job limits at runtime by editing the allocation 
-      config file. The scheduler will reload this file 10-15 seconds after it 
+      It is possible to modify minimum shares, limits, weights, preemption
+      timeouts and pool scheduling modes at runtime by editing the allocation
+      file. The scheduler will reload this file 10-15 seconds after it 
       sees that it was modified.
       sees that it was modified.
      </li>
      </li>
      <li>
      <li>
      Current jobs, pools, and fair shares  can be examined through the 
      Current jobs, pools, and fair shares  can be examined through the 
-     JobTracker's web interface, at  http://&lt;jobtracker URL&gt;/scheduler. 
+     JobTracker's web interface, at
+     <em>http://&lt;JobTracker URL&gt;/scheduler</em>. 
      On this interface, it is also possible to modify jobs' priorities or 
      On this interface, it is also possible to modify jobs' priorities or 
      move jobs from one pool to another and see the effects on the fair 
      move jobs from one pool to another and see the effects on the fair 
      shares (this requires JavaScript).
      shares (this requires JavaScript).
@@ -312,24 +505,36 @@
      the job has had, but on average it will get its fair share amount.</li>
      the job has had, but on average it will get its fair share amount.</li>
      </ul>
      </ul>
      <p>
      <p>
-     In addition, it is possible to turn on an "advanced" view for the web UI,
-     by going to http://&lt;jobtracker URL&gt;/scheduler?advanced. This view shows 
-     four more columns used for calculations internally:
+     In addition, it is possible to view an "advanced" version of the web 
+     UI by going to <em>http://&lt;JobTracker URL&gt;/scheduler?advanced</em>. 
+     This view shows two more columns:
      </p>
      </p>
      <ul>
      <ul>
      <li><em>Maps/Reduce Weight</em>: Weight of the job in the fair sharing 
      <li><em>Maps/Reduce Weight</em>: Weight of the job in the fair sharing 
      calculations. This depends on priority and potentially also on 
      calculations. This depends on priority and potentially also on 
      job size and job age if the <em>sizebasedweight</em> and 
      job size and job age if the <em>sizebasedweight</em> and 
      <em>NewJobWeightBooster</em> are enabled.</li>
      <em>NewJobWeightBooster</em> are enabled.</li>
-     <li><em>Map/Reduce Deficit</em>: The job's scheduling deficit in machine-
-     seconds - the amount of resources it should have gotten according to 
-     its fair share, minus how many it actually got. Positive deficit means
-      the job will be scheduled again in the near future because it needs to 
-      catch up to its fair share. The scheduler schedules jobs with higher 
-      deficit ahead of others. Please see the Implementation section of 
-      this document for details.</li>
      </ul>
      </ul>
     </section>
     </section>
+    <section>
+      <title>Metrics</title>
+      <p>
+        The fair scheduler can export metrics using the Hadoop metrics interface.
+        This can be enabled by adding an entry to <code>hadoop-metrics.properties</code>
+        to enable the <code>fairscheduler</code> metrics context. For example, to
+        simply retain the metrics in memory so they may be viewed in the <code>/metrics</code>
+        servlet:
+      </p>
+      <p>
+        <code>fairscheduler.class=org.apache.hadoop.metrics.spi.NoEmitMetricsContext</code>
+      </p>
+      <p>
+        Metrics are generated for each pool and job, and contain the same information that
+        is visible on the <code>/scheduler</code> web page.
+      </p>
+    </section>
+
+    <!--
     <section>
     <section>
     <title>Implementation</title>
     <title>Implementation</title>
     <p>There are two aspects to implementing fair scheduling: Calculating 
     <p>There are two aspects to implementing fair scheduling: Calculating 
@@ -359,13 +564,31 @@
      This capacity is divided among the jobs in that pool according again to 
      This capacity is divided among the jobs in that pool according again to 
      their weights.
      their weights.
      </p>
      </p>
-     <p>Finally, when limits on a user's running jobs or a pool's running jobs 
+     <p>When limits on a user's running jobs or a pool's running jobs 
      are in place, we choose which jobs get to run by sorting all jobs in order 
      are in place, we choose which jobs get to run by sorting all jobs in order 
      of priority and then submit time, as in the standard Hadoop scheduler. Any 
      of priority and then submit time, as in the standard Hadoop scheduler. Any 
      jobs that fall after the user/pool's limit in this ordering are queued up 
      jobs that fall after the user/pool's limit in this ordering are queued up 
      and wait idle until they can be run. During this time, they are ignored 
      and wait idle until they can be run. During this time, they are ignored 
      from the fair sharing calculations and do not gain or lose deficit (their 
      from the fair sharing calculations and do not gain or lose deficit (their 
      fair share is set to zero).</p>
      fair share is set to zero).</p>
+     <p>
+     Preemption is implemented by periodically checking whether jobs are
+     below their minimum share or below half their fair share. If a job has
+     been below its share for sufficiently long, it is allowed to kill
+     other jobs' tasks. The tasks chosen are the most-recently-launched
+     tasks from over-allocated jobs, to minimize the amount of wasted
+     computation.
+     </p>
+     <p>
+     Finally, the fair scheduler provides several extension points where
+     the basic functionality can be extended. For example, the weight
+     calculation can be modified to give a priority boost to new jobs,
+     implementing a "shortest job first" policy which reduces response
+     times for interactive jobs even further.
+     These extension points are listed in
+     <a href="#Scheduler+Parameters+in+mapred-site.xml">Advanced Parameters</a>.
+     </p>
     </section>
     </section>
+    -->
   </body>  
   </body>  
 </document>
 </document>

+ 4 - 0
src/docs/src/documentation/content/xdocs/site.xml

@@ -36,6 +36,10 @@ See http://forrest.apache.org/docs/linking.html for more info.
     <single   label="Single Node Setup"  href="single_node_setup.html" />
     <single   label="Single Node Setup"  href="single_node_setup.html" />
     <cluster  label="Cluster Setup"  href="cluster_setup.html" />
     <cluster  label="Cluster Setup"  href="cluster_setup.html" />
   </docs>  
   </docs>  
+
+   <docs label="Guides">
+    <http_authentication label="Authentication for Hadoop HTTP web-consoles" href="HttpAuthentication.html"/>
+   </docs>
        
        
   <docs label="MapReduce">
   <docs label="MapReduce">
     <mapred     label="MapReduce Tutorial"   href="mapred_tutorial.html" />
     <mapred     label="MapReduce Tutorial"   href="mapred_tutorial.html" />

Деякі файли не було показано, через те що забагато файлів було змінено