DistCp.md.vm 32 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591
  1. <!---
  2. Licensed under the Apache License, Version 2.0 (the "License");
  3. you may not use this file except in compliance with the License.
  4. You may obtain a copy of the License at
  5. http://www.apache.org/licenses/LICENSE-2.0
  6. Unless required by applicable law or agreed to in writing, software
  7. distributed under the License is distributed on an "AS IS" BASIS,
  8. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  9. See the License for the specific language governing permissions and
  10. limitations under the License. See accompanying LICENSE file.
  11. -->
  12. #set ( $H3 = '###' )
  13. DistCp Guide
  14. =====================
  15. ---
  16. - [Overview](#Overview)
  17. - [Usage](#Usage)
  18. - [Basic Usage](#Basic_Usage)
  19. - [Update and Overwrite](#Update_and_Overwrite)
  20. - [Command Line Options](#Command_Line_Options)
  21. - [Architecture of DistCp](#Architecture_of_DistCp)
  22. - [DistCp Driver](#DistCp_Driver)
  23. - [Copy-listing Generator](#Copy-listing_Generator)
  24. - [InputFormats and MapReduce Components](#InputFormats_and_MapReduce_Components)
  25. - [Appendix](#Appendix)
  26. - [Map sizing](#Map_sizing)
  27. - [Copying Between Versions of HDFS](#Copying_Between_Versions_of_HDFS)
  28. - [MapReduce and other side-effects](#MapReduce_and_other_side-effects)
  29. - [Frequently Asked Questions](#Frequently_Asked_Questions)
  30. ---
  31. Overview
  32. --------
  33. DistCp (distributed copy) is a tool used for large
  34. inter/intra-cluster copying. It uses MapReduce to effect its distribution,
  35. error handling and recovery, and reporting. It expands a list of files and
  36. directories into input to map tasks, each of which will copy a partition of
  37. the files specified in the source list.
  38. [The erstwhile implementation of DistCp]
  39. (http://hadoop.apache.org/docs/r1.2.1/distcp.html) has its share of quirks
  40. and drawbacks, both in its usage, as well as its extensibility and
  41. performance. The purpose of the DistCp refactor was to fix these
  42. shortcomings, enabling it to be used and extended programmatically. New
  43. paradigms have been introduced to improve runtime and setup performance,
  44. while simultaneously retaining the legacy behaviour as default.
  45. This document aims to describe the design of the new DistCp, its spanking new
  46. features, their optimal use, and any deviance from the legacy implementation.
  47. Usage
  48. -----
  49. $H3 Basic Usage
  50. The most common invocation of DistCp is an inter-cluster copy:
  51. bash$ hadoop distcp hdfs://nn1:8020/foo/bar \
  52. hdfs://nn2:8020/bar/foo
  53. This will expand the namespace under `/foo/bar` on nn1 into a temporary file,
  54. partition its contents among a set of map tasks, and start a copy on each
  55. NodeManager from `nn1` to `nn2`.
  56. One can also specify multiple source directories on the command line:
  57. bash$ hadoop distcp hdfs://nn1:8020/foo/a \
  58. hdfs://nn1:8020/foo/b \
  59. hdfs://nn2:8020/bar/foo
  60. Or, equivalently, from a file using the -f option:
  61. bash$ hadoop distcp -f hdfs://nn1:8020/srclist \
  62. hdfs://nn2:8020/bar/foo
  63. Where `srclist` contains
  64. hdfs://nn1:8020/foo/a
  65. hdfs://nn1:8020/foo/b
  66. When copying from multiple sources, DistCp will abort the copy with an error
  67. message if two sources collide, but collisions at the destination are
  68. resolved per the [options](#Command_Line_Options) specified. By default,
  69. files already existing at the destination are skipped (i.e. not replaced by
  70. the source file). A count of skipped files is reported at the end of each
  71. job, but it may be inaccurate if a copier failed for some subset of its
  72. files, but succeeded on a later attempt.
  73. It is important that each NodeManager can reach and communicate with both the
  74. source and destination file systems. For HDFS, both the source and
  75. destination must be running the same version of the protocol or use a
  76. backwards-compatible protocol; see [Copying Between Versions]
  77. (#Copying_Between_Versions_of_HDFS).
  78. After a copy, it is recommended that one generates and cross-checks a listing
  79. of the source and destination to verify that the copy was truly successful.
  80. Since DistCp employs both Map/Reduce and the FileSystem API, issues in or
  81. between any of the three could adversely and silently affect the copy. Some
  82. have had success running with `-update` enabled to perform a second pass, but
  83. users should be acquainted with its semantics before attempting this.
  84. It's also worth noting that if another client is still writing to a source
  85. file, the copy will likely fail. Attempting to overwrite a file being written
  86. at the destination should also fail on HDFS. If a source file is (re)moved
  87. before it is copied, the copy will fail with a `FileNotFoundException`.
  88. Please refer to the detailed Command Line Reference for information on all
  89. the options available in DistCp.
  90. $H3 Update and Overwrite
  91. `-update` is used to copy files from source that don't exist at the target
  92. or differ from the target version. `-overwrite` overwrites target-files that
  93. exist at the target.
  94. The Update and Overwrite options warrant special attention since their
  95. handling of source-paths varies from the defaults in a very subtle manner.
  96. Consider a copy from `/source/first/` and `/source/second/` to `/target/`,
  97. where the source paths have the following contents:
  98. hdfs://nn1:8020/source/first/1
  99. hdfs://nn1:8020/source/first/2
  100. hdfs://nn1:8020/source/second/10
  101. hdfs://nn1:8020/source/second/20
  102. When DistCp is invoked without `-update` or `-overwrite`, the DistCp defaults
  103. would create directories `first/` and `second/`, under `/target`. Thus:
  104. distcp hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target
  105. would yield the following contents in `/target`:
  106. hdfs://nn2:8020/target/first/1
  107. hdfs://nn2:8020/target/first/2
  108. hdfs://nn2:8020/target/second/10
  109. hdfs://nn2:8020/target/second/20
  110. When either `-update` or `-overwrite` is specified, the **contents** of the
  111. source-directories are copied to target, and not the source directories
  112. themselves. Thus:
  113. distcp -update hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target
  114. would yield the following contents in `/target`:
  115. hdfs://nn2:8020/target/1
  116. hdfs://nn2:8020/target/2
  117. hdfs://nn2:8020/target/10
  118. hdfs://nn2:8020/target/20
  119. By extension, if both source folders contained a file with the same name
  120. (say, `0`), then both sources would map an entry to `/target/0` at the
  121. destination. Rather than to permit this conflict, DistCp will abort.
  122. Now, consider the following copy operation:
  123. distcp hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target
  124. With sources/sizes:
  125. hdfs://nn1:8020/source/first/1 32
  126. hdfs://nn1:8020/source/first/2 32
  127. hdfs://nn1:8020/source/second/10 64
  128. hdfs://nn1:8020/source/second/20 32
  129. And destination/sizes:
  130. hdfs://nn2:8020/target/1 32
  131. hdfs://nn2:8020/target/10 32
  132. hdfs://nn2:8020/target/20 64
  133. Will effect:
  134. hdfs://nn2:8020/target/1 32
  135. hdfs://nn2:8020/target/2 32
  136. hdfs://nn2:8020/target/10 64
  137. hdfs://nn2:8020/target/20 32
  138. `1` is skipped because the file-length and contents match. `2` is copied
  139. because it doesn't exist at the target. `10` and `20` are overwritten since
  140. the contents don't match the source.
  141. If `-update` is used, `1` is skipped because the file-length and contents match. `2` is copied because it doesn’t exist at the target. `10` and `20` are overwritten since the contents don’t match the source. However, if `-append` is additionally used, then only `10` is overwritten (source length less than destination) and `20` is appended with the change in file (if the files match up to the destination's original length).
  142. If `-overwrite` is used, `1` is overwritten as well.
  143. $H3 raw Namespace Extended Attribute Preservation
  144. This section only applies to HDFS.
  145. If the target and all of the source pathnames are in the `/.reserved/raw`
  146. hierarchy, then 'raw' namespace extended attributes will be preserved.
  147. 'raw' xattrs are used by the system for internal functions such as encryption
  148. meta data. They are only visible to users when accessed through the
  149. `/.reserved/raw` hierarchy.
  150. raw xattrs are preserved based solely on whether /.reserved/raw prefixes are
  151. supplied. The -p (preserve, see below) flag does not impact preservation of
  152. raw xattrs.
  153. To prevent raw xattrs from being preserved, simply do not use the
  154. `/.reserved/raw` prefix on any of the source and target paths.
  155. If the `/.reserved/raw `prefix is specified on only a subset of the source and
  156. target paths, an error will be displayed and a non-0 exit code returned.
  157. Command Line Options
  158. --------------------
  159. | Flag | Description | Notes |
  160. | ----------------- | ------------------------------------ | -------- |
  161. | `-p[rbugpcaxt]` | Preserve r: replication number b: block size u: user g: group p: permission c: checksum-type a: ACL x: XAttr t: timestamp | When `-update` is specified, status updates will **not** be synchronized unless the file sizes also differ (i.e. unless the file is re-created). If -pa is specified, DistCp preserves the permissions also because ACLs are a super-set of permissions. The option -pr is only valid if both source and target directory are not erasure coded. |
  162. | `-i` | Ignore failures | As explained in the Appendix, this option will keep more accurate statistics about the copy than the default case. It also preserves logs from failed copies, which can be valuable for debugging. Finally, a failing map will not cause the job to fail before all splits are attempted. |
  163. | `-log <logdir>` | Write logs to \<logdir\> | DistCp keeps logs of each file it attempts to copy as map output. If a map fails, the log output will not be retained if it is re-executed. |
  164. | `-v` | Log additional info (path, size) in the SKIP/COPY log | This option can only be used with -log option. |
  165. | `-m <num_maps>` | Maximum number of simultaneous copies | Specify the number of maps to copy data. Note that more maps may not necessarily improve throughput. |
  166. | `-overwrite` | Overwrite destination | If a map fails and `-i` is not specified, all the files in the split, not only those that failed, will be recopied. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully. |
  167. | `-update` | Overwrite if source and destination differ in size, blocksize, or checksum | As noted in the preceding, this is not a "sync" operation. The criteria examined are the source and destination file sizes, blocksizes, and checksums; if they differ, the source file replaces the destination file. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully. |
  168. | `-append` | Incremental copy of file with same name but different length | If the source file is greater in length than the destination file, the checksum of the common length part is compared. If the checksum matches, only the difference is copied using read and append functionalities. The -append option only works with `-update` without `-skipcrccheck` |
  169. | `-f <urilist_uri>` | Use list at \<urilist_uri\> as src list | This is equivalent to listing each source on the command line. The `urilist_uri` list should be a fully qualified URI. |
  170. | `-filters` | The path to a file containing a list of pattern strings, one string per line, such that paths matching the pattern will be excluded from the copy. | Support regular expressions specified by java.util.regex.Pattern. |
  171. | `-filelimit <n>` | Limit the total number of files to be <= n | **Deprecated!** Ignored in the new DistCp. |
  172. | `-sizelimit <n>` | Limit the total size to be <= n bytes | **Deprecated!** Ignored in the new DistCp. |
  173. | `-delete` | Delete the files existing in the dst but not in src | The deletion is done by FS Shell. So the trash will be used, if it is enable. Delete is applicable only with update or overwrite options. |
  174. | `-strategy {dynamic|uniformsize}` | Choose the copy-strategy to be used in DistCp. | By default, uniformsize is used. (i.e. Maps are balanced on the total size of files copied by each map. Similar to legacy.) If "dynamic" is specified, `DynamicInputFormat` is used instead. (This is described in the Architecture section, under InputFormats.) |
  175. | `-bandwidth` | Specify bandwidth per map, in MB/second. | Each map will be restricted to consume only the specified bandwidth. This is not always exact. The map throttles back its bandwidth consumption during a copy, such that the **net** bandwidth used tends towards the specified value. |
  176. | `-atomic {-tmp <tmp_dir>}` | Specify atomic commit, with optional tmp directory. | `-atomic` instructs DistCp to copy the source data to a temporary target location, and then move the temporary target to the final-location atomically. Data will either be available at final target in a complete and consistent form, or not at all. Optionally, `-tmp` may be used to specify the location of the tmp-target. If not specified, a default is chosen. **Note:** tmp_dir must be on the final target cluster. |
  177. | `-async` | Run DistCp asynchronously. Quits as soon as the Hadoop Job is launched. | The Hadoop Job-id is logged, for tracking. |
  178. | `-diff <oldSnapshot> <newSnapshot>` | Use snapshot diff report between given two snapshots to identify the difference between source and target, and apply the diff to the target to make it in sync with source. | This option is valid only with `-update` option and the following conditions should be satisfied. <ol><li> Both the source and the target FileSystem must be DistributedFileSystem.</li> <li> Two snapshots `<oldSnapshot>` and `<newSnapshot>` have been created on the source FS, and `<oldSnapshot>` is older than `<newSnapshot>`. </li> <li> The target has the same snapshot `<oldSnapshot>`. No changes have been made on the target since `<oldSnapshot>` was created, thus `<oldSnapshot>` has the same content as the current state of the target. All the files/directories in the target are the same with source's `<oldSnapshot>`.</li></ol> |
  179. | `-rdiff <newSnapshot> <oldSnapshot>` | Use snapshot diff report between given two snapshots to identify what has been changed on the target since the snapshot `<oldSnapshot>` was created on the target, and apply the diff reversely to the target, and copy modified files from the source's `<oldSnapshot>`, to make the target the same as `<oldSnapshot>`. | This option is valid only with `-update` option and the following conditions should be satisfied. <ol><li>Both the source and the target FileSystem must be DistributedFileSystem. The source and the target can be two different clusters/paths, or they can be exactly the same cluster/path. In the latter case, modified files are copied from target's `<oldSnapshot>` to target's current state).</li> <li> Two snapshots `<newSnapshot>` and `<oldSnapshot>` have been created on the target FS, and `<oldSnapshot>` is older than `<newSnapshot>`. No change has been made on target since `<newSnapshot>` was created on the target. </li> <li> The source has the same snapshot `<oldSnapshot>`, which has the same content as the `<oldSnapshot>` on the target. All the files/directories in the target's `<oldSnapshot>` are the same with source's `<oldSnapshot>`.</li> </ol> |
  180. | `-numListstatusThreads` | Number of threads to use for building file listing | At most 40 threads. |
  181. | `-skipcrccheck` | Whether to skip CRC checks between source and target paths. | |
  182. | `-blocksperchunk <blocksperchunk>` | Number of blocks per chunk. When specified, split files into chunks to copy in parallel | If set to a positive value, files with more blocks than this value will be split into chunks of `<blocksperchunk>` blocks to be transferred in parallel, and reassembled on the destination. By default, `<blocksperchunk>` is 0 and the files will be transmitted in their entirety without splitting. This switch is only applicable when the source file system implements getBlockLocations method and the target file system implements concat method. |
  183. | `-copybuffersize <copybuffersize>` | Size of the copy buffer to use. By default, `<copybuffersize>` is set to 8192B | |
  184. | `-xtrack <path>` | Save information about missing source files to the specified path. | This option is only valid with `-update` option. This is an experimental property and it cannot be used with `-atomic` option. |
  185. | `-direct` | Write directly to destination paths | Useful for avoiding potentially very expensive temporary file rename operations when the destination is an object store |
  186. Architecture of DistCp
  187. ----------------------
  188. The components of the new DistCp may be classified into the following
  189. categories:
  190. * DistCp Driver
  191. * Copy-listing generator
  192. * Input-formats and Map-Reduce components
  193. $H3 DistCp Driver
  194. The DistCp Driver components are responsible for:
  195. * Parsing the arguments passed to the DistCp command on the command-line,
  196. via:
  197. * OptionsParser, and
  198. * DistCpOptionsSwitch
  199. * Assembling the command arguments into an appropriate DistCpOptions object,
  200. and initializing DistCp. These arguments include:
  201. * Source-paths
  202. * Target location
  203. * Copy options (e.g. whether to update-copy, overwrite, which
  204. file-attributes to preserve, etc.)
  205. * Orchestrating the copy operation by:
  206. * Invoking the copy-listing-generator to create the list of files to be
  207. copied.
  208. * Setting up and launching the Hadoop Map-Reduce Job to carry out the
  209. copy.
  210. * Based on the options, either returning a handle to the Hadoop MR Job
  211. immediately, or waiting till completion.
  212. The parser-elements are exercised only from the command-line (or if
  213. DistCp::run() is invoked). The DistCp class may also be used
  214. programmatically, by constructing the DistCpOptions object, and initializing
  215. a DistCp object appropriately.
  216. $H3 Copy-listing Generator
  217. The copy-listing-generator classes are responsible for creating the list of
  218. files/directories to be copied from source. They examine the contents of the
  219. source-paths (files/directories, including wild-cards), and record all paths
  220. that need copy into a SequenceFile, for consumption by the DistCp Hadoop
  221. Job. The main classes in this module include:
  222. 1. `CopyListing`: The interface that should be implemented by any
  223. copy-listing-generator implementation. Also provides the factory method by
  224. which the concrete CopyListing implementation is chosen.
  225. 2. `SimpleCopyListing`: An implementation of `CopyListing` that accepts multiple
  226. source paths (files/directories), and recursively lists all the individual
  227. files and directories under each, for copy.
  228. 3. `GlobbedCopyListing`: Another implementation of `CopyListing` that expands
  229. wild-cards in the source paths.
  230. 4. `FileBasedCopyListing`: An implementation of `CopyListing` that reads the
  231. source-path list from a specified file.
  232. Based on whether a source-file-list is specified in the DistCpOptions, the
  233. source-listing is generated in one of the following ways:
  234. 1. If there's no source-file-list, the `GlobbedCopyListing` is used. All
  235. wild-cards are expanded, and all the expansions are forwarded to the
  236. SimpleCopyListing, which in turn constructs the listing (via recursive
  237. descent of each path).
  238. 2. If a source-file-list is specified, the `FileBasedCopyListing` is used.
  239. Source-paths are read from the specified file, and then forwarded to the
  240. `GlobbedCopyListing`. The listing is then constructed as described above.
  241. One may customize the method by which the copy-listing is constructed by
  242. providing a custom implementation of the CopyListing interface. The behaviour
  243. of DistCp differs here from the legacy DistCp, in how paths are considered
  244. for copy.
  245. The legacy implementation only lists those paths that must definitely be
  246. copied on to target. E.g. if a file already exists at the target (and
  247. `-overwrite` isn't specified), the file isn't even considered in the
  248. MapReduce Copy Job. Determining this during setup (i.e. before the MapReduce
  249. Job) involves file-size and checksum-comparisons that are potentially
  250. time-consuming.
  251. The new DistCp postpones such checks until the MapReduce Job, thus reducing
  252. setup time. Performance is enhanced further since these checks are
  253. parallelized across multiple maps.
  254. $H3 InputFormats and MapReduce Components
  255. The InputFormats and MapReduce components are responsible for the actual copy
  256. of files and directories from the source to the destination path. The
  257. listing-file created during copy-listing generation is consumed at this
  258. point, when the copy is carried out. The classes of interest here include:
  259. * **UniformSizeInputFormat:**
  260. This implementation of org.apache.hadoop.mapreduce.InputFormat provides
  261. equivalence with Legacy DistCp in balancing load across maps. The aim of
  262. the UniformSizeInputFormat is to make each map copy roughly the same number
  263. of bytes. Apropos, the listing file is split into groups of paths, such
  264. that the sum of file-sizes in each InputSplit is nearly equal to every
  265. other map. The splitting isn't always perfect, but its trivial
  266. implementation keeps the setup-time low.
  267. * **DynamicInputFormat and DynamicRecordReader:**
  268. The DynamicInputFormat implements `org.apache.hadoop.mapreduce.InputFormat`,
  269. and is new to DistCp. The listing-file is split into several "chunk-files",
  270. the exact number of chunk-files being a multiple of the number of maps
  271. requested for in the Hadoop Job. Each map task is "assigned" one of the
  272. chunk-files (by renaming the chunk to the task's id), before the Job is
  273. launched.
  274. Paths are read from each chunk using the `DynamicRecordReader`, and
  275. processed in the CopyMapper. After all the paths in a chunk are processed,
  276. the current chunk is deleted and a new chunk is acquired. The process
  277. continues until no more chunks are available.
  278. This "dynamic" approach allows faster map-tasks to consume more paths than
  279. slower ones, thus speeding up the DistCp job overall.
  280. * **CopyMapper:**
  281. This class implements the physical file-copy. The input-paths are checked
  282. against the input-options (specified in the Job's Configuration), to
  283. determine whether a file needs copy. A file will be copied only if at least
  284. one of the following is true:
  285. * A file with the same name doesn't exist at target.
  286. * A file with the same name exists at target, but has a different file
  287. size.
  288. * A file with the same name exists at target, but has a different
  289. checksum, and `-skipcrccheck` isn't mentioned.
  290. * A file with the same name exists at target, but `-overwrite` is
  291. specified.
  292. * A file with the same name exists at target, but differs in block-size
  293. (and block-size needs to be preserved.
  294. * **CopyCommitter:** This class is responsible for the commit-phase of the
  295. DistCp job, including:
  296. * Preservation of directory-permissions (if specified in the options)
  297. * Clean-up of temporary-files, work-directories, etc.
  298. Appendix
  299. --------
  300. $H3 Map sizing
  301. By default, DistCp makes an attempt to size each map comparably so that each
  302. copies roughly the same number of bytes. Note that files are the finest level
  303. of granularity, so increasing the number of simultaneous copiers (i.e. maps)
  304. may not always increase the number of simultaneous copies nor the overall
  305. throughput.
  306. The new DistCp also provides a strategy to "dynamically" size maps, allowing
  307. faster data-nodes to copy more bytes than slower nodes. Using `-strategy
  308. dynamic` (explained in the Architecture), rather than to assign a fixed set
  309. of source-files to each map-task, files are instead split into several sets.
  310. The number of sets exceeds the number of maps, usually by a factor of 2-3.
  311. Each map picks up and copies all files listed in a chunk. When a chunk is
  312. exhausted, a new chunk is acquired and processed, until no more chunks
  313. remain.
  314. By not assigning a source-path to a fixed map, faster map-tasks (i.e.
  315. data-nodes) are able to consume more chunks, and thus copy more data, than
  316. slower nodes. While this distribution isn't uniform, it is fair with regard
  317. to each mapper's capacity.
  318. The dynamic-strategy is implemented by the `DynamicInputFormat`. It provides
  319. superior performance under most conditions.
  320. Tuning the number of maps to the size of the source and destination clusters,
  321. the size of the copy, and the available bandwidth is recommended for
  322. long-running and regularly run jobs.
  323. $H3 Copying Between Versions of HDFS
  324. For copying between two different major versions of Hadoop (e.g. between 1.X
  325. and 2.X), one will usually use WebHdfsFileSystem. Unlike the previous
  326. HftpFileSystem, as webhdfs is available for both read and write operations,
  327. DistCp can be run on both source and destination cluster.
  328. Remote cluster is specified as `webhdfs://<namenode_hostname>:<http_port>`.
  329. When copying between same major versions of Hadoop cluster (e.g. between 2.X
  330. and 2.X), use hdfs protocol for better performance.
  331. $H3 Secure Copy over the wire with distcp
  332. Use the "`swebhdfs://`" scheme when webhdfs is secured with SSL. For more information see [SSL Configurations for SWebHDFS](../hadoop-project-dist/hadoop-hdfs/WebHDFS.html#SSL_Configurations_for_SWebHDFS).
  333. $H3 MapReduce and other side-effects
  334. As has been mentioned in the preceding, should a map fail to copy one of its
  335. inputs, there will be several side-effects.
  336. * Unless `-overwrite` is specified, files successfully copied by a previous
  337. map on a re-execution will be marked as "skipped".
  338. * If a map fails `mapreduce.map.maxattempts` times, the remaining map tasks
  339. will be killed (unless `-i` is set).
  340. * If `mapreduce.map.speculative` is set set final and true, the result of the
  341. copy is undefined.
  342. $H3 DistCp and Object Stores
  343. DistCp works with Object Stores such as Amazon S3, Azure WASB and OpenStack Swift.
  344. Prequisites
  345. 1. The JAR containing the object store implementation is on the classpath,
  346. along with all of its dependencies.
  347. 1. Unless the JAR automatically registers its bundled filesystem clients,
  348. the configuration may need to be modified to state the class which
  349. implements the filesystem schema. All of the ASF's own object store clients
  350. are self-registering.
  351. 1. The relevant object store access credentials must be available in the cluster
  352. configuration, or be otherwise available in all cluster hosts.
  353. DistCp can be used to upload data
  354. ```bash
  355. hadoop distcp -direct hdfs://nn1:8020/datasets/set1 s3a://bucket/datasets/set1
  356. ```
  357. To download data
  358. ```bash
  359. hadoop distcp s3a://bucket/generated/results hdfs://nn1:8020/results
  360. ```
  361. To copy data between object stores
  362. ```bash
  363. hadoop distcp s3a://bucket/generated/results \
  364. wasb://updates@example.blob.core.windows.net
  365. ```
  366. And do copy data within an object store
  367. ```bash
  368. hadoop distcp wasb://updates@example.blob.core.windows.net/current \
  369. wasb://updates@example.blob.core.windows.net/old
  370. ```
  371. And to use `-update` to only copy changed files.
  372. ```bash
  373. hadoop distcp -update -numListstatusThreads 20 \
  374. swift://history.cluster1/2016 \
  375. hdfs://nn1:8020/history/2016
  376. ```
  377. Because object stores are slow to list files, consider setting the `-numListstatusThreads` option when performing a `-update` operation
  378. on a large directory tree (the limit is 40 threads).
  379. When `DistCp -update` is used with object stores,
  380. generally only the modification time and length of the individual files are compared,
  381. not any checksums. The fact that most object stores do have valid timestamps
  382. for directories is irrelevant; only the file timestamps are compared.
  383. However, it is important to have the clock of the client computers close
  384. to that of the infrastructure, so that timestamps are consistent between
  385. the client/HDFS cluster and that of the object store. Otherwise, changed files may be
  386. missed/copied too often.
  387. **Notes**
  388. * The `-atomic` option causes a rename of the temporary data, so significantly
  389. increases the time to commit work at the end of the operation. Furthermore,
  390. as Object Stores other than (optionally) `wasb://` do not offer atomic renames of directories
  391. the `-atomic` operation doesn't actually deliver what is promised. *Avoid*.
  392. * The `-append` option is not supported.
  393. * The `-diff` and `rdiff` options are not supported
  394. * CRC checking will not be performed, irrespective of the value of the `-skipCrc`
  395. flag.
  396. * All `-p` options, including those to preserve permissions, user and group information, attributes
  397. checksums and replication are generally ignored. The `wasb://` connector will
  398. preserve the information, but not enforce the permissions.
  399. * Some object store connectors offer an option for in-memory buffering of
  400. output —for example the S3A connector. Using such option while copying
  401. large files may trigger some form of out of memory event,
  402. be it a heap overflow or a YARN container termination.
  403. This is particularly common if the network bandwidth
  404. between the cluster and the object store is limited (such as when working
  405. with remote object stores). It is best to disable/avoid such options and
  406. rely on disk buffering.
  407. * Copy operations within a single object store still take place in the Hadoop cluster
  408. —even when the object store implements a more efficient COPY operation internally
  409. That is, an operation such as
  410. hadoop distcp s3a://bucket/datasets/set1 s3a://bucket/datasets/set2
  411. Copies each byte down to the Hadoop worker nodes and back to the
  412. bucket. As well as being slow, it means that charges may be incurred.
  413. * The `-direct` option can be used to write to object store target paths directly,
  414. avoiding the potentially very expensive temporary file rename operations that would
  415. otherwise occur.
  416. Frequently Asked Questions
  417. --------------------------
  418. 1. **Why does -update not create the parent source-directory under a pre-existing target directory?**
  419. The behaviour of `-update` and `-overwrite` is described in detail in the
  420. Usage section of this document. In short, if either option is used with a
  421. pre-existing destination directory, the **contents** of each source
  422. directory is copied over, rather than the source-directory itself. This
  423. behaviour is consistent with the legacy DistCp implementation as well.
  424. 2. **How does the new DistCp differ in semantics from the Legacy DistCp?**
  425. * Files that are skipped during copy used to also have their
  426. file-attributes (permissions, owner/group info, etc.) unchanged, when
  427. copied with Legacy DistCp. These are now updated, even if the file-copy
  428. is skipped.
  429. * Empty root directories among the source-path inputs were not created at
  430. the target, in Legacy DistCp. These are now created.
  431. 3. **Why does the new DistCp use more maps than legacy DistCp?**
  432. Legacy DistCp works by figuring out what files need to be actually copied
  433. to target before the copy-job is launched, and then launching as many maps
  434. as required for copy. So if a majority of the files need to be skipped
  435. (because they already exist, for example), fewer maps will be needed. As a
  436. consequence, the time spent in setup (i.e. before the M/R job) is higher.
  437. The new DistCp calculates only the contents of the source-paths. It
  438. doesn't try to filter out what files can be skipped. That decision is put
  439. off till the M/R job runs. This is much faster (vis-a-vis execution-time),
  440. but the number of maps launched will be as specified in the `-m` option,
  441. or 20 (default) if unspecified.
  442. 4. **Why does DistCp not run faster when more maps are specified?**
  443. At present, the smallest unit of work for DistCp is a file. i.e., a file
  444. is processed by only one map. Increasing the number of maps to a value
  445. exceeding the number of files would yield no performance benefit. The
  446. number of maps launched would equal the number of files.
  447. 5. **Why does DistCp run out of memory?**
  448. If the number of individual files/directories being copied from the source
  449. path(s) is extremely large (e.g. 1,000,000 paths), DistCp might run out of
  450. memory while determining the list of paths for copy. This is not unique to
  451. the new DistCp implementation.
  452. To get around this, consider changing the `-Xmx` JVM heap-size parameters,
  453. as follows:
  454. bash$ export HADOOP_CLIENT_OPTS="-Xms64m -Xmx1024m"
  455. bash$ hadoop distcp /source /target