文章目录

  • 前言
  • Nfly link模式的由来
  • Nfly link实现细节分析
  • 引用

前言


在多集群模式下,为了保证数据的一定冗余性要求,我们有时会跨集群或跨data center去备份一些重要的数据。这样可以避免某天一旦一个cluster或者data center处于不可用状态时,从而影响集群正常的数据服务。如果在不额外实现此功能代码的情况下,我们可以采用简单直接的Distcp工具来做集群间的数据拷贝。不过这种方式无法做到实时的数据replication,我们可以按照实际的使用场景做到一天同步一次或者小时级别的同步。不过本文笔者要介绍与此相关的一个重要的ViewFs的新特性:Nfly link模式。

Nfly link模式的由来


社区在JIRA HADOOP-12077:Provide a multi-URI replication Inode for ViewFs 中提出了在ViewFs模式下能够通过多uri地址的方式做跨集群的replication。而这里提到的multi-URL mount point link即Nfly模式,这里的N指的是N个data center。

在社区JIRA里,将数据冗余备份在不同的data center(cluster)里,保持high availability是一方面,可以到时出问题时应以可以做failover到下一个URI地址读写数据。另外一方面,它还能够减少用户application的迁移可能带来的影响。比如用户的application原本运行在cluster1上,它会优先于读取cluster1上的C数据。但是当某一天,这个用户的application migration到cluster2上后,这个时候它则会优先读取cluster2上的数据了。这里其实就避免了跨集群读取数据的过程了,概括来说,多集群备份数据还具有data access locality的好处。

OK,下面我们来深入地了解下ViewFs Nfly模式的实现细节部分的内容。

Nfly link实现细节分析


Nfly link在实现上属于ViewFs的一种新的mount point类型,相比较原始mount point 一个path对一个single cluster地址的方式,Nfly则是对应对个地址,类似如下配置所示:

 <property><name>fs.viewfs.mounttable.global.linkNfly../ads</name><value>uri1,uri2,uri3</value></property>

实际样例配置如下:

  <property><name>fs.viewfs.mounttable.global.linkNfly.minReplication=3,readMostRecent=true,repairOnRead=false./ads</name><value>hdfs://datacenter-east/ads,hdfs://datacenter-west/ads,s3a://models-us-west/ads,file:/tmp/ads</value></property>

其次如上所示,在Nfly内部实现中,还能够支持下列特殊属性要求:

  • minReplication,最少副本数要求,replication出去的文件要达到规定成功的数量才能算写成功文件。
  • readMostRecent,读文件时选取一个最近一次被更改的文件数据。
  • repairOnRead,如果副本间的数据状态不一致,则在读操作的时候做检查并进行恢复(从其它健康的副本数据拷贝覆盖掉stale的那个副本数据)。
  • 如果不考虑上述read属性,Nfly模式在读文件操作时会返回client一个最closest的副本数据,根据传入topology位置来做。

注意上面提到的副本和replication指的是夸集群location的replication,副本,可不是单集群内的。

当Nfly模式从开始写一个文件到最后文件关闭的时候,它基于的流程如下图所示:

上述C@DC1和C@DC2代表不同集群地址。

1)第一步,会创建对应不同集群的FSOutputStream实例对象,创建BitSet来记录每个FSOutputStream的数据写出情况,为了做后面minReplication判断用。
2)创建nfly tmp文件路径为临时写出路径,后续的数据写出都是先写到这个带有nfly tmp标识的路径下。
3)每次执行数据写出操作时,依次遍历写出数据到上述FSOutputStream中去,如果失败了则标记BitSet对应为false。
4) 关闭文件时,如果发现BitSet中记录的成功数达到minReplication时,则进行nfly tmp路径的rename操作,rename成最后的原目录路径名称。随后统一这些replication file的last modified time。否则,删除nfly tmp临时文件,意为这次写过程失败了。这个阶段为Nfly写过程的commit阶段。

相关代码如下:

  private final class NflyOutputStream extends OutputStream {// 实际路径,最终需要rename成的路径private final Path nflyPath;// 临时路径private final Path tmpPath;// broadcast setprivate final FSDataOutputStream[] outputStreams;// status set: 1 working, 0 problemprivate final BitSet opSet;private final boolean useOverwrite;private NflyOutputStream(Path f, FsPermission permission, boolean overwrite,int bufferSize, short replication, long blockSize,Progressable progress) throws IOException {nflyPath = f;tmpPath = getNflyTmpPath(f);// 初始化多个FSDataOutputStream,每个FSDataOutputStream对应所属独立的集群地址outputStreams = new FSDataOutputStream[nodes.length];for (int i = 0; i < outputStreams.length; i++) {outputStreams[i] = nodes[i].fs.create(tmpPath, permission, true,bufferSize, replication, blockSize, progress);}// BitSet用来标记写数据是否写成功的标记位opSet = new BitSet(outputStreams.length);opSet.set(0, outputStreams.length);useOverwrite = false;}

其中的write方法如下所示:

    @Overridepublic void write(byte[] bytes, int offset, int len) throws IOException {final List<IOException> ioExceptions = new ArrayList<IOException>();// 遍历outputStreams,进行数据的写出for (int i = opSet.nextSetBit(0);i >= 0;i = opSet.nextSetBit(i + 1)) {try {outputStreams[i].write(bytes, offset, len);} catch (Throwable t) {// 写第i个outputStreams过程出错,则进行此位置下标BitSet的重置osException(i, "write", t, ioExceptions);}}// 这期间如果已经达不到最小副本数要求,则抛出IO异常mayThrow(ioExceptions);}

close/commit阶段过程的代码:

    @Overridepublic void close() throws IOException {final List<IOException> ioExceptions = new ArrayList<IOException>();for (int i = opSet.nextSetBit(0);i >= 0;i = opSet.nextSetBit(i + 1)) {try {outputStreams[i].close();} catch (Throwable t) {osException(i, "close", t, ioExceptions);}}// 如果close后,没有达到最小副本数要求,则清理nfly tmp文件if (opSet.cardinality() < minReplication) {cleanupAllTmpFiles();throw new IOException("Failed to sufficiently replicate: min="+ minReplication + " actual=" + opSet.cardinality());} else {// 如果达到最小副本要求,则进行rename操作commit();}}private void cleanupAllTmpFiles() throws IOException {for (int i = 0; i < outputStreams.length; i++) {try {nodes[i].fs.delete(tmpPath);} catch (Throwable t) {processThrowable(nodes[i], "delete", t, null, tmpPath);}}}private void commit() throws IOException {final List<IOException> ioExceptions = new ArrayList<IOException>();for (int i = opSet.nextSetBit(0);i >= 0;i = opSet.nextSetBit(i + 1)) {final NflyNode nflyNode = nodes[i];try {// 是否需要overwriteif (useOverwrite) {nflyNode.fs.delete(nflyPath);}// 进行rename操作nflyNode.fs.rename(tmpPath, nflyPath);} catch (Throwable t) {osException(i, "commit", t, ioExceptions);}}if (opSet.cardinality() < minReplication) {// cleanup should be done outside. If rename failed, it's unlikely that// delete will work either. It's the same kind of metadata-only op//throw MultipleIOException.createIOException(ioExceptions);}// 获取当前时间作为这些 replication file的last modified timefinal long commitTime = System.currentTimeMillis();for (int i = opSet.nextSetBit(0);i >= 0;i = opSet.nextSetBit(i + 1)) {try {nodes[i].fs.setTimes(nflyPath, commitTime, commitTime);} catch (Throwable t) {LOG.info("Failed to set timestamp: " + nodes[i] + " " + nflyPath);}}}}

readMostRecent相关代码如下:

  public FSDataInputStream open(Path f, int bufferSize) throws IOException {// TODO proxy stream for readsfinal List<IOException> ioExceptions =new ArrayList<IOException>(nodes.length);int numNotFounds = 0;final MRNflyNode[] mrNodes = workSet();// naively iterate until one can be opened...// 如果指定了readMostRecent,则会进行last modified time的排序// 未指定则会按照topology distance排序,if (nflyFlags.contains(NflyKey.readMostRecent)) {// sort from most recent to least recentArrays.sort(mrNodes);}final FSDataInputStream fsdisAfterRepair = repairAndOpen(mrNodes, f,bufferSize);if (fsdisAfterRepair != null) {return fsdisAfterRepair;}mayThrowFileNotFound(ioExceptions, numNotFounds);throw MultipleIOException.createIOException(ioExceptions);}

上述只是部分代码的细节,更多细节可以阅读源码相关类NflyFSystem。

引用


[1].https://issues.apache.org/jira/browse/HADOOP-12077
[2].https://blog.twitter.com/engineering/en_us/a/2015/hadoop-filesystem-at-twitter.html

Hadoop ViewFs的多Replication模式:Nfly link模式相关推荐

  1. Hadoop ViewFs允许hdfs schema的重载

    文章目录 前言 Hadoop ViewFs的问题痛点 Hadoop ViewFs的重载hdfs schema方式 ViewFs的mount point中心化管理问题 引用 前言 在大数据时代,随着业务 ...

  2. hadoop的安装与配置(伪分布式模式安装)

    最近偶然在研究hadoop看完网络上的配置文章居然在linux下安装成功的hadoop就把别人的文章进行说明一下,提供给需要的读者. hadoop 的安装分为本地模式.伪分布模式.集群模式.本地模式是 ...

  3. Hadoop 的三种运行模式_本地模式_伪分布式模式

    演示的版本是:2.7.2 官方文档 Hadoop运行模式 Hadoop运行模式包括:本地模式.伪分布式模式以及完全分布式模式. Hadoop官方网站:http://hadoop.apache.org/ ...

  4. Solaris IPMP Link模式

    Solaris 10以后IPMP可以配置Link模式. 以Solaris 10位环境,配置 配置网卡ce1,之前该接口没有任何配置,如果有 ifconfig ce1 unplumb 全部清除,如果有子 ...

  5. Apache Hadoop 2.9.2 的YARN High Available 模式部署

    Apache Hadoop 2.9.2 的YARN High Available 模式部署 作者:尹正杰 版权声明:原创作品,谢绝转载!否则将追究法律责任. 一.环境准备 1>.官方文档(htt ...

  6. java基础巩固-宇宙第一AiYWM:为了维持生计,Redis基础Part7(Redis常见使用(部署)方式:单机模式、主从模式、哨兵模式、集群模式)~整起

    Redis持久化:RDB.AOF是针对存储在一台服务器上的数据由于存储过程被打断而发生丢失的情况的.此时,咱们肯定要考虑到,所有鸡蛋都放在一个篮子里是会出问题的. 如果服务器发生了宕机,由于数据恢复是 ...

  7. Redis 高级特性(5)— 集群模式(主从模式、哨兵模式、cluster 集群模式)

    Redis 是如何做到高可用的呢? 它主要通过支持主从模式.哨兵模式.集群模式这三种模式,来满足不同业务特点和可用等级的需求. 其中,主从模式部署最简单,用得也最多,集群模式比较复杂,但可用性最高. ...

  8. 任务和特权级保护(三)——《x86汇编语言:从实模式到保护模式》读书笔记34

    任务和特权级保护(三)--<x86汇编语言:从实模式到保护模式>读书笔记34 5.2.7 在GDT中创建LDT描述符 处理器要求在GDT中安装每个LDT的描述符.当要使用这些LDT时,可以 ...

  9. 32位x86处理器编程导入——《x86汇编语言:从实模式到保护模式》读书笔记08

    在说正题之前,我们先看2个概念. 1.指令集架构(ISA) ISA 的全称是 instruction set architecture,中文就是指令集架构,是指对程序员实际"可见" ...

最新文章

  1. Google 开源的依赖注入库,比 Spring 更小更快!
  2. SpringBoot中提示:Consider marking one of the beans as @Primary, updating the consumer to accept multipl
  3. C语言eigen存为txt文件,如何为使用Eigen(C线性代数的模板库)的C项目编写一个makefile?...
  4. android 支付宝 40247,支付宝 app支付异常摘记 -- ALI40247
  5. as3 访问远程计算机,本地swf不能访问网络的解决办法
  6. 938. 二叉搜索树的范围和
  7. 对话框式activity
  8. 窗口及元素尺寸大小位置获取方法记录
  9. Redis 3.0正式版发布,正式支持Redis集群
  10. android+制作开机动画,Android 开机动画制作详解
  11. 【笔记】深入理解 Java 虚拟机:类文件结构
  12. 如何选择一款好的倾斜摄影相机
  13. 微型计算机日常的应用领域,计算机的应用领域有哪些?
  14. HTTP(9):新增功能协议
  15. 美创科技入选第九届CNCERT网络安全应急服务支撑单位
  16. 有互联网药品信息服务资格证是否可以销售药品?
  17. 关于short 类型
  18. Google advertiser api开发概述
  19. React 生命周期总结(新旧对比)
  20. 应聘者面试的收获-说说工作的正确姿势(20170108)

热门文章

  1. Linux 备忘录
  2. verilog_边学边练习(6) 相邻点累加(时序逻辑电路)
  3. AVI音视频封装格式学习(二)——AVI RIFF文件参考
  4. ue4 怎么修改骨骼动画_【UE4】神器!!!动画师必备!!!基于物理的动画制作软件 Cascadeur 使用指南!...
  5. 阿里巴巴2024届校园招聘
  6. 二手书交易系统设计模式实验
  7. 【跃迁之路】【701天】程序员高效学习方法论探索系列(实验阶段458-2019.1.21)...
  8. 【IoT】产品设计:WIFI 芯片 RTL8710BX 基础资源详解
  9. flash 火狐总是崩溃_Firefox插件Flash经常崩溃的原因及解决方法
  10. Qt下Unix时间10进制格式和实际时间的相互转换