hadoop git commit: MAPREDUCE-5583. Ability to limit running map and reduce tasks. Contributed by Jason Lowe.
Repository: hadoop Updated Branches: refs/heads/trunk 9ae7f9eb7 - 4228de940 MAPREDUCE-5583. Ability to limit running map and reduce tasks. Contributed by Jason Lowe. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4228de94 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4228de94 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4228de94 Branch: refs/heads/trunk Commit: 4228de94028f1e10ca59ce23e963e488fe566909 Parents: 9ae7f9e Author: Junping Du junping...@apache.org Authored: Tue Mar 3 02:01:04 2015 -0800 Committer: Junping Du junping...@apache.org Committed: Tue Mar 3 02:02:28 2015 -0800 -- hadoop-mapreduce-project/CHANGES.txt| 3 + .../v2/app/rm/RMContainerAllocator.java | 65 +- .../v2/app/rm/RMContainerRequestor.java | 74 ++- .../v2/app/rm/TestRMContainerAllocator.java | 214 +++ .../apache/hadoop/mapreduce/MRJobConfig.java| 8 + .../src/main/resources/mapred-default.xml | 16 ++ 6 files changed, 363 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/4228de94/hadoop-mapreduce-project/CHANGES.txt -- diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 5524b14..7a2eff3 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -258,6 +258,9 @@ Release 2.7.0 - UNRELEASED MAPREDUCE-6228. Add truncate operation to SLive. (Plamen Jeliazkov via shv) +MAPREDUCE-5583. Ability to limit running map and reduce tasks. +(Jason Lowe via junping_du) + IMPROVEMENTS MAPREDUCE-6149. Document override log4j.properties in MR job. http://git-wip-us.apache.org/repos/asf/hadoop/blob/4228de94/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java index 1acfeec..efea674 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java @@ -99,9 +99,9 @@ public class RMContainerAllocator extends RMContainerRequestor public static final float DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART = 0.05f; - private static final Priority PRIORITY_FAST_FAIL_MAP; - private static final Priority PRIORITY_REDUCE; - private static final Priority PRIORITY_MAP; + static final Priority PRIORITY_FAST_FAIL_MAP; + static final Priority PRIORITY_REDUCE; + static final Priority PRIORITY_MAP; @VisibleForTesting public static final String RAMPDOWN_DIAGNOSTIC = Reducer preempted @@ -166,6 +166,8 @@ public class RMContainerAllocator extends RMContainerRequestor */ private long allocationDelayThresholdMs = 0; private float reduceSlowStart = 0; + private int maxRunningMaps = 0; + private int maxRunningReduces = 0; private long retryInterval; private long retrystartTime; private Clock clock; @@ -201,6 +203,10 @@ public class RMContainerAllocator extends RMContainerRequestor allocationDelayThresholdMs = conf.getInt( MRJobConfig.MR_JOB_REDUCER_PREEMPT_DELAY_SEC, MRJobConfig.DEFAULT_MR_JOB_REDUCER_PREEMPT_DELAY_SEC) * 1000;//sec - ms +maxRunningMaps = conf.getInt(MRJobConfig.JOB_RUNNING_MAP_LIMIT, +MRJobConfig.DEFAULT_JOB_RUNNING_MAP_LIMIT); +maxRunningReduces = conf.getInt(MRJobConfig.JOB_RUNNING_REDUCE_LIMIT, +MRJobConfig.DEFAULT_JOB_RUNNING_REDUCE_LIMIT); RackResolver.init(conf); retryInterval = getConfig().getLong(MRJobConfig.MR_AM_TO_RM_WAIT_INTERVAL_MS, MRJobConfig.DEFAULT_MR_AM_TO_RM_WAIT_INTERVAL_MS); @@ -664,6 +670,8 @@ public class RMContainerAllocator extends RMContainerRequestor @SuppressWarnings(unchecked) private ListContainer getResources() throws Exception { +applyConcurrentTaskLimits(); + // will be null the first time Resource headRoom = getAvailableResources() == null ? Resources.none() : @@ -778,6 +786,43 @@ public class RMContainerAllocator extends RMContainerRequestor return newContainers; }
[2/2] hadoop git commit: MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA.
MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9ae7f9eb Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9ae7f9eb Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9ae7f9eb Branch: refs/heads/trunk Commit: 9ae7f9eb7baeb244e1b95aabc93ad8124870b9a9 Parents: 742f9d9 Author: Tsuyoshi Ozawa oz...@apache.org Authored: Tue Mar 3 18:06:26 2015 +0900 Committer: Tsuyoshi Ozawa oz...@apache.org Committed: Tue Mar 3 18:06:26 2015 +0900 -- hadoop-mapreduce-project/CHANGES.txt| 3 ++ .../hadoop/mapred/TaskAttemptListenerImpl.java | 4 +- .../hadoop/mapreduce/v2/app/JobEndNotifier.java | 1 - .../apache/hadoop/mapreduce/v2/util/MRApps.java | 2 +- .../hadoop/filecache/DistributedCache.java | 2 +- .../org/apache/hadoop/mapred/ClusterStatus.java | 4 +- .../apache/hadoop/mapred/FileOutputFormat.java | 2 +- .../java/org/apache/hadoop/mapred/IFile.java| 2 +- .../apache/hadoop/mapred/JobACLsManager.java| 1 - .../org/apache/hadoop/mapred/JobClient.java | 8 ++-- .../java/org/apache/hadoop/mapred/JobConf.java | 49 +--- .../java/org/apache/hadoop/mapred/Mapper.java | 2 +- .../org/apache/hadoop/mapred/QueueManager.java | 30 ++-- .../org/apache/hadoop/mapred/RecordReader.java | 2 +- .../java/org/apache/hadoop/mapred/Reducer.java | 14 +++--- .../hadoop/mapred/TaskUmbilicalProtocol.java| 1 - .../apache/hadoop/mapred/lib/ChainMapper.java | 40 .../apache/hadoop/mapred/lib/ChainReducer.java | 44 +- .../hadoop/mapred/lib/MultipleOutputs.java | 29 +--- .../hadoop/mapred/lib/TokenCountMapper.java | 2 +- .../lib/aggregate/ValueAggregatorJob.java | 2 +- .../lib/aggregate/ValueAggregatorReducer.java | 3 +- .../hadoop/mapred/lib/db/DBInputFormat.java | 4 +- .../org/apache/hadoop/mapreduce/Cluster.java| 1 + .../apache/hadoop/mapreduce/ClusterMetrics.java | 6 +-- .../apache/hadoop/mapreduce/CryptoUtils.java| 10 ++-- .../java/org/apache/hadoop/mapreduce/Job.java | 2 +- .../org/apache/hadoop/mapreduce/JobContext.java | 2 - .../hadoop/mapreduce/JobSubmissionFiles.java| 2 +- .../org/apache/hadoop/mapreduce/Mapper.java | 9 ++-- .../org/apache/hadoop/mapreduce/Reducer.java| 12 ++--- .../mapreduce/filecache/DistributedCache.java | 5 +- .../lib/aggregate/ValueAggregatorJob.java | 2 +- .../hadoop/mapreduce/lib/chain/Chain.java | 4 +- .../hadoop/mapreduce/lib/chain/ChainMapper.java | 10 ++-- .../mapreduce/lib/chain/ChainReducer.java | 14 +++--- .../hadoop/mapreduce/lib/db/DBInputFormat.java | 2 +- .../hadoop/mapreduce/lib/db/DBWritable.java | 2 +- .../mapreduce/lib/join/TupleWritable.java | 2 +- .../mapreduce/lib/map/MultithreadedMapper.java | 6 +-- .../mapreduce/lib/output/FileOutputFormat.java | 2 +- .../mapreduce/lib/output/MultipleOutputs.java | 11 ++--- .../lib/partition/BinaryPartitioner.java| 2 +- .../hadoop/mapreduce/task/JobContextImpl.java | 2 - .../hadoop/mapreduce/RandomTextWriter.java | 4 +- .../apache/hadoop/mapreduce/RandomWriter.java | 5 +- .../hadoop/examples/MultiFileWordCount.java | 2 +- .../apache/hadoop/examples/QuasiMonteCarlo.java | 4 +- .../hadoop/examples/RandomTextWriter.java | 4 +- .../apache/hadoop/examples/RandomWriter.java| 5 +- .../apache/hadoop/examples/SecondarySort.java | 2 +- .../org/apache/hadoop/examples/pi/DistBbp.java | 2 +- .../apache/hadoop/examples/pi/math/Modular.java | 2 +- .../hadoop/examples/terasort/GenSort.java | 2 +- .../org/apache/hadoop/tools/CopyListing.java| 14 +++--- .../java/org/apache/hadoop/tools/DistCp.java| 4 +- .../apache/hadoop/tools/DistCpOptionSwitch.java | 2 +- .../org/apache/hadoop/tools/OptionsParser.java | 2 +- .../hadoop/tools/mapred/CopyCommitter.java | 4 +- .../apache/hadoop/tools/mapred/CopyMapper.java | 5 +- .../hadoop/tools/mapred/CopyOutputFormat.java | 4 +- .../tools/mapred/RetriableFileCopyCommand.java | 3 +- .../tools/mapred/UniformSizeInputFormat.java| 4 +- .../tools/mapred/lib/DynamicInputFormat.java| 4 +- .../tools/mapred/lib/DynamicRecordReader.java | 12 ++--- .../apache/hadoop/tools/util/DistCpUtils.java | 2 +- .../hadoop/tools/util/RetriableCommand.java | 2 +- .../hadoop/tools/util/ThrottledInputStream.java | 8 ++-- .../java/org/apache/hadoop/tools/Logalyzer.java | 4 +- .../ResourceUsageEmulatorPlugin.java| 2 +- .../fs/swift/http/RestClientBindings.java | 6 +-- .../hadoop/fs/swift/http/SwiftRestClient.java | 6 +-- .../fs/swift/snative/SwiftNativeFileSystem.java | 6 +--
[1/2] hadoop git commit: MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA.
Repository: hadoop Updated Branches: refs/heads/trunk 742f9d90c - 9ae7f9eb7 http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ae7f9eb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java index fa3708e..2c69542 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java @@ -181,7 +181,7 @@ public static final String OUTDIR = mapreduce.output.fileoutputformat.outputdir * Get the {@link Path} to the task's temporary output directory * for the map-reduce job * - * h4 id=SideEffectFilesTasks' Side-Effect Files/h4 + * b id=SideEffectFilesTasks' Side-Effect Files/b * * pSome applications need to create/write-to side-files, which differ from * the actual job-outputs. http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ae7f9eb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java index 24baa59..c31cab7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java @@ -81,7 +81,7 @@ import java.util.*; * p * Usage in Reducer: * pre - * K, V String generateFileName(K k, V v) { + * lt;K, Vgt; String generateFileName(K k, V v) { * return k.toString() + _ + v.toString(); * } * @@ -124,16 +124,16 @@ import java.util.*; * /p * * pre - * private MultipleOutputsText, Text out; + * private MultipleOutputslt;Text, Textgt; out; * * public void setup(Context context) { - * out = new MultipleOutputsText, Text(context); + * out = new MultipleOutputslt;Text, Textgt;(context); * ... * } * - * public void reduce(Text key, IterableText values, Context context) throws IOException, InterruptedException { + * public void reduce(Text key, Iterablelt;Textgt; values, Context context) throws IOException, InterruptedException { * for (Text t : values) { - * out.write(key, t, generateFileName(iparameter list.../i)); + * out.write(key, t, generateFileName(lt;iparameter list.../igt;)); * } * } * @@ -294,7 +294,6 @@ public class MultipleOutputsKEYOUT, VALUEOUT { /** * Adds a named output for the job. - * p/ * * @param job job to add the named output * @param namedOutput named output name, it has to be a word, letters http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ae7f9eb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java index 4a40840..2a89908 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java @@ -64,7 +64,7 @@ import org.apache.hadoop.mapreduce.Partitioner; * li{@link #setOffsets}/li * li{@link #setLeftOffset}/li * li{@link #setRightOffset}/li - * /ul/p + * /ul */ @InterfaceAudience.Public @InterfaceStability.Evolving
[37/43] hadoop git commit: MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ae7f9eb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java index fa3708e..2c69542 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java @@ -181,7 +181,7 @@ public static final String OUTDIR = mapreduce.output.fileoutputformat.outputdir * Get the {@link Path} to the task's temporary output directory * for the map-reduce job * - * h4 id=SideEffectFilesTasks' Side-Effect Files/h4 + * b id=SideEffectFilesTasks' Side-Effect Files/b * * pSome applications need to create/write-to side-files, which differ from * the actual job-outputs. http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ae7f9eb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java index 24baa59..c31cab7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java @@ -81,7 +81,7 @@ import java.util.*; * p * Usage in Reducer: * pre - * K, V String generateFileName(K k, V v) { + * lt;K, Vgt; String generateFileName(K k, V v) { * return k.toString() + _ + v.toString(); * } * @@ -124,16 +124,16 @@ import java.util.*; * /p * * pre - * private MultipleOutputsText, Text out; + * private MultipleOutputslt;Text, Textgt; out; * * public void setup(Context context) { - * out = new MultipleOutputsText, Text(context); + * out = new MultipleOutputslt;Text, Textgt;(context); * ... * } * - * public void reduce(Text key, IterableText values, Context context) throws IOException, InterruptedException { + * public void reduce(Text key, Iterablelt;Textgt; values, Context context) throws IOException, InterruptedException { * for (Text t : values) { - * out.write(key, t, generateFileName(iparameter list.../i)); + * out.write(key, t, generateFileName(lt;iparameter list.../igt;)); * } * } * @@ -294,7 +294,6 @@ public class MultipleOutputsKEYOUT, VALUEOUT { /** * Adds a named output for the job. - * p/ * * @param job job to add the named output * @param namedOutput named output name, it has to be a word, letters http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ae7f9eb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java index 4a40840..2a89908 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java @@ -64,7 +64,7 @@ import org.apache.hadoop.mapreduce.Partitioner; * li{@link #setOffsets}/li * li{@link #setLeftOffset}/li * li{@link #setRightOffset}/li - * /ul/p + * /ul */ @InterfaceAudience.Public @InterfaceStability.Evolving
[04/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md new file mode 100644 index 000..5e4df9f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md @@ -0,0 +1,591 @@ +!--- + 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. +-- + +Hadoop: Writing YARN Applications += + +* [Purpose](#Purpose) +* [Concepts and Flow](#Concepts_and_Flow) +* [Interfaces](#Interfaces) +* [Writing a Simple Yarn Application](#Writing_a_Simple_Yarn_Application) +* [Writing a simple Client](#Writing_a_simple_Client) +* [Writing an ApplicationMaster (AM)](#Writing_an_ApplicationMaster_AM) +* [FAQ](#FAQ) +* [How can I distribute my application's jars to all of the nodes in the YARN cluster that need it?](#How_can_I_distribute_my_applications_jars_to_all_of_the_nodes_in_the_YARN_cluster_that_need_it) +* [How do I get the ApplicationMaster's ApplicationAttemptId?](#How_do_I_get_the_ApplicationMasters_ApplicationAttemptId) +* [Why my container is killed by the NodeManager?](#Why_my_container_is_killed_by_the_NodeManager) +* [How do I include native libraries?](#How_do_I_include_native_libraries) +* [Useful Links](#Useful_Links) +* [Sample Code](#Sample_Code) + +Purpose +--- + +This document describes, at a high-level, the way to implement new Applications for YARN. + +Concepts and Flow +- + +The general concept is that an *application submission client* submits an *application* to the YARN *ResourceManager* (RM). This can be done through setting up a `YarnClient` object. After `YarnClient` is started, the client can then set up application context, prepare the very first container of the application that contains the *ApplicationMaster* (AM), and then submit the application. You need to provide information such as the details about the local files/jars that need to be available for your application to run, the actual command that needs to be executed (with the necessary command line arguments), any OS environment settings (optional), etc. Effectively, you need to describe the Unix process(es) that needs to be launched for your ApplicationMaster. + +The YARN ResourceManager will then launch the ApplicationMaster (as specified) on an allocated container. The ApplicationMaster communicates with YARN cluster, and handles application execution. It performs operations in an asynchronous fashion. During application launch time, the main tasks of the ApplicationMaster are: a) communicating with the ResourceManager to negotiate and allocate resources for future containers, and b) after container allocation, communicating YARN *NodeManager*s (NMs) to launch application containers on them. Task a) can be performed asynchronously through an `AMRMClientAsync` object, with event handling methods specified in a `AMRMClientAsync.CallbackHandler` type of event handler. The event handler needs to be set to the client explicitly. Task b) can be performed by launching a runnable object that then launches containers when there are containers allocated. As part of launching this container, the AM has to specify the `ContainerLaunchContext` that has the launch information such as command line specification, environment, etc. + +During the execution of an application, the ApplicationMaster communicates NodeManagers through `NMClientAsync` object. All container events are handled by `NMClientAsync.CallbackHandler`, associated with `NMClientAsync`. A typical callback handler handles client start, stop, status update and error. ApplicationMaster also reports execution progress to ResourceManager by handling the `getProgress()` method of `AMRMClientAsync.CallbackHandler`. + +Other than asynchronous clients, there are synchronous versions for certain workflows (`AMRMClient` and `NMClient`). The asynchronous clients are recommended because of (subjectively) simpler usages, and this article will mainly cover the asynchronous clients. Please refer to `AMRMClient` and `NMClient` for more information on
[42/43] hadoop git commit: YARN-3210. Refactored timeline aggregator according to new code organization proposed in YARN-3166. Contributed by Li Lu.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3ff7f06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestPerNodeAggregatorServer.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestPerNodeAggregatorServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestPerNodeAggregatorServer.java deleted file mode 100644 index 902047d..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestPerNodeAggregatorServer.java +++ /dev/null @@ -1,149 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * License); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an AS IS BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.aggregator; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.server.api.ContainerInitializationContext; -import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; -import org.junit.Test; - -public class TestPerNodeAggregatorServer { - private ApplicationAttemptId appAttemptId; - - public TestPerNodeAggregatorServer() { -ApplicationId appId = -ApplicationId.newInstance(System.currentTimeMillis(), 1); -appAttemptId = ApplicationAttemptId.newInstance(appId, 1); - } - - @Test - public void testAddApplication() throws Exception { -PerNodeAggregatorServer aggregator = createAggregatorAndAddApplication(); -// aggregator should have a single app -assertTrue(aggregator.hasApplication( -appAttemptId.getApplicationId().toString())); -aggregator.close(); - } - - @Test - public void testAddApplicationNonAMContainer() throws Exception { -PerNodeAggregatorServer aggregator = createAggregator(); - -ContainerId containerId = getContainerId(2L); // not an AM -ContainerInitializationContext context = -mock(ContainerInitializationContext.class); -when(context.getContainerId()).thenReturn(containerId); -aggregator.initializeContainer(context); -// aggregator should not have that app -assertFalse(aggregator.hasApplication( -appAttemptId.getApplicationId().toString())); - } - - @Test - public void testRemoveApplication() throws Exception { -PerNodeAggregatorServer aggregator = createAggregatorAndAddApplication(); -// aggregator should have a single app -String appIdStr = appAttemptId.getApplicationId().toString(); -assertTrue(aggregator.hasApplication(appIdStr)); - -ContainerId containerId = getAMContainerId(); -ContainerTerminationContext context = -mock(ContainerTerminationContext.class); -when(context.getContainerId()).thenReturn(containerId); -aggregator.stopContainer(context); -// aggregator should not have that app -assertFalse(aggregator.hasApplication(appIdStr)); -aggregator.close(); - } - - @Test - public void testRemoveApplicationNonAMContainer() throws Exception { -PerNodeAggregatorServer aggregator = createAggregatorAndAddApplication(); -// aggregator should have a single app -String appIdStr = appAttemptId.getApplicationId().toString(); -assertTrue(aggregator.hasApplication(appIdStr)); - -ContainerId containerId = getContainerId(2L); // not an AM -ContainerTerminationContext context = -
[36/43] hadoop git commit: MAPREDUCE-6268. Fix typo in Task Attempt API's URL. Contributed by Ryu Kobayashi.
MAPREDUCE-6268. Fix typo in Task Attempt API's URL. Contributed by Ryu Kobayashi. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/742f9d90 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/742f9d90 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/742f9d90 Branch: refs/heads/YARN-2928 Commit: 742f9d90c00f823ad7fea7e79702fcf238fa5721 Parents: d1c6acc Author: Tsuyoshi Ozawa oz...@apache.org Authored: Tue Mar 3 16:21:16 2015 +0900 Committer: Tsuyoshi Ozawa oz...@apache.org Committed: Tue Mar 3 16:21:16 2015 +0900 -- hadoop-mapreduce-project/CHANGES.txt | 3 +++ .../src/site/markdown/HistoryServerRest.md| 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/742f9d90/hadoop-mapreduce-project/CHANGES.txt -- diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index ccd24a6..5fd7d30 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -399,6 +399,9 @@ Release 2.7.0 - UNRELEASED MAPREDUCE-6223. TestJobConf#testNegativeValueForTaskVmem failures. (Varun Saxena via kasha) +MAPREDUCE-6268. Fix typo in Task Attempt API's URL. (Ryu Kobayashi +via ozawa) + Release 2.6.1 - UNRELEASED INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/742f9d90/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/markdown/HistoryServerRest.md -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/markdown/HistoryServerRest.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/markdown/HistoryServerRest.md index 8a78754..b4ce00a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/markdown/HistoryServerRest.md +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/markdown/HistoryServerRest.md @@ -1889,7 +1889,7 @@ A Task Attempt resource contains information about a particular task attempt wit Use the following URI to obtain an Task Attempt Object, from a task identified by the attemptid value. - * http://history server http address:port/ws/v1/history/mapreduce/jobs/{jobid}/tasks/{taskid}/attempt/{attemptid} + * http://history server http address:port/ws/v1/history/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid} HTTP Operations Supported
hadoop git commit: YARN-3272. Surface container locality info in RM web UI (Jian He via wangda)
Repository: hadoop Updated Branches: refs/heads/branch-2 7e6624c21 - 0a502c665 YARN-3272. Surface container locality info in RM web UI (Jian He via wangda) (cherry picked from commit e17e5ba9d7e2bd45ba6884f59f8045817594b284) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0a502c66 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0a502c66 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0a502c66 Branch: refs/heads/branch-2 Commit: 0a502c665b51e114ad3624062cd440d26b0fa6e3 Parents: 7e6624c Author: Wangda Tan wan...@apache.org Authored: Tue Mar 3 11:49:01 2015 -0800 Committer: Wangda Tan wan...@apache.org Committed: Tue Mar 3 11:49:45 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 3 + .../dev-support/findbugs-exclude.xml| 7 ++ .../rmapp/attempt/RMAppAttemptMetrics.java | 21 - .../resourcemanager/scheduler/NodeType.java | 9 +- .../scheduler/SchedulerApplicationAttempt.java | 15 +++- .../scheduler/capacity/LeafQueue.java | 95 +--- .../server/resourcemanager/webapp/AppBlock.java | 45 +- .../scheduler/capacity/TestReservations.java| 8 +- 8 files changed, 163 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a502c66/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 73bcaf0..ae7b942 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -303,6 +303,9 @@ Release 2.7.0 - UNRELEASED YARN-3281. Added RMStateStore to StateMachine visualization list. (Chengbing Liu via jianhe) +YARN-3272. Surface container locality info in RM web UI. +(Jian He via wangda) + OPTIMIZATIONS YARN-2990. FairScheduler's delay-scheduling always waits for node-local and http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a502c66/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml -- diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 70f1a71..1c3f201 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -63,6 +63,13 @@ Bug pattern=BC_UNCONFIRMED_CAST / /Match Match +Class name=~org\.apache\.hadoop\.yarn\.server\.resourcemanager\.rmapp\.attempt\.RMAppAttemptMetrics / +Method name=getLocalityStatistics / +Bug pattern=EI_EXPOSE_REP / +Method name=incNumAllocatedContainers/ +Bug pattern=VO_VOLATILE_INCREMENT / + /Match + Match Class name=org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl$AppRejectedTransition / Bug pattern=BC_UNCONFIRMED_CAST / /Match http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a502c66/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java index 0e60fd5..bc22073 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.util.resource.Resources; public class RMAppAttemptMetrics { @@ -49,6 +50,10 @@ public class RMAppAttemptMetrics { private AtomicLong finishedVcoreSeconds = new AtomicLong(0); private RMContext rmContext; + private int[][] localityStatistics = + new int[NodeType.values().length][NodeType.values().length]; +
[2/2] hadoop git commit: HDFS-7757. Misleading error messages in FSImage.java. (Contributed by Brahma Reddy Battula)
HDFS-7757. Misleading error messages in FSImage.java. (Contributed by Brahma Reddy Battula) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7e6624c2 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7e6624c2 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7e6624c2 Branch: refs/heads/branch-2 Commit: 7e6624c218f759ca0a6cf58c6fff177e2a6ecd5b Parents: a9613bc Author: Arpit Agarwal a...@apache.org Authored: Tue Mar 3 10:55:22 2015 -0800 Committer: Arpit Agarwal a...@apache.org Committed: Tue Mar 3 10:55:32 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt| 3 +++ .../java/org/apache/hadoop/hdfs/server/namenode/FSImage.java | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6624c2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 354b99b..df5520e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -768,6 +768,9 @@ Release 2.7.0 - UNRELEASED HDFS-7871. NameNodeEditLogRoller can keep printing Swallowing exception message. (jing9) +HDFS-7757. Misleading error messages in FSImage.java. (Brahma Reddy Battula +via Arpit Agarwal) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6624c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java index 56d946f..c108886 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java @@ -887,7 +887,7 @@ public class FSImage implements Closeable { final long namespace = counts.getNameSpace() - parentNamespace; final long nsQuota = q.getNameSpace(); if (Quota.isViolated(nsQuota, namespace)) { -LOG.error(BUG: Namespace quota violation in image for +LOG.warn(Namespace quota violation in image for + dir.getFullPathName() + quota = + nsQuota + consumed = + namespace); } @@ -895,7 +895,7 @@ public class FSImage implements Closeable { final long ssConsumed = counts.getStorageSpace() - parentStoragespace; final long ssQuota = q.getStorageSpace(); if (Quota.isViolated(ssQuota, ssConsumed)) { -LOG.error(BUG: Storagespace quota violation in image for +LOG.warn(Storagespace quota violation in image for + dir.getFullPathName() + quota = + ssQuota + consumed = + ssConsumed); } @@ -907,7 +907,7 @@ public class FSImage implements Closeable { parentTypeSpaces.get(t); final long typeQuota = q.getTypeSpaces().get(t); if (Quota.isViolated(typeQuota, typeSpace)) { - LOG.error(BUG: Storage type quota violation in image for + LOG.warn(Storage type quota violation in image for + dir.getFullPathName() + type = + t.toString() + quota = + typeQuota + consumed + typeSpace);
[32/43] hadoop git commit: HDFS-7871. NameNodeEditLogRoller can keep printing 'Swallowing exception' message. Contributed by Jing Zhao.
HDFS-7871. NameNodeEditLogRoller can keep printing 'Swallowing exception' message. Contributed by Jing Zhao. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b442aeec Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b442aeec Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b442aeec Branch: refs/heads/YARN-2928 Commit: b442aeec95abfa1c6f835a116dfe6e186b0d841d Parents: b18d383 Author: Jing Zhao ji...@apache.org Authored: Mon Mar 2 20:22:04 2015 -0800 Committer: Jing Zhao ji...@apache.org Committed: Mon Mar 2 20:22:04 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 8 +--- 2 files changed, 8 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/b442aeec/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 52e5d3c..fe78097 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1071,6 +1071,9 @@ Release 2.7.0 - UNRELEASED HDFS-7785. Improve diagnostics information for HttpPutFailedException. (Chengbing Liu via wheat9) +HDFS-7871. NameNodeEditLogRoller can keep printing Swallowing exception +message. (jing9) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/b442aeec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 7cd194e..d2b48f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -4558,14 +4558,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, + rollThreshold); rollEditLog(); } +} catch (Exception e) { + FSNamesystem.LOG.error(Swallowing exception in + + NameNodeEditLogRoller.class.getSimpleName() + :, e); +} +try { Thread.sleep(sleepIntervalMs); } catch (InterruptedException e) { FSNamesystem.LOG.info(NameNodeEditLogRoller.class.getSimpleName() + was interrupted, exiting); break; -} catch (Exception e) { - FSNamesystem.LOG.error(Swallowing exception in - + NameNodeEditLogRoller.class.getSimpleName() + :, e); } } }
[09/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm deleted file mode 100644 index a08c19d..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm +++ /dev/null @@ -1,298 +0,0 @@ -~~ 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. - - --- - ResourceManager Restart - --- - --- - ${maven.build.timestamp} - -ResourceManager Restart - -%{toc|section=1|fromDepth=0} - -* {Overview} - - ResourceManager is the central authority that manages resources and schedules - applications running atop of YARN. Hence, it is potentially a single point of - failure in a Apache YARN cluster. - - This document gives an overview of ResourceManager Restart, a feature that - enhances ResourceManager to keep functioning across restarts and also makes - ResourceManager down-time invisible to end-users. - - ResourceManager Restart feature is divided into two phases: - - ResourceManager Restart Phase 1 (Non-work-preserving RM restart): - Enhance RM to persist application/attempt state - and other credentials information in a pluggable state-store. RM will reload - this information from state-store upon restart and re-kick the previously - running applications. Users are not required to re-submit the applications. - - ResourceManager Restart Phase 2 (Work-preserving RM restart): - Focus on re-constructing the running state of ResourceManager by combining - the container statuses from NodeManagers and container requests from ApplicationMasters - upon restart. The key difference from phase 1 is that previously running applications - will not be killed after RM restarts, and so applications won't lose its work - because of RM outage. - -* {Feature} - -** Phase 1: Non-work-preserving RM restart - - As of Hadoop 2.4.0 release, only ResourceManager Restart Phase 1 is implemented which - is described below. - - The overall concept is that RM will persist the application metadata - (i.e. ApplicationSubmissionContext) in - a pluggable state-store when client submits an application and also saves the final status - of the application such as the completion state (failed, killed, finished) - and diagnostics when the application completes. Besides, RM also saves - the credentials like security keys, tokens to work in a secure environment. - Any time RM shuts down, as long as the required information (i.e.application metadata - and the alongside credentials if running in a secure environment) is available - in the state-store, when RM restarts, it can pick up the application metadata - from the state-store and re-submit the application. RM won't re-submit the - applications if they were already completed (i.e. failed, killed, finished) - before RM went down. - - NodeManagers and clients during the down-time of RM will keep polling RM until - RM comes up. When RM becomes alive, it will send a re-sync command to - all the NodeManagers and ApplicationMasters it was talking to via heartbeats. - As of Hadoop 2.4.0 release, the behaviors for NodeManagers and ApplicationMasters to handle this command - are: NMs will kill all its managed containers and re-register with RM. From the - RM's perspective, these re-registered NodeManagers are similar to the newly joining NMs. - AMs(e.g. MapReduce AM) are expected to shutdown when they receive the re-sync command. - After RM restarts and loads all the application metadata, credentials from state-store - and populates them into memory, it will create a new - attempt (i.e. ApplicationMaster) for each application that was not yet completed - and re-kick that application as usual. As described before, the previously running - applications' work is lost in this manner since they are essentially killed by - RM via the re-sync command on restart. - -** Phase 2: Work-preserving RM restart - - As of Hadoop 2.6.0, we further enhanced RM restart feature to address the problem - to not kill any applications running on YARN cluster if RM restarts. - - Beyond all the groundwork that has been done in Phase 1
[23/43] hadoop git commit: HDFS-7789. DFSck should resolve the path to support cross-FS symlinks. (gera)
HDFS-7789. DFSck should resolve the path to support cross-FS symlinks. (gera) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cbb49257 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cbb49257 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cbb49257 Branch: refs/heads/YARN-2928 Commit: cbb492578ef09300821b7199de54c6508f9d7fe8 Parents: 67ed593 Author: Gera Shegalov g...@apache.org Authored: Thu Feb 12 04:32:43 2015 -0800 Committer: Gera Shegalov g...@apache.org Committed: Mon Mar 2 00:55:35 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 ++ .../org/apache/hadoop/hdfs/tools/DFSck.java | 31 +--- .../hadoop/hdfs/server/namenode/TestFsck.java | 14 ++--- .../namenode/TestFsckWithMultipleNameNodes.java | 20 + 4 files changed, 53 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbb49257/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 5ca16af..d5208da 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -697,6 +697,9 @@ Release 2.7.0 - UNRELEASED HDFS-7439. Add BlockOpResponseProto's message to the exception messages. (Takanobu Asanuma via szetszwo) +HDFS-7789. DFSck should resolve the path to support cross-FS symlinks. +(gera) + OPTIMIZATIONS HDFS-7454. Reduce memory footprint for AclEntries in NameNode. http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbb49257/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java index ec83a90..dc6d9d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java @@ -225,6 +225,14 @@ public class DFSck extends Configured implements Tool { return errCode; } + + private Path getResolvedPath(String dir) throws IOException { +Configuration conf = getConf(); +Path dirPath = new Path(dir); +FileSystem fs = dirPath.getFileSystem(conf); +return fs.resolvePath(dirPath); + } + /** * Derive the namenode http address from the current file system, * either default or as set by -fs in the generic options. @@ -236,19 +244,12 @@ public class DFSck extends Configured implements Tool { Configuration conf = getConf(); //get the filesystem object to verify it is an HDFS system -final FileSystem fs; -try { - fs = target.getFileSystem(conf); -} catch (IOException ioe) { - System.err.println(FileSystem is inaccessible due to:\n - + StringUtils.stringifyException(ioe)); - return null; -} +final FileSystem fs = target.getFileSystem(conf); if (!(fs instanceof DistributedFileSystem)) { System.err.println(FileSystem is + fs.getUri()); return null; } - + return DFSUtil.getInfoServer(HAUtil.getAddressOfActive(fs), conf, DFSUtil.getHttpClientScheme(conf)); } @@ -303,8 +304,16 @@ public class DFSck extends Configured implements Tool { dir = /; } -final Path dirpath = new Path(dir); -final URI namenodeAddress = getCurrentNamenodeAddress(dirpath); +Path dirpath = null; +URI namenodeAddress = null; +try { + dirpath = getResolvedPath(dir); + namenodeAddress = getCurrentNamenodeAddress(dirpath); +} catch (IOException ioe) { + System.err.println(FileSystem is inaccessible due to:\n + + StringUtils.stringifyException(ioe)); +} + if (namenodeAddress == null) { //Error message already output in {@link #getCurrentNamenodeAddress()} System.err.println(DFSck exiting.); http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbb49257/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java index 1053b5f..409fffc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java +++
[34/43] hadoop git commit: HADOOP-11602. Fix toUpperCase/toLowerCase to use Locale.ENGLISH. (ozawa)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c6accb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java index 92a16cd..e6cf16c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -79,19 +80,19 @@ public class GetConf extends Configured implements Tool { private static final MapString, CommandHandler map; static { map = new HashMapString, CommandHandler(); - map.put(NAMENODE.getName().toLowerCase(), + map.put(StringUtils.toLowerCase(NAMENODE.getName()), new NameNodesCommandHandler()); - map.put(SECONDARY.getName().toLowerCase(), + map.put(StringUtils.toLowerCase(SECONDARY.getName()), new SecondaryNameNodesCommandHandler()); - map.put(BACKUP.getName().toLowerCase(), + map.put(StringUtils.toLowerCase(BACKUP.getName()), new BackupNodesCommandHandler()); - map.put(INCLUDE_FILE.getName().toLowerCase(), + map.put(StringUtils.toLowerCase(INCLUDE_FILE.getName()), new CommandHandler(DFSConfigKeys.DFS_HOSTS)); - map.put(EXCLUDE_FILE.getName().toLowerCase(), + map.put(StringUtils.toLowerCase(EXCLUDE_FILE.getName()), new CommandHandler(DFSConfigKeys.DFS_HOSTS_EXCLUDE)); - map.put(NNRPCADDRESSES.getName().toLowerCase(), + map.put(StringUtils.toLowerCase(NNRPCADDRESSES.getName()), new NNRpcAddressesCommandHandler()); - map.put(CONFKEY.getName().toLowerCase(), + map.put(StringUtils.toLowerCase(CONFKEY.getName()), new PrintConfKeyCommandHandler()); } @@ -116,7 +117,7 @@ public class GetConf extends Configured implements Tool { } public static CommandHandler getHandler(String cmd) { - return map.get(cmd.toLowerCase()); + return map.get(StringUtils.toLowerCase(cmd)); } } http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c6accb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java index c4b8424..de3aceb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitorFactory.java @@ -24,6 +24,7 @@ import java.io.OutputStream; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.StringUtils; /** * EditsVisitorFactory for different implementations of EditsVisitor @@ -43,7 +44,7 @@ public class OfflineEditsVisitorFactory { */ static public OfflineEditsVisitor getEditsVisitor(String filename, String processor, boolean printToScreen) throws IOException { -if(processor.toLowerCase().equals(binary)) { +if(StringUtils.equalsIgnoreCase(binary, processor)) { return new BinaryEditsVisitor(filename); } OfflineEditsVisitor vis; @@ -59,9 +60,9 @@ public class OfflineEditsVisitorFactory { outs[1] = System.out; out = new TeeOutputStream(outs); } - if(processor.toLowerCase().equals(xml)) { + if(StringUtils.equalsIgnoreCase(xml, processor)) { vis = new XmlEditsVisitor(out); - } else if(processor.toLowerCase().equals(stats)) { + } else if(StringUtils.equalsIgnoreCase(stats, processor)) { vis = new StatisticsEditsVisitor(out); } else { throw new IOException(Unknown proccesor + processor + http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c6accb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java -- diff --git
[03/43] hadoop git commit: YARN-3262. Surface application outstanding resource requests table in RM web UI. (Jian He via wangda)
YARN-3262. Surface application outstanding resource requests table in RM web UI. (Jian He via wangda) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/edcecedc Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/edcecedc Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/edcecedc Branch: refs/heads/YARN-2928 Commit: edcecedc1c39d54db0f86a1325b4db26c38d2d4d Parents: cf51ff2 Author: Wangda Tan wan...@apache.org Authored: Fri Feb 27 16:13:32 2015 -0800 Committer: Wangda Tan wan...@apache.org Committed: Fri Feb 27 16:13:32 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 3 ++ .../records/impl/pb/ResourceRequestPBImpl.java | 4 +- .../scheduler/AbstractYarnScheduler.java| 9 .../scheduler/AppSchedulingInfo.java| 33 +++--- .../scheduler/SchedulerApplicationAttempt.java | 6 ++- .../server/resourcemanager/webapp/AppBlock.java | 46 +++- .../server/resourcemanager/webapp/AppPage.java | 4 ++ .../resourcemanager/webapp/AppsBlock.java | 5 ++- .../webapp/FairSchedulerAppsBlock.java | 5 ++- .../resourcemanager/webapp/RMWebServices.java | 6 +-- .../resourcemanager/webapp/dao/AppInfo.java | 17 +++- .../webapp/TestRMWebAppFairScheduler.java | 10 - .../webapp/TestRMWebServicesApps.java | 3 +- 13 files changed, 118 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/edcecedc/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 38dd9fa..e7af84b 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -336,6 +336,9 @@ Release 2.7.0 - UNRELEASED YARN-2820. Retry in FileSystemRMStateStore when FS's operations fail due to IOException. (Zhihai Xu via ozawa) +YARN-3262. Surface application outstanding resource requests table +in RM web UI. (Jian He via wangda) + OPTIMIZATIONS YARN-2990. FairScheduler's delay-scheduling always waits for node-local and http://git-wip-us.apache.org/repos/asf/hadoop/blob/edcecedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java index 0c8491f..27fb5ae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java @@ -140,13 +140,13 @@ public class ResourceRequestPBImpl extends ResourceRequest { this.capability = capability; } @Override - public int getNumContainers() { + public synchronized int getNumContainers() { ResourceRequestProtoOrBuilder p = viaProto ? proto : builder; return (p.getNumContainers()); } @Override - public void setNumContainers(int numContainers) { + public synchronized void setNumContainers(int numContainers) { maybeInitBuilder(); builder.setNumContainers((numContainers)); } http://git-wip-us.apache.org/repos/asf/hadoop/blob/edcecedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java index 04b3452..968a767 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java @@ -658,4 +658,13 @@ public abstract class AbstractYarnScheduler maxAllocWriteLock.unlock(); } } + + public
[10/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm deleted file mode 100644 index 69728fb..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm +++ /dev/null @@ -1,3104 +0,0 @@ -~~ 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. - - --- - ResourceManager REST API's. - --- - --- - ${maven.build.timestamp} - -ResourceManager REST API's. - -%{toc|section=1|fromDepth=0|toDepth=2} - -* Overview - - The ResourceManager REST API's allow the user to get information about the cluster - status on the cluster, metrics on the cluster, scheduler information, information about nodes in the cluster, and information about applications on the cluster. - -* Cluster Information API - - The cluster information resource provides overall information about the cluster. - -** URI - - Both of the following URI's give you the cluster information. - --- - * http://rm http address:port/ws/v1/cluster - * http://rm http address:port/ws/v1/cluster/info --- - -** HTTP Operations Supported - --- - * GET --- - -** Query Parameters Supported - --- - None --- - -** Elements of the clusterInfo object - -*---+--+---+ -|| Item || Data Type || Description | -*---+--+---+ -| id| long | The cluster id | -*---+--+---+ -| startedOn | long | The time the cluster started (in ms since epoch)| -*---+--+---+ -| state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED| -*---+--+---+ -| haState | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED| -*---+--+---+ -| resourceManagerVersion | string | Version of the ResourceManager | -*---+--+---+ -| resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum | -*---+--+---+ -| resourceManagerVersionBuiltOn | string | Timestamp when ResourceManager was built (in ms since epoch)| -*---+--+---+ -| hadoopVersion | string | Version of hadoop common | -*---+--+---+ -| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum | -*---+--+---+ -| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch)| -*---+--+---+ - -** Response Examples - - JSON response - - HTTP Request: - --- - GET http://rm http address:port/ws/v1/cluster/info --- - - Response Header: - -+---+ - HTTP/1.1 200 OK - Content-Type: application/json - Transfer-Encoding: chunked - Server: Jetty(6.1.26) -+---+ - - Response Body: - -+---+ -{ - clusterInfo: - { -id:1324053971963, -startedOn:1324053971963, -state:STARTED, -resourceManagerVersion:0.23.1-SNAPSHOT, -resourceManagerBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693, -resourceManagerVersionBuiltOn:Tue Dec 13 22:12:48 CST 2011, -hadoopVersion:0.23.1-SNAPSHOT, -hadoopBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328, -hadoopVersionBuiltOn:Tue Dec 13 22:12:26 CST 2011 - } -} -+---+ - - XML response - - HTTP Request: - -- - Accept: application/xml - GET http://rm http address:port/ws/v1/cluster/info -- - - Response Header: - -+---+ - HTTP/1.1 200 OK - Content-Type: application/xml - Content-Length: 712 - Server: Jetty(6.1.26) -+---+ - - Response
[17/43] hadoop git commit: HDFS-4681. TestBlocksWithNotEnoughRacks#testCorruptBlockRereplicatedAcrossRacks fails using IBM java (Ayappan via aw)
HDFS-4681. TestBlocksWithNotEnoughRacks#testCorruptBlockRereplicatedAcrossRacks fails using IBM java (Ayappan via aw) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dbc9b643 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dbc9b643 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dbc9b643 Branch: refs/heads/YARN-2928 Commit: dbc9b6433e9276057181cf4927cedf321acd354e Parents: b01d343 Author: Allen Wittenauer a...@apache.org Authored: Sat Feb 28 23:32:09 2015 -0800 Committer: Allen Wittenauer a...@apache.org Committed: Sat Feb 28 23:32:09 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../test/java/org/apache/hadoop/hdfs/DFSTestUtil.java | 12 .../java/org/apache/hadoop/hdfs/MiniDFSCluster.java | 10 ++ .../blockmanagement/TestBlocksWithNotEnoughRacks.java | 7 --- 4 files changed, 29 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbc9b643/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 2a8da43..16fe394 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -306,6 +306,9 @@ Trunk (Unreleased) HDFS-7803. Wrong command mentioned in HDFSHighAvailabilityWithQJM documentation (Arshad Mohammad via aw) +HDFS-4681. TestBlocksWithNotEnoughRacks#testCorruptBlockRereplicatedAcrossRacks +fails using IBM java (Ayappan via aw) + Release 2.7.0 - UNRELEASED INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbc9b643/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java index 5b391c5..7e7ff39 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java @@ -251,6 +251,12 @@ public class DFSTestUtil { public void createFiles(FileSystem fs, String topdir) throws IOException { createFiles(fs, topdir, (short)3); } + + public static byte[] readFileAsBytes(FileSystem fs, Path fileName) throws IOException { +ByteArrayOutputStream os = new ByteArrayOutputStream(); +IOUtils.copyBytes(fs.open(fileName), os, 1024, true); +return os.toByteArray(); + } /** create nFiles with random names and directory hierarchies * with random (but reproducible) data in them. @@ -723,6 +729,12 @@ public class DFSTestUtil { return b.toString(); } + public static byte[] readFileAsBytes(File f) throws IOException { +ByteArrayOutputStream os = new ByteArrayOutputStream(); +IOUtils.copyBytes(new FileInputStream(f), os, 1024, true); +return os.toByteArray(); + } + /* Write the given string to the given file */ public static void writeFile(FileSystem fs, Path p, String s) throws IOException { http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbc9b643/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index 5297ba2..2c1d07e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -1869,6 +1869,16 @@ public class MiniDFSCluster { return null; } + public byte[] readBlockOnDataNodeAsBytes(int i, ExtendedBlock block) + throws IOException { +assert (i = 0 i dataNodes.size()) : Invalid datanode +i; +File blockFile = getBlockFile(i, block); +if (blockFile != null blockFile.exists()) { + return DFSTestUtil.readFileAsBytes(blockFile); +} +return null; + } + /** * Corrupt a block on a particular datanode. * http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbc9b643/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java -- diff --git
[27/43] hadoop git commit: YARN-3270. Fix node label expression not getting set in ApplicationSubmissionContext (Rohit Agarwal via wangda)
YARN-3270. Fix node label expression not getting set in ApplicationSubmissionContext (Rohit Agarwal via wangda) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/abac6eb9 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/abac6eb9 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/abac6eb9 Branch: refs/heads/YARN-2928 Commit: abac6eb9d530bb1e6ff58ec3c75b17d840a0ee3f Parents: c5eac9c Author: Wangda Tan wan...@apache.org Authored: Mon Mar 2 17:21:19 2015 -0800 Committer: Wangda Tan wan...@apache.org Committed: Mon Mar 2 17:21:19 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 3 +++ .../hadoop/yarn/api/records/ApplicationSubmissionContext.java | 1 + 2 files changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/abac6eb9/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index c7dac60..d07aa26 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -683,6 +683,9 @@ Release 2.7.0 - UNRELEASED all Schedulers even when using ParameterizedSchedulerTestBase. (Anubhav Dhoot via devaraj) +YARN-3270. Fix node label expression not getting set in +ApplicationSubmissionContext (Rohit Agarwal via wangda) + Release 2.6.0 - 2014-11-18 INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/abac6eb9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java index f1ebbfe..c4014fc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java @@ -155,6 +155,7 @@ public abstract class ApplicationSubmissionContext { context.setMaxAppAttempts(maxAppAttempts); context.setApplicationType(applicationType); context.setKeepContainersAcrossApplicationAttempts(keepContainers); +context.setNodeLabelExpression(appLabelExpression); context.setAMContainerResourceRequest(resourceRequest); return context; }
[08/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm deleted file mode 100644 index 57a47fd..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm +++ /dev/null @@ -1,757 +0,0 @@ -~~ 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. - - --- - Hadoop Map Reduce Next Generation-${project.version} - Writing YARN - Applications - --- - --- - ${maven.build.timestamp} - -Hadoop MapReduce Next Generation - Writing YARN Applications - -%{toc|section=1|fromDepth=0} - -* Purpose - - This document describes, at a high-level, the way to implement new - Applications for YARN. - -* Concepts and Flow - - The general concept is that an application submission client submits an - application to the YARN ResourceManager (RM). This can be done through - setting up a YarnClient object. After YarnClient is started, the - client can then set up application context, prepare the very first container of - the application that contains the ApplicationMaster (AM), and then submit - the application. You need to provide information such as the details about the - local files/jars that need to be available for your application to run, the - actual command that needs to be executed (with the necessary command line - arguments), any OS environment settings (optional), etc. Effectively, you - need to describe the Unix process(es) that needs to be launched for your - ApplicationMaster. - - The YARN ResourceManager will then launch the ApplicationMaster (as - specified) on an allocated container. The ApplicationMaster communicates with - YARN cluster, and handles application execution. It performs operations in an - asynchronous fashion. During application launch time, the main tasks of the - ApplicationMaster are: a) communicating with the ResourceManager to negotiate - and allocate resources for future containers, and b) after container - allocation, communicating YARN NodeManagers (NMs) to launch application - containers on them. Task a) can be performed asynchronously through an - AMRMClientAsync object, with event handling methods specified in a - AMRMClientAsync.CallbackHandler type of event handler. The event handler - needs to be set to the client explicitly. Task b) can be performed by launching - a runnable object that then launches containers when there are containers - allocated. As part of launching this container, the AM has to - specify the ContainerLaunchContext that has the launch information such as - command line specification, environment, etc. - - During the execution of an application, the ApplicationMaster communicates - NodeManagers through NMClientAsync object. All container events are - handled by NMClientAsync.CallbackHandler, associated with - NMClientAsync. A typical callback handler handles client start, stop, - status update and error. ApplicationMaster also reports execution progress to - ResourceManager by handling the getProgress() method of - AMRMClientAsync.CallbackHandler. - - Other than asynchronous clients, there are synchronous versions for certain - workflows (AMRMClient and NMClient). The asynchronous clients are - recommended because of (subjectively) simpler usages, and this article - will mainly cover the asynchronous clients. Please refer to AMRMClient - and NMClient for more information on synchronous clients. - -* Interfaces - - The interfaces you'd most like be concerned with are: - - * Client\--\ResourceManager\ -By using YarnClient objects. - - * ApplicationMaster\--\ResourceManager\ -By using AMRMClientAsync objects, handling events asynchronously by -AMRMClientAsync.CallbackHandler - - * ApplicationMaster\--\NodeManager\ -Launch containers. Communicate with NodeManagers -by using NMClientAsync objects, handling container events by -NMClientAsync.CallbackHandler - - [] - - Note - -* The three main protocols for YARN application (ApplicationClientProtocol, - ApplicationMasterProtocol and ContainerManagementProtocol) are still - preserved. The 3
[35/43] hadoop git commit: HADOOP-11602. Fix toUpperCase/toLowerCase to use Locale.ENGLISH. (ozawa)
HADOOP-11602. Fix toUpperCase/toLowerCase to use Locale.ENGLISH. (ozawa) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d1c6accb Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d1c6accb Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d1c6accb Branch: refs/heads/YARN-2928 Commit: d1c6accb6f87b08975175580e15f1ff1fe29ab04 Parents: b442aee Author: Tsuyoshi Ozawa oz...@apache.org Authored: Tue Mar 3 14:12:34 2015 +0900 Committer: Tsuyoshi Ozawa oz...@apache.org Committed: Tue Mar 3 14:17:52 2015 +0900 -- .../classification/tools/StabilityOptions.java | 5 ++- .../AltKerberosAuthenticationHandler.java | 6 ++- .../authentication/util/TestKerberosUtil.java | 14 --- hadoop-common-project/hadoop-common/CHANGES.txt | 2 + .../org/apache/hadoop/conf/Configuration.java | 6 +-- .../org/apache/hadoop/crypto/CipherSuite.java | 3 +- .../hadoop/crypto/key/JavaKeyStoreProvider.java | 3 +- .../java/org/apache/hadoop/fs/FileSystem.java | 7 +++- .../java/org/apache/hadoop/fs/StorageType.java | 3 +- .../apache/hadoop/fs/permission/AclEntry.java | 5 ++- .../apache/hadoop/fs/shell/XAttrCommands.java | 2 +- .../org/apache/hadoop/fs/shell/find/Name.java | 5 ++- .../io/compress/CompressionCodecFactory.java| 7 ++-- .../hadoop/metrics2/impl/MetricsConfig.java | 7 ++-- .../hadoop/metrics2/impl/MetricsSystemImpl.java | 5 ++- .../hadoop/security/SaslPropertiesResolver.java | 3 +- .../apache/hadoop/security/SecurityUtil.java| 12 +++--- .../hadoop/security/WhitelistBasedResolver.java | 3 +- .../security/ssl/FileBasedKeyStoresFactory.java | 4 +- .../apache/hadoop/security/ssl/SSLFactory.java | 5 ++- .../security/ssl/SSLHostnameVerifier.java | 10 +++-- .../DelegationTokenAuthenticationHandler.java | 3 +- .../web/DelegationTokenAuthenticator.java | 3 +- .../apache/hadoop/util/ComparableVersion.java | 3 +- .../org/apache/hadoop/util/StringUtils.java | 40 +++- .../hadoop/fs/FileSystemContractBaseTest.java | 4 +- .../java/org/apache/hadoop/ipc/TestIPC.java | 2 +- .../java/org/apache/hadoop/ipc/TestSaslRPC.java | 2 +- .../hadoop/security/TestSecurityUtil.java | 10 +++-- .../security/TestUserGroupInformation.java | 5 ++- .../hadoop/test/TimedOutTestsListener.java | 6 ++- .../org/apache/hadoop/util/TestStringUtils.java | 21 ++ .../org/apache/hadoop/util/TestWinUtils.java| 6 ++- .../java/org/apache/hadoop/nfs/NfsExports.java | 5 ++- .../server/CheckUploadContentTypeFilter.java| 4 +- .../hadoop/fs/http/server/FSOperations.java | 7 +++- .../http/server/HttpFSParametersProvider.java | 4 +- .../org/apache/hadoop/lib/server/Server.java| 3 +- .../service/hadoop/FileSystemAccessService.java | 6 ++- .../org/apache/hadoop/lib/wsrs/EnumParam.java | 2 +- .../apache/hadoop/lib/wsrs/EnumSetParam.java| 3 +- .../hadoop/lib/wsrs/ParametersProvider.java | 3 +- .../org/apache/hadoop/hdfs/XAttrHelper.java | 19 ++ .../hadoop/hdfs/protocol/HdfsConstants.java | 3 +- .../BlockStoragePolicySuite.java| 4 +- .../hdfs/server/common/HdfsServerConstants.java | 5 ++- .../hdfs/server/datanode/StorageLocation.java | 4 +- .../hdfs/server/namenode/FSEditLogOp.java | 3 +- .../namenode/QuotaByStorageTypeEntry.java | 3 +- .../hdfs/server/namenode/SecondaryNameNode.java | 2 +- .../org/apache/hadoop/hdfs/tools/GetConf.java | 17 + .../OfflineEditsVisitorFactory.java | 7 ++-- .../offlineImageViewer/FSImageHandler.java | 4 +- .../org/apache/hadoop/hdfs/web/AuthFilter.java | 3 +- .../org/apache/hadoop/hdfs/web/ParamFilter.java | 3 +- .../hadoop/hdfs/web/WebHdfsFileSystem.java | 5 ++- .../hadoop/hdfs/web/resources/EnumParam.java| 3 +- .../hadoop/hdfs/web/resources/EnumSetParam.java | 3 +- .../namenode/snapshot/TestSnapshotManager.java | 6 +-- .../jobhistory/JobHistoryEventHandler.java | 3 +- .../mapreduce/v2/app/webapp/AppController.java | 6 +-- .../apache/hadoop/mapreduce/TypeConverter.java | 3 +- .../apache/hadoop/mapreduce/v2/util/MRApps.java | 4 +- .../hadoop/mapreduce/TestTypeConverter.java | 6 ++- .../java/org/apache/hadoop/mapred/Task.java | 2 +- .../counters/FileSystemCounterGroup.java| 4 +- .../mapreduce/filecache/DistributedCache.java | 4 +- .../hadoop/mapreduce/lib/db/DBInputFormat.java | 5 ++- .../org/apache/hadoop/mapreduce/tools/CLI.java | 9 +++-- .../java/org/apache/hadoop/fs/TestDFSIO.java| 18 - .../org/apache/hadoop/fs/TestFileSystem.java| 4 +- .../org/apache/hadoop/fs/slive/Constants.java | 6 ++- .../apache/hadoop/fs/slive/OperationData.java | 3 +- .../apache/hadoop/fs/slive/OperationOutput.java | 4 +-
[11/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm deleted file mode 100644 index 36b8621..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm +++ /dev/null @@ -1,645 +0,0 @@ -~~ 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. - - --- - NodeManager REST API's. - --- - --- - ${maven.build.timestamp} - -NodeManager REST API's. - -%{toc|section=1|fromDepth=0|toDepth=2} - -* Overview - - The NodeManager REST API's allow the user to get status on the node and information about applications and containers running on that node. - -* NodeManager Information API - - The node information resource provides overall information about that particular node. - -** URI - - Both of the following URI's give you the cluster information. - --- - * http://nm http address:port/ws/v1/node - * http://nm http address:port/ws/v1/node/info --- - -** HTTP Operations Supported - --- - * GET --- - -** Query Parameters Supported - --- - None --- - -** Elements of the nodeInfo object - -*---+--+---+ -|| Item || Data Type || Description | -*---+--+---+ -| id| long | The NodeManager id | -*---+--+---+ -| nodeHostName | string | The host name of the NodeManager | -*---+--+---+ -| totalPmemAllocatedContainersMB | long | The amount of physical memory allocated for use by containers in MB | -*---+--+---+ -| totalVmemAllocatedContainersMB | long | The amount of virtual memory allocated for use by containers in MB | -*---+--+---+ -| totalVCoresAllocatedContainers | long | The number of virtual cores allocated for use by containers | -*---+--+---+ -| lastNodeUpdateTime | long | The last timestamp at which the health report was received (in ms since epoch)| -*---+--+---+ -| healthReport | string | The diagnostic health report of the node | -*---+--+---+ -| nodeHealthy | boolean | true/false indicator of if the node is healthy| -*---+--+---+ -| nodeManagerVersion | string | Version of the NodeManager | -*---+--+---+ -| nodeManagerBuildVersion | string | NodeManager build string with build version, user, and checksum | -*---+--+---+ -| nodeManagerVersionBuiltOn | string | Timestamp when NodeManager was built(in ms since epoch) | -*---+--+---+ -| hadoopVersion | string | Version of hadoop common | -*---+--+---+ -| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum | -*---+--+---+ -| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch) | -*---+--+---+ - -** Response Examples - - JSON response - - HTTP Request: - --- - GET http://nm http address:port/ws/v1/node/info --- - - Response Header: - -+---+ - HTTP/1.1 200 OK - Content-Type: application/json - Transfer-Encoding: chunked - Server: Jetty(6.1.26) -+---+ - - Response Body: - -+---+ -{ - nodeInfo : { - hadoopVersionBuiltOn : Mon Jan 9 14:58:42 UTC 2012, - nodeManagerBuildVersion : 0.23.1-SNAPSHOT from 1228355 by user1 source checksum 20647f76c36430e888cc7204826a445c, - lastNodeUpdateTime : 132666126, - totalVmemAllocatedContainersMB : 17203, - totalVCoresAllocatedContainers : 8, - nodeHealthy
[22/43] hadoop git commit: HDFS-7439. Add BlockOpResponseProto's message to the exception messages. Contributed by Takanobu Asanuma
HDFS-7439. Add BlockOpResponseProto's message to the exception messages. Contributed by Takanobu Asanuma Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/67ed5934 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/67ed5934 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/67ed5934 Branch: refs/heads/YARN-2928 Commit: 67ed59348d638d56e6752ba2c71fdcd69567546d Parents: dd9cd07 Author: Tsz-Wo Nicholas Sze szets...@hortonworks.com Authored: Mon Mar 2 15:03:58 2015 +0800 Committer: Tsz-Wo Nicholas Sze szets...@hortonworks.com Committed: Mon Mar 2 15:03:58 2015 +0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../java/org/apache/hadoop/hdfs/DFSClient.java | 26 ++-- .../org/apache/hadoop/hdfs/DFSOutputStream.java | 15 --- .../apache/hadoop/hdfs/RemoteBlockReader2.java | 24 ++ .../datatransfer/DataTransferProtoUtil.java | 26 .../hadoop/hdfs/server/balancer/Dispatcher.java | 9 +++ .../hdfs/server/datanode/DataXceiver.java | 14 +++ 7 files changed, 55 insertions(+), 62 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ed5934/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index ce35ea2..5ca16af 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -694,6 +694,9 @@ Release 2.7.0 - UNRELEASED HDFS-5853. Add hadoop.user.group.metrics.percentiles.intervals to hdfs-default.xml. (aajisaka) +HDFS-7439. Add BlockOpResponseProto's message to the exception messages. +(Takanobu Asanuma via szetszwo) + OPTIMIZATIONS HDFS-7454. Reduce memory footprint for AclEntries in NameNode. http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ed5934/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 792c2dd..abcd847 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -174,6 +174,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure; @@ -2260,15 +2261,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, final BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); - if (reply.getStatus() != Status.SUCCESS) { -if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) { - throw new InvalidBlockTokenException(); -} else { - throw new IOException(Bad response + reply + for block - + block + from datanode + datanodes[j]); -} - } - + String logInfo = for block + block + from datanode + datanodes[j]; + DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); + OpBlockChecksumResponseProto checksumData = reply.getChecksumResponse(); @@ -2425,16 +2420,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, 0, 1, true, CachingStrategy.newDefaultStrategy()); final BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in)); - - if (reply.getStatus() != Status.SUCCESS) { -if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) { - throw new InvalidBlockTokenException(); -} else { - throw new IOException(Bad response + reply + trying to read - + lb.getBlock() + from datanode + dn); -} - } - + String logInfo = trying to read + lb.getBlock() + from datanode + dn; + DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); + return PBHelper.convert(reply.getReadOpChecksumInfo().getChecksum().getType()); } finally {
[14/43] hadoop git commit: move HADOOP-10976 to 2.7
move HADOOP-10976 to 2.7 Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/915bec3e Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/915bec3e Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/915bec3e Branch: refs/heads/YARN-2928 Commit: 915bec3e84f4da913dd7b7ad0f389eb69fc064c6 Parents: 8472d72 Author: Akira Ajisaka aajis...@apache.org Authored: Sat Feb 28 17:15:13 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Sat Feb 28 17:15:13 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/915bec3e/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 6d4da77..74bf558 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -163,9 +163,6 @@ Trunk (Unreleased) HADOOP-11346. Rewrite sls/rumen to use new shell framework (John Smith via aw) -HADOOP-10976. moving the source code of hadoop-tools docs to the -directory under hadoop-tools (Masatake Iwasaki via aw) - HADOOP-7713. dfs -count -q should label output column (Jonathan Allen via aw) @@ -636,6 +633,9 @@ Release 2.7.0 - UNRELEASED HADOOP-11632. Cleanup Find.java to remove SupressWarnings annotations. (Akira Ajisaka via ozawa) +HADOOP-10976. moving the source code of hadoop-tools docs to the +directory under hadoop-tools (Masatake Iwasaki via aw) + OPTIMIZATIONS HADOOP-11323. WritableComparator#compare keeps reference to byte array.
[01/43] hadoop git commit: HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in hdfs-default.xml. Contributed by Kai Sasaki.
Repository: hadoop Updated Branches: refs/heads/YARN-2928 bf08f7f0e - d3ff7f06c HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in hdfs-default.xml. Contributed by Kai Sasaki. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8719cdd4 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8719cdd4 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8719cdd4 Branch: refs/heads/YARN-2928 Commit: 8719cdd4f68abb91bf9459bca2a5467dafb6b5ae Parents: 01a1621 Author: Akira Ajisaka aajis...@apache.org Authored: Fri Feb 27 12:17:34 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Fri Feb 27 12:17:34 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../hadoop-hdfs/src/main/resources/hdfs-default.xml | 11 +++ 2 files changed, 14 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/8719cdd4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index b2422d6..b4b0087 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -685,6 +685,9 @@ Release 2.7.0 - UNRELEASED HDFS-7308. Change the packet chunk size computation in DFSOutputStream in order to enforce packet size = 64kB. (Takuya Fukudome via szetszwo) +HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in +hdfs-default.xml. (Kai Sasaki via aajisaka) + OPTIMIZATIONS HDFS-7454. Reduce memory footprint for AclEntries in NameNode. http://git-wip-us.apache.org/repos/asf/hadoop/blob/8719cdd4/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 85d2273..66fe86c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -145,6 +145,17 @@ /property property + namedfs.namenode.heartbeat.recheck-interval/name + value30/value + description +This time decides the interval to check for expired datanodes. +With this value and dfs.heartbeat.interval, the interval of +deciding the datanode is stale or not is also calculated. +The unit of this configuration is millisecond. + /description +/property + +property namedfs.http.policy/name valueHTTP_ONLY/value descriptionDecide if HTTPS(SSL) is supported on HDFS
[31/43] hadoop git commit: HADOOP-11605. FilterFileSystem#create with ChecksumOpt should propagate it to wrapped FS. (gera)
HADOOP-11605. FilterFileSystem#create with ChecksumOpt should propagate it to wrapped FS. (gera) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b18d3830 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b18d3830 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b18d3830 Branch: refs/heads/YARN-2928 Commit: b18d3830aca00f44d31e42839578f24eecedf2c8 Parents: 431e7d8 Author: Gera Shegalov g...@apache.org Authored: Tue Feb 17 00:24:37 2015 -0800 Committer: Gera Shegalov g...@apache.org Committed: Mon Mar 2 18:09:23 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++ .../src/main/java/org/apache/hadoop/fs/FilterFileSystem.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/b18d3830/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index b8ed286..ebe23c7 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -1031,6 +1031,9 @@ Release 2.7.0 - UNRELEASED HADOOP-11615. Update ServiceLevelAuth.md for YARN. (Brahma Reddy Battula via aajisaka) +HADOOP-11605. FilterFileSystem#create with ChecksumOpt should propagate it +to wrapped FS. (gera) + Release 2.6.1 - UNRELEASED INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/b18d3830/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java -- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java index d4080ad..d14a272 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java @@ -190,7 +190,7 @@ public class FilterFileSystem extends FileSystem { Progressable progress, ChecksumOpt checksumOpt) throws IOException { return fs.create(f, permission, - flags, bufferSize, replication, blockSize, progress); + flags, bufferSize, replication, blockSize, progress, checksumOpt); } @Override
[39/43] hadoop git commit: MAPREDUCE-5583. Ability to limit running map and reduce tasks. Contributed by Jason Lowe.
MAPREDUCE-5583. Ability to limit running map and reduce tasks. Contributed by Jason Lowe. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4228de94 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4228de94 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4228de94 Branch: refs/heads/YARN-2928 Commit: 4228de94028f1e10ca59ce23e963e488fe566909 Parents: 9ae7f9e Author: Junping Du junping...@apache.org Authored: Tue Mar 3 02:01:04 2015 -0800 Committer: Junping Du junping...@apache.org Committed: Tue Mar 3 02:02:28 2015 -0800 -- hadoop-mapreduce-project/CHANGES.txt| 3 + .../v2/app/rm/RMContainerAllocator.java | 65 +- .../v2/app/rm/RMContainerRequestor.java | 74 ++- .../v2/app/rm/TestRMContainerAllocator.java | 214 +++ .../apache/hadoop/mapreduce/MRJobConfig.java| 8 + .../src/main/resources/mapred-default.xml | 16 ++ 6 files changed, 363 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/4228de94/hadoop-mapreduce-project/CHANGES.txt -- diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 5524b14..7a2eff3 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -258,6 +258,9 @@ Release 2.7.0 - UNRELEASED MAPREDUCE-6228. Add truncate operation to SLive. (Plamen Jeliazkov via shv) +MAPREDUCE-5583. Ability to limit running map and reduce tasks. +(Jason Lowe via junping_du) + IMPROVEMENTS MAPREDUCE-6149. Document override log4j.properties in MR job. http://git-wip-us.apache.org/repos/asf/hadoop/blob/4228de94/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java index 1acfeec..efea674 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java @@ -99,9 +99,9 @@ public class RMContainerAllocator extends RMContainerRequestor public static final float DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART = 0.05f; - private static final Priority PRIORITY_FAST_FAIL_MAP; - private static final Priority PRIORITY_REDUCE; - private static final Priority PRIORITY_MAP; + static final Priority PRIORITY_FAST_FAIL_MAP; + static final Priority PRIORITY_REDUCE; + static final Priority PRIORITY_MAP; @VisibleForTesting public static final String RAMPDOWN_DIAGNOSTIC = Reducer preempted @@ -166,6 +166,8 @@ public class RMContainerAllocator extends RMContainerRequestor */ private long allocationDelayThresholdMs = 0; private float reduceSlowStart = 0; + private int maxRunningMaps = 0; + private int maxRunningReduces = 0; private long retryInterval; private long retrystartTime; private Clock clock; @@ -201,6 +203,10 @@ public class RMContainerAllocator extends RMContainerRequestor allocationDelayThresholdMs = conf.getInt( MRJobConfig.MR_JOB_REDUCER_PREEMPT_DELAY_SEC, MRJobConfig.DEFAULT_MR_JOB_REDUCER_PREEMPT_DELAY_SEC) * 1000;//sec - ms +maxRunningMaps = conf.getInt(MRJobConfig.JOB_RUNNING_MAP_LIMIT, +MRJobConfig.DEFAULT_JOB_RUNNING_MAP_LIMIT); +maxRunningReduces = conf.getInt(MRJobConfig.JOB_RUNNING_REDUCE_LIMIT, +MRJobConfig.DEFAULT_JOB_RUNNING_REDUCE_LIMIT); RackResolver.init(conf); retryInterval = getConfig().getLong(MRJobConfig.MR_AM_TO_RM_WAIT_INTERVAL_MS, MRJobConfig.DEFAULT_MR_AM_TO_RM_WAIT_INTERVAL_MS); @@ -664,6 +670,8 @@ public class RMContainerAllocator extends RMContainerRequestor @SuppressWarnings(unchecked) private ListContainer getResources() throws Exception { +applyConcurrentTaskLimits(); + // will be null the first time Resource headRoom = getAvailableResources() == null ? Resources.none() : @@ -778,6 +786,43 @@ public class RMContainerAllocator extends RMContainerRequestor return newContainers; } + private void applyConcurrentTaskLimits() { +int numScheduledMaps =
[24/43] hadoop git commit: HADOOP-11658. Externalize io.compression.codecs property. Contributed by Kai Zheng.
HADOOP-11658. Externalize io.compression.codecs property. Contributed by Kai Zheng. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ca1c00bf Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ca1c00bf Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ca1c00bf Branch: refs/heads/YARN-2928 Commit: ca1c00bf814a8b8290a81d06b1f4918c36c7d9e0 Parents: cbb4925 Author: Akira Ajisaka aajis...@apache.org Authored: Mon Mar 2 01:09:54 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Mon Mar 2 01:12:44 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++ .../hadoop/fs/CommonConfigurationKeys.java | 17 +++- .../io/compress/CompressionCodecFactory.java| 21 +--- .../hadoop/io/compress/TestCodecFactory.java| 3 ++- 4 files changed, 31 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca1c00bf/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 4c0c375..b8ed286 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -642,6 +642,9 @@ Release 2.7.0 - UNRELEASED HADOOP-10976. moving the source code of hadoop-tools docs to the directory under hadoop-tools (Masatake Iwasaki via aw) +HADOOP-11658. Externalize io.compression.codecs property. +(Kai Zheng via aajisaka) + OPTIMIZATIONS HADOOP-11323. WritableComparator#compare keeps reference to byte array. http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca1c00bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java -- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 442dc7d..7575496 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -91,17 +91,24 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { public static final String IPC_CALLQUEUE_IMPL_KEY = callqueue.impl; public static final String IPC_CALLQUEUE_IDENTITY_PROVIDER_KEY = identity-provider.impl; + /** This is for specifying the implementation for the mappings from + * hostnames to the racks they belong to + */ + public static final String NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY = + net.topology.configured.node.mapping; + + /** + * Supported compression codec classes + */ + public static final String IO_COMPRESSION_CODECS_KEY = io.compression.codecs; + /** Internal buffer size for Lzo compressor/decompressors */ public static final String IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY = io.compression.codec.lzo.buffersize; + /** Default value for IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY */ public static final int IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT = 64*1024; - /** This is for specifying the implementation for the mappings from - * hostnames to the racks they belong to - */ - public static final String NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY = -net.topology.configured.node.mapping; /** Internal buffer size for Snappy compressor/decompressors */ public static final String IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY = http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca1c00bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java -- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java index eb35759..7476a15 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.Path;
[16/43] hadoop git commit: HADOOP-10774. Update KerberosTestUtils for hadoop-auth tests when using IBM Java (sangamesh via aw)
HADOOP-10774. Update KerberosTestUtils for hadoop-auth tests when using IBM Java (sangamesh via aw) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b01d3433 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b01d3433 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b01d3433 Branch: refs/heads/YARN-2928 Commit: b01d3433aefb68a0f66a48ac9cae7d32463ab95e Parents: 039366e Author: Allen Wittenauer a...@apache.org Authored: Sat Feb 28 23:22:06 2015 -0800 Committer: Allen Wittenauer a...@apache.org Committed: Sat Feb 28 23:22:06 2015 -0800 -- .../authentication/KerberosTestUtils.java | 40 ++-- hadoop-common-project/hadoop-common/CHANGES.txt | 3 ++ 2 files changed, 32 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/b01d3433/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java -- diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java index 7629a30..8fc08e2 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java @@ -32,12 +32,14 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; +import static org.apache.hadoop.util.PlatformName.IBM_JAVA; + /** * Test helper class for Java Kerberos setup. */ public class KerberosTestUtils { private static String keytabFile = new File(System.getProperty(test.dir, target), - UUID.randomUUID().toString()).toString(); + UUID.randomUUID().toString()).getAbsolutePath(); public static String getRealm() { return EXAMPLE.COM; @@ -65,18 +67,34 @@ public class KerberosTestUtils { @Override public AppConfigurationEntry[] getAppConfigurationEntry(String name) { MapString, String options = new HashMapString, String(); - options.put(keyTab, KerberosTestUtils.getKeytabFile()); - 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, true); + if (IBM_JAVA) { +options.put(useKeytab, KerberosTestUtils.getKeytabFile().startsWith(file://) ? +KerberosTestUtils.getKeytabFile() : file:// + KerberosTestUtils.getKeytabFile()); +options.put(principal, principal); +options.put(refreshKrb5Config, true); +options.put(credsType, both); + } else { +options.put(keyTab, KerberosTestUtils.getKeytabFile()); +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, true); + } String ticketCache = System.getenv(KRB5CCNAME); if (ticketCache != null) { -options.put(ticketCache, ticketCache); +if (IBM_JAVA) { + // IBM JAVA only respect system property and not env variable + // The first value searched when useDefaultCcache is used. + System.setProperty(KRB5CCNAME, ticketCache); + options.put(useDefaultCcache, true); + options.put(renewTGT, true); +} else { + options.put(ticketCache, ticketCache); +} } options.put(debug, true); http://git-wip-us.apache.org/repos/asf/hadoop/blob/b01d3433/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 74bf558..3c4dc99 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -403,6 +403,9 @@ Trunk (Unreleased) HADOOP-11637. bash location hard-coded in shell scripts (aw) +HADOOP-10774. Update KerberosTestUtils for hadoop-auth tests when using +IBM Java (sangamesh via aw) + OPTIMIZATIONS HADOOP-7761. Improve the performance of raw comparisons. (todd)
[26/43] hadoop git commit: HDFS-7785. Improve diagnostics information for HttpPutFailedException. Contributed by Chengbing Liu.
HDFS-7785. Improve diagnostics information for HttpPutFailedException. Contributed by Chengbing Liu. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c5eac9c6 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c5eac9c6 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c5eac9c6 Branch: refs/heads/YARN-2928 Commit: c5eac9c6fe937ff32f4efed89d34c06974fac4d6 Parents: 5d0bae5 Author: Haohui Mai whe...@apache.org Authored: Mon Mar 2 15:35:02 2015 -0800 Committer: Haohui Mai whe...@apache.org Committed: Mon Mar 2 15:35:02 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5eac9c6/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index d5208da..43505d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1064,6 +1064,9 @@ Release 2.7.0 - UNRELEASED HDFS-6753. Initialize checkDisk when DirectoryScanner not able to get files list for scanning (J.Andreina via vinayakumarb) +HDFS-7785. Improve diagnostics information for HttpPutFailedException. +(Chengbing Liu via wheat9) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5eac9c6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java index c1e9d7f..0d32758 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java @@ -291,7 +291,9 @@ public class TransferFsImage { int responseCode = connection.getResponseCode(); if (responseCode != HttpURLConnection.HTTP_OK) { -throw new HttpPutFailedException(connection.getResponseMessage(), +throw new HttpPutFailedException(String.format( +Image uploading failed, status: %d, url: %s, message: %s, +responseCode, urlWithParams, connection.getResponseMessage()), responseCode); } } catch (AuthenticationException e) {
[38/43] hadoop git commit: MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA.
MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9ae7f9eb Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9ae7f9eb Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9ae7f9eb Branch: refs/heads/YARN-2928 Commit: 9ae7f9eb7baeb244e1b95aabc93ad8124870b9a9 Parents: 742f9d9 Author: Tsuyoshi Ozawa oz...@apache.org Authored: Tue Mar 3 18:06:26 2015 +0900 Committer: Tsuyoshi Ozawa oz...@apache.org Committed: Tue Mar 3 18:06:26 2015 +0900 -- hadoop-mapreduce-project/CHANGES.txt| 3 ++ .../hadoop/mapred/TaskAttemptListenerImpl.java | 4 +- .../hadoop/mapreduce/v2/app/JobEndNotifier.java | 1 - .../apache/hadoop/mapreduce/v2/util/MRApps.java | 2 +- .../hadoop/filecache/DistributedCache.java | 2 +- .../org/apache/hadoop/mapred/ClusterStatus.java | 4 +- .../apache/hadoop/mapred/FileOutputFormat.java | 2 +- .../java/org/apache/hadoop/mapred/IFile.java| 2 +- .../apache/hadoop/mapred/JobACLsManager.java| 1 - .../org/apache/hadoop/mapred/JobClient.java | 8 ++-- .../java/org/apache/hadoop/mapred/JobConf.java | 49 +--- .../java/org/apache/hadoop/mapred/Mapper.java | 2 +- .../org/apache/hadoop/mapred/QueueManager.java | 30 ++-- .../org/apache/hadoop/mapred/RecordReader.java | 2 +- .../java/org/apache/hadoop/mapred/Reducer.java | 14 +++--- .../hadoop/mapred/TaskUmbilicalProtocol.java| 1 - .../apache/hadoop/mapred/lib/ChainMapper.java | 40 .../apache/hadoop/mapred/lib/ChainReducer.java | 44 +- .../hadoop/mapred/lib/MultipleOutputs.java | 29 +--- .../hadoop/mapred/lib/TokenCountMapper.java | 2 +- .../lib/aggregate/ValueAggregatorJob.java | 2 +- .../lib/aggregate/ValueAggregatorReducer.java | 3 +- .../hadoop/mapred/lib/db/DBInputFormat.java | 4 +- .../org/apache/hadoop/mapreduce/Cluster.java| 1 + .../apache/hadoop/mapreduce/ClusterMetrics.java | 6 +-- .../apache/hadoop/mapreduce/CryptoUtils.java| 10 ++-- .../java/org/apache/hadoop/mapreduce/Job.java | 2 +- .../org/apache/hadoop/mapreduce/JobContext.java | 2 - .../hadoop/mapreduce/JobSubmissionFiles.java| 2 +- .../org/apache/hadoop/mapreduce/Mapper.java | 9 ++-- .../org/apache/hadoop/mapreduce/Reducer.java| 12 ++--- .../mapreduce/filecache/DistributedCache.java | 5 +- .../lib/aggregate/ValueAggregatorJob.java | 2 +- .../hadoop/mapreduce/lib/chain/Chain.java | 4 +- .../hadoop/mapreduce/lib/chain/ChainMapper.java | 10 ++-- .../mapreduce/lib/chain/ChainReducer.java | 14 +++--- .../hadoop/mapreduce/lib/db/DBInputFormat.java | 2 +- .../hadoop/mapreduce/lib/db/DBWritable.java | 2 +- .../mapreduce/lib/join/TupleWritable.java | 2 +- .../mapreduce/lib/map/MultithreadedMapper.java | 6 +-- .../mapreduce/lib/output/FileOutputFormat.java | 2 +- .../mapreduce/lib/output/MultipleOutputs.java | 11 ++--- .../lib/partition/BinaryPartitioner.java| 2 +- .../hadoop/mapreduce/task/JobContextImpl.java | 2 - .../hadoop/mapreduce/RandomTextWriter.java | 4 +- .../apache/hadoop/mapreduce/RandomWriter.java | 5 +- .../hadoop/examples/MultiFileWordCount.java | 2 +- .../apache/hadoop/examples/QuasiMonteCarlo.java | 4 +- .../hadoop/examples/RandomTextWriter.java | 4 +- .../apache/hadoop/examples/RandomWriter.java| 5 +- .../apache/hadoop/examples/SecondarySort.java | 2 +- .../org/apache/hadoop/examples/pi/DistBbp.java | 2 +- .../apache/hadoop/examples/pi/math/Modular.java | 2 +- .../hadoop/examples/terasort/GenSort.java | 2 +- .../org/apache/hadoop/tools/CopyListing.java| 14 +++--- .../java/org/apache/hadoop/tools/DistCp.java| 4 +- .../apache/hadoop/tools/DistCpOptionSwitch.java | 2 +- .../org/apache/hadoop/tools/OptionsParser.java | 2 +- .../hadoop/tools/mapred/CopyCommitter.java | 4 +- .../apache/hadoop/tools/mapred/CopyMapper.java | 5 +- .../hadoop/tools/mapred/CopyOutputFormat.java | 4 +- .../tools/mapred/RetriableFileCopyCommand.java | 3 +- .../tools/mapred/UniformSizeInputFormat.java| 4 +- .../tools/mapred/lib/DynamicInputFormat.java| 4 +- .../tools/mapred/lib/DynamicRecordReader.java | 12 ++--- .../apache/hadoop/tools/util/DistCpUtils.java | 2 +- .../hadoop/tools/util/RetriableCommand.java | 2 +- .../hadoop/tools/util/ThrottledInputStream.java | 8 ++-- .../java/org/apache/hadoop/tools/Logalyzer.java | 4 +- .../ResourceUsageEmulatorPlugin.java| 2 +- .../fs/swift/http/RestClientBindings.java | 6 +-- .../hadoop/fs/swift/http/SwiftRestClient.java | 6 +-- .../fs/swift/snative/SwiftNativeFileSystem.java | 6 +--
[02/43] hadoop git commit: recommit HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir. (cherry picked from commit 7c6b6547eeed110e1a842e503bfd33afe04fa814)
recommit HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir. (cherry picked from commit 7c6b6547eeed110e1a842e503bfd33afe04fa814) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cf51ff2f Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cf51ff2f Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cf51ff2f Branch: refs/heads/YARN-2928 Commit: cf51ff2fe8f0f08060dd1a9d96dac0c032277f77 Parents: 8719cdd Author: Tsz-Wo Nicholas Sze szets...@hortonworks.com Authored: Tue Feb 10 17:48:57 2015 -0800 Committer: Konstantin V Shvachko s...@apache.org Committed: Fri Feb 27 14:30:41 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../hadoop-hdfs/src/test/resources/testHDFSConf.xml | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf51ff2f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index b4b0087..2a8da43 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -981,6 +981,9 @@ Release 2.7.0 - UNRELEASED HDFS-7714. Simultaneous restart of HA NameNodes and DataNode can cause DataNode to register successfully with only one NameNode.(vinayakumarb) +HDFS-7769. TestHDFSCLI should not create files in hdfs project root dir. +(szetszwo) + HDFS-7753. Fix Multithreaded correctness Warnings in BackupImage. (Rakesh R and shv) http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf51ff2f/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml index e59b05a..2d3de1f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml @@ -16483,8 +16483,8 @@ command-fs NAMENODE -mkdir -p /user/USERNAME/dir1/command command-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1/command command-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1/command -command-fs NAMENODE -getmerge /user/USERNAME/dir1 data/command -command-cat data/command +command-fs NAMENODE -getmerge /user/USERNAME/dir1 CLITEST_DATA/file/command +command-cat CLITEST_DATA/file/command /test-commands cleanup-commands command-fs NAMENODE -rm -r /user/USERNAME/command
[18/43] hadoop git commit: HDFS-5853. Add hadoop.user.group.metrics.percentiles.intervals to hdfs-default.xml (aajisaka)
HDFS-5853. Add hadoop.user.group.metrics.percentiles.intervals to hdfs-default.xml (aajisaka) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/aa55fd30 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aa55fd30 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aa55fd30 Branch: refs/heads/YARN-2928 Commit: aa55fd3096442f186aebc5a767d7e271b7224b51 Parents: dbc9b64 Author: Akira Ajisaka aajis...@apache.org Authored: Sun Mar 1 01:16:36 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Sun Mar 1 01:16:36 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../hadoop-hdfs/src/main/resources/hdfs-default.xml | 11 +++ 2 files changed, 14 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa55fd30/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 16fe394..ce35ea2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -691,6 +691,9 @@ Release 2.7.0 - UNRELEASED HDFS-7685. Document dfs.namenode.heartbeat.recheck-interval in hdfs-default.xml. (Kai Sasaki via aajisaka) +HDFS-5853. Add hadoop.user.group.metrics.percentiles.intervals to +hdfs-default.xml. (aajisaka) + OPTIMIZATIONS HDFS-7454. Reduce memory footprint for AclEntries in NameNode. http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa55fd30/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 66fe86c..7eacfc5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -1518,6 +1518,17 @@ /property property + namehadoop.user.group.metrics.percentiles.intervals/name + value/value + description +A comma-separated list of the granularity in seconds for the metrics +which describe the 50/75/90/95/99th percentile latency for group resolution +in milliseconds. +By default, percentile latency metrics are disabled. + /description +/property + +property namedfs.encrypt.data.transfer/name valuefalse/value description
[07/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md new file mode 100644 index 000..1812a44 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md @@ -0,0 +1,233 @@ +!--- + 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. +-- + +Hadoop: Fair Scheduler +== + +* [Purpose](#Purpose) +* [Introduction](#Introduction) +* [Hierarchical queues with pluggable policies](#Hierarchical_queues_with_pluggable_policies) +* [Automatically placing applications in queues](#Automatically_placing_applications_in_queues) +* [Installation](#Installation) +* [Configuration](#Configuration) +* [Properties that can be placed in yarn-site.xml](#Properties_that_can_be_placed_in_yarn-site.xml) +* [Allocation file format](#Allocation_file_format) +* [Queue Access Control Lists](#Queue_Access_Control_Lists) +* [Administration](#Administration) +* [Modifying configuration at runtime](#Modifying_configuration_at_runtime) +* [Monitoring through web UI](#Monitoring_through_web_UI) +* [Moving applications between queues](#Moving_applications_between_queues) + +##Purpose + +This document describes the `FairScheduler`, a pluggable scheduler for Hadoop that allows YARN applications to share resources in large clusters fairly. + +##Introduction + +Fair scheduling is a method of assigning resources to applications such that all apps get, on average, an equal share of resources over time. Hadoop NextGen is capable of scheduling multiple resource types. By default, the Fair Scheduler bases scheduling fairness decisions only on memory. It can be configured to schedule with both memory and CPU, using the notion of Dominant Resource Fairness developed by Ghodsi et al. When there is a single app running, that app uses the entire cluster. When other apps are submitted, resources that free up are assigned to the new apps, so that each app eventually on gets roughly the same amount of resources. Unlike the default Hadoop scheduler, which forms a queue of apps, this lets short apps finish in reasonable time while not starving long-lived apps. It is also a reasonable way to share a cluster between a number of users. Finally, fair sharing can also work with app priorities - the priorities are used as weights to determine the fraction of t otal resources that each app should get. + +The scheduler organizes apps further into queues, and shares resources fairly between these queues. By default, all users share a single queue, named default. If an app specifically lists a queue in a container resource request, the request is submitted to that queue. It is also possible to assign queues based on the user name included with the request through configuration. Within each queue, a scheduling policy is used to share resources between the running apps. The default is memory-based fair sharing, but FIFO and multi-resource with Dominant Resource Fairness can also be configured. Queues can be arranged in a hierarchy to divide resources and configured with weights to share the cluster in specific proportions. + +In addition to providing fair sharing, the Fair Scheduler allows assigning guaranteed minimum shares to queues, which is useful for ensuring that certain users, groups or production applications always get sufficient resources. When a queue contains apps, it gets at least its minimum share, but when the queue does not need its full guaranteed share, the excess is split between other running apps. This lets the scheduler guarantee capacity for queues while utilizing resources efficiently when these queues don't contain applications. + +The Fair Scheduler lets all apps run by default, but it is also possible to limit the number of running apps per user and per queue through the config file. This can be useful when a user must submit hundreds of apps at once, or in general to improve performance if running too many apps at once would cause too much intermediate data to be created or too much context-switching. Limiting the apps does not cause any subsequently submitted apps
[21/43] hadoop git commit: HADOOP-11615. Update ServiceLevelAuth.md for YARN. Contributed by Brahma Reddy Battula.
HADOOP-11615. Update ServiceLevelAuth.md for YARN. Contributed by Brahma Reddy Battula. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dd9cd079 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dd9cd079 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dd9cd079 Branch: refs/heads/YARN-2928 Commit: dd9cd0797c265edfa7c3f18d2efce7c8f2801a6d Parents: 30e73eb Author: Akira Ajisaka aajis...@apache.org Authored: Sun Mar 1 22:16:06 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Sun Mar 1 22:16:06 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt| 3 +++ .../src/site/markdown/ServiceLevelAuth.md | 17 - 2 files changed, 11 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd9cd079/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index b1a7a7d..4c0c375 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -1025,6 +1025,9 @@ Release 2.7.0 - UNRELEASED HADOOP-11634. Description of webhdfs' principal/keytab should switch places each other. (Brahma Reddy Battula via ozawa) +HADOOP-11615. Update ServiceLevelAuth.md for YARN. +(Brahma Reddy Battula via aajisaka) + Release 2.6.1 - UNRELEASED INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd9cd079/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md -- diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md b/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md index ae41b47..e0017d4 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/ServiceLevelAuth.md @@ -68,10 +68,9 @@ This section lists the various Hadoop services and their configuration knobs: | security.datanode.protocol.acl | ACL for DatanodeProtocol, which is used by datanodes to communicate with the namenode. | | security.inter.datanode.protocol.acl | ACL for InterDatanodeProtocol, the inter-datanode protocol for updating generation timestamp. | | security.namenode.protocol.acl | ACL for NamenodeProtocol, the protocol used by the secondary namenode to communicate with the namenode. | -| security.inter.tracker.protocol.acl | ACL for InterTrackerProtocol, used by the tasktrackers to communicate with the jobtracker. | -| security.job.submission.protocol.acl | ACL for JobSubmissionProtocol, used by job clients to communciate with the jobtracker for job submission, querying job status etc. | -| security.task.umbilical.protocol.acl | ACL for TaskUmbilicalProtocol, used by the map and reduce tasks to communicate with the parent tasktracker. | -| security.refresh.policy.protocol.acl | ACL for RefreshAuthorizationPolicyProtocol, used by the dfsadmin and mradmin commands to refresh the security policy in-effect. | +| security.job.client.protocol.acl | ACL for JobSubmissionProtocol, used by job clients to communciate with the resourcemanager for job submission, querying job status etc. | +| security.job.task.protocol.acl | ACL for TaskUmbilicalProtocol, used by the map and reduce tasks to communicate with the parent nodemanager. | +| security.refresh.policy.protocol.acl | ACL for RefreshAuthorizationPolicyProtocol, used by the dfsadmin and rmadmin commands to refresh the security policy in-effect. | | security.ha.service.protocol.acl | ACL for HAService protocol used by HAAdmin to manage the active and stand-by states of namenode. | ### Access Control Lists @@ -98,15 +97,15 @@ If access control list is not defined for a service, the value of `security.serv ### Refreshing Service Level Authorization Configuration -The service-level authorization configuration for the NameNode and JobTracker can be changed without restarting either of the Hadoop master daemons. The cluster administrator can change `$HADOOP_CONF_DIR/hadoop-policy.xml` on the master nodes and instruct the NameNode and JobTracker to reload their respective configurations via the `-refreshServiceAcl` switch to `dfsadmin` and `mradmin` commands respectively. +The service-level authorization configuration for the NameNode and ResourceManager can be changed without restarting either of the Hadoop master daemons. The cluster administrator can change `$HADOOP_CONF_DIR/hadoop-policy.xml` on the master nodes and instruct the NameNode and ResourceManager to reload their respective
[29/43] hadoop git commit: YARN-3265. Fixed a deadlock in CapacityScheduler by always passing a queue's available resource-limit from the parent queue. Contributed by Wangda Tan.
YARN-3265. Fixed a deadlock in CapacityScheduler by always passing a queue's available resource-limit from the parent queue. Contributed by Wangda Tan. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/14dd647c Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/14dd647c Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/14dd647c Branch: refs/heads/YARN-2928 Commit: 14dd647c556016d351f425ee956ccf800ccb9ce2 Parents: abac6eb Author: Vinod Kumar Vavilapalli vino...@apache.org Authored: Mon Mar 2 17:52:47 2015 -0800 Committer: Vinod Kumar Vavilapalli vino...@apache.org Committed: Mon Mar 2 17:52:47 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 3 + .../scheduler/ResourceLimits.java | 40 +++ .../scheduler/ResourceUsage.java| 61 ++--- .../scheduler/capacity/AbstractCSQueue.java | 24 +- .../scheduler/capacity/CSQueue.java | 11 +- .../scheduler/capacity/CSQueueUtils.java| 48 .../capacity/CapacityHeadroomProvider.java | 16 +- .../scheduler/capacity/CapacityScheduler.java | 30 ++- .../scheduler/capacity/LeafQueue.java | 131 +- .../scheduler/capacity/ParentQueue.java | 53 +++- .../yarn/server/resourcemanager/MockAM.java | 11 +- .../scheduler/TestResourceUsage.java| 2 +- .../capacity/TestApplicationLimits.java | 32 +-- .../scheduler/capacity/TestCSQueueUtils.java| 250 --- .../capacity/TestCapacityScheduler.java | 85 ++- .../scheduler/capacity/TestChildQueueOrder.java | 36 ++- .../scheduler/capacity/TestLeafQueue.java | 221 ++-- .../scheduler/capacity/TestParentQueue.java | 106 .../scheduler/capacity/TestReservations.java| 100 +--- 19 files changed, 646 insertions(+), 614 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/14dd647c/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index d07aa26..0850f0b 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -686,6 +686,9 @@ Release 2.7.0 - UNRELEASED YARN-3270. Fix node label expression not getting set in ApplicationSubmissionContext (Rohit Agarwal via wangda) +YARN-3265. Fixed a deadlock in CapacityScheduler by always passing a queue's +available resource-limit from the parent queue. (Wangda Tan via vinodkv) + Release 2.6.0 - 2014-11-18 INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/14dd647c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java new file mode 100644 index 000..12333e8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler; + +import org.apache.hadoop.yarn.api.records.Resource; + +/** + * Resource limits for queues/applications, this means max overall (please note + * that, it's not extra) resource you can get. + */ +public class ResourceLimits { + public ResourceLimits(Resource limit) { +this.limit = limit; + } + + volatile Resource limit; + public Resource getLimit() { +
[05/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md new file mode 100644 index 000..e516afb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md @@ -0,0 +1,181 @@ +!--- + 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. +-- + +ResourceManger Restart +== + +* [Overview](#Overview) +* [Feature](#Feature) +* [Configurations](#Configurations) +* [Enable RM Restart](#Enable_RM_Restart) +* [Configure the state-store for persisting the RM state](#Configure_the_state-store_for_persisting_the_RM_state) +* [How to choose the state-store implementation](#How_to_choose_the_state-store_implementation) +* [Configurations for Hadoop FileSystem based state-store implementation](#Configurations_for_Hadoop_FileSystem_based_state-store_implementation) +* [Configurations for ZooKeeper based state-store implementation](#Configurations_for_ZooKeeper_based_state-store_implementation) +* [Configurations for LevelDB based state-store implementation](#Configurations_for_LevelDB_based_state-store_implementation) +* [Configurations for work-preserving RM recovery](#Configurations_for_work-preserving_RM_recovery) +* [Notes](#Notes) +* [Sample Configurations](#Sample_Configurations) + +Overview + + +ResourceManager is the central authority that manages resources and schedules applications running atop of YARN. Hence, it is potentially a single point of failure in a Apache YARN cluster. +` +This document gives an overview of ResourceManager Restart, a feature that enhances ResourceManager to keep functioning across restarts and also makes ResourceManager down-time invisible to end-users. + +ResourceManager Restart feature is divided into two phases: + +* **ResourceManager Restart Phase 1 (Non-work-preserving RM restart)**: Enhance RM to persist application/attempt state and other credentials information in a pluggable state-store. RM will reload this information from state-store upon restart and re-kick the previously running applications. Users are not required to re-submit the applications. + +* **ResourceManager Restart Phase 2 (Work-preserving RM restart)**: Focus on re-constructing the running state of ResourceManager by combining the container statuses from NodeManagers and container requests from ApplicationMasters upon restart. The key difference from phase 1 is that previously running applications will not be killed after RM restarts, and so applications won't lose its work because of RM outage. + +Feature +--- + +* **Phase 1: Non-work-preserving RM restart** + + As of Hadoop 2.4.0 release, only ResourceManager Restart Phase 1 is implemented which is described below. + + The overall concept is that RM will persist the application metadata (i.e. ApplicationSubmissionContext) in a pluggable state-store when client submits an application and also saves the final status of the application such as the completion state (failed, killed, finished) and diagnostics when the application completes. Besides, RM also saves the credentials like security keys, tokens to work in a secure environment. Any time RM shuts down, as long as the required information (i.e.application metadata and the alongside credentials if running in a secure environment) is available in the state-store, when RM restarts, it can pick up the application metadata from the state-store and re-submit the application. RM won't re-submit the applications if they were already completed (i.e. failed, killed, finished) before RM went down. + + NodeManagers and clients during the down-time of RM will keep polling RM until RM comes up. When RM becomes alive, it will send a re-sync command to all the NodeManagers and ApplicationMasters it was talking to via heartbeats. As of Hadoop 2.4.0 release, the behaviors for NodeManagers and ApplicationMasters to handle this command are: NMs will kill all its managed containers and re-register with RM. From the RM's perspective, these re-registered NodeManagers are similar to the
[06/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md new file mode 100644 index 000..b1591bb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md @@ -0,0 +1,2640 @@ +!--- + 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. +-- + +ResourceManager REST API's. +=== + +* [Overview](#Overview) +* [Cluster Information API](#Cluster_Information_API) +* [Cluster Metrics API](#Cluster_Metrics_API) +* [Cluster Scheduler API](#Cluster_Scheduler_API) +* [Cluster Applications API](#Cluster_Applications_API) +* [Cluster Application Statistics API](#Cluster_Application_Statistics_API) +* [Cluster Application API](#Cluster_Application_API) +* [Cluster Application Attempts API](#Cluster_Application_Attempts_API) +* [Cluster Nodes API](#Cluster_Nodes_API) +* [Cluster Node API](#Cluster_Node_API) +* [Cluster Writeable APIs](#Cluster_Writeable_APIs) +* [Cluster New Application API](#Cluster_New_Application_API) +* [Cluster Applications API(Submit Application)](#Cluster_Applications_APISubmit_Application) +* [Cluster Application State API](#Cluster_Application_State_API) +* [Cluster Application Queue API](#Cluster_Application_Queue_API) +* [Cluster Delegation Tokens API](#Cluster_Delegation_Tokens_API) + +Overview + + +The ResourceManager REST API's allow the user to get information about the cluster - status on the cluster, metrics on the cluster, scheduler information, information about nodes in the cluster, and information about applications on the cluster. + +Cluster Information API +--- + +The cluster information resource provides overall information about the cluster. + +### URI + +Both of the following URI's give you the cluster information. + + * http://rm http address:port/ws/v1/cluster + * http://rm http address:port/ws/v1/cluster/info + +### HTTP Operations Supported + + * GET + +### Query Parameters Supported + + None + +### Elements of the *clusterInfo* object + +| Item | Data Type | Description | +|: |: |: | +| id | long | The cluster id | +| startedOn | long | The time the cluster started (in ms since epoch) | +| state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED | +| haState | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED | +| resourceManagerVersion | string | Version of the ResourceManager | +| resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum | +| resourceManagerVersionBuiltOn | string | Timestamp when ResourceManager was built (in ms since epoch) | +| hadoopVersion | string | Version of hadoop common | +| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum | +| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch) | + +### Response Examples + +**JSON response** + +HTTP Request: + + GET http://rm http address:port/ws/v1/cluster/info + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/json + Transfer-Encoding: chunked + Server: Jetty(6.1.26) + +Response Body: + +```json +{ + clusterInfo: + { +id:1324053971963, +startedOn:1324053971963, +state:STARTED, +resourceManagerVersion:0.23.1-SNAPSHOT, +resourceManagerBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693, +resourceManagerVersionBuiltOn:Tue Dec 13 22:12:48 CST 2011, +hadoopVersion:0.23.1-SNAPSHOT, +hadoopBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328, +hadoopVersionBuiltOn:Tue Dec 13 22:12:26 CST 2011 + } +} +``` + +**XML response** + +HTTP Request: + + Accept: application/xml + GET http://rm http address:port/ws/v1/cluster/info + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/xml + Content-Length: 712 + Server: Jetty(6.1.26) + +Response Body: + +```xml +?xml
[43/43] hadoop git commit: YARN-3210. Refactored timeline aggregator according to new code organization proposed in YARN-3166. Contributed by Li Lu.
YARN-3210. Refactored timeline aggregator according to new code organization proposed in YARN-3166. Contributed by Li Lu. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d3ff7f06 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d3ff7f06 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d3ff7f06 Branch: refs/heads/YARN-2928 Commit: d3ff7f06cbc66d3a23c2551e7d4c752689f46afe Parents: e4d81eb Author: Zhijie Shen zjs...@apache.org Authored: Tue Mar 3 11:21:03 2015 -0800 Committer: Zhijie Shen zjs...@apache.org Committed: Tue Mar 3 11:25:17 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 3 + .../distributedshell/TestDistributedShell.java | 4 +- .../hadoop-yarn-server-nodemanager/pom.xml | 5 - .../server/nodemanager/webapp/WebServer.java| 3 - .../TestTimelineServiceClientIntegration.java | 12 +- .../aggregator/AppLevelAggregatorService.java | 57 .../aggregator/AppLevelServiceManager.java | 136 -- .../AppLevelServiceManagerProvider.java | 33 --- .../aggregator/AppLevelTimelineAggregator.java | 57 .../aggregator/BaseAggregatorService.java | 107 .../aggregator/PerNodeAggregatorServer.java | 268 --- .../aggregator/PerNodeAggregatorWebService.java | 180 - .../PerNodeTimelineAggregatorsAuxService.java | 212 +++ .../aggregator/TimelineAggregator.java | 107 .../TimelineAggregatorWebService.java | 180 + .../TimelineAggregatorsCollection.java | 203 ++ .../TestAppLevelAggregatorService.java | 23 -- .../aggregator/TestAppLevelServiceManager.java | 102 --- .../TestAppLevelTimelineAggregator.java | 23 ++ .../aggregator/TestBaseAggregatorService.java | 23 -- .../aggregator/TestPerNodeAggregatorServer.java | 149 --- ...estPerNodeTimelineAggregatorsAuxService.java | 150 +++ .../aggregator/TestTimelineAggregator.java | 23 ++ .../TestTimelineAggregatorsCollection.java | 108 24 files changed, 1074 insertions(+), 1094 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3ff7f06/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index b13475a..0548460 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -23,6 +23,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1 YARN-3125. Made the distributed shell use timeline service next gen and add an integration test for it. (Junping Du and Li Lu via zjshen) +YARN-3210. Refactored timeline aggregator according to new code +organization proposed in YARN-3166. (Li Lu via zjshen) + IMPROVEMENTS OPTIMIZATIONS http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3ff7f06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java index 71466cb..313dc97 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java @@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; -import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeAggregatorServer; +import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeTimelineAggregatorsAuxService; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -96,7 +96,7 @@ public class TestDistributedShell { // enable aux-service based timeline aggregators conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
[13/43] hadoop git commit: YARN-3199. Fair Scheduler documentation improvements (Rohit Agarwal via aw)
YARN-3199. Fair Scheduler documentation improvements (Rohit Agarwal via aw) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8472d729 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8472d729 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8472d729 Branch: refs/heads/YARN-2928 Commit: 8472d729974ea3ccf9fff5ce4f5309aa8e43a49e Parents: 2e44b75 Author: Allen Wittenauer a...@apache.org Authored: Sat Feb 28 11:36:15 2015 -0800 Committer: Allen Wittenauer a...@apache.org Committed: Sat Feb 28 11:36:15 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 5 - .../hadoop-yarn-site/src/site/markdown/FairScheduler.md | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/8472d729/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 02b1831..cef1758 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -1,6 +1,6 @@ Hadoop YARN Change Log -Trunk - Unreleased +Trunk - Unreleased INCOMPATIBLE CHANGES @@ -23,6 +23,9 @@ Trunk - Unreleased YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw) +YARN-3199. Fair Scheduler documentation improvements (Rohit Agarwal via +aw) + OPTIMIZATIONS BUG FIXES http://git-wip-us.apache.org/repos/asf/hadoop/blob/8472d729/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md index 1812a44..a58b3d3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md @@ -85,6 +85,8 @@ Customizing the Fair Scheduler typically involves altering two files. First, sch | `yarn.scheduler.fair.locality.threshold.rack` | For applications that request containers on particular racks, the number of scheduling opportunities since the last container assignment to wait before accepting a placement on another rack. Expressed as a float between 0 and 1, which, as a fraction of the cluster size, is the number of scheduling opportunities to pass up. The default value of -1.0 means don't pass up any scheduling opportunities. | | `yarn.scheduler.fair.allow-undeclared-pools` | If this is true, new queues can be created at application submission time, whether because they are specified as the application's queue by the submitter or because they are placed there by the user-as-default-queue property. If this is false, any time an app would be placed in a queue that is not specified in the allocations file, it is placed in the default queue instead. Defaults to true. If a queue placement policy is given in the allocations file, this property is ignored. | | `yarn.scheduler.fair.update-interval-ms` | The interval at which to lock the scheduler and recalculate fair shares, recalculate demand, and check whether anything is due for preemption. Defaults to 500 ms. | +| `yarn.scheduler.increment-allocation-mb` | The fairscheduler grants memory in increments of this value. If you submit a task with resource request that is not a multiple of increment-allocation-mb, the request will be rounded up to the nearest increment. Defaults to 1024 MB. | +| `yarn.scheduler.increment-allocation-vcores` | The fairscheduler grants vcores in increments of this value. If you submit a task with resource request that is not a multiple of increment-allocation-vcores, the request will be rounded up to the nearest increment. Defaults to 1. | ###Allocation file format
[25/43] hadoop git commit: YARN-3281. Added RMStateStore to StateMachine visualization list. Contributed by Chengbing Liu
YARN-3281. Added RMStateStore to StateMachine visualization list. Contributed by Chengbing Liu Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5d0bae55 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5d0bae55 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5d0bae55 Branch: refs/heads/YARN-2928 Commit: 5d0bae550f5b9a6005aa1d373cfe1ec80513dbd9 Parents: ca1c00b Author: Jian He jia...@apache.org Authored: Mon Mar 2 14:39:49 2015 -0800 Committer: Jian He jia...@apache.org Committed: Mon Mar 2 14:39:49 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 3 +++ .../hadoop-yarn-server-resourcemanager/pom.xml| 7 --- 2 files changed, 7 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d0bae55/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index cef1758..c7dac60 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -345,6 +345,9 @@ Release 2.7.0 - UNRELEASED YARN-3262. Surface application outstanding resource requests table in RM web UI. (Jian He via wangda) +YARN-3281. Added RMStateStore to StateMachine visualization list. +(Chengbing Liu via jianhe) + OPTIMIZATIONS YARN-2990. FairScheduler's delay-scheduling always waits for node-local and http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d0bae55/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index ff429cc..aaa0de5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -278,7 +278,7 @@ source directory${basedir}/src/main/proto/directory includes - includeyarn_server_resourcemanager_recovery.proto/include + includeyarn_server_resourcemanager_recovery.proto/include /includes /source output${project.build.directory}/generated-sources/java/output @@ -331,10 +331,11 @@ /goals configuration mainClassorg.apache.hadoop.yarn.state.VisualizeStateMachine/mainClass - classpathScopecompile/classpathScope + classpathScopecompile/classpathScope arguments argumentResourceManager/argument - argumentorg.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl, + argumentorg.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore, + org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl, org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl, org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl, org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl/argument
[20/43] hadoop git commit: HADOOP-11657. Align the output of `hadoop fs -du` to be more Unix-like. (aajisaka)
HADOOP-11657. Align the output of `hadoop fs -du` to be more Unix-like. (aajisaka) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/30e73ebc Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/30e73ebc Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/30e73ebc Branch: refs/heads/YARN-2928 Commit: 30e73ebc77654ff941bcae5b6fb11d52c6d74d2e Parents: e9ac88a Author: Akira Ajisaka aajis...@apache.org Authored: Sun Mar 1 21:09:15 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Sun Mar 1 21:09:15 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 3 ++ .../org/apache/hadoop/fs/shell/FsUsage.java | 12 ++-- .../org/apache/hadoop/hdfs/TestDFSShell.java| 29 3 files changed, 42 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/30e73ebc/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index f1d48bc..b1a7a7d 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -13,6 +13,9 @@ Trunk (Unreleased) HADOOP-10950. rework heap management vars (John Smith via aw) +HADOOP-11657. Align the output of `hadoop fs -du` to be more Unix-like. +(aajisaka) + NEW FEATURES HADOOP-6590. Add a username check for hadoop sub-commands (John Smith via aw) http://git-wip-us.apache.org/repos/asf/hadoop/blob/30e73ebc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java -- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java index 5c1dbf0..765b181 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java @@ -132,15 +132,23 @@ class FsUsage extends FsCommand { } @Override -protected void processPathArgument(PathData item) throws IOException { +protected void processArguments(LinkedListPathData args) +throws IOException { usagesTable = new TableBuilder(3); + super.processArguments(args); + if (!usagesTable.isEmpty()) { +usagesTable.printToStream(out); + } +} + +@Override +protected void processPathArgument(PathData item) throws IOException { // go one level deep on dirs from cmdline unless in summary mode if (!summary item.stat.isDirectory()) { recursePath(item); } else { super.processPathArgument(item); } - usagesTable.printToStream(out); } @Override http://git-wip-us.apache.org/repos/asf/hadoop/blob/30e73ebc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java index ee04076..0a88208 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java @@ -95,6 +95,14 @@ public class TestDFSShell { return f; } + static Path writeByte(FileSystem fs, Path f) throws IOException { +DataOutputStream out = fs.create(f); +out.writeByte(1); +out.close(); +assertTrue(fs.exists(f)); +return f; + } + static Path mkdir(FileSystem fs, Path p) throws IOException { assertTrue(fs.mkdirs(p)); assertTrue(fs.exists(p)); @@ -272,6 +280,27 @@ public class TestDFSShell { Long combinedDiskUsed = myFileDiskUsed + myFile2DiskUsed; assertThat(returnString, containsString(combinedLength.toString())); assertThat(returnString, containsString(combinedDiskUsed.toString())); + + // Check if output is rendered properly with multiple input paths + Path myFile3 = new Path(/test/dir/file3); + writeByte(fs, myFile3); + assertTrue(fs.exists(myFile3)); + args = new String[3]; + args[0] = -du; + args[1] = /test/dir/file3; + args[2] = /test/dir/file2; + val = -1; + try { +val = shell.run(args); + } catch (Exception e) { +System.err.println(Exception raised from DFSShell.run + +e.getLocalizedMessage()); + } +
[41/43] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into YARN-2928
Merge remote-tracking branch 'apache/trunk' into YARN-2928 Conflicts: hadoop-yarn-project/CHANGES.txt Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e4d81ebb Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e4d81ebb Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e4d81ebb Branch: refs/heads/YARN-2928 Commit: e4d81ebb335a928d4806cffe556db35208cfd9a9 Parents: bf08f7f 1004473 Author: Zhijie Shen zjs...@apache.org Authored: Tue Mar 3 11:11:41 2015 -0800 Committer: Zhijie Shen zjs...@apache.org Committed: Tue Mar 3 11:11:41 2015 -0800 -- .../classification/tools/StabilityOptions.java |5 +- .../AltKerberosAuthenticationHandler.java |6 +- .../authentication/KerberosTestUtils.java | 40 +- .../authentication/util/TestKerberosUtil.java | 14 +- hadoop-common-project/hadoop-common/CHANGES.txt | 26 +- .../org/apache/hadoop/conf/Configuration.java |6 +- .../org/apache/hadoop/crypto/CipherSuite.java |3 +- .../hadoop/crypto/key/JavaKeyStoreProvider.java |3 +- .../hadoop/fs/CommonConfigurationKeys.java | 17 +- .../java/org/apache/hadoop/fs/FileSystem.java |7 +- .../org/apache/hadoop/fs/FilterFileSystem.java |2 +- .../java/org/apache/hadoop/fs/StorageType.java |3 +- .../apache/hadoop/fs/permission/AclEntry.java |5 +- .../org/apache/hadoop/fs/shell/FsUsage.java | 12 +- .../apache/hadoop/fs/shell/XAttrCommands.java |2 +- .../org/apache/hadoop/fs/shell/find/Name.java |5 +- .../io/compress/CompressionCodecFactory.java| 28 +- .../hadoop/metrics2/impl/MetricsConfig.java |7 +- .../hadoop/metrics2/impl/MetricsSystemImpl.java |5 +- .../hadoop/security/SaslPropertiesResolver.java |3 +- .../apache/hadoop/security/SecurityUtil.java| 12 +- .../hadoop/security/WhitelistBasedResolver.java |3 +- .../security/ssl/FileBasedKeyStoresFactory.java |4 +- .../apache/hadoop/security/ssl/SSLFactory.java |5 +- .../security/ssl/SSLHostnameVerifier.java | 10 +- .../DelegationTokenAuthenticationHandler.java |3 +- .../web/DelegationTokenAuthenticator.java |3 +- .../apache/hadoop/util/ComparableVersion.java |3 +- .../org/apache/hadoop/util/StringUtils.java | 40 +- .../src/site/markdown/SecureMode.md |4 +- .../src/site/markdown/ServiceLevelAuth.md | 17 +- .../hadoop/fs/FileSystemContractBaseTest.java |4 +- .../hadoop/io/compress/TestCodecFactory.java|3 +- .../java/org/apache/hadoop/ipc/TestIPC.java |2 +- .../java/org/apache/hadoop/ipc/TestSaslRPC.java |2 +- .../hadoop/security/TestSecurityUtil.java | 10 +- .../security/TestUserGroupInformation.java |5 +- .../hadoop/test/TimedOutTestsListener.java |6 +- .../org/apache/hadoop/util/TestStringUtils.java | 21 + .../org/apache/hadoop/util/TestWinUtils.java|6 +- .../java/org/apache/hadoop/nfs/NfsExports.java |5 +- .../server/CheckUploadContentTypeFilter.java|4 +- .../hadoop/fs/http/server/FSOperations.java |7 +- .../http/server/HttpFSParametersProvider.java |4 +- .../org/apache/hadoop/lib/server/Server.java|3 +- .../service/hadoop/FileSystemAccessService.java |6 +- .../org/apache/hadoop/lib/wsrs/EnumParam.java |2 +- .../apache/hadoop/lib/wsrs/EnumSetParam.java|3 +- .../hadoop/lib/wsrs/ParametersProvider.java |3 +- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 31 + .../java/org/apache/hadoop/hdfs/DFSClient.java | 26 +- .../org/apache/hadoop/hdfs/DFSOutputStream.java | 15 +- .../apache/hadoop/hdfs/RemoteBlockReader2.java | 24 +- .../org/apache/hadoop/hdfs/XAttrHelper.java | 19 +- .../hadoop/hdfs/protocol/HdfsConstants.java |3 +- .../datatransfer/DataTransferProtoUtil.java | 26 + .../hadoop/hdfs/server/balancer/Dispatcher.java |9 +- .../BlockStoragePolicySuite.java|4 +- .../hdfs/server/common/HdfsServerConstants.java | 15 +- .../hdfs/server/datanode/DataXceiver.java | 14 +- .../hdfs/server/datanode/StorageLocation.java |4 +- .../hdfs/server/namenode/FSEditLogLoader.java |3 - .../hdfs/server/namenode/FSEditLogOp.java |3 +- .../hadoop/hdfs/server/namenode/FSImage.java| 10 +- .../hdfs/server/namenode/FSNamesystem.java | 11 +- .../namenode/QuotaByStorageTypeEntry.java |3 +- .../hdfs/server/namenode/SecondaryNameNode.java |2 +- .../hdfs/server/namenode/TransferFsImage.java |4 +- .../org/apache/hadoop/hdfs/tools/DFSck.java | 31 +- .../org/apache/hadoop/hdfs/tools/GetConf.java | 17 +- .../OfflineEditsVisitorFactory.java |7 +- .../offlineImageViewer/FSImageHandler.java |4 +-
[40/43] hadoop git commit: HDFS-7757. Misleading error messages in FSImage.java. (Contributed by Brahma Reddy Battula)
HDFS-7757. Misleading error messages in FSImage.java. (Contributed by Brahma Reddy Battula) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1004473a Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1004473a Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1004473a Branch: refs/heads/YARN-2928 Commit: 1004473aa612ee3703394943f25687aa5bef47ea Parents: 4228de9 Author: Arpit Agarwal a...@apache.org Authored: Tue Mar 3 10:55:22 2015 -0800 Committer: Arpit Agarwal a...@apache.org Committed: Tue Mar 3 10:55:22 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt| 3 +++ .../java/org/apache/hadoop/hdfs/server/namenode/FSImage.java | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/1004473a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index fe78097..42430ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1074,6 +1074,9 @@ Release 2.7.0 - UNRELEASED HDFS-7871. NameNodeEditLogRoller can keep printing Swallowing exception message. (jing9) +HDFS-7757. Misleading error messages in FSImage.java. (Brahma Reddy Battula +via Arpit Agarwal) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/1004473a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java index 44c41d0..e589eea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java @@ -883,7 +883,7 @@ public class FSImage implements Closeable { final long namespace = counts.getNameSpace() - parentNamespace; final long nsQuota = q.getNameSpace(); if (Quota.isViolated(nsQuota, namespace)) { -LOG.error(BUG: Namespace quota violation in image for +LOG.warn(Namespace quota violation in image for + dir.getFullPathName() + quota = + nsQuota + consumed = + namespace); } @@ -891,7 +891,7 @@ public class FSImage implements Closeable { final long ssConsumed = counts.getStorageSpace() - parentStoragespace; final long ssQuota = q.getStorageSpace(); if (Quota.isViolated(ssQuota, ssConsumed)) { -LOG.error(BUG: Storagespace quota violation in image for +LOG.warn(Storagespace quota violation in image for + dir.getFullPathName() + quota = + ssQuota + consumed = + ssConsumed); } @@ -903,7 +903,7 @@ public class FSImage implements Closeable { parentTypeSpaces.get(t); final long typeQuota = q.getTypeSpaces().get(t); if (Quota.isViolated(typeQuota, typeSpace)) { - LOG.error(BUG: Storage type quota violation in image for + LOG.warn(Storage type quota violation in image for + dir.getFullPathName() + type = + t.toString() + quota = + typeQuota + consumed + typeSpace);
[19/43] hadoop git commit: HADOOP-11634. Description of webhdfs' principal/keytab should switch places each other. Contributed by Brahma Reddy Battula.
HADOOP-11634. Description of webhdfs' principal/keytab should switch places each other. Contributed by Brahma Reddy Battula. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e9ac88aa Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e9ac88aa Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e9ac88aa Branch: refs/heads/YARN-2928 Commit: e9ac88aac77dd98508854de445793c2180466ee8 Parents: aa55fd3 Author: Tsuyoshi Ozawa oz...@apache.org Authored: Mon Mar 2 04:18:07 2015 +0900 Committer: Tsuyoshi Ozawa oz...@apache.org Committed: Mon Mar 2 04:18:07 2015 +0900 -- hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++ .../hadoop-common/src/site/markdown/SecureMode.md| 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9ac88aa/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 3c4dc99..f1d48bc 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -1019,6 +1019,9 @@ Release 2.7.0 - UNRELEASED HADOOP-9922. hadoop windows native build will fail in 32 bit machine. (Kiran Kumar M R via cnauroth) +HADOOP-11634. Description of webhdfs' principal/keytab should switch places +each other. (Brahma Reddy Battula via ozawa) + Release 2.6.1 - UNRELEASED INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9ac88aa/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md -- diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md index 0004d25..cb27e29 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md @@ -289,8 +289,8 @@ The following properties should be in the `core-site.xml` of all the nodes in th | Parameter | Value | Notes | |: |: |: | -| `dfs.web.authentication.kerberos.principal` | http/\_h...@realm.tld | Kerberos keytab file for the WebHDFS. | -| `dfs.web.authentication.kerberos.keytab` | */etc/security/keytab/http.service.keytab* | Kerberos principal name for WebHDFS. | +| `dfs.web.authentication.kerberos.principal` | http/\_h...@realm.tld | Kerberos principal name for the WebHDFS. | +| `dfs.web.authentication.kerberos.keytab` | */etc/security/keytab/http.service.keytab* | Kerberos keytab file for WebHDFS. | ### ResourceManager
[12/43] hadoop git commit: YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw)
YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty via aw) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2e44b75f Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2e44b75f Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2e44b75f Branch: refs/heads/YARN-2928 Commit: 2e44b75f729009d33e309d1366bf86746443db81 Parents: edceced Author: Allen Wittenauer a...@apache.org Authored: Fri Feb 27 20:39:44 2015 -0800 Committer: Allen Wittenauer a...@apache.org Committed: Fri Feb 27 20:39:44 2015 -0800 -- hadoop-yarn-project/CHANGES.txt |3 + .../src/site/apt/CapacityScheduler.apt.vm | 368 --- .../src/site/apt/DockerContainerExecutor.apt.vm | 204 -- .../src/site/apt/FairScheduler.apt.vm | 483 --- .../src/site/apt/NodeManager.apt.vm | 64 - .../src/site/apt/NodeManagerCgroups.apt.vm | 77 - .../src/site/apt/NodeManagerRest.apt.vm | 645 .../src/site/apt/NodeManagerRestart.apt.vm | 86 - .../src/site/apt/ResourceManagerHA.apt.vm | 233 -- .../src/site/apt/ResourceManagerRest.apt.vm | 3104 -- .../src/site/apt/ResourceManagerRestart.apt.vm | 298 -- .../src/site/apt/SecureContainer.apt.vm | 176 - .../src/site/apt/TimelineServer.apt.vm | 260 -- .../src/site/apt/WebApplicationProxy.apt.vm | 49 - .../src/site/apt/WebServicesIntro.apt.vm| 593 .../src/site/apt/WritingYarnApplications.apt.vm | 757 - .../hadoop-yarn-site/src/site/apt/YARN.apt.vm | 77 - .../src/site/apt/YarnCommands.apt.vm| 369 --- .../hadoop-yarn-site/src/site/apt/index.apt.vm | 82 - .../src/site/markdown/CapacityScheduler.md | 186 ++ .../site/markdown/DockerContainerExecutor.md.vm | 154 + .../src/site/markdown/FairScheduler.md | 233 ++ .../src/site/markdown/NodeManager.md| 57 + .../src/site/markdown/NodeManagerCgroups.md | 57 + .../src/site/markdown/NodeManagerRest.md| 543 +++ .../src/site/markdown/NodeManagerRestart.md | 53 + .../src/site/markdown/ResourceManagerHA.md | 140 + .../src/site/markdown/ResourceManagerRest.md| 2640 +++ .../src/site/markdown/ResourceManagerRestart.md | 181 + .../src/site/markdown/SecureContainer.md| 135 + .../src/site/markdown/TimelineServer.md | 231 ++ .../src/site/markdown/WebApplicationProxy.md| 24 + .../src/site/markdown/WebServicesIntro.md | 569 .../site/markdown/WritingYarnApplications.md| 591 .../hadoop-yarn-site/src/site/markdown/YARN.md | 42 + .../src/site/markdown/YarnCommands.md | 272 ++ .../hadoop-yarn-site/src/site/markdown/index.md | 75 + 37 files changed, 6186 insertions(+), 7925 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index e7af84b..02b1831 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -20,6 +20,9 @@ Trunk - Unreleased YARN-2980. Move health check script related functionality to hadoop-common (Varun Saxena via aw) +YARN-3168. Convert site documentation from apt to markdown (Gururaj Shetty +via aw) + OPTIMIZATIONS BUG FIXES http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e44b75f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm deleted file mode 100644 index 8528c1a..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm +++ /dev/null @@ -1,368 +0,0 @@ -~~ 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. - - --- - Hadoop Map Reduce Next Generation-${project.version} - Capacity Scheduler - --- - --- - ${maven.build.timestamp} - -Hadoop MapReduce Next Generation - Capacity
[15/43] hadoop git commit: MAPREDUCE-5653. DistCp does not honour config-overrides for mapreduce.[map, reduce].memory.mb (Ratandeep Ratti via aw)
MAPREDUCE-5653. DistCp does not honour config-overrides for mapreduce.[map,reduce].memory.mb (Ratandeep Ratti via aw) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/039366e3 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/039366e3 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/039366e3 Branch: refs/heads/YARN-2928 Commit: 039366e3b430ff7d9a7ff30405a0431292069a8a Parents: 915bec3 Author: Allen Wittenauer a...@apache.org Authored: Sat Feb 28 22:53:38 2015 -0800 Committer: Allen Wittenauer a...@apache.org Committed: Sat Feb 28 22:53:38 2015 -0800 -- hadoop-mapreduce-project/CHANGES.txt | 3 +++ .../hadoop-distcp/src/main/resources/distcp-default.xml | 10 -- 2 files changed, 3 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/039366e3/hadoop-mapreduce-project/CHANGES.txt -- diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index f509d4e..ccd24a6 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -6,6 +6,9 @@ Trunk (Unreleased) MAPREDUCE-5785. Derive heap size or mapreduce.*.memory.mb automatically. (Gera Shegalov and Karthik Kambatla via gera) +MAPREDUCE-5653. DistCp does not honour config-overrides for +mapreduce.[map,reduce].memory.mb (Ratandeep Ratti via aw) + NEW FEATURES MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk) http://git-wip-us.apache.org/repos/asf/hadoop/blob/039366e3/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml -- diff --git a/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml b/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml index f50..6e1154e 100644 --- a/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml +++ b/hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml @@ -32,16 +32,6 @@ /property property -namemapred.job.map.memory.mb/name -value1024/value -/property - -property -namemapred.job.reduce.memory.mb/name -value1024/value -/property - -property namemapred.reducer.new-api/name valuetrue/value /property
[33/43] hadoop git commit: HADOOP-11602. Fix toUpperCase/toLowerCase to use Locale.ENGLISH. (ozawa)
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c6accb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 46b45f8..21d70b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -46,6 +46,7 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; @@ -756,7 +757,7 @@ public class ClientRMService extends AbstractService implements if (applicationTypes != null !applicationTypes.isEmpty()) { String appTypeToMatch = caseSensitive ? application.getApplicationType() -: application.getApplicationType().toLowerCase(); +: StringUtils.toLowerCase(application.getApplicationType()); if (!applicationTypes.contains(appTypeToMatch)) { continue; } http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c6accb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java index 230f9a9..d6e9e45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java @@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.resource; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Evolving; +import org.apache.hadoop.util.StringUtils; @Private @Evolving @@ -61,7 +62,7 @@ public class ResourceWeights { sb.append(, ); } ResourceType resourceType = ResourceType.values()[i]; - sb.append(resourceType.name().toLowerCase()); + sb.append(StringUtils.toLowerCase(resourceType.name())); sb.append(String.format( weight=%.1f, getWeight(resourceType))); } sb.append(); http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c6accb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java index 3528c2d..102e553 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java @@ -394,7 +394,7 @@ public class
[28/43] hadoop git commit: YARN-3265. Fixed a deadlock in CapacityScheduler by always passing a queue's available resource-limit from the parent queue. Contributed by Wangda Tan.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/14dd647c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java index ead5719..a5a2e5f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java @@ -73,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; @@ -294,11 +295,13 @@ public class TestLeafQueue { //Verify the value for getAMResourceLimit for queues with .1 maxcap Resource clusterResource = Resource.newInstance(50 * GB, 50); - a.updateClusterResource(clusterResource); +a.updateClusterResource(clusterResource, +new ResourceLimits(clusterResource)); assertEquals(Resource.newInstance(1 * GB, 1), a.getAMResourceLimit()); - b.updateClusterResource(clusterResource); + b.updateClusterResource(clusterResource, +new ResourceLimits(clusterResource)); assertEquals(Resource.newInstance(5 * GB, 1), b.getAMResourceLimit()); } @@ -347,7 +350,8 @@ public class TestLeafQueue { // Start testing... // Only 1 container -a.assignContainers(clusterResource, node_0, false); +a.assignContainers(clusterResource, node_0, false, +new ResourceLimits(clusterResource)); assertEquals( (int)(node_0.getTotalResource().getMemory() * a.getCapacity()) - (1*GB), a.getMetrics().getAvailableMB()); @@ -482,7 +486,8 @@ public class TestLeafQueue { // Start testing... // Only 1 container -a.assignContainers(clusterResource, node_0, false); +a.assignContainers(clusterResource, node_0, false, +new ResourceLimits(clusterResource)); assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -492,7 +497,8 @@ public class TestLeafQueue { // Also 2nd - minCapacity = 1024 since (.1 * 8G) minAlloc, also // you can get one container more than user-limit -a.assignContainers(clusterResource, node_0, false); +a.assignContainers(clusterResource, node_0, false, +new ResourceLimits(clusterResource)); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -500,7 +506,8 @@ public class TestLeafQueue { assertEquals(2*GB, a.getMetrics().getAllocatedMB()); // Can't allocate 3rd due to user-limit -a.assignContainers(clusterResource, node_0, false); +a.assignContainers(clusterResource, node_0, false, +new ResourceLimits(clusterResource)); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -509,7 +516,8 @@ public class TestLeafQueue { // Bump up user-limit-factor, now allocate should work a.setUserLimitFactor(10); -a.assignContainers(clusterResource, node_0, false); +a.assignContainers(clusterResource, node_0, false, +new ResourceLimits(clusterResource)); assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -517,7 +525,8 @@ public class TestLeafQueue {
[1/2] hadoop git commit: HDFS-7757. Misleading error messages in FSImage.java. (Contributed by Brahma Reddy Battula)
Repository: hadoop Updated Branches: refs/heads/branch-2 a9613bcc9 - 7e6624c21 refs/heads/trunk 4228de940 - 1004473aa HDFS-7757. Misleading error messages in FSImage.java. (Contributed by Brahma Reddy Battula) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1004473a Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1004473a Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1004473a Branch: refs/heads/trunk Commit: 1004473aa612ee3703394943f25687aa5bef47ea Parents: 4228de9 Author: Arpit Agarwal a...@apache.org Authored: Tue Mar 3 10:55:22 2015 -0800 Committer: Arpit Agarwal a...@apache.org Committed: Tue Mar 3 10:55:22 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt| 3 +++ .../java/org/apache/hadoop/hdfs/server/namenode/FSImage.java | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/1004473a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index fe78097..42430ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1074,6 +1074,9 @@ Release 2.7.0 - UNRELEASED HDFS-7871. NameNodeEditLogRoller can keep printing Swallowing exception message. (jing9) +HDFS-7757. Misleading error messages in FSImage.java. (Brahma Reddy Battula +via Arpit Agarwal) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/1004473a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java index 44c41d0..e589eea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java @@ -883,7 +883,7 @@ public class FSImage implements Closeable { final long namespace = counts.getNameSpace() - parentNamespace; final long nsQuota = q.getNameSpace(); if (Quota.isViolated(nsQuota, namespace)) { -LOG.error(BUG: Namespace quota violation in image for +LOG.warn(Namespace quota violation in image for + dir.getFullPathName() + quota = + nsQuota + consumed = + namespace); } @@ -891,7 +891,7 @@ public class FSImage implements Closeable { final long ssConsumed = counts.getStorageSpace() - parentStoragespace; final long ssQuota = q.getStorageSpace(); if (Quota.isViolated(ssQuota, ssConsumed)) { -LOG.error(BUG: Storagespace quota violation in image for +LOG.warn(Storagespace quota violation in image for + dir.getFullPathName() + quota = + ssQuota + consumed = + ssConsumed); } @@ -903,7 +903,7 @@ public class FSImage implements Closeable { parentTypeSpaces.get(t); final long typeQuota = q.getTypeSpaces().get(t); if (Quota.isViolated(typeQuota, typeSpace)) { - LOG.error(BUG: Storage type quota violation in image for + LOG.warn(Storage type quota violation in image for + dir.getFullPathName() + type = + t.toString() + quota = + typeQuota + consumed + typeSpace);
[30/43] hadoop git commit: HDFS-7302. Remove downgrade from namenode -rollingUpgrade startup option since it may incorrectly finalize an ongoing rolling upgrade. Contributed by Kai Sasaki
HDFS-7302. Remove downgrade from namenode -rollingUpgrade startup option since it may incorrectly finalize an ongoing rolling upgrade. Contributed by Kai Sasaki Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/431e7d84 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/431e7d84 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/431e7d84 Branch: refs/heads/YARN-2928 Commit: 431e7d84c7b68b34ff18de19afe8e46637047fa6 Parents: 14dd647 Author: Tsz-Wo Nicholas Sze szets...@hortonworks.com Authored: Tue Mar 3 10:04:08 2015 +0800 Committer: Tsz-Wo Nicholas Sze szets...@hortonworks.com Committed: Tue Mar 3 10:04:08 2015 +0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 4 .../hadoop/hdfs/server/common/HdfsServerConstants.java | 10 +- .../hadoop/hdfs/server/namenode/FSEditLogLoader.java| 3 --- .../org/apache/hadoop/hdfs/server/namenode/FSImage.java | 4 .../hadoop/hdfs/server/namenode/FSNamesystem.java | 3 +-- .../hadoop-hdfs/src/site/markdown/HDFSCommands.md | 2 +- .../hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml| 11 +-- .../apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java | 12 .../hdfs/server/datanode/TestHdfsServerConstants.java | 3 --- .../hdfs/server/namenode/TestNameNodeOptionParsing.java | 8 10 files changed, 28 insertions(+), 32 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/431e7d84/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 43505d7..52e5d3c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -14,6 +14,10 @@ Trunk (Unreleased) HDFS-2538. option to disable fsck dots (Mohammad Kamrul Islam via aw) +HDFS-7302. Remove downgrade from namenode -rollingUpgrade startup +option since it may incorrectly finalize an ongoing rolling upgrade. +(Kai Sasaki via szetszwo) + NEW FEATURES HDFS-3125. Add JournalService to enable Journal Daemon. (suresh) http://git-wip-us.apache.org/repos/asf/hadoop/blob/431e7d84/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java index 9bba2c9..ff64524 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java @@ -49,7 +49,7 @@ public final class HdfsServerConstants { /** Startup options for rolling upgrade. */ public static enum RollingUpgradeStartupOption{ -ROLLBACK, DOWNGRADE, STARTED; +ROLLBACK, STARTED; public String getOptionString() { return StartupOption.ROLLINGUPGRADE.getName() + @@ -64,6 +64,14 @@ public final class HdfsServerConstants { private static final RollingUpgradeStartupOption[] VALUES = values(); static RollingUpgradeStartupOption fromString(String s) { + if (downgrade.equalsIgnoreCase(s)) { +throw new IllegalArgumentException( +The \downgrade\ option is no longer supported ++ since it may incorrectly finalize an ongoing rolling upgrade. ++ For downgrade instruction, please see the documentation ++ (http://hadoop.apache.org/docs/current/hadoop-project-dist/; ++ hadoop-hdfs/HdfsRollingUpgrade.html#Downgrade).); + } for(RollingUpgradeStartupOption opt : VALUES) { if (opt.name().equalsIgnoreCase(s)) { return opt; http://git-wip-us.apache.org/repos/asf/hadoop/blob/431e7d84/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java index a09df82..51c167a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java @@ -793,9 +793,6 @@ public class
hadoop git commit: YARN-3272. Surface container locality info in RM web UI (Jian He via wangda)
Repository: hadoop Updated Branches: refs/heads/trunk 1004473aa - e17e5ba9d YARN-3272. Surface container locality info in RM web UI (Jian He via wangda) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e17e5ba9 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e17e5ba9 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e17e5ba9 Branch: refs/heads/trunk Commit: e17e5ba9d7e2bd45ba6884f59f8045817594b284 Parents: 1004473 Author: Wangda Tan wan...@apache.org Authored: Tue Mar 3 11:49:01 2015 -0800 Committer: Wangda Tan wan...@apache.org Committed: Tue Mar 3 11:49:01 2015 -0800 -- hadoop-yarn-project/CHANGES.txt | 3 + .../dev-support/findbugs-exclude.xml| 7 ++ .../rmapp/attempt/RMAppAttemptMetrics.java | 21 - .../resourcemanager/scheduler/NodeType.java | 9 +- .../scheduler/SchedulerApplicationAttempt.java | 15 +++- .../scheduler/capacity/LeafQueue.java | 95 +--- .../server/resourcemanager/webapp/AppBlock.java | 45 +- .../scheduler/capacity/TestReservations.java| 8 +- 8 files changed, 163 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/e17e5ba9/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 0850f0b..5eaf4f4 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -348,6 +348,9 @@ Release 2.7.0 - UNRELEASED YARN-3281. Added RMStateStore to StateMachine visualization list. (Chengbing Liu via jianhe) +YARN-3272. Surface container locality info in RM web UI. +(Jian He via wangda) + OPTIMIZATIONS YARN-2990. FairScheduler's delay-scheduling always waits for node-local and http://git-wip-us.apache.org/repos/asf/hadoop/blob/e17e5ba9/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml -- diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 70f1a71..1c3f201 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -63,6 +63,13 @@ Bug pattern=BC_UNCONFIRMED_CAST / /Match Match +Class name=~org\.apache\.hadoop\.yarn\.server\.resourcemanager\.rmapp\.attempt\.RMAppAttemptMetrics / +Method name=getLocalityStatistics / +Bug pattern=EI_EXPOSE_REP / +Method name=incNumAllocatedContainers/ +Bug pattern=VO_VOLATILE_INCREMENT / + /Match + Match Class name=org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl$AppRejectedTransition / Bug pattern=BC_UNCONFIRMED_CAST / /Match http://git-wip-us.apache.org/repos/asf/hadoop/blob/e17e5ba9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java index 0e60fd5..bc22073 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.util.resource.Resources; public class RMAppAttemptMetrics { @@ -49,6 +50,10 @@ public class RMAppAttemptMetrics { private AtomicLong finishedVcoreSeconds = new AtomicLong(0); private RMContext rmContext; + private int[][] localityStatistics = + new int[NodeType.values().length][NodeType.values().length]; + private volatile int totalAllocatedContainers; + public
hadoop git commit: YARN-3222. Fixed NPE on RMNodeImpl#ReconnectNodeTransition when a node is reconnected with a different port. Contributed by Rohith Sharmaks
Repository: hadoop Updated Branches: refs/heads/trunk 15b7076ad - b2f1ec312 YARN-3222. Fixed NPE on RMNodeImpl#ReconnectNodeTransition when a node is reconnected with a different port. Contributed by Rohith Sharmaks Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b2f1ec31 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b2f1ec31 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b2f1ec31 Branch: refs/heads/trunk Commit: b2f1ec312ee431aef762cfb49cb29cd6f4661e86 Parents: 15b7076 Author: Jian He jia...@apache.org Authored: Tue Mar 3 16:25:57 2015 -0800 Committer: Jian He jia...@apache.org Committed: Tue Mar 3 16:28:28 2015 -0800 -- .../resourcemanager/rmnode/RMNodeImpl.java | 34 +++- .../yarn/server/resourcemanager/MockNM.java | 6 +++- .../TestResourceTrackerService.java | 17 +- 3 files changed, 39 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2f1ec31/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index 9701775..c556b80 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -571,12 +571,12 @@ public class RMNodeImpl implements RMNode, EventHandlerRMNodeEvent { rmNode.nodeUpdateQueue.clear(); rmNode.context.getDispatcher().getEventHandler().handle( new NodeRemovedSchedulerEvent(rmNode)); - + if (rmNode.getHttpPort() == newNode.getHttpPort()) { // Reset heartbeat ID since node just restarted. rmNode.getLastNodeHeartBeatResponse().setResponseId(0); - if (rmNode.getState() != NodeState.UNHEALTHY) { -// Only add new node if old state is not UNHEALTHY + if (rmNode.getState().equals(NodeState.RUNNING)) { +// Only add new node if old state is RUNNING rmNode.context.getDispatcher().getEventHandler().handle( new NodeAddedSchedulerEvent(newNode)); } @@ -599,30 +599,32 @@ public class RMNodeImpl implements RMNode, EventHandlerRMNodeEvent { } else { rmNode.httpPort = newNode.getHttpPort(); rmNode.httpAddress = newNode.getHttpAddress(); -rmNode.totalCapability = newNode.getTotalCapability(); +boolean isCapabilityChanged = false; +if (rmNode.getTotalCapability() != newNode.getTotalCapability()) { + rmNode.totalCapability = newNode.getTotalCapability(); + isCapabilityChanged = true; +} handleNMContainerStatus(reconnectEvent.getNMContainerStatuses(), rmNode); // Reset heartbeat ID since node just restarted. rmNode.getLastNodeHeartBeatResponse().setResponseId(0); - } - if (null != reconnectEvent.getRunningApplications()) { for (ApplicationId appId : reconnectEvent.getRunningApplications()) { handleRunningAppOnNode(rmNode, rmNode.context, appId, rmNode.nodeId); } - } - rmNode.context.getDispatcher().getEventHandler().handle( - new NodesListManagerEvent( - NodesListManagerEventType.NODE_USABLE, rmNode)); - if (rmNode.getState().equals(NodeState.RUNNING)) { -// Update scheduler node's capacity for reconnect node. -rmNode.context.getDispatcher().getEventHandler().handle( -new NodeResourceUpdateSchedulerEvent(rmNode, -ResourceOption.newInstance(newNode.getTotalCapability(), -1))); +if (isCapabilityChanged + rmNode.getState().equals(NodeState.RUNNING)) { + // Update scheduler node's capacity for reconnect node. + rmNode.context + .getDispatcher() + .getEventHandler() + .handle( + new NodeResourceUpdateSchedulerEvent(rmNode, ResourceOption + .newInstance(newNode.getTotalCapability(), -1))); +} } - } private void handleNMContainerStatus(
hadoop git commit: YARN-3222. Fixed NPE on RMNodeImpl#ReconnectNodeTransition when a node is reconnected with a different port. Contributed by Rohith Sharmaks (cherry picked from commit b2f1ec312ee431
Repository: hadoop Updated Branches: refs/heads/branch-2 824c32de1 - 888a44563 YARN-3222. Fixed NPE on RMNodeImpl#ReconnectNodeTransition when a node is reconnected with a different port. Contributed by Rohith Sharmaks (cherry picked from commit b2f1ec312ee431aef762cfb49cb29cd6f4661e86) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/888a4456 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/888a4456 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/888a4456 Branch: refs/heads/branch-2 Commit: 888a44563819ba910dc3cc10d10ee0fb8f05db61 Parents: 824c32d Author: Jian He jia...@apache.org Authored: Tue Mar 3 16:25:57 2015 -0800 Committer: Jian He jia...@apache.org Committed: Tue Mar 3 16:28:55 2015 -0800 -- .../resourcemanager/rmnode/RMNodeImpl.java | 34 +++- .../yarn/server/resourcemanager/MockNM.java | 6 +++- .../TestResourceTrackerService.java | 17 +- 3 files changed, 39 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/888a4456/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index 9701775..c556b80 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -571,12 +571,12 @@ public class RMNodeImpl implements RMNode, EventHandlerRMNodeEvent { rmNode.nodeUpdateQueue.clear(); rmNode.context.getDispatcher().getEventHandler().handle( new NodeRemovedSchedulerEvent(rmNode)); - + if (rmNode.getHttpPort() == newNode.getHttpPort()) { // Reset heartbeat ID since node just restarted. rmNode.getLastNodeHeartBeatResponse().setResponseId(0); - if (rmNode.getState() != NodeState.UNHEALTHY) { -// Only add new node if old state is not UNHEALTHY + if (rmNode.getState().equals(NodeState.RUNNING)) { +// Only add new node if old state is RUNNING rmNode.context.getDispatcher().getEventHandler().handle( new NodeAddedSchedulerEvent(newNode)); } @@ -599,30 +599,32 @@ public class RMNodeImpl implements RMNode, EventHandlerRMNodeEvent { } else { rmNode.httpPort = newNode.getHttpPort(); rmNode.httpAddress = newNode.getHttpAddress(); -rmNode.totalCapability = newNode.getTotalCapability(); +boolean isCapabilityChanged = false; +if (rmNode.getTotalCapability() != newNode.getTotalCapability()) { + rmNode.totalCapability = newNode.getTotalCapability(); + isCapabilityChanged = true; +} handleNMContainerStatus(reconnectEvent.getNMContainerStatuses(), rmNode); // Reset heartbeat ID since node just restarted. rmNode.getLastNodeHeartBeatResponse().setResponseId(0); - } - if (null != reconnectEvent.getRunningApplications()) { for (ApplicationId appId : reconnectEvent.getRunningApplications()) { handleRunningAppOnNode(rmNode, rmNode.context, appId, rmNode.nodeId); } - } - rmNode.context.getDispatcher().getEventHandler().handle( - new NodesListManagerEvent( - NodesListManagerEventType.NODE_USABLE, rmNode)); - if (rmNode.getState().equals(NodeState.RUNNING)) { -// Update scheduler node's capacity for reconnect node. -rmNode.context.getDispatcher().getEventHandler().handle( -new NodeResourceUpdateSchedulerEvent(rmNode, -ResourceOption.newInstance(newNode.getTotalCapability(), -1))); +if (isCapabilityChanged + rmNode.getState().equals(NodeState.RUNNING)) { + // Update scheduler node's capacity for reconnect node. + rmNode.context + .getDispatcher() + .getEventHandler() + .handle( + new NodeResourceUpdateSchedulerEvent(rmNode, ResourceOption + .newInstance(newNode.getTotalCapability(), -1))); +}
hadoop git commit: MAPREDUCE-6248. Exposed the internal MapReduce job's information as a public API in DistCp. Contributed by Jing Zhao.
Repository: hadoop Updated Branches: refs/heads/trunk b2f1ec312 - 5af693fde MAPREDUCE-6248. Exposed the internal MapReduce job's information as a public API in DistCp. Contributed by Jing Zhao. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5af693fd Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5af693fd Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5af693fd Branch: refs/heads/trunk Commit: 5af693fde26755b6f175bd65f93cf4a80de0d1e0 Parents: b2f1ec3 Author: Vinod Kumar Vavilapalli vino...@apache.org Authored: Tue Mar 3 16:28:22 2015 -0800 Committer: Vinod Kumar Vavilapalli vino...@apache.org Committed: Tue Mar 3 16:28:41 2015 -0800 -- hadoop-mapreduce-project/CHANGES.txt| 3 ++ .../java/org/apache/hadoop/tools/DistCp.java| 47 +++- 2 files changed, 39 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/5af693fd/hadoop-mapreduce-project/CHANGES.txt -- diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 7a2eff3..b2ae9d9 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -320,6 +320,9 @@ Release 2.7.0 - UNRELEASED MAPREDUCE-5612. Add javadoc for TaskCompletionEvent.Status. (Chris Palmer via aajisaka) +MAPREDUCE-6248. Exposed the internal MapReduce job's information as a public +API in DistCp. (Jing Zhao via vinodkv) + OPTIMIZATIONS MAPREDUCE-6169. MergeQueue should release reference to the current item http://git-wip-us.apache.org/repos/asf/hadoop/blob/5af693fd/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java -- diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java index 28535a7..b80aeb8 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java @@ -20,6 +20,8 @@ package org.apache.hadoop.tools; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; @@ -51,12 +53,14 @@ import com.google.common.annotations.VisibleForTesting; * launch the copy-job. DistCp may alternatively be sub-classed to fine-tune * behaviour. */ +@InterfaceAudience.Public +@InterfaceStability.Evolving public class DistCp extends Configured implements Tool { /** - * Priority of the ResourceManager shutdown hook. + * Priority of the shutdown hook. */ - public static final int SHUTDOWN_HOOK_PRIORITY = 30; + static final int SHUTDOWN_HOOK_PRIORITY = 30; private static final Log LOG = LogFactory.getLog(DistCp.class); @@ -66,7 +70,7 @@ public class DistCp extends Configured implements Tool { private static final String PREFIX = _distcp; private static final String WIP_PREFIX = ._WIP_; private static final String DISTCP_DEFAULT_XML = distcp-default.xml; - public static final Random rand = new Random(); + static final Random rand = new Random(); private boolean submitted; private FileSystem jobFS; @@ -90,7 +94,7 @@ public class DistCp extends Configured implements Tool { * To be used with the ToolRunner. Not for public consumption. */ @VisibleForTesting - public DistCp() {} + DistCp() {} /** * Implementation of Tool::run(). Orchestrates the copy of source file(s) @@ -100,6 +104,7 @@ public class DistCp extends Configured implements Tool { * @param argv List of arguments passed to DistCp, from the ToolRunner. * @return On success, it returns 0. Else, -1. */ + @Override public int run(String[] argv) { if (argv.length 1) { OptionsParser.usage(); @@ -145,9 +150,21 @@ public class DistCp extends Configured implements Tool { * @throws Exception */ public Job execute() throws Exception { +Job job = createAndSubmitJob(); + +if (inputOptions.shouldBlock()) { + waitForJobCompletion(job); +} +return job; + } + + /** + * Create and submit the mapreduce job. + * @return The mapreduce job object that has been submitted + */ + public Job createAndSubmitJob() throws Exception { assert inputOptions != null; assert getConf() != null; - Job job = null; try { synchronized(this) { @@ -169,16 +186,24 @@ public class
hadoop git commit: MAPREDUCE-6248. Exposed the internal MapReduce job's information as a public API in DistCp. Contributed by Jing Zhao.
Repository: hadoop Updated Branches: refs/heads/branch-2 888a44563 - 6604613bc MAPREDUCE-6248. Exposed the internal MapReduce job's information as a public API in DistCp. Contributed by Jing Zhao. (cherry picked from commit 5af693fde26755b6f175bd65f93cf4a80de0d1e0) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6604613b Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6604613b Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6604613b Branch: refs/heads/branch-2 Commit: 6604613bc5778564b66a8bfda542c0086cada17f Parents: 888a445 Author: Vinod Kumar Vavilapalli vino...@apache.org Authored: Tue Mar 3 16:28:22 2015 -0800 Committer: Vinod Kumar Vavilapalli vino...@apache.org Committed: Tue Mar 3 16:29:36 2015 -0800 -- hadoop-mapreduce-project/CHANGES.txt| 3 ++ .../java/org/apache/hadoop/tools/DistCp.java| 47 +++- 2 files changed, 39 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/6604613b/hadoop-mapreduce-project/CHANGES.txt -- diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index ad74745..ed3022a 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -72,6 +72,9 @@ Release 2.7.0 - UNRELEASED MAPREDUCE-5612. Add javadoc for TaskCompletionEvent.Status. (Chris Palmer via aajisaka) +MAPREDUCE-6248. Exposed the internal MapReduce job's information as a public +API in DistCp. (Jing Zhao via vinodkv) + OPTIMIZATIONS MAPREDUCE-6169. MergeQueue should release reference to the current item http://git-wip-us.apache.org/repos/asf/hadoop/blob/6604613b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java -- diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java index d202f0a..08c88bf 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java @@ -20,6 +20,8 @@ package org.apache.hadoop.tools; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; @@ -51,12 +53,14 @@ import com.google.common.annotations.VisibleForTesting; * launch the copy-job. DistCp may alternatively be sub-classed to fine-tune * behaviour. */ +@InterfaceAudience.Public +@InterfaceStability.Evolving public class DistCp extends Configured implements Tool { /** - * Priority of the ResourceManager shutdown hook. + * Priority of the shutdown hook. */ - public static final int SHUTDOWN_HOOK_PRIORITY = 30; + static final int SHUTDOWN_HOOK_PRIORITY = 30; private static final Log LOG = LogFactory.getLog(DistCp.class); @@ -66,7 +70,7 @@ public class DistCp extends Configured implements Tool { private static final String PREFIX = _distcp; private static final String WIP_PREFIX = ._WIP_; private static final String DISTCP_DEFAULT_XML = distcp-default.xml; - public static final Random rand = new Random(); + static final Random rand = new Random(); private boolean submitted; private FileSystem jobFS; @@ -90,7 +94,7 @@ public class DistCp extends Configured implements Tool { * To be used with the ToolRunner. Not for public consumption. */ @VisibleForTesting - public DistCp() {} + DistCp() {} /** * Implementation of Tool::run(). Orchestrates the copy of source file(s) @@ -100,6 +104,7 @@ public class DistCp extends Configured implements Tool { * @param argv List of arguments passed to DistCp, from the ToolRunner. * @return On success, it returns 0. Else, -1. */ + @Override public int run(String[] argv) { if (argv.length 1) { OptionsParser.usage(); @@ -145,9 +150,21 @@ public class DistCp extends Configured implements Tool { * @throws Exception, on failure. */ public Job execute() throws Exception { +Job job = createAndSubmitJob(); + +if (inputOptions.shouldBlock()) { + waitForJobCompletion(job); +} +return job; + } + + /** + * Create and submit the mapreduce job. + * @return The mapreduce job object that has been submitted + */ + public Job createAndSubmitJob() throws Exception { assert inputOptions != null; assert getConf() != null; -
[2/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md new file mode 100644 index 000..e516afb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md @@ -0,0 +1,181 @@ +!--- + 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. +-- + +ResourceManger Restart +== + +* [Overview](#Overview) +* [Feature](#Feature) +* [Configurations](#Configurations) +* [Enable RM Restart](#Enable_RM_Restart) +* [Configure the state-store for persisting the RM state](#Configure_the_state-store_for_persisting_the_RM_state) +* [How to choose the state-store implementation](#How_to_choose_the_state-store_implementation) +* [Configurations for Hadoop FileSystem based state-store implementation](#Configurations_for_Hadoop_FileSystem_based_state-store_implementation) +* [Configurations for ZooKeeper based state-store implementation](#Configurations_for_ZooKeeper_based_state-store_implementation) +* [Configurations for LevelDB based state-store implementation](#Configurations_for_LevelDB_based_state-store_implementation) +* [Configurations for work-preserving RM recovery](#Configurations_for_work-preserving_RM_recovery) +* [Notes](#Notes) +* [Sample Configurations](#Sample_Configurations) + +Overview + + +ResourceManager is the central authority that manages resources and schedules applications running atop of YARN. Hence, it is potentially a single point of failure in a Apache YARN cluster. +` +This document gives an overview of ResourceManager Restart, a feature that enhances ResourceManager to keep functioning across restarts and also makes ResourceManager down-time invisible to end-users. + +ResourceManager Restart feature is divided into two phases: + +* **ResourceManager Restart Phase 1 (Non-work-preserving RM restart)**: Enhance RM to persist application/attempt state and other credentials information in a pluggable state-store. RM will reload this information from state-store upon restart and re-kick the previously running applications. Users are not required to re-submit the applications. + +* **ResourceManager Restart Phase 2 (Work-preserving RM restart)**: Focus on re-constructing the running state of ResourceManager by combining the container statuses from NodeManagers and container requests from ApplicationMasters upon restart. The key difference from phase 1 is that previously running applications will not be killed after RM restarts, and so applications won't lose its work because of RM outage. + +Feature +--- + +* **Phase 1: Non-work-preserving RM restart** + + As of Hadoop 2.4.0 release, only ResourceManager Restart Phase 1 is implemented which is described below. + + The overall concept is that RM will persist the application metadata (i.e. ApplicationSubmissionContext) in a pluggable state-store when client submits an application and also saves the final status of the application such as the completion state (failed, killed, finished) and diagnostics when the application completes. Besides, RM also saves the credentials like security keys, tokens to work in a secure environment. Any time RM shuts down, as long as the required information (i.e.application metadata and the alongside credentials if running in a secure environment) is available in the state-store, when RM restarts, it can pick up the application metadata from the state-store and re-submit the application. RM won't re-submit the applications if they were already completed (i.e. failed, killed, finished) before RM went down. + + NodeManagers and clients during the down-time of RM will keep polling RM until RM comes up. When RM becomes alive, it will send a re-sync command to all the NodeManagers and ApplicationMasters it was talking to via heartbeats. As of Hadoop 2.4.0 release, the behaviors for NodeManagers and ApplicationMasters to handle this command are: NMs will kill all its managed containers and re-register with RM. From the RM's perspective, these re-registered NodeManagers are similar to the
[7/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm deleted file mode 100644 index 69728fb..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm +++ /dev/null @@ -1,3104 +0,0 @@ -~~ 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. - - --- - ResourceManager REST API's. - --- - --- - ${maven.build.timestamp} - -ResourceManager REST API's. - -%{toc|section=1|fromDepth=0|toDepth=2} - -* Overview - - The ResourceManager REST API's allow the user to get information about the cluster - status on the cluster, metrics on the cluster, scheduler information, information about nodes in the cluster, and information about applications on the cluster. - -* Cluster Information API - - The cluster information resource provides overall information about the cluster. - -** URI - - Both of the following URI's give you the cluster information. - --- - * http://rm http address:port/ws/v1/cluster - * http://rm http address:port/ws/v1/cluster/info --- - -** HTTP Operations Supported - --- - * GET --- - -** Query Parameters Supported - --- - None --- - -** Elements of the clusterInfo object - -*---+--+---+ -|| Item || Data Type || Description | -*---+--+---+ -| id| long | The cluster id | -*---+--+---+ -| startedOn | long | The time the cluster started (in ms since epoch)| -*---+--+---+ -| state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED| -*---+--+---+ -| haState | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED| -*---+--+---+ -| resourceManagerVersion | string | Version of the ResourceManager | -*---+--+---+ -| resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum | -*---+--+---+ -| resourceManagerVersionBuiltOn | string | Timestamp when ResourceManager was built (in ms since epoch)| -*---+--+---+ -| hadoopVersion | string | Version of hadoop common | -*---+--+---+ -| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum | -*---+--+---+ -| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch)| -*---+--+---+ - -** Response Examples - - JSON response - - HTTP Request: - --- - GET http://rm http address:port/ws/v1/cluster/info --- - - Response Header: - -+---+ - HTTP/1.1 200 OK - Content-Type: application/json - Transfer-Encoding: chunked - Server: Jetty(6.1.26) -+---+ - - Response Body: - -+---+ -{ - clusterInfo: - { -id:1324053971963, -startedOn:1324053971963, -state:STARTED, -resourceManagerVersion:0.23.1-SNAPSHOT, -resourceManagerBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693, -resourceManagerVersionBuiltOn:Tue Dec 13 22:12:48 CST 2011, -hadoopVersion:0.23.1-SNAPSHOT, -hadoopBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328, -hadoopVersionBuiltOn:Tue Dec 13 22:12:26 CST 2011 - } -} -+---+ - - XML response - - HTTP Request: - -- - Accept: application/xml - GET http://rm http address:port/ws/v1/cluster/info -- - - Response Header: - -+---+ - HTTP/1.1 200 OK - Content-Type: application/xml - Content-Length: 712 - Server: Jetty(6.1.26) -+---+ - - Response
[1/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
Repository: hadoop Updated Branches: refs/heads/branch-2 6604613bc - aafe57139 http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md new file mode 100644 index 000..5e4df9f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md @@ -0,0 +1,591 @@ +!--- + 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. +-- + +Hadoop: Writing YARN Applications += + +* [Purpose](#Purpose) +* [Concepts and Flow](#Concepts_and_Flow) +* [Interfaces](#Interfaces) +* [Writing a Simple Yarn Application](#Writing_a_Simple_Yarn_Application) +* [Writing a simple Client](#Writing_a_simple_Client) +* [Writing an ApplicationMaster (AM)](#Writing_an_ApplicationMaster_AM) +* [FAQ](#FAQ) +* [How can I distribute my application's jars to all of the nodes in the YARN cluster that need it?](#How_can_I_distribute_my_applications_jars_to_all_of_the_nodes_in_the_YARN_cluster_that_need_it) +* [How do I get the ApplicationMaster's ApplicationAttemptId?](#How_do_I_get_the_ApplicationMasters_ApplicationAttemptId) +* [Why my container is killed by the NodeManager?](#Why_my_container_is_killed_by_the_NodeManager) +* [How do I include native libraries?](#How_do_I_include_native_libraries) +* [Useful Links](#Useful_Links) +* [Sample Code](#Sample_Code) + +Purpose +--- + +This document describes, at a high-level, the way to implement new Applications for YARN. + +Concepts and Flow +- + +The general concept is that an *application submission client* submits an *application* to the YARN *ResourceManager* (RM). This can be done through setting up a `YarnClient` object. After `YarnClient` is started, the client can then set up application context, prepare the very first container of the application that contains the *ApplicationMaster* (AM), and then submit the application. You need to provide information such as the details about the local files/jars that need to be available for your application to run, the actual command that needs to be executed (with the necessary command line arguments), any OS environment settings (optional), etc. Effectively, you need to describe the Unix process(es) that needs to be launched for your ApplicationMaster. + +The YARN ResourceManager will then launch the ApplicationMaster (as specified) on an allocated container. The ApplicationMaster communicates with YARN cluster, and handles application execution. It performs operations in an asynchronous fashion. During application launch time, the main tasks of the ApplicationMaster are: a) communicating with the ResourceManager to negotiate and allocate resources for future containers, and b) after container allocation, communicating YARN *NodeManager*s (NMs) to launch application containers on them. Task a) can be performed asynchronously through an `AMRMClientAsync` object, with event handling methods specified in a `AMRMClientAsync.CallbackHandler` type of event handler. The event handler needs to be set to the client explicitly. Task b) can be performed by launching a runnable object that then launches containers when there are containers allocated. As part of launching this container, the AM has to specify the `ContainerLaunchContext` that has the launch information such as command line specification, environment, etc. + +During the execution of an application, the ApplicationMaster communicates NodeManagers through `NMClientAsync` object. All container events are handled by `NMClientAsync.CallbackHandler`, associated with `NMClientAsync`. A typical callback handler handles client start, stop, status update and error. ApplicationMaster also reports execution progress to ResourceManager by handling the `getProgress()` method of `AMRMClientAsync.CallbackHandler`. + +Other than asynchronous clients, there are synchronous versions for certain workflows (`AMRMClient` and `NMClient`). The asynchronous clients are recommended because of (subjectively) simpler usages, and this article will mainly cover the asynchronous
[9/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/aafe5713 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aafe5713 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aafe5713 Branch: refs/heads/branch-2 Commit: aafe571398e9c64fd418a939f1e2b715396fce4e Parents: 6604613 Author: Jian He jia...@apache.org Authored: Tue Mar 3 16:42:06 2015 -0800 Committer: Jian He jia...@apache.org Committed: Tue Mar 3 16:44:59 2015 -0800 -- hadoop-yarn-project/CHANGES.txt |3 + .../src/site/apt/CapacityScheduler.apt.vm | 368 --- .../src/site/apt/DockerContainerExecutor.apt.vm | 204 -- .../src/site/apt/FairScheduler.apt.vm | 483 --- .../src/site/apt/NodeManager.apt.vm | 64 - .../src/site/apt/NodeManagerCgroups.apt.vm | 77 - .../src/site/apt/NodeManagerRest.apt.vm | 645 .../src/site/apt/NodeManagerRestart.apt.vm | 86 - .../src/site/apt/ResourceManagerHA.apt.vm | 233 -- .../src/site/apt/ResourceManagerRest.apt.vm | 3104 -- .../src/site/apt/ResourceManagerRestart.apt.vm | 298 -- .../src/site/apt/SecureContainer.apt.vm | 176 - .../src/site/apt/TimelineServer.apt.vm | 260 -- .../src/site/apt/WebApplicationProxy.apt.vm | 49 - .../src/site/apt/WebServicesIntro.apt.vm| 593 .../src/site/apt/WritingYarnApplications.apt.vm | 757 - .../hadoop-yarn-site/src/site/apt/YARN.apt.vm | 77 - .../src/site/apt/YarnCommands.apt.vm| 286 -- .../hadoop-yarn-site/src/site/apt/index.apt.vm | 82 - .../src/site/markdown/CapacityScheduler.md | 186 ++ .../site/markdown/DockerContainerExecutor.md.vm | 154 + .../src/site/markdown/FairScheduler.md | 233 ++ .../src/site/markdown/NodeManager.md| 57 + .../src/site/markdown/NodeManagerCgroups.md | 57 + .../src/site/markdown/NodeManagerRest.md| 543 +++ .../src/site/markdown/NodeManagerRestart.md | 53 + .../src/site/markdown/ResourceManagerHA.md | 140 + .../src/site/markdown/ResourceManagerRest.md| 2640 +++ .../src/site/markdown/ResourceManagerRestart.md | 181 + .../src/site/markdown/SecureContainer.md| 135 + .../src/site/markdown/TimelineServer.md | 231 ++ .../src/site/markdown/WebApplicationProxy.md| 24 + .../src/site/markdown/WebServicesIntro.md | 569 .../site/markdown/WritingYarnApplications.md| 591 .../hadoop-yarn-site/src/site/markdown/YARN.md | 42 + .../src/site/markdown/YarnCommands.md | 258 ++ .../hadoop-yarn-site/src/site/markdown/index.md | 75 + 37 files changed, 6172 insertions(+), 7842 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/CHANGES.txt -- diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index ae7b942..03a4a82 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -306,6 +306,9 @@ Release 2.7.0 - UNRELEASED YARN-3272. Surface container locality info in RM web UI. (Jian He via wangda) +YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to +markdown. (Masatake Iwasaki via jianhe) + OPTIMIZATIONS YARN-2990. FairScheduler's delay-scheduling always waits for node-local and http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm deleted file mode 100644 index 8528c1a..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm +++ /dev/null @@ -1,368 +0,0 @@ -~~ 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. - - --- - Hadoop Map Reduce Next Generation-${project.version} -
[3/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md new file mode 100644 index 000..b1591bb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md @@ -0,0 +1,2640 @@ +!--- + 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. +-- + +ResourceManager REST API's. +=== + +* [Overview](#Overview) +* [Cluster Information API](#Cluster_Information_API) +* [Cluster Metrics API](#Cluster_Metrics_API) +* [Cluster Scheduler API](#Cluster_Scheduler_API) +* [Cluster Applications API](#Cluster_Applications_API) +* [Cluster Application Statistics API](#Cluster_Application_Statistics_API) +* [Cluster Application API](#Cluster_Application_API) +* [Cluster Application Attempts API](#Cluster_Application_Attempts_API) +* [Cluster Nodes API](#Cluster_Nodes_API) +* [Cluster Node API](#Cluster_Node_API) +* [Cluster Writeable APIs](#Cluster_Writeable_APIs) +* [Cluster New Application API](#Cluster_New_Application_API) +* [Cluster Applications API(Submit Application)](#Cluster_Applications_APISubmit_Application) +* [Cluster Application State API](#Cluster_Application_State_API) +* [Cluster Application Queue API](#Cluster_Application_Queue_API) +* [Cluster Delegation Tokens API](#Cluster_Delegation_Tokens_API) + +Overview + + +The ResourceManager REST API's allow the user to get information about the cluster - status on the cluster, metrics on the cluster, scheduler information, information about nodes in the cluster, and information about applications on the cluster. + +Cluster Information API +--- + +The cluster information resource provides overall information about the cluster. + +### URI + +Both of the following URI's give you the cluster information. + + * http://rm http address:port/ws/v1/cluster + * http://rm http address:port/ws/v1/cluster/info + +### HTTP Operations Supported + + * GET + +### Query Parameters Supported + + None + +### Elements of the *clusterInfo* object + +| Item | Data Type | Description | +|: |: |: | +| id | long | The cluster id | +| startedOn | long | The time the cluster started (in ms since epoch) | +| state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED | +| haState | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED | +| resourceManagerVersion | string | Version of the ResourceManager | +| resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum | +| resourceManagerVersionBuiltOn | string | Timestamp when ResourceManager was built (in ms since epoch) | +| hadoopVersion | string | Version of hadoop common | +| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum | +| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch) | + +### Response Examples + +**JSON response** + +HTTP Request: + + GET http://rm http address:port/ws/v1/cluster/info + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/json + Transfer-Encoding: chunked + Server: Jetty(6.1.26) + +Response Body: + +```json +{ + clusterInfo: + { +id:1324053971963, +startedOn:1324053971963, +state:STARTED, +resourceManagerVersion:0.23.1-SNAPSHOT, +resourceManagerBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 050cd664439d931c8743a6428fd6a693, +resourceManagerVersionBuiltOn:Tue Dec 13 22:12:48 CST 2011, +hadoopVersion:0.23.1-SNAPSHOT, +hadoopBuildVersion:0.23.1-SNAPSHOT from 1214049 by user1 source checksum 11458df3bb77342dca5f917198fad328, +hadoopVersionBuiltOn:Tue Dec 13 22:12:26 CST 2011 + } +} +``` + +**XML response** + +HTTP Request: + + Accept: application/xml + GET http://rm http address:port/ws/v1/cluster/info + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/xml + Content-Length: 712 + Server: Jetty(6.1.26) + +Response Body: + +```xml +?xml
[8/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm deleted file mode 100644 index 36b8621..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRest.apt.vm +++ /dev/null @@ -1,645 +0,0 @@ -~~ 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. - - --- - NodeManager REST API's. - --- - --- - ${maven.build.timestamp} - -NodeManager REST API's. - -%{toc|section=1|fromDepth=0|toDepth=2} - -* Overview - - The NodeManager REST API's allow the user to get status on the node and information about applications and containers running on that node. - -* NodeManager Information API - - The node information resource provides overall information about that particular node. - -** URI - - Both of the following URI's give you the cluster information. - --- - * http://nm http address:port/ws/v1/node - * http://nm http address:port/ws/v1/node/info --- - -** HTTP Operations Supported - --- - * GET --- - -** Query Parameters Supported - --- - None --- - -** Elements of the nodeInfo object - -*---+--+---+ -|| Item || Data Type || Description | -*---+--+---+ -| id| long | The NodeManager id | -*---+--+---+ -| nodeHostName | string | The host name of the NodeManager | -*---+--+---+ -| totalPmemAllocatedContainersMB | long | The amount of physical memory allocated for use by containers in MB | -*---+--+---+ -| totalVmemAllocatedContainersMB | long | The amount of virtual memory allocated for use by containers in MB | -*---+--+---+ -| totalVCoresAllocatedContainers | long | The number of virtual cores allocated for use by containers | -*---+--+---+ -| lastNodeUpdateTime | long | The last timestamp at which the health report was received (in ms since epoch)| -*---+--+---+ -| healthReport | string | The diagnostic health report of the node | -*---+--+---+ -| nodeHealthy | boolean | true/false indicator of if the node is healthy| -*---+--+---+ -| nodeManagerVersion | string | Version of the NodeManager | -*---+--+---+ -| nodeManagerBuildVersion | string | NodeManager build string with build version, user, and checksum | -*---+--+---+ -| nodeManagerVersionBuiltOn | string | Timestamp when NodeManager was built(in ms since epoch) | -*---+--+---+ -| hadoopVersion | string | Version of hadoop common | -*---+--+---+ -| hadoopBuildVersion | string | Hadoop common build string with build version, user, and checksum | -*---+--+---+ -| hadoopVersionBuiltOn | string | Timestamp when hadoop common was built(in ms since epoch) | -*---+--+---+ - -** Response Examples - - JSON response - - HTTP Request: - --- - GET http://nm http address:port/ws/v1/node/info --- - - Response Header: - -+---+ - HTTP/1.1 200 OK - Content-Type: application/json - Transfer-Encoding: chunked - Server: Jetty(6.1.26) -+---+ - - Response Body: - -+---+ -{ - nodeInfo : { - hadoopVersionBuiltOn : Mon Jan 9 14:58:42 UTC 2012, - nodeManagerBuildVersion : 0.23.1-SNAPSHOT from 1228355 by user1 source checksum 20647f76c36430e888cc7204826a445c, - lastNodeUpdateTime : 132666126, - totalVmemAllocatedContainersMB : 17203, - totalVCoresAllocatedContainers : 8, - nodeHealthy
[6/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm deleted file mode 100644 index a08c19d..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRestart.apt.vm +++ /dev/null @@ -1,298 +0,0 @@ -~~ 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. - - --- - ResourceManager Restart - --- - --- - ${maven.build.timestamp} - -ResourceManager Restart - -%{toc|section=1|fromDepth=0} - -* {Overview} - - ResourceManager is the central authority that manages resources and schedules - applications running atop of YARN. Hence, it is potentially a single point of - failure in a Apache YARN cluster. - - This document gives an overview of ResourceManager Restart, a feature that - enhances ResourceManager to keep functioning across restarts and also makes - ResourceManager down-time invisible to end-users. - - ResourceManager Restart feature is divided into two phases: - - ResourceManager Restart Phase 1 (Non-work-preserving RM restart): - Enhance RM to persist application/attempt state - and other credentials information in a pluggable state-store. RM will reload - this information from state-store upon restart and re-kick the previously - running applications. Users are not required to re-submit the applications. - - ResourceManager Restart Phase 2 (Work-preserving RM restart): - Focus on re-constructing the running state of ResourceManager by combining - the container statuses from NodeManagers and container requests from ApplicationMasters - upon restart. The key difference from phase 1 is that previously running applications - will not be killed after RM restarts, and so applications won't lose its work - because of RM outage. - -* {Feature} - -** Phase 1: Non-work-preserving RM restart - - As of Hadoop 2.4.0 release, only ResourceManager Restart Phase 1 is implemented which - is described below. - - The overall concept is that RM will persist the application metadata - (i.e. ApplicationSubmissionContext) in - a pluggable state-store when client submits an application and also saves the final status - of the application such as the completion state (failed, killed, finished) - and diagnostics when the application completes. Besides, RM also saves - the credentials like security keys, tokens to work in a secure environment. - Any time RM shuts down, as long as the required information (i.e.application metadata - and the alongside credentials if running in a secure environment) is available - in the state-store, when RM restarts, it can pick up the application metadata - from the state-store and re-submit the application. RM won't re-submit the - applications if they were already completed (i.e. failed, killed, finished) - before RM went down. - - NodeManagers and clients during the down-time of RM will keep polling RM until - RM comes up. When RM becomes alive, it will send a re-sync command to - all the NodeManagers and ApplicationMasters it was talking to via heartbeats. - As of Hadoop 2.4.0 release, the behaviors for NodeManagers and ApplicationMasters to handle this command - are: NMs will kill all its managed containers and re-register with RM. From the - RM's perspective, these re-registered NodeManagers are similar to the newly joining NMs. - AMs(e.g. MapReduce AM) are expected to shutdown when they receive the re-sync command. - After RM restarts and loads all the application metadata, credentials from state-store - and populates them into memory, it will create a new - attempt (i.e. ApplicationMaster) for each application that was not yet completed - and re-kick that application as usual. As described before, the previously running - applications' work is lost in this manner since they are essentially killed by - RM via the re-sync command on restart. - -** Phase 2: Work-preserving RM restart - - As of Hadoop 2.6.0, we further enhanced RM restart feature to address the problem - to not kill any applications running on YARN cluster if RM restarts. - - Beyond all the groundwork that has been done in Phase 1
[5/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm deleted file mode 100644 index 57a47fd..000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm +++ /dev/null @@ -1,757 +0,0 @@ -~~ 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. - - --- - Hadoop Map Reduce Next Generation-${project.version} - Writing YARN - Applications - --- - --- - ${maven.build.timestamp} - -Hadoop MapReduce Next Generation - Writing YARN Applications - -%{toc|section=1|fromDepth=0} - -* Purpose - - This document describes, at a high-level, the way to implement new - Applications for YARN. - -* Concepts and Flow - - The general concept is that an application submission client submits an - application to the YARN ResourceManager (RM). This can be done through - setting up a YarnClient object. After YarnClient is started, the - client can then set up application context, prepare the very first container of - the application that contains the ApplicationMaster (AM), and then submit - the application. You need to provide information such as the details about the - local files/jars that need to be available for your application to run, the - actual command that needs to be executed (with the necessary command line - arguments), any OS environment settings (optional), etc. Effectively, you - need to describe the Unix process(es) that needs to be launched for your - ApplicationMaster. - - The YARN ResourceManager will then launch the ApplicationMaster (as - specified) on an allocated container. The ApplicationMaster communicates with - YARN cluster, and handles application execution. It performs operations in an - asynchronous fashion. During application launch time, the main tasks of the - ApplicationMaster are: a) communicating with the ResourceManager to negotiate - and allocate resources for future containers, and b) after container - allocation, communicating YARN NodeManagers (NMs) to launch application - containers on them. Task a) can be performed asynchronously through an - AMRMClientAsync object, with event handling methods specified in a - AMRMClientAsync.CallbackHandler type of event handler. The event handler - needs to be set to the client explicitly. Task b) can be performed by launching - a runnable object that then launches containers when there are containers - allocated. As part of launching this container, the AM has to - specify the ContainerLaunchContext that has the launch information such as - command line specification, environment, etc. - - During the execution of an application, the ApplicationMaster communicates - NodeManagers through NMClientAsync object. All container events are - handled by NMClientAsync.CallbackHandler, associated with - NMClientAsync. A typical callback handler handles client start, stop, - status update and error. ApplicationMaster also reports execution progress to - ResourceManager by handling the getProgress() method of - AMRMClientAsync.CallbackHandler. - - Other than asynchronous clients, there are synchronous versions for certain - workflows (AMRMClient and NMClient). The asynchronous clients are - recommended because of (subjectively) simpler usages, and this article - will mainly cover the asynchronous clients. Please refer to AMRMClient - and NMClient for more information on synchronous clients. - -* Interfaces - - The interfaces you'd most like be concerned with are: - - * Client\--\ResourceManager\ -By using YarnClient objects. - - * ApplicationMaster\--\ResourceManager\ -By using AMRMClientAsync objects, handling events asynchronously by -AMRMClientAsync.CallbackHandler - - * ApplicationMaster\--\NodeManager\ -Launch containers. Communicate with NodeManagers -by using NMClientAsync objects, handling container events by -NMClientAsync.CallbackHandler - - [] - - Note - -* The three main protocols for YARN application (ApplicationClientProtocol, - ApplicationMasterProtocol and ContainerManagementProtocol) are still - preserved. The 3
[4/9] hadoop git commit: YARN-3285. (Backport YARN-3168) Convert branch-2 .apt.vm files of YARN to markdown. Contributed by Masatake Iwasaki
http://git-wip-us.apache.org/repos/asf/hadoop/blob/aafe5713/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md -- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md new file mode 100644 index 000..1812a44 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md @@ -0,0 +1,233 @@ +!--- + 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. +-- + +Hadoop: Fair Scheduler +== + +* [Purpose](#Purpose) +* [Introduction](#Introduction) +* [Hierarchical queues with pluggable policies](#Hierarchical_queues_with_pluggable_policies) +* [Automatically placing applications in queues](#Automatically_placing_applications_in_queues) +* [Installation](#Installation) +* [Configuration](#Configuration) +* [Properties that can be placed in yarn-site.xml](#Properties_that_can_be_placed_in_yarn-site.xml) +* [Allocation file format](#Allocation_file_format) +* [Queue Access Control Lists](#Queue_Access_Control_Lists) +* [Administration](#Administration) +* [Modifying configuration at runtime](#Modifying_configuration_at_runtime) +* [Monitoring through web UI](#Monitoring_through_web_UI) +* [Moving applications between queues](#Moving_applications_between_queues) + +##Purpose + +This document describes the `FairScheduler`, a pluggable scheduler for Hadoop that allows YARN applications to share resources in large clusters fairly. + +##Introduction + +Fair scheduling is a method of assigning resources to applications such that all apps get, on average, an equal share of resources over time. Hadoop NextGen is capable of scheduling multiple resource types. By default, the Fair Scheduler bases scheduling fairness decisions only on memory. It can be configured to schedule with both memory and CPU, using the notion of Dominant Resource Fairness developed by Ghodsi et al. When there is a single app running, that app uses the entire cluster. When other apps are submitted, resources that free up are assigned to the new apps, so that each app eventually on gets roughly the same amount of resources. Unlike the default Hadoop scheduler, which forms a queue of apps, this lets short apps finish in reasonable time while not starving long-lived apps. It is also a reasonable way to share a cluster between a number of users. Finally, fair sharing can also work with app priorities - the priorities are used as weights to determine the fraction of t otal resources that each app should get. + +The scheduler organizes apps further into queues, and shares resources fairly between these queues. By default, all users share a single queue, named default. If an app specifically lists a queue in a container resource request, the request is submitted to that queue. It is also possible to assign queues based on the user name included with the request through configuration. Within each queue, a scheduling policy is used to share resources between the running apps. The default is memory-based fair sharing, but FIFO and multi-resource with Dominant Resource Fairness can also be configured. Queues can be arranged in a hierarchy to divide resources and configured with weights to share the cluster in specific proportions. + +In addition to providing fair sharing, the Fair Scheduler allows assigning guaranteed minimum shares to queues, which is useful for ensuring that certain users, groups or production applications always get sufficient resources. When a queue contains apps, it gets at least its minimum share, but when the queue does not need its full guaranteed share, the excess is split between other running apps. This lets the scheduler guarantee capacity for queues while utilizing resources efficiently when these queues don't contain applications. + +The Fair Scheduler lets all apps run by default, but it is also possible to limit the number of running apps per user and per queue through the config file. This can be useful when a user must submit hundreds of apps at once, or in general to improve performance if running too many apps at once would cause too much intermediate data to be created or too much context-switching. Limiting the apps does not cause any subsequently submitted apps
[2/2] hadoop git commit: HADOOP-11183. Memory-based S3AOutputstream. (Thomas Demoor via stevel)
HADOOP-11183. Memory-based S3AOutputstream. (Thomas Demoor via stevel) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/15b7076a Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/15b7076a Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/15b7076a Branch: refs/heads/trunk Commit: 15b7076ad5f2ae92d231140b2f8cebc392a92c87 Parents: e17e5ba Author: Steve Loughran ste...@apache.org Authored: Tue Mar 3 16:18:39 2015 -0800 Committer: Steve Loughran ste...@apache.org Committed: Tue Mar 3 16:18:51 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 2 + .../src/main/resources/core-default.xml | 20 +- .../org/apache/hadoop/fs/s3a/Constants.java | 8 + .../hadoop/fs/s3a/S3AFastOutputStream.java | 413 +++ .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 24 +- .../src/site/markdown/tools/hadoop-aws/index.md | 46 ++- .../hadoop/fs/s3a/TestS3AFastOutputStream.java | 74 7 files changed, 570 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/15b7076a/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 11785f2..cb5cd4d 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -667,6 +667,8 @@ Release 2.7.0 - UNRELEASED HADOOP-11620. Add support for load balancing across a group of KMS for HA. (Arun Suresh via wang) +HADOOP-11183. Memory-based S3AOutputstream. (Thomas Demoor via stevel) + BUG FIXES HADOOP-11512. Use getTrimmedStrings when reading serialization keys http://git-wip-us.apache.org/repos/asf/hadoop/blob/15b7076a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml -- diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 80dd15b..74390d8 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -763,13 +763,13 @@ for ldap providers in the same way as above does. property namefs.s3a.connection.establish.timeout/name value5000/value - descriptionSocket connection setup timeout in seconds./description + descriptionSocket connection setup timeout in milliseconds./description /property property namefs.s3a.connection.timeout/name value5/value - descriptionSocket connection timeout in seconds./description + descriptionSocket connection timeout in milliseconds./description /property property @@ -846,6 +846,22 @@ for ldap providers in the same way as above does. /property property + namefs.s3a.fast.upload/name + valuefalse/value + descriptionUpload directly from memory instead of buffering to +disk first. Memory usage and parallelism can be controlled as up to +fs.s3a.multipart.size memory is consumed for each (part)upload actively +uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)/description +/property + + property + namefs.s3a.fast.buffer.size/name + value1048576/value + descriptionSize of initial memory buffer in bytes allocated for an +upload. No effect if fs.s3a.fast.upload is false./description +/property + +property namefs.s3a.impl/name valueorg.apache.hadoop.fs.s3a.S3AFileSystem/value descriptionThe implementation class of the S3A Filesystem/description http://git-wip-us.apache.org/repos/asf/hadoop/blob/15b7076a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java -- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 1d4f67b..e7462dc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -83,6 +83,14 @@ public class Constants { // comma separated list of directories public static final String BUFFER_DIR = fs.s3a.buffer.dir; + // should we upload directly from memory rather than using a file buffer + public static final String FAST_UPLOAD = fs.s3a.fast.upload; + public static final boolean DEFAULT_FAST_UPLOAD = false; + + //initial size of memory buffer for a fast upload + public static final String FAST_BUFFER_SIZE = fs.s3a.fast.buffer.size; + public static final int
[1/2] hadoop git commit: HADOOP-11183. Memory-based S3AOutputstream. (Thomas Demoor via stevel)
Repository: hadoop Updated Branches: refs/heads/branch-2 0a502c665 - 824c32de1 refs/heads/trunk e17e5ba9d - 15b7076ad HADOOP-11183. Memory-based S3AOutputstream. (Thomas Demoor via stevel) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/824c32de Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/824c32de Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/824c32de Branch: refs/heads/branch-2 Commit: 824c32de1ab34111f3430e26c3e07e0e5d510153 Parents: 0a502c6 Author: Steve Loughran ste...@apache.org Authored: Tue Mar 3 16:18:39 2015 -0800 Committer: Steve Loughran ste...@apache.org Committed: Tue Mar 3 16:18:39 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 2 + .../src/main/resources/core-default.xml | 20 +- .../org/apache/hadoop/fs/s3a/Constants.java | 8 + .../hadoop/fs/s3a/S3AFastOutputStream.java | 413 +++ .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 24 +- .../src/site/markdown/tools/hadoop-aws/index.md | 46 ++- .../hadoop/fs/s3a/TestS3AFastOutputStream.java | 74 7 files changed, 570 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/824c32de/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 107940e..6394f4f 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -249,6 +249,8 @@ Release 2.7.0 - UNRELEASED HADOOP-11620. Add support for load balancing across a group of KMS for HA. (Arun Suresh via wang) +HADOOP-11183. Memory-based S3AOutputstream. (Thomas Demoor via stevel) + BUG FIXES HADOOP-11512. Use getTrimmedStrings when reading serialization keys http://git-wip-us.apache.org/repos/asf/hadoop/blob/824c32de/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml -- diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index bc0f064..690a22c 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -755,13 +755,13 @@ for ldap providers in the same way as above does. property namefs.s3a.connection.establish.timeout/name value5000/value - descriptionSocket connection setup timeout in seconds./description + descriptionSocket connection setup timeout in milliseconds./description /property property namefs.s3a.connection.timeout/name value5/value - descriptionSocket connection timeout in seconds./description + descriptionSocket connection timeout in milliseconds./description /property property @@ -838,6 +838,22 @@ for ldap providers in the same way as above does. /property property + namefs.s3a.fast.upload/name + valuefalse/value + descriptionUpload directly from memory instead of buffering to +disk first. Memory usage and parallelism can be controlled as up to +fs.s3a.multipart.size memory is consumed for each (part)upload actively +uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)/description +/property + + property + namefs.s3a.fast.buffer.size/name + value1048576/value + descriptionSize of initial memory buffer in bytes allocated for an +upload. No effect if fs.s3a.fast.upload is false./description +/property + +property namefs.s3a.impl/name valueorg.apache.hadoop.fs.s3a.S3AFileSystem/value descriptionThe implementation class of the S3A Filesystem/description http://git-wip-us.apache.org/repos/asf/hadoop/blob/824c32de/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java -- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 1d4f67b..e7462dc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -83,6 +83,14 @@ public class Constants { // comma separated list of directories public static final String BUFFER_DIR = fs.s3a.buffer.dir; + // should we upload directly from memory rather than using a file buffer + public static final String FAST_UPLOAD = fs.s3a.fast.upload; + public static final boolean DEFAULT_FAST_UPLOAD = false; + + //initial size of memory buffer for a fast
[Hadoop Wiki] Update of AmazonS3 by SteveLoughran
Dear Wiki user, You have subscribed to a wiki page or wiki category on Hadoop Wiki for change notification. The AmazonS3 page has been changed by SteveLoughran: https://wiki.apache.org/hadoop/AmazonS3?action=diffrev1=15rev2=16 Comment: s3a monthly for storage and data transfer. Transfer between S3 and [[AmazonEC2]] is free. This makes use of S3 attractive for Hadoop users who run clusters on EC2. - Hadoop provides two filesystems that use S3. + Hadoop provides multiple filesystem clients for reading and writing to and from Amazon S3 or compatible service. S3 Native FileSystem (URI scheme: s3n):: A native filesystem for reading and writing regular files on S3. The advantage of this filesystem is that you can access files on S3 that were written with other tools. Conversely, other tools can access files written using Hadoop. The disadvantage is the 5GB limit on file size imposed by S3. + + S3A (URI scheme: s3a):: + A successor to the S3 Native, s3n fs, the S3a: system uses Amazon's libraries to interact with S3. This allows S3a to support larger files (no more 5GB limit), higher performance operations and more. The filesystem is intended to be a replacement for/successor to S3 Native: all objects accessible from s3n:// URLs should also be accessible from s3a simply by replacing the URL schema. S3 Block FileSystem (URI scheme: s3):: A block-based filesystem backed by S3. Files are stored as blocks, just like they are in HDFS. This permits efficient implementation of renames. This filesystem requires you to dedicate a bucket for the filesystem - you should not use an existing bucket containing files, or write other files to the same bucket. The files stored by this filesystem can be larger than 5GB, but they are not interoperable with other S3 tools. @@ -20, +23 @@ = History = * The S3 block filesystem was introduced in Hadoop 0.10.0 ([[http://issues.apache.org/jira/browse/HADOOP-574|HADOOP-574]]), but this had a few bugs so you should use Hadoop 0.10.1 or later. * The S3 native filesystem was introduced in Hadoop 0.18.0 ([[http://issues.apache.org/jira/browse/HADOOP-930|HADOOP-930]]) and rename support was added in Hadoop 0.19.0 ([[https://issues.apache.org/jira/browse/HADOOP-3361|HADOOP-3361]]). + * The S3A filesystem was introduced in Hadoop 2.6.0. = Why you cannot use S3 as a replacement for HDFS = You cannot use either of the S3 filesystems as a drop-in replacement for HDFS. Amazon S3 is an object store with * eventual consistency: changes made by one application (creation, updates and deletions) will not be visible until some undefined time. - * s3n: non-atomic rename and delete operations. Renaming or deleting large directories takes time proportional to the number of entries -and visible to other processes during this time, and indeed, until the eventual consistency has been resolved. + * s3n and s3a: non-atomic rename and delete operations. Renaming or deleting large directories takes time proportional to the number of entries -and visible to other processes during this time, and indeed, until the eventual consistency has been resolved. S3 is not a filesystem. The Hadoop S3 filesystem bindings make it pretend to be a filesystem, but it is not. It can act as a source of data, and as a destination -though in the latter case, you must remember that the output may not be immediately visible. @@ -73, +77 @@ = Security = - Your Amazon Secret Access Key is that: secret. If it gets known you have to go to the [[https://portal.aws.amazon.com/gp/aws/securityCredentials|Security Credentials]] page and revoke it. Try and avoid printing it in logs, or checking the XML configuration files into revision control. + Your Amazon Secret Access Key is that: secret. If it gets known you have to go to the [[https://portal.aws.amazon.com/gp/aws/securityCredentials|Security Credentials]] page and revoke it. Try and avoid printing it in logs, or checking the XML configuration files into revision control. Do not ever check it in to revision control systems. = Running bulk copies in and out of S3 = @@ -90, +94 @@ Flip the arguments if you want to run the copy in the opposite direction. - Other schemes supported by `distcp` are `file` (for local), and `http`. + Other schemes supported by `distcp` include `file:` (for local), and `http:`.
hadoop git commit: HADOOP-11666. Revert the format change of du output introduced by HADOOP-6857. Contributed by Byron Wong.
Repository: hadoop Updated Branches: refs/heads/branch-2 bce3d442f - 31b3f8460 HADOOP-11666. Revert the format change of du output introduced by HADOOP-6857. Contributed by Byron Wong. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/31b3f846 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/31b3f846 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/31b3f846 Branch: refs/heads/branch-2 Commit: 31b3f84601ab48be5bb002e4396b53461d85c8a0 Parents: bce3d44 Author: Akira Ajisaka aajis...@apache.org Authored: Tue Mar 3 21:46:28 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Tue Mar 3 21:46:28 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 3 + .../org/apache/hadoop/fs/shell/FsUsage.java | 19 +- .../src/test/resources/testConf.xml | 2 +- .../org/apache/hadoop/hdfs/TestDFSShell.java| 10 +- .../src/test/resources/testHDFSConf.xml | 214 +-- 5 files changed, 123 insertions(+), 125 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b3f846/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 6394f4f..f4728b2 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -628,6 +628,9 @@ Release 2.7.0 - UNRELEASED HADOOP-11605. FilterFileSystem#create with ChecksumOpt should propagate it to wrapped FS. (gera) +HADOOP-11666. Revert the format change of du output introduced by +HADOOP-6857. (Byron Wong via aajisaka) + Release 2.6.1 - UNRELEASED INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b3f846/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java -- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java index 5c1dbf0..43a950e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java @@ -26,7 +26,6 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FsStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.StringUtils; @@ -118,7 +117,7 @@ class FsUsage extends FsCommand { Note that, even without the -s option, this only shows size summaries + one level deep into a directory.\n\n + The output is in the form \n + -\tsize\tdisk space consumed\tname(full path)\n; +\tsize\tname(full path)\n; protected boolean summary = false; @@ -133,7 +132,7 @@ class FsUsage extends FsCommand { @Override protected void processPathArgument(PathData item) throws IOException { - usagesTable = new TableBuilder(3); + usagesTable = new TableBuilder(2); // go one level deep on dirs from cmdline unless in summary mode if (!summary item.stat.isDirectory()) { recursePath(item); @@ -145,12 +144,16 @@ class FsUsage extends FsCommand { @Override protected void processPath(PathData item) throws IOException { - ContentSummary contentSummary = item.fs.getContentSummary(item.path); - long length = contentSummary.getLength(); - long spaceConsumed = contentSummary.getSpaceConsumed(); - usagesTable.addRow(formatSize(length), formatSize(spaceConsumed), item); + long length; + if (item.stat.isDirectory()) { +length = item.fs.getContentSummary(item.path).getLength(); + } else { +length = item.stat.getLen(); + } + usagesTable.addRow(formatSize(length), item); } } + /** show disk usage summary */ public static class Dus extends Du { public static final String NAME = dus; @@ -259,4 +262,4 @@ class FsUsage extends FsCommand { return size() == 0; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b3f846/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml -- diff --git a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml index 5c667e1..52da00b 100644 ---
hadoop git commit: HDFS-6565. Use jackson instead jetty json in hdfs-client. Contributed by Akira AJISAKA.
Repository: hadoop Updated Branches: refs/heads/trunk 5af693fde - e2262d3d1 HDFS-6565. Use jackson instead jetty json in hdfs-client. Contributed by Akira AJISAKA. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e2262d3d Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e2262d3d Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e2262d3d Branch: refs/heads/trunk Commit: e2262d3d18c6d5c2aa20f96920104dc07271b869 Parents: 5af693f Author: Haohui Mai whe...@apache.org Authored: Tue Mar 3 17:54:13 2015 -0800 Committer: Haohui Mai whe...@apache.org Committed: Tue Mar 3 17:54:13 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 + .../org/apache/hadoop/hdfs/web/JsonUtil.java| 217 +-- .../hadoop/hdfs/web/WebHdfsFileSystem.java | 21 +- .../apache/hadoop/hdfs/web/TestJsonUtil.java| 22 +- 4 files changed, 127 insertions(+), 136 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2262d3d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 42430ef..4e7b919 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1077,6 +1077,9 @@ Release 2.7.0 - UNRELEASED HDFS-7757. Misleading error messages in FSImage.java. (Brahma Reddy Battula via Arpit Agarwal) +HDFS-6565. Use jackson instead jetty json in hdfs-client. +(Akira Ajisaka via wheat9) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2262d3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java index aa6100c..2e67848 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.*; @@ -35,7 +34,8 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.StringUtils; -import org.mortbay.util.ajax.JSON; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.ObjectReader; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -95,59 +95,6 @@ public class JsonUtil { return (TokenBlockTokenIdentifier)toToken(m); } - /** Convert a Token[] to a JSON array. */ - private static Object[] toJsonArray(final Token? extends TokenIdentifier[] array - ) throws IOException { -if (array == null) { - return null; -} else if (array.length == 0) { - return EMPTY_OBJECT_ARRAY; -} else { - final Object[] a = new Object[array.length]; - for(int i = 0; i array.length; i++) { -a[i] = toJsonMap(array[i]); - } - return a; -} - } - - /** Convert a token object to a JSON string. */ - public static String toJsonString(final Token? extends TokenIdentifier[] tokens - ) throws IOException { -if (tokens == null) { - return null; -} - -final MapString, Object m = new TreeMapString, Object(); -m.put(Token.class.getSimpleName(), toJsonArray(tokens)); -return toJsonString(Token.class.getSimpleName() + s, m); - } - - /** Convert an Object[] to a ListToken?. */ - private static ListToken? toTokenList(final Object[] objects) throws IOException { -if (objects == null) { - return null; -} else if (objects.length == 0) { - return Collections.emptyList(); -} else { - final ListToken? list = new ArrayListToken?(objects.length); - for(int i = 0; i objects.length; i++) { -list.add(toToken((Map?, ?)objects[i])); - } - return list; -} - } - - /** Convert a JSON map to a ListToken?. */ - public static ListToken? toTokenList(final Map?, ? json) throws IOException { -if (json == null) { - return
hadoop git commit: HDFS-6565. Use jackson instead jetty json in hdfs-client. Contributed by Akira AJISAKA.
Repository: hadoop Updated Branches: refs/heads/branch-2 aafe57139 - 65bfde552 HDFS-6565. Use jackson instead jetty json in hdfs-client. Contributed by Akira AJISAKA. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/65bfde55 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/65bfde55 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/65bfde55 Branch: refs/heads/branch-2 Commit: 65bfde552a968d358c269484a1b8a1af5b65c228 Parents: aafe571 Author: Haohui Mai whe...@apache.org Authored: Tue Mar 3 17:54:13 2015 -0800 Committer: Haohui Mai whe...@apache.org Committed: Tue Mar 3 17:58:21 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 + .../org/apache/hadoop/hdfs/web/JsonUtil.java| 224 +-- .../hadoop/hdfs/web/WebHdfsFileSystem.java | 22 +- .../apache/hadoop/hdfs/web/TestJsonUtil.java| 22 +- 4 files changed, 131 insertions(+), 140 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/65bfde55/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index df5520e..69a410f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -771,6 +771,9 @@ Release 2.7.0 - UNRELEASED HDFS-7757. Misleading error messages in FSImage.java. (Brahma Reddy Battula via Arpit Agarwal) +HDFS-6565. Use jackson instead jetty json in hdfs-client. +(Akira Ajisaka via wheat9) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/65bfde55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java index 69ef926..edaa4a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.XAttrHelper; import org.apache.hadoop.hdfs.protocol.*; @@ -35,7 +34,8 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.StringUtils; -import org.mortbay.util.ajax.JSON; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.ObjectReader; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -95,59 +95,6 @@ public class JsonUtil { return (TokenBlockTokenIdentifier)toToken(m); } - /** Convert a Token[] to a JSON array. */ - private static Object[] toJsonArray(final Token? extends TokenIdentifier[] array - ) throws IOException { -if (array == null) { - return null; -} else if (array.length == 0) { - return EMPTY_OBJECT_ARRAY; -} else { - final Object[] a = new Object[array.length]; - for(int i = 0; i array.length; i++) { -a[i] = toJsonMap(array[i]); - } - return a; -} - } - - /** Convert a token object to a JSON string. */ - public static String toJsonString(final Token? extends TokenIdentifier[] tokens - ) throws IOException { -if (tokens == null) { - return null; -} - -final MapString, Object m = new TreeMapString, Object(); -m.put(Token.class.getSimpleName(), toJsonArray(tokens)); -return toJsonString(Token.class.getSimpleName() + s, m); - } - - /** Convert an Object[] to a ListToken?. */ - private static ListToken? toTokenList(final Object[] objects) throws IOException { -if (objects == null) { - return null; -} else if (objects.length == 0) { - return Collections.emptyList(); -} else { - final ListToken? list = new ArrayListToken?(objects.length); - for(int i = 0; i objects.length; i++) { -list.add(toToken((Map?, ?)objects[i])); - } - return list; -} - } - - /** Convert a JSON map to a ListToken?. */ - public static ListToken? toTokenList(final Map?, ? json) throws IOException { -if (json == null) { - return
[1/2] hadoop git commit: HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file includes non-snapshotted content. Contributed by Charles Lamb.
Repository: hadoop Updated Branches: refs/heads/branch-2 65bfde552 - bce3d442f refs/heads/trunk e2262d3d1 - f2d7a67a2 HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file includes non-snapshotted content. Contributed by Charles Lamb. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f2d7a67a Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f2d7a67a Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f2d7a67a Branch: refs/heads/trunk Commit: f2d7a67a2c1d9dde10ed3171fdec65dff885afcc Parents: e2262d3 Author: Aaron T. Myers a...@apache.org Authored: Tue Mar 3 18:08:59 2015 -0800 Committer: Aaron T. Myers a...@apache.org Committed: Tue Mar 3 18:08:59 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../java/org/apache/hadoop/hdfs/DFSClient.java | 3 +++ .../snapshot/TestSnapshotFileLength.java| 25 +--- 3 files changed, 28 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2d7a67a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 4e7b919..7ff3c78 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1080,6 +1080,9 @@ Release 2.7.0 - UNRELEASED HDFS-6565. Use jackson instead jetty json in hdfs-client. (Akira Ajisaka via wheat9) +HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file +includes non-snapshotted content. (Charles Lamb via atm) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2d7a67a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index abcd847..aac7b51 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -2220,6 +2220,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, // get block checksum for each block long remaining = length; +if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) { + remaining = Math.min(length, blockLocations.getFileLength()); +} for(int i = 0; i locatedblocks.size() remaining 0; i++) { if (refetchBlocks) { // refetch to get fresh tokens blockLocations = callGetBlockLocations(namenode, src, 0, length); http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2d7a67a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java index 98aafc1..d53140f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java @@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot; import java.io.ByteArrayOutputStream; import java.io.PrintStream; - import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hdfs.AppendTestUtil; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -29,8 +29,9 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.*; - +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -103,17 +104,35 @@ public class TestSnapshotFileLength { Path file1snap1 = SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name); +final FileChecksum snapChksum1 = hdfs.getFileChecksum(file1snap1); +assertThat(file and snapshot file checksums are
[2/2] hadoop git commit: HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file includes non-snapshotted content. Contributed by Charles Lamb. (cherry picked from commit f2d7a67a2c
HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file includes non-snapshotted content. Contributed by Charles Lamb. (cherry picked from commit f2d7a67a2c1d9dde10ed3171fdec65dff885afcc) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bce3d442 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bce3d442 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bce3d442 Branch: refs/heads/branch-2 Commit: bce3d442ff08ee1e730b0bac112439d6a6931917 Parents: 65bfde5 Author: Aaron T. Myers a...@apache.org Authored: Tue Mar 3 18:08:59 2015 -0800 Committer: Aaron T. Myers a...@apache.org Committed: Tue Mar 3 18:09:31 2015 -0800 -- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../java/org/apache/hadoop/hdfs/DFSClient.java | 3 +++ .../snapshot/TestSnapshotFileLength.java| 25 +--- 3 files changed, 28 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/bce3d442/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 69a410f..bff45bb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -774,6 +774,9 @@ Release 2.7.0 - UNRELEASED HDFS-6565. Use jackson instead jetty json in hdfs-client. (Akira Ajisaka via wheat9) +HDFS-7682. {{DistributedFileSystem#getFileChecksum}} of a snapshotted file +includes non-snapshotted content. (Charles Lamb via atm) + BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS HDFS-7720. Quota by Storage Type API, tools and ClientNameNode http://git-wip-us.apache.org/repos/asf/hadoop/blob/bce3d442/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 6f96126..ba6a1d1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -2219,6 +2219,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, // get block checksum for each block long remaining = length; +if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) { + remaining = Math.min(length, blockLocations.getFileLength()); +} for(int i = 0; i locatedblocks.size() remaining 0; i++) { if (refetchBlocks) { // refetch to get fresh tokens blockLocations = callGetBlockLocations(namenode, src, 0, length); http://git-wip-us.apache.org/repos/asf/hadoop/blob/bce3d442/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java -- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java index 98aafc1..d53140f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java @@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot; import java.io.ByteArrayOutputStream; import java.io.PrintStream; - import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hdfs.AppendTestUtil; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -29,8 +29,9 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.*; - +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -103,17 +104,35 @@ public class TestSnapshotFileLength { Path file1snap1 = SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name); +final FileChecksum snapChksum1 = hdfs.getFileChecksum(file1snap1); +assertThat(file and snapshot file checksums are not equal, +hdfs.getFileChecksum(file1),
hadoop git commit: Move HADOOP-6857 to 3.0.0.
Repository: hadoop Updated Branches: refs/heads/trunk f2d7a67a2 - 29bb68986 Move HADOOP-6857 to 3.0.0. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/29bb6898 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/29bb6898 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/29bb6898 Branch: refs/heads/trunk Commit: 29bb6898654199a809f1c3e8e536a63fb0d4f073 Parents: f2d7a67 Author: Akira Ajisaka aajis...@apache.org Authored: Tue Mar 3 21:52:37 2015 -0800 Committer: Akira Ajisaka aajis...@apache.org Committed: Tue Mar 3 21:52:37 2015 -0800 -- hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/29bb6898/hadoop-common-project/hadoop-common/CHANGES.txt -- diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index cb5cd4d..d518d9f 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -13,6 +13,9 @@ Trunk (Unreleased) HADOOP-10950. rework heap management vars (John Smith via aw) +HADOOP-6857. FsShell should report raw disk usage including replication +factor. (Byron Wong via shv) + HADOOP-11657. Align the output of `hadoop fs -du` to be more Unix-like. (aajisaka) @@ -465,9 +468,6 @@ Release 2.7.0 - UNRELEASED HADOOP-10748. HttpServer2 should not load JspServlet. (wheat9) -HADOOP-6857. FsShell should report raw disk usage including replication -factor. (Byron Wong via shv) - HADOOP-10847. Remove the usage of sun.security.x509.* in testing code. (Pascal Oliva via wheat9)
hadoop git commit: Ability to limit running map and reduce tasks. Contributed by Jason Lowe.
Repository: hadoop Updated Branches: refs/heads/branch-2 b1e7f9c58 - a9613bcc9 Ability to limit running map and reduce tasks. Contributed by Jason Lowe. (cherry picked from commit 4228de94028f1e10ca59ce23e963e488fe566909) Conflicts: hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a9613bcc Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a9613bcc Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a9613bcc Branch: refs/heads/branch-2 Commit: a9613bcc945fd5d9ce19f8d839afcb13ba38d441 Parents: b1e7f9c Author: Junping Du junping...@apache.org Authored: Tue Mar 3 02:01:04 2015 -0800 Committer: Junping Du junping...@apache.org Committed: Tue Mar 3 02:23:00 2015 -0800 -- hadoop-mapreduce-project/CHANGES.txt| 3 + .../v2/app/rm/RMContainerAllocator.java | 65 +- .../v2/app/rm/RMContainerRequestor.java | 74 ++- .../v2/app/rm/TestRMContainerAllocator.java | 214 +++ .../apache/hadoop/mapreduce/MRJobConfig.java| 8 + .../src/main/resources/mapred-default.xml | 15 ++ 6 files changed, 362 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9613bcc/hadoop-mapreduce-project/CHANGES.txt -- diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 47152f9..ad74745 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -10,6 +10,9 @@ Release 2.7.0 - UNRELEASED MAPREDUCE-6228. Add truncate operation to SLive. (Plamen Jeliazkov via shv) +MAPREDUCE-5583. Ability to limit running map and reduce tasks. +(Jason Lowe via junping_du) + IMPROVEMENTS MAPREDUCE-6149. Document override log4j.properties in MR job. http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9613bcc/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java -- diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java index b903a5e..8d35d79 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java @@ -97,9 +97,9 @@ public class RMContainerAllocator extends RMContainerRequestor public static final float DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART = 0.05f; - private static final Priority PRIORITY_FAST_FAIL_MAP; - private static final Priority PRIORITY_REDUCE; - private static final Priority PRIORITY_MAP; + static final Priority PRIORITY_FAST_FAIL_MAP; + static final Priority PRIORITY_REDUCE; + static final Priority PRIORITY_MAP; @VisibleForTesting public static final String RAMPDOWN_DIAGNOSTIC = Reducer preempted @@ -164,6 +164,8 @@ public class RMContainerAllocator extends RMContainerRequestor */ private long allocationDelayThresholdMs = 0; private float reduceSlowStart = 0; + private int maxRunningMaps = 0; + private int maxRunningReduces = 0; private long retryInterval; private long retrystartTime; private Clock clock; @@ -195,6 +197,10 @@ public class RMContainerAllocator extends RMContainerRequestor allocationDelayThresholdMs = conf.getInt( MRJobConfig.MR_JOB_REDUCER_PREEMPT_DELAY_SEC, MRJobConfig.DEFAULT_MR_JOB_REDUCER_PREEMPT_DELAY_SEC) * 1000;//sec - ms +maxRunningMaps = conf.getInt(MRJobConfig.JOB_RUNNING_MAP_LIMIT, +MRJobConfig.DEFAULT_JOB_RUNNING_MAP_LIMIT); +maxRunningReduces = conf.getInt(MRJobConfig.JOB_RUNNING_REDUCE_LIMIT, +MRJobConfig.DEFAULT_JOB_RUNNING_REDUCE_LIMIT); RackResolver.init(conf); retryInterval = getConfig().getLong(MRJobConfig.MR_AM_TO_RM_WAIT_INTERVAL_MS, MRJobConfig.DEFAULT_MR_AM_TO_RM_WAIT_INTERVAL_MS); @@ -654,6 +660,8 @@ public class RMContainerAllocator extends RMContainerRequestor @SuppressWarnings(unchecked) private ListContainer getResources() throws Exception { +applyConcurrentTaskLimits(); + // will be null the first time Resource