DistCp(分布式复制)是一种用于大规模集群间/集群内复制的工具。它使用 MapReduce 来实现其分布、错误处理和恢复以及报告。它将文件和目录列表扩展为映射任务的输入,每个任务都将复制源列表中指定的文件的一部分。
[DistCp 的早期实现] (https://hadoop.apache.org/docs/r1.2.1/distcp.html) 在其用法、可扩展性和性能方面都存在一些怪癖和缺点。DistCp 重构的目的是修复这些缺点,使其能够以编程方式使用和扩展。已经引入了新的范例来提高运行时和设置性能,同时保留默认的传统行为。
本文档旨在描述新 DistCp 的设计、其全新功能、最佳用法以及与传统实现的任何偏差。
DistCp 最常见的调用是集群间复制
bash$ hadoop distcp hdfs://nn1:8020/foo/bar \ hdfs://nn2:8020/bar/foo
这会将 nn1 上 /foo/bar
下的命名空间扩展到一个临时文件,将其内容划分为一组映射任务,并从 nn1
到 nn2
在每个 NodeManager 上启动一个副本。
还可以在命令行上指定多个源目录
bash$ hadoop distcp hdfs://nn1:8020/foo/a \ hdfs://nn1:8020/foo/b \ hdfs://nn2:8020/bar/foo
或者,等效地,从使用 -f 选项的文件中指定
bash$ hadoop distcp -f hdfs://nn1:8020/srclist \ hdfs://nn2:8020/bar/foo
其中 srclist
包含
hdfs://nn1:8020/foo/a hdfs://nn1:8020/foo/b
从多个源复制时,如果两个源发生冲突,DistCp 将中止复制并显示一条错误消息,但目标位置的冲突将根据指定的选项解决。默认情况下,目标位置已存在的文件将被跳过(即不会被源文件替换)。每个作业结束时都会报告跳过文件数,但如果复制器对某些文件子集失败,但在稍后的尝试中成功,则报告可能不准确。
每个 NodeManager 都能够访问并与源文件系统和目标文件系统通信非常重要。对于 HDFS,源和目标都必须运行相同版本的协议或使用向后兼容的协议;请参阅 [不同 HDFS 版本之间的复制] (#Copying_Between_Versions_of_HDFS)。
复制后,建议生成源和目标的列表并进行交叉检查,以验证复制是否真正成功。由于 DistCp 同时使用 Map/Reduce 和 FileSystem API,因此这三者中的任何一个或它们之间的任何问题都可能对复制产生不利影响且不会发出通知。有些人成功地运行了启用了 -update
的功能来执行第二次传递,但在尝试此操作之前,用户应了解其语义。
还值得注意的是,如果另一个客户端仍在向源文件写入,则复制可能会失败。尝试覆盖在目标位置写入的文件在 HDFS 上也应该会失败。如果源文件在复制之前被(重新)移动,则复制将失败并显示 FileNotFoundException
。
请参阅详细的命令行参考,了解 DistCp 中所有可用选项的信息。
-update
用于复制目标位置不存在或与目标版本不同的源文件。-overwrite
覆盖目标位置存在的目标文件。
更新和覆盖选项需要特别注意,因为它们对源路径的处理方式与默认方式有非常细微的不同。考虑从 /source/first/
和 /source/second/
复制到 /target/
,其中源路径具有以下内容
hdfs://nn1:8020/source/first/1 hdfs://nn1:8020/source/first/2 hdfs://nn1:8020/source/second/10 hdfs://nn1:8020/source/second/20
在不使用 -update
或 -overwrite
的情况下调用 DistCp 时,DistCp 默认会在 /target
下创建目录 first/
和 second/
。因此
distcp hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target
将在 /target
中生成以下内容
hdfs://nn2:8020/target/first/1 hdfs://nn2:8020/target/first/2 hdfs://nn2:8020/target/second/10 hdfs://nn2:8020/target/second/20
当指定 -update
或 -overwrite
时,源目录的内容将被复制到目标,而不是源目录本身。因此
distcp -update hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target
将在 /target
中生成以下内容
hdfs://nn2:8020/target/1 hdfs://nn2:8020/target/2 hdfs://nn2:8020/target/10 hdfs://nn2:8020/target/20
进一步说,如果两个源文件夹包含同名文件(例如,0
),那么两个源都会将一个条目映射到目标中的 /target/0
。为了避免这种冲突,DistCp 将中止。
现在,考虑以下复制操作
distcp hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target
源/大小
hdfs://nn1:8020/source/first/1 32 hdfs://nn1:8020/source/first/2 32 hdfs://nn1:8020/source/second/10 64 hdfs://nn1:8020/source/second/20 32
目标/大小
hdfs://nn2:8020/target/1 32 hdfs://nn2:8020/target/10 32 hdfs://nn2:8020/target/20 64
将生效
hdfs://nn2:8020/target/1 32 hdfs://nn2:8020/target/2 32 hdfs://nn2:8020/target/10 64 hdfs://nn2:8020/target/20 32
1
被跳过,因为文件长度和内容匹配。2
被复制,因为它在目标中不存在。10
和 20
被覆盖,因为内容与源不匹配。
如果使用 -update
,则 1
被跳过,因为文件长度和内容匹配。2
被复制,因为它在目标中不存在。10
和 20
被覆盖,因为内容与源不匹配。但是,如果还使用了 -append
,则仅覆盖 10
(源长度小于目标)并且 20
附加了文件中的更改(如果文件与目标的原始长度匹配)。
如果使用 -overwrite
,则 1
也将被覆盖。
-diff
选项使用快照差异将文件从源集群同步到目标集群。它复制、重命名和删除快照差异列表中的文件。
当使用 -diff
选项时,必须包含 -update
选项。
目前,大多数云提供商无法很好地配合同步。
用法
hadoop distcp -update -diff <from_snapshot> <to_snapshot> <source> <destination>
示例
hadoop distcp -update -diff snap1 snap2 /src/ /dst/
上面的命令将快照 snap1
到 snap2
的更改(即从 snap1
到 snap2
的快照差异)应用于 /src/
到 /dst/
。显然,它要求 /src/
同时具有快照 snap1
和 snap2
。但目标 /dst/
也必须具有与 <from_snapshot>
同名的快照,在本例中为 snap1
。目标 /dst/
自 snap1
以来不应有新的文件操作(创建、重命名、删除)。请注意,当此命令完成时,不会在 /dst/
创建新的快照 snap2
。
使用 -diff
选项需要 -update
。
例如,在 /src/
中,如果在创建 snap1
之后和创建 snap2
之前添加了 1.txt
并删除了 2.txt
,则上面的命令将从 /src/
复制 1.txt
到 /dst/
,并从 /dst/
删除 2.txt
。
以下实验将详细说明同步行为。
开始之前做一些准备工作。
# Create source and destination directories hdfs dfs -mkdir /src/ /dst/ # Allow snapshot on source hdfs dfsadmin -allowSnapshot /src/ # Create a snapshot (empty one) hdfs dfs -createSnapshot /src/ snap1 # Allow snapshot on destination hdfs dfsadmin -allowSnapshot /dst/ # Create a from_snapshot with the same name hdfs dfs -createSnapshot /dst/ snap1 # Put one text file under /src/ echo "This is the 1st text file." > 1.txt hdfs dfs -put 1.txt /src/ # Create the second snapshot hdfs dfs -createSnapshot /src/ snap2 # Put another text file under /src/ echo "This is the 2nd text file." > 2.txt hdfs dfs -put 2.txt /src/ # Create the third snapshot hdfs dfs -createSnapshot /src/ snap3
然后运行 distcp 同步
hadoop distcp -update -diff snap1 snap2 /src/ /dst/
上述命令应该会成功。1.txt
将从 /src/
复制到 /dst/
。同样,需要 -update
选项。
如果再次运行相同的命令,我们将收到 DistCp sync failed
异常,因为自 snap1
以来,目标已添加了一个新文件 1.txt
。也就是说,如果我们从 /dst/
手动删除 1.txt
并运行同步,该命令将成功。
首先从实验 1 中进行清理。
hdfs dfs -rm -skipTrash /dst/1.txt
运行同步命令,请注意 <to_snapshot>
已从实验 1 中的 snap2
更改为 snap3
。
hadoop distcp -update -diff snap1 snap3 /src/ /dst/
1.txt
和 2.txt
都将复制到 /dst/
。
从实验 2 的末尾继续
hdfs dfs -rm -skipTrash /dst/2.txt # Create snap2 at destination, it contains 1.txt hdfs dfs -createSnapshot /dst/ snap2 # Delete 1.txt from source hdfs dfs -rm -skipTrash /src/1.txt # Create snap4 at source, it only contains 2.txt hdfs dfs -createSnapshot /src/ snap4
立即运行同步命令
hadoop distcp -update -diff snap2 snap4 /src/ /dst/
2.txt
已复制,1.txt
已在 /dst/
下删除。
请注意,虽然 /src/
和 /dst/
都具有名称为 snap2
的快照,但这些快照不必具有相同的内容。这意味着,如果 /dst/
的 snap2
中有 1.txt
,但它们的内容不同,1.txt
仍将从 /dst/
中删除。同步命令不会检查要删除的文件的内容。它只是遵循 <from_snapshot>
和 <to_snapshot> 之间的快照差异列表。
此外,如果我们在上述步骤中在 /dst/
上创建 snap2
之前从 /dst/
中删除 1.txt
,以便 /dst/
的 snap2
在运行同步命令之前没有 1.txt
,该命令仍将成功。它不会在尝试从不存在的 /dst/
中删除 1.txt
时引发异常。
本部分仅适用于 HDFS。
如果目标和所有源路径名位于 /.reserved/raw
层次结构中,则将保留“raw”命名空间扩展属性。“raw”xattr 由系统用于内部功能,例如加密元数据。只有通过 /.reserved/raw
层次结构访问时,用户才能看到它们。
raw xattr 的保留仅基于是否提供了 /.reserved/raw 前缀。-p(保留,见下文)标志不会影响 raw xattr 的保留。
若要防止保留 raw xattr,只需不在任何源和目标路径上使用 /.reserved/raw
前缀。
如果仅在源和目标路径的子集中指定了 /.reserved/raw
前缀,则会显示错误并返回非 0 退出代码。
标志 | 说明 | 注释 |
---|---|---|
-p[rbugpcaxt] |
保留 r:复制数 b:块大小 u:用户 g:组 p:权限 c:校验和类型 a:ACL x:XAttr t:时间戳 | 当指定 -update 时,除非文件大小也有所不同(即除非重新创建文件),否则不会同步状态更新。如果指定了 -pa,DistCp 还将保留权限,因为 ACL 是权限的超集。如果源目录和目标目录都不是擦除编码,则选项 -pr 才有效。 |
-i |
忽略失败 | 如附录中所述,此选项将保留比默认情况下更准确的复制统计信息。它还保留失败复制的日志,这对于调试很有价值。最后,失败的映射不会导致在尝试所有拆分之前作业失败。 |
-log <logdir> |
将日志写入 <logdir> | DistCp 保留它尝试复制的每个文件的日志作为映射输出。如果映射失败,则在重新执行时不会保留日志输出。 |
-v |
在 SKIP/COPY 日志中记录其他信息(路径、大小) | 此选项只能与 -log 选项一起使用。 |
-m <num_maps> |
同时进行的最大复制数 | 指定复制数据的映射数。请注意,更多映射不一定能提高吞吐量。 |
-overwrite |
覆盖目标 | 如果映射失败,并且未指定 -i ,则将重新复制拆分中的所有文件,而不仅仅是失败的文件。如使用文档中所述,它还更改了生成目标路径的语义,因此用户应谨慎使用。 |
-update |
如果源和目标在大小、块大小或校验和方面不同,则覆盖 | 如前所述,这不是“同步”操作。检查的标准是源和目标文件的大小、块大小和校验和;如果它们不同,则源文件将替换目标文件。如使用文档中所述,它还更改了生成目标路径的语义,因此用户应谨慎使用。 |
-append |
具有相同名称但不同长度的文件的增量复制 | 如果源文件比目标文件长度更大,则比较相同长度部分的校验和。如果校验和匹配,则仅使用读取和追加功能复制差异。-append 选项仅适用于不带 -skipcrccheck 的 -update |
-f <urilist_uri> |
将 <urilist_uri> 中的列表用作 src 列表 | 这相当于在命令行中列出每个源。urilist_uri 列表应为完全限定的 URI。 |
-filters |
包含模式字符串列表的文件的路径,每行一个字符串,以便与模式匹配的路径将从复制中排除。 | 支持 java.util.regex.Pattern 指定的正则表达式。 |
-filelimit <n> |
将要复制的文件总数限制为 <= n | 已弃用!在新 DistCp 中忽略。 |
-sizelimit <n> |
将总大小限制为 <= n 字节 | 已弃用!在新 DistCp 中忽略。 |
-delete |
删除 dst 中存在但 src 中不存在的文件 | 删除由 FS Shell 执行。因此,如果启用,将使用回收站。删除仅适用于更新或覆盖选项。 |
-strategy {dynamic|uniformsize} |
选择要在 DistCp 中使用的复制策略。 | 默认情况下,使用 uniformsize。(即,映射根据每个映射复制的文件的总大小进行平衡。类似于旧版。)如果指定“dynamic”,则改用 DynamicInputFormat 。(这在架构部分的 InputFormats 下进行了描述。) |
-bandwidth |
以 MB/秒为单位指定每个映射的带宽。 | 每个映射将仅限于消耗指定的带宽。这并不总是准确的。映射会在复制过程中限制其带宽消耗,以便使用的净带宽趋向于指定的值。 |
-atomic {-tmp <tmp_dir>} |
指定原子提交,带可选的临时目录。 | -atomic 指示 DistCp 将源数据复制到临时目标位置,然后将临时目标原子地移动到最终位置。数据将以完整且一致的形式在最终目标中可用,或者根本不可用。或者,可以使用 -tmp 指定临时目标的位置。如果未指定,则选择默认值。注意:tmp_dir 必须位于最终目标集群中。 |
-async |
异步运行 DistCp。在 Hadoop 作业启动后立即退出。 | 记录 Hadoop 作业 ID 以便跟踪。 |
-diff <oldSnapshot> <newSnapshot> |
使用给定的两个快照之间的快照差异报告来识别源和目标之间的差异,并将差异应用于目标,使其与源同步。 | 此选项仅与 -update 选项有效,并且应满足以下条件。
|
-rdiff <newSnapshot> <oldSnapshot> |
使用给定的两个快照之间的快照差异报告来识别自快照 <oldSnapshot> 在目标上创建以来在目标上已更改的内容,并将差异反向应用于目标,并将修改后的文件从源的 <oldSnapshot> 复制,以使目标与 <oldSnapshot> 相同。 |
此选项仅与 -update 选项有效,并且应满足以下条件。
|
-numListstatusThreads |
用于构建文件列表的线程数 | 最多 40 个线程。 |
-skipcrccheck |
是否跳过源路径和目标路径之间的 CRC 检查。 | |
-blocksperchunk <blocksperchunk> |
每个块中的块数。指定后,将文件拆分成块,并行复制 | 如果设置为正值,则块数多于此值的文件将被拆分成 <blocksperchunk> 个块,并行传输,并在目标位置重新组装。默认情况下,<blocksperchunk> 为 0,文件将被完整传输,而不会拆分。仅当源文件系统实现了 getBlockLocations 方法,并且目标文件系统实现了 concat 方法时,此开关才适用。 |
-copybuffersize <copybuffersize> |
要使用的复制缓冲区大小。默认情况下,<copybuffersize> 设置为 8192B |
|
-xtrack <path> |
将有关丢失源文件的信息保存到指定路径。 | 此选项仅与 -update 选项一起使用。这是一个实验性属性,不能与 -atomic 选项一起使用。 |
-direct |
直接写入目标路径 | 当目标是对象存储时,可用于避免潜在的非常昂贵的临时文件重命名操作 |
-useiterator |
使用单线程 listStatusIterator 来构建列表 | 可用于节省客户端内存。使用此选项将忽略 numListstatusThreads 选项 |
新 DistCp 的组件可归类为以下类别
DistCp 驱动程序组件负责
通过以下方式解析传递给命令行上 DistCp 命令的参数:
将命令参数组装到适当的 DistCpOptions 对象中,并初始化 DistCp。这些参数包括
通过以下方式协调复制操作:
解析器元素仅从命令行(或调用 DistCp::run() 时)执行。还可通过构造 DistCpOptions 对象并适当地初始化 DistCp 对象,以编程方式使用 DistCp 类。
复制列表生成器类负责创建要从源复制的文件/目录列表。它们检查源路径的内容(包括文件/目录、通配符),并将需要复制的所有路径记录到 SequenceFile 中,供 DistCp Hadoop 作业使用。此模块中的主要类包括
CopyListing
:任何复制列表生成器实现都应实现的接口。还提供了选择具体 CopyListing 实现的工厂方法。SimpleCopyListing
:CopyListing
的实现,接受多个源路径(文件/目录),并递归列出每个路径下的所有单个文件和目录,以进行复制。GlobbedCopyListing
:CopyListing
的另一个实现,可扩展源路径中的通配符。FileBasedCopyListing
:CopyListing
的实现,从指定文件中读取源路径列表。根据 DistCpOptions 中是否指定了源文件列表,源列表将通过以下方式之一生成
GlobbedCopyListing
。所有通配符都将展开,所有展开内容都将转发到 SimpleCopyListing,后者又构造列表(通过每个路径的递归下降)。FileBasedCopyListing
。源路径从指定文件中读取,然后转发到 GlobbedCopyListing
。然后如上所述构造列表。可以通过提供 CopyListing 接口的自定义实现,来自定义构造复制列表的方法。DistCp 的行为在此处与旧版 DistCp 不同,在于如何考虑路径进行复制。
还可以通过传递 CopyFilter 接口当前支持的实现或编写新实现,来自定义不应复制的文件的筛选。这可以通过在 DistCpOptions 中设置 distcp.filters.class
来指定
distcp.filters.class
为 “RegexCopyFilter”。如果您使用此实现,则必须传递包含用于筛选的正则表达式的 “CopyFilter” distcp.filters.file
。支持 java.util.regex.Pattern 指定的正则表达式。distcp.filters.class
为 “RegexpInConfigurationFilter”。如果您使用此实现,则还必须使用 “DistCpOptions” 中的 distcp.exclude-file-regex
参数传递正则表达式。支持 java.util.regex.Pattern 指定的正则表达式。与 “RegexCopyFilter” 相比,这是一种更动态的方法。distcp.filters.class
为 “TrueCopyFilter”。如果未指定上述任何选项,则将其用作默认实现。旧版实现仅列出必须复制到目标的路径。例如,如果文件已存在于目标中(并且未指定-overwrite
),则该文件甚至不会在 MapReduce 复制作业中考虑。在设置期间(即在 MapReduce 作业之前)确定这一点涉及文件大小和校验和比较,这可能会耗时。
新的 DistCp 将此类检查推迟到 MapReduce 作业,从而减少设置时间。由于这些检查在多个映射中并行化,因此性能进一步提高。
InputFormats 和 MapReduce 组件负责将文件和目录从源实际复制到目标路径。在执行复制时,会使用在复制列表生成期间创建的列表文件。此处涉及的类包括
UniformSizeInputFormat:此 org.apache.hadoop.mapreduce.InputFormat 实现提供与旧版 DistCp 相等的负载均衡功能。UniformSizeInputFormat 的目的是使每个映射复制大致相同数量的字节。顺便说一下,列表文件被分成几组路径,使得每个 InputSplit 中的文件大小之和几乎等于每个其他映射。拆分并不总是完美的,但其简单的实现使设置时间保持较低。
DynamicInputFormat 和 DynamicRecordReader:DynamicInputFormat 实现org.apache.hadoop.mapreduce.InputFormat
,并且是 DistCp 中的新功能。列表文件被分成几个“块文件”,块文件的准确数量是 Hadoop 作业中请求的映射数量的倍数。在启动作业之前,每个映射任务都被“分配”一个块文件(通过将块重命名为任务的 ID)。使用DynamicRecordReader
从每个块读取路径,并在 CopyMapper 中处理。在处理块中的所有路径后,将删除当前块并获取新块。此过程将持续到没有更多块可用为止。这种“动态”方法允许更快的映射任务比更慢的任务消耗更多路径,从而总体上加快 DistCp 作业的速度。
CopyMapper:此类实现物理文件复制。输入路径根据输入选项(在作业的配置中指定)进行检查,以确定是否需要复制文件。仅当满足以下至少一项条件时,才会复制文件
-skipcrccheck
。-overwrite
。CopyCommitter:此类负责 DistCp 作业的提交阶段,包括
默认情况下,DistCp 会尝试对每个 Map 进行大小调整,以便每个 Map 大致复制相同数量的字节。请注意,文件是粒度最细的级别,因此增加同时进行复制的副本数(即 Map)可能并不总是会增加同时进行的复制次数或整体吞吐量。
新的 DistCp 还提供了一种“动态”调整 Map 大小的策略,允许速度更快的 Data 节点复制比速度较慢的节点更多的字节。使用 -strategy dynamic
(在架构中进行了解释),而不是为每个 Map 任务分配一组固定的源文件,而是将文件拆分为多个集合。集合数超过 Map 数,通常为 2-3 倍。每个 Map 都会提取并复制一个块中列出的所有文件。当一个块用尽时,将获取并处理一个新块,直到没有更多块为止。
通过不将源路径分配给固定的 Map,速度更快的 Map 任务(即 Data 节点)能够消耗更多块,从而复制比速度较慢的节点更多的数据。虽然这种分布并不均匀,但它对于每个映射器的容量来说是公平的。
动态策略由 DynamicInputFormat
实现。在大多数情况下,它都能提供卓越的性能。
建议根据源集群和目标集群的大小、复制的大小以及可用带宽调整 Map 的数量,以用于长时间运行和定期运行作业。
对于在两个不同主要版本的 Hadoop(例如 1.X 和 2.X)之间进行复制,通常会使用 WebHdfsFileSystem。与之前的 HftpFileSystem 不同,由于 webhdfs 可用于读写操作,因此可以在源集群和目标集群上运行 DistCp。远程集群指定为 webhdfs://<namenode_hostname>:<http_port>
。在同一主要版本的 Hadoop 集群(例如 2.X 和 2.X)之间进行复制时,使用 hdfs 协议以获得更好的性能。
当 webhdfs 使用 SSL 保护时,请使用“swebhdfs://
”方案。有关更多信息,请参阅 SWebHDFS 的 SSL 配置。
如前所述,如果一个映射未能复制其一个输入,将出现几个副作用。
-overwrite
,否则在重新执行时由先前映射成功复制的文件将标记为“已跳过”。mapreduce.map.maxattempts
次,将终止剩余的映射任务(除非设置 -i
)。mapreduce.map.speculative
设置为最终且为真,则复制结果未定义。DistCp 可用于对象存储,例如 Amazon S3、Azure ABFS 和 Google GCS。
先决条件
DistCp 可用于上传数据
hadoop distcp -direct hdfs://nn1:8020/datasets/set1 s3a://bucket/datasets/set1
下载数据
hadoop distcp s3a://bucket/generated/results hdfs://nn1:8020/results
在对象存储之间复制数据
hadoop distcp s3a://bucket/generated/results \ wasb://[email protected]
以及在对象存储内复制数据
hadoop distcp wasb://[email protected]/current \ wasb://[email protected]/old
以及使用 -update
仅复制已更改的文件。
hadoop distcp -update -numListstatusThreads 20 \ s3a://history/2016 \ hdfs://nn1:8020/history/2016
由于对象存储列出文件的速度较慢,因此请考虑在对大型目录树执行 -update
操作时设置 -numListstatusThreads
选项(限制为 40 个线程)。
当 DistCp -update
与对象存储一起使用时,通常只比较各个文件的修改时间和长度,而不比较任何校验和(如果两个存储之间的校验和算法不同)。
具有不同校验和算法的两个对象存储之间的 distcp -update
会比较源文件和目标文件的修改时间以及文件大小,以确定是否跳过文件复制。该行为由属性 distcp.update.modification.time
控制,该属性默认设置为 true。如果源文件的修改时间比目标文件新,则假定内容已更改,并且应更新文件。我们需要确保机器之间没有时钟偏差。大多数对象存储确实具有目录的有效时间戳这一事实无关紧要;只比较文件时间戳。但是,重要的是让客户端计算机的时钟接近基础设施的时钟,以便在客户端/HDFS 群集和对象存储之间保持时间戳一致。否则,可能会错过/过于频繁地复制已更改的文件。
distcp.update.modification.time
仅在两个存储库均没有校验和验证,导致两个存储库之间的校验和比较不兼容时才使用。即使将该属性设置为 true,如果两个存储库之间存在有效的校验和比较,也不会使用该属性。
若要关闭修改时间检查,请在 core-site.xml 中进行以下设置
<property> <name>distcp.update.modification.time</name> <value>false</value> </property>
注释
-atomic
选项会导致临时数据重命名,从而显著增加操作结束时提交工作的时间。此外,由于除(可选)wasb://
之外的对象存储库不提供目录的原子重命名,因此 -atomic
操作实际上并未兑现承诺。避免。
不支持 -append
选项。
不支持 -diff
和 rdiff
选项
无论 -skipCrc
标志的值如何,都不会执行 CRC 检查。
通常会忽略所有 -p
选项,包括保留权限、用户和组信息、属性校验和和复制的选项。wasb://
连接器将保留信息,但不会强制执行权限。
某些对象存储库连接器提供输出内存缓冲选项,例如 S3A 连接器。在复制大文件时使用此类选项可能会触发某种形式的内存不足事件,无论是堆栈溢出还是 YARN 容器终止。如果群集和对象存储库之间的网络带宽受限(例如在处理远程对象存储库时),这种情况尤其常见。最好禁用/避免此类选项并依赖磁盘缓冲。
即使对象存储库在内部实施了更有效的 COPY 操作,单个对象存储库内的复制操作仍会在 Hadoop 群集中进行
也就是说,诸如
hadoop distcp s3a://bucket/datasets/set1 s3a://bucket/datasets/set2
这样的操作将每个字节复制到 Hadoop 工作节点,然后复制回存储桶。除了速度慢之外,还意味着可能会产生费用。
可以使用 -direct
选项直接写入对象存储库目标路径,从而避免可能非常昂贵的临时文件重命名操作(否则会发生此类操作)。
为什么 -update 不会在已存在的目标目录下创建父源目录? -update
和 -overwrite
的行为已在此文档的使用部分中进行了详细描述。简而言之,如果任一选项与已存在的目标目录一起使用,则会复制每个源目录的内容,而不是源目录本身。此行为也与旧版 DistCp 实现保持一致。
新 DistCp 在语义上与旧版 DistCp 有何不同?
为什么新 DistCp 使用的地图比旧版 DistCp 多? 旧版 DistCp 通过在启动复制作业之前找出实际上需要复制到目标的文件来工作,然后启动复制所需的地图。因此,如果需要跳过大多数文件(例如,因为它们已经存在),则需要的地图会更少。因此,设置中花费的时间(即在 M/R 作业之前)会更长。新 DistCp 仅计算源路径的内容。它不会尝试过滤掉可以跳过的文件。此决策被推迟到 M/R 作业运行时。这在执行时间方面要快得多,但启动的地图数将如 -m
选项中指定的那样,或者在未指定时为 20(默认值)。
为什么在指定更多地图时 DistCp 不会运行得更快? 目前,DistCp 的最小工作单元是一个文件。即,一个文件仅由一个地图处理。将地图数增加到超过文件数的值不会产生任何性能优势。启动的地图数将等于文件数。
为什么 DistCp 会耗尽内存? 如果从源路径复制的各个文件/目录数量非常大(例如 1,000,000 个路径),则 DistCp 在确定要复制的路径列表时可能会耗尽内存。这并不是新 DistCp 实现独有的问题。要解决此问题,请考虑更改 -Xmx
JVM 堆大小参数,如下所示
bash$ export HADOOP_CLIENT_OPTS="-Xms64m -Xmx1024m" bash$ hadoop distcp /source /target