├── .gitignore ├── .gitmodules ├── README.md ├── build.sh ├── build.xml ├── doc └── images │ ├── HDFS_DATA_WRITE_FLOW.PNG │ └── R4H_DATA_WRITE_FLOW.PNG ├── journal.txt ├── manifest.sh └── src ├── common-hadoop_2.3_2.4 └── src │ └── com │ └── mellanox │ └── r4h │ ├── DataXceiver.java │ ├── DistributedFileSystem.java │ └── R4HPipelineAck.java ├── common-hadoop_2.5_2.6 └── src │ └── com │ └── mellanox │ └── r4h │ └── R4HPipelineAck.java ├── common └── src │ ├── com │ └── mellanox │ │ └── r4h │ │ ├── AsyncFileOutputStream.java │ │ ├── BRBInetrface.java │ │ ├── ByteBufferInputStream.java │ │ ├── ByteBufferOutputStream.java │ │ ├── CorruptFileBlockIterator.java │ │ ├── DataXceiverBase.java │ │ ├── DataXceiverServer.java │ │ ├── DummyDataOutputStream.java │ │ ├── IOBufferSupplier.java │ │ ├── JXIOClientResource.java │ │ ├── JXOutputStream.java │ │ ├── MessageAction.java │ │ ├── PacketMessageContext.java │ │ ├── R4HDatanodePlugin.java │ │ ├── R4HEventHandler.java │ │ ├── R4HExecutor.java │ │ ├── R4HProtocol.java │ │ ├── ServerPortalWorker.java │ │ └── WriteOprHeader.java │ └── org │ └── apache │ └── hadoop │ └── hdfs │ ├── BlockStorageLocationUtilBridge.java │ ├── ClientBridge.java │ ├── DFSClientConfBridge.java │ └── server │ └── datanode │ ├── DataNodeBridge.java │ └── R4HBlockReceiver.java ├── r4h-cdh5.1.2 ├── src │ ├── com │ │ └── mellanox │ │ │ └── r4h │ │ │ ├── DFSClient.java │ │ │ ├── DFSInputStream.java │ │ │ ├── DFSOutputStream.java │ │ │ ├── LeaseRenewer.java │ │ │ └── client │ │ │ ├── HdfsDataInputStream.java │ │ │ └── HdfsDataOutputStream.java │ └── org │ │ └── apache │ │ └── hadoop │ │ └── hdfs │ │ └── server │ │ └── datanode │ │ └── R4HBlockReceiverBase.java └── test │ ├── com │ └── mellanox │ │ └── r4h │ │ ├── MiniDFSCluster.java │ │ ├── MiniDFSClusterManager.java │ │ ├── TestFSOutputSummer.java │ │ ├── TestHFlush.java │ │ ├── TestReadWhileWriting.java │ │ └── TestWriteRead.java │ ├── hdfs │ └── server │ │ └── datanode │ │ └── fsdataset │ │ └── impl │ │ └── FsDataSetBridge.java │ └── org │ └── apache │ └── hadoop │ └── hdfs │ └── MiniDFSClusterBridge.java ├── r4h-cdh5.3.1 ├── src │ ├── com │ │ └── mellanox │ │ │ └── r4h │ │ │ ├── DFSClient.java │ │ │ ├── DFSInputStream.java │ │ │ ├── DFSOutputStream.java │ │ │ ├── DataXceiver.java │ │ │ ├── DistributedFileSystem.java │ │ │ ├── LeaseRenewer.java │ │ │ ├── WriteOprHeader2_5.java │ │ │ └── client │ │ │ ├── HdfsDataInputStream.java │ │ │ └── HdfsDataOutputStream.java │ └── org │ │ └── apache │ │ └── hadoop │ │ └── hdfs │ │ ├── DFSInotifyEventInputStreamBridge.java │ │ └── server │ │ └── datanode │ │ └── R4HBlockReceiverBase.java └── test │ ├── com │ └── mellanox │ │ └── r4h │ │ ├── MiniDFSCluster.java │ │ └── MiniDFSClusterManager.java │ └── org │ └── apache │ └── hadoop │ └── hdfs │ └── MiniDFSClusterBridge.java ├── r4h-hdp2.1.2 ├── src │ ├── com │ │ └── mellanox │ │ │ └── r4h │ │ │ ├── DFSClient.java │ │ │ ├── DFSInputStream.java │ │ │ ├── DFSOutputStream.java │ │ │ ├── LeaseRenewer.java │ │ │ └── client │ │ │ ├── HdfsDataInputStream.java │ │ │ └── HdfsDataOutputStream.java │ └── org │ │ └── apache │ │ └── hadoop │ │ └── hdfs │ │ └── server │ │ └── datanode │ │ └── R4HBlockReceiverBase.java └── test │ └── TBD ├── r4h-hdp2.2.0 ├── src │ ├── com │ │ └── mellanox │ │ │ └── r4h │ │ │ ├── DFSClient.java │ │ │ ├── DFSInputStream.java │ │ │ ├── DFSOutputStream.java │ │ │ ├── DataXceiver.java │ │ │ ├── DistributedFileSystem.java │ │ │ ├── LeaseRenewer.java │ │ │ ├── WriteOprHeader2_6.java │ │ │ └── client │ │ │ ├── HdfsDataInputStream.java │ │ │ └── HdfsDataOutputStream.java │ └── org │ │ └── apache │ │ └── hadoop │ │ └── hdfs │ │ ├── DFSInotifyEventInputStreamBridge.java │ │ └── server │ │ └── datanode │ │ └── R4HBlockReceiverBase.java └── test │ └── TBD └── r4h-hdp2.3.0 ├── src ├── com │ └── mellanox │ │ └── r4h │ │ ├── DFSClient.java │ │ ├── DFSInputStream.java │ │ ├── DFSOutputStream.java │ │ ├── DataXceiver.java │ │ ├── DistributedFileSystem.java │ │ ├── LeaseRenewer.java │ │ ├── R4HPipelineAck.java │ │ ├── WriteOprHeader2_7.java │ │ └── client │ │ ├── HdfsDataInputStream.java │ │ └── HdfsDataOutputStream.java └── org │ └── apache │ └── hadoop │ └── hdfs │ ├── DFSClientConfBridge2_7.java │ ├── DFSInotifyEventInputStreamBridge.java │ └── server │ └── datanode │ └── R4HBlockReceiverBase.java └── test └── TBD /.gitignore: -------------------------------------------------------------------------------- 1 | build 2 | .classpath 3 | .project 4 | bin 5 | 6 | 7 | 8 | -------------------------------------------------------------------------------- /.gitmodules: -------------------------------------------------------------------------------- 1 | [submodule "jxio"] 2 | path = jxio 3 | url = https://github.com/accelio/JXIO 4 | ignore = dirty 5 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # This repository is DEPRECATED. 2 | 3 | # R4H 4 | RDMA for HDFS (R4H) is a plugin for Hadoop Distributed FileSystem (HDFS) which accelerates HDFS by using RDMA (Remote Direct Memory Access) technology. 5 | R4H enables HDFS write operations over RDMA and is tested using Mellanox ConnectX® interconnect for Ethernet and InfiniBand fabrics. 6 | 7 | 8 | ### Architecture Highlights 9 | The R4H plugin works side-by-side with other HDFS communication layers, and does not replace or intervene with the TCP activity and other HDFS core tasks. 10 | 11 | You can choose to use: 12 | * The existing HDFS over TCP 13 | * The faster R4H with RDMA, to get faster writes with higher bandwidth and lower CPU utilization 14 | 15 | Upon startup, every DataNode loads R4H plugin in addition to standard HDFS code. 16 | If the client application uses the standard HDFS jar, the R4H plugin does not process any data transfer. The client DFS will use the standard TCP connection to that respectful DataNode according to the Namenode's pipeline. 17 | When the client application uses the R4H plugin jar, the connection is initiated over RDMA using JXIO framework to the DataNode. With R4H the client utilizes RDMA connectivity for all write operations. All other client communication uses the TCP/IP connectivity. Similarly, the server R4H plugin process all incoming write operations from the clients. All other communications between the server and Namenode are handled by the TCP stack. 18 | 19 | R4H provides significant CPU utilization by using Mellanox ConnectX® interconnect for Ethernet and InfiniBand fabrics to transfer the data over RDMA (Remote Direct Memory Access) technology with kernel bypass and zero-copy. 20 | 21 | #### More design details [here] (https://github.com/Mellanox/r4h/wiki) 22 | -------------------------------------------------------------------------------- /build.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # COLORS ARE GOOD FOR YOU 4 | 5 | export NONE='\033[00m' 6 | export RED='\033[01;31m' 7 | export GREEN='\033[01;32m' 8 | export YELLOW='\033[01;33m' 9 | export PURPLE='\033[01;35m' 10 | export CYAN='\033[01;36m' 11 | export WHITE='\033[01;37m' 12 | export BOLD='\033[1m' 13 | export UNDERLINE='\033[4m' 14 | 15 | 16 | prefix="$(basename $0)" 17 | 18 | which ant &> /dev/null 19 | 20 | if (($? != 0)); then 21 | echo -e "$prefix: ${RED}ERROR Please add ant to one of your PATH directories and try again.${NONE}" 22 | exit 1 23 | fi 24 | 25 | echo -e "$prefix: ${UNDERLINE}Cleaning old JAR...${NONE}" 26 | echo "$prefix: ant clean" 27 | ant clean &> /dev/null 28 | 29 | echo -e "$prefix: ${UNDERLINE}Preparing JXIO submodule...${NONE}" 30 | echo "$prefix: git submodule update --init" 31 | git submodule update --init 32 | 33 | echo -e "$prefix: ${UNDERLINE}Building JXIO...${NONE}" 34 | echo "$prefix: cd jxio" 35 | cd jxio 36 | echo "$prefix: bash build.sh" 37 | bash build.sh 38 | 39 | if (($? != 0)); then 40 | echo -e "$prefix: ${RED}ERROR Building JXIO - Exiting${NONE}" 41 | exit 1 42 | fi 43 | 44 | echo -e "$prefix: ${UNDERLINE}Building HDFS JAR...${NONE}" 45 | echo "$prefix: cd .." 46 | cd .. 47 | echo "$prefix: ant $1" 48 | ant $1 49 | 50 | if (($? != 0)); then 51 | echo -e "$prefix: ${RED}ERROR Building HDFS JAR - Exiting${NONE}" 52 | exit 1 53 | fi 54 | 55 | echo -e "$prefix: ${GREEN}DONE${NONE}" 56 | echo "$prefix: JXIO JAR available at: `pwd`/jxio/bin/jxio.jar" 57 | echo "$prefix: HDFS JAR available at: `pwd`/build/r4h.jar" 58 | 59 | -------------------------------------------------------------------------------- /build.xml: -------------------------------------------------------------------------------- 1 | 2 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | -------------------------------------------------------------------------------- /doc/images/HDFS_DATA_WRITE_FLOW.PNG: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Mellanox/R4H/db73b0ebceb55cc73691685a76933eb5b6b707f8/doc/images/HDFS_DATA_WRITE_FLOW.PNG -------------------------------------------------------------------------------- /doc/images/R4H_DATA_WRITE_FLOW.PNG: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Mellanox/R4H/db73b0ebceb55cc73691685a76933eb5b6b707f8/doc/images/R4H_DATA_WRITE_FLOW.PNG -------------------------------------------------------------------------------- /journal.txt: -------------------------------------------------------------------------------- 1 | # 2 | # R4H Change Log 3 | # 4 | 5 | ================================ 6 | Version: 1.3 - 2015-04-26 7 | ================================ 8 | 9 | NEW FEATURES: 10 | 11 | Feature #511015 - Support CDH v5.3.0 and CDH v5.3.1 12 | 13 | Feature #492233 - Support HDP v2.2.0 14 | 15 | Feature #492230 - Improve scalability with new IO thread model 16 | 17 | FIXED: 18 | 19 | Bug #492342 - YCSB (HBASE benchmark) hangs when using r4h plug-in 20 | 21 | KNOWN LIMITATIONS: 22 | 23 | Same limitations as in v1.2 24 | 25 | ================================ 26 | Version: 1.2 - 2014-12-31 27 | ================================ 28 | 29 | CHANGES: 30 | 31 | Stability and failure handling improvments 32 | 33 | Task #425202 - Improved streamer's batch processing logic 34 | 35 | Task #449726 - Avoid allocating new IO thread on clean flow 36 | 37 | Task #449728 - Workers return to pool earlier, after sending last reply 38 | 39 | FIXED: 40 | 41 | Bug #428539 - Client excludes all datanodes upon failure 42 | 43 | Bug #425912 - Large job fails due to absence of getAdditionalMsgPool callback 44 | 45 | Bug #425948 - No cleaning of async tasks submitted for IO thread on failure flow 46 | 47 | Bug #448205 - memory leak messages in Hive shell on exit when R4H is enabled 48 | 49 | Bug #431417 - Accelio resource leak on YARN's Application Master 50 | 51 | Bug #429514 - Different callbacks share the same client session 52 | 53 | Bug #424522 - User object leak on client side 54 | 55 | Bug #468918 - Header ack timeout on first request of 0.1% of new block sessions (Accelio bugs fix) 56 | 57 | 58 | KNOWN LIMITATIONS: 59 | 60 | Not supported by applications which explicitly require org.apache.hadoop.hdfs.DistributedFileSystem 61 | 62 | - HBase master fails to boot with R4H when HDFS is in safe mode 63 | Work around: wait for HDFS to leave safe mode and only then start Hbase 64 | 65 | - Impala 66 | 67 | - DFSAdmin 68 | 69 | KNOWN ISSUES: 70 | 71 | Risk of unexpected behavior for applications that fork after allocating JXIO/Accelio/RDMA resources, though such applications were not observed 72 | 73 | ================================ 74 | Version: 1.1 - 2014-09-22 75 | ================================ 76 | 77 | NEW FEATURES: 78 | 79 | Feature #389935 - Move to JXIO forward model 80 | 81 | Feature #398747 - Support CDH v5.0.2 82 | 83 | Feature #398748 - Support HDP v2.1.2 84 | 85 | Feature #398759 - Support more than 100 parallel files and overall jobs bigger than 1TB 86 | 87 | Feature #419484 - Support CDH v5.1.2 88 | 89 | FIXED: 90 | 91 | Bug #389599 - Network thread crashes because of NullPointerException 92 | 93 | Bug #416334 - Memory leak when DataXceivers aren't removed from list 94 | 95 | Bug #416382 - Failing to retrieve worker from session->worker hashtable 96 | 97 | Bug #416341 - Race condition on free workers between JXIO and DataXceiverServer 98 | 99 | Bug #423640 - YARN Application Master hangs when using EQH caching 100 | 101 | LIMITATIONS: 102 | 103 | No longer supporting CDH v4.4 104 | 105 | KNOWN ISSUES: 106 | 107 | Client hangs waiting for header ack once in every ~10K connections 108 | 109 | ================================ 110 | Version: 1.0 - 2014-05-07 111 | ================================ 112 | 113 | R4H is a plugin for Hadoop Distributed FileSystem (HDFS) which accelerates HDFS by 114 | using RDMA (Remote Direct Memory Access) technology as a parallel data path to the 115 | current TCP implementation. Currently R4H enables HDFS Write operations over RDMA 116 | using Mellanox ConnectX® interconnect for Ethernet and InfiniBand fabrics. 117 | 118 | LIMITATIONS: 119 | 120 | Currently supports only Cloudera Hadoop (CDH) v4.4 121 | 122 | KNOWN ISSUES: 123 | 124 | Unable to run jobs with more than 100 parallel files and over 1TB overall 125 | 126 | -------------------------------------------------------------------------------- /manifest.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | echo "----------- R4H -----------" 4 | unzip -q -c build/r4h.jar META-INF/MANIFEST.MF 5 | echo "----------- JXIO -----------" 6 | unzip -q -c jxio/bin/jxio.jar META-INF/MANIFEST.MF 7 | -------------------------------------------------------------------------------- /src/common-hadoop_2.3_2.4/src/com/mellanox/r4h/DataXceiver.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; 4 | 5 | import java.io.DataInputStream; 6 | import java.io.DataOutputStream; 7 | import java.io.IOException; 8 | 9 | import org.accelio.jxio.ServerSession.SessionKey; 10 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 11 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 12 | import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; 13 | import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; 14 | import org.apache.hadoop.hdfs.protocolPB.PBHelper; 15 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 16 | 17 | import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto; 18 | 19 | public class DataXceiver extends DataXceiverBase { 20 | 21 | DataXceiver(DataXceiverServer dxcs, ServerPortalWorker spw, SessionKey sKey, R4HExecutor ioExec, R4HExecutor auxExec) { 22 | super(dxcs, spw, sKey, ioExec, auxExec); 23 | } 24 | 25 | @Override 26 | void parseOpWriteBlock(DataInputStream in) throws IOException { 27 | final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in)); 28 | oprHeader = new WriteOprHeader(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convert(proto.getHeader() 29 | .getBaseHeader().getToken()), proto.getHeader().getClientName(), PBHelper.convert(proto.getTargetsList()), PBHelper.convert(proto 30 | .getSource()), BlockConstructionStage.valueOf(proto.getStage().name()), proto.getPipelineSize(), proto.getMinBytesRcvd(), 31 | proto.getMaxBytesRcvd(), proto.getLatestGenerationStamp(), fromProto(proto.getRequestedChecksum()), 32 | (proto.hasCachingStrategy() ? getCachingStrategy(proto.getCachingStrategy()) : CachingStrategy.newDefaultStrategy())); 33 | } 34 | 35 | @Override 36 | void senderWriteBlock(DataOutputStream out, ExtendedBlock origBlk) throws IOException { 37 | new Sender(out).writeBlock(origBlk, oprHeader.getBlockToken(), oprHeader.getClientName(), oprHeader.getTargets(), oprHeader.getSrcDataNode(), 38 | oprHeader.getStage(), oprHeader.getPipelineSize(), oprHeader.getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), 39 | oprHeader.getLatestGenerationStamp(), oprHeader.getRequestedChecksum(), oprHeader.getCachingStrategy()); 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /src/common-hadoop_2.3_2.4/src/com/mellanox/r4h/R4HPipelineAck.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck; 21 | import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; 22 | 23 | public class R4HPipelineAck extends PipelineAck { 24 | 25 | /** 26 | * Constructor assuming no next DN in pipeline 27 | * @param seqno sequence number 28 | * @param replies an array of replies 29 | */ 30 | public R4HPipelineAck(long seqno, Status[] replies) { 31 | super(seqno, replies); 32 | } 33 | 34 | /** 35 | * Constructor 36 | * @param seqno sequence number 37 | * @param replies an array of replies 38 | * @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline 39 | */ 40 | public R4HPipelineAck(long seqno, Status[] replies, long downstreamAckTimeNanos) { 41 | super(seqno, replies, downstreamAckTimeNanos); 42 | } 43 | 44 | /** default constructor **/ 45 | public R4HPipelineAck() { 46 | super(); 47 | } 48 | 49 | /** 50 | * get the ith reply - The api has been changed since hadoop 2.6 51 | * @return the the ith reply 52 | */ 53 | public Status getReply(int i) { 54 | return super.getReply(i); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/common-hadoop_2.5_2.6/src/com/mellanox/r4h/R4HPipelineAck.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck; 21 | import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; 22 | 23 | public class R4HPipelineAck extends PipelineAck { 24 | 25 | /** 26 | * Constructor assuming no next DN in pipeline 27 | * @param seqno sequence number 28 | * @param replies an array of replies 29 | */ 30 | public R4HPipelineAck(long seqno, Status[] replies) { 31 | super(seqno, replies); 32 | } 33 | 34 | /** 35 | * Constructor 36 | * @param seqno sequence number 37 | * @param replies an array of replies 38 | * @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline 39 | */ 40 | public R4HPipelineAck(long seqno, Status[] replies, long downstreamAckTimeNanos) { 41 | super(seqno, replies, downstreamAckTimeNanos); 42 | } 43 | 44 | /** default constructor **/ 45 | public R4HPipelineAck() { 46 | super(); 47 | } 48 | 49 | /** 50 | * get the ith reply - The api has been changed since hadoop 2.6 51 | * @return the the ith reply 52 | */ 53 | public Status getReply(int i) { 54 | return super.getReply(i); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/AsyncFileOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.io.File; 21 | import java.io.FileDescriptor; 22 | import java.io.FileNotFoundException; 23 | import java.io.FileOutputStream; 24 | import java.io.IOException; 25 | 26 | import org.accelio.jxio.Msg; 27 | 28 | public class AsyncFileOutputStream extends FileOutputStream { 29 | private final R4HExecutor ioExecutor; 30 | private final AsyncWriteCompletion writeCompleteCallBack; 31 | private Object userContext = null; 32 | private long limiterThreadID = 0; 33 | private final MessageAction msgCallbacks; 34 | private Msg currMsg = null; 35 | 36 | public AsyncFileOutputStream(File file, R4HExecutor ioExecutor, MessageAction msgCallbacks, AsyncWriteCompletion writeCompleteCallBack) 37 | throws FileNotFoundException { 38 | super(file); 39 | this.ioExecutor = ioExecutor; 40 | this.writeCompleteCallBack = writeCompleteCallBack; 41 | this.msgCallbacks = msgCallbacks; 42 | } 43 | 44 | public AsyncFileOutputStream(FileDescriptor fd, R4HExecutor ioExecutor, MessageAction msgCallbacks, AsyncWriteCompletion writeCompleteCallBack) 45 | throws FileNotFoundException { 46 | super(fd); 47 | this.ioExecutor = ioExecutor; 48 | this.writeCompleteCallBack = writeCompleteCallBack; 49 | this.msgCallbacks = msgCallbacks; 50 | } 51 | 52 | public void setUserContext(Object userContext) { 53 | this.userContext = userContext; 54 | } 55 | 56 | public void limitAsyncIObyThreadID(long limiterThreadId) { 57 | this.limiterThreadID = limiterThreadId; 58 | } 59 | 60 | public void setCurrMsg(Msg msg) { 61 | this.currMsg = msg; 62 | } 63 | 64 | public class AsyncWrite implements Runnable { 65 | private final byte[] b; 66 | private final int off; 67 | private final int len; 68 | private final Object context; 69 | 70 | public AsyncWrite(Object context, byte[] b, int off, int len) { 71 | this.context = context; 72 | this.b = b; 73 | this.off = off; 74 | this.len = len; 75 | } 76 | 77 | @Override 78 | public void run() { 79 | 80 | try { 81 | AsyncFileOutputStream.super.write(b, off, len); 82 | } catch (IOException e) { 83 | onWriteComplete(this, e); 84 | } catch (Throwable t) { 85 | onWriteComplete(this, new IOException(t)); 86 | } 87 | onWriteComplete(this); 88 | } 89 | 90 | @Override 91 | public String toString() { 92 | return String.format("AsyncWrite{ context='%s', offset='%d', length='%d' }", context, off, len); 93 | } 94 | 95 | } 96 | 97 | @Override 98 | public void write(byte[] b, int off, int len) throws IOException { 99 | if ((limiterThreadID == 0) || (limiterThreadID == Thread.currentThread().getId())) { 100 | ioExecutor.execute(currMsg, msgCallbacks, new AsyncWrite(this.userContext, b, off, len)); 101 | } else { 102 | super.write(b, off, len); 103 | } 104 | } 105 | 106 | @Override 107 | public void write(byte[] b) throws IOException { 108 | if ((limiterThreadID == 0) || (limiterThreadID == Thread.currentThread().getId())) { 109 | ioExecutor.execute(currMsg, msgCallbacks, new AsyncWrite(this, b, 0, b.length)); 110 | } else { 111 | super.write(b, 0, b.length); 112 | } 113 | } 114 | 115 | private void onWriteComplete(AsyncWrite asyncWrite) { 116 | onWriteComplete(asyncWrite, null); 117 | } 118 | 119 | private void onWriteComplete(AsyncWrite asyncWrite, IOException e) { 120 | if (writeCompleteCallBack != null) 121 | writeCompleteCallBack.onWriteComplete(asyncWrite, e); 122 | } 123 | 124 | public static interface AsyncWriteCompletion { 125 | void onWriteComplete(Object context, IOException e); 126 | } 127 | 128 | } 129 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/BRBInetrface.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import java.io.IOException; 4 | 5 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 6 | import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface; 7 | 8 | public interface BRBInetrface { 9 | void closeBlock(); 10 | 11 | ReplicaInPipelineInterface getReplicaInfo(); 12 | 13 | ExtendedBlock getBlock(); 14 | 15 | String getStorageID() throws IOException; 16 | } 17 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/ByteBufferInputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.io.IOException; 21 | import java.io.InputStream; 22 | import java.nio.ByteBuffer; 23 | 24 | /** 25 | * A weak implementation of InputStream to a ByteBuffer. "weak" because it can only read the byteBuffer byte after byte 26 | * instead of using ByteBuffer API to read several bytes at once for primitives types 27 | */ 28 | public class ByteBufferInputStream extends InputStream { 29 | private final ByteBuffer buff; 30 | 31 | public ByteBufferInputStream(ByteBuffer buff) { 32 | this.buff = buff; 33 | 34 | } 35 | 36 | @Override 37 | public int read() throws IOException { 38 | if (!buff.hasRemaining()) { 39 | return -1; 40 | } 41 | 42 | // return buff.get(); // BAD! XXX - must be value=0-255 and casting breaks it 43 | return buff.get() & 0xFF; 44 | } 45 | 46 | @Override 47 | public int read(byte[] bytes, int off, int len) throws IOException { 48 | if (!buff.hasRemaining()) { 49 | return -1; 50 | } 51 | 52 | len = Math.min(len, buff.remaining()); 53 | buff.get(bytes, off, len); 54 | return len; 55 | } 56 | 57 | } 58 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/ByteBufferOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.io.IOException; 21 | import java.io.OutputStream; 22 | import java.nio.ByteBuffer; 23 | 24 | /** 25 | * A weak implementation of OutputStream to a ByteBuffer. "weak" because it can only write to the byteBuffer byte after 26 | * byte instead of using ByteBuffer API to write several bytes at once for primitives types 27 | */ 28 | public class ByteBufferOutputStream extends OutputStream { 29 | 30 | ByteBuffer buff; 31 | 32 | public ByteBufferOutputStream(ByteBuffer buf) { 33 | this.buff = buf; 34 | } 35 | 36 | @Override 37 | public void write(int b) throws IOException { 38 | buff.put((byte) b); 39 | } 40 | 41 | @Override 42 | public void write(byte[] bytes, int off, int len) throws IOException { 43 | buff.put(bytes, off, len); 44 | } 45 | 46 | } 47 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/CorruptFileBlockIterator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.mellanox.r4h; 20 | 21 | import java.io.IOException; 22 | import java.util.NoSuchElementException; 23 | 24 | import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; 25 | import org.apache.hadoop.fs.Path; 26 | import org.apache.hadoop.fs.RemoteIterator; 27 | 28 | /** 29 | * Provides an iterator interface for listCorruptFileBlocks. 30 | * This class is used by DistributedFileSystem and Hdfs. 31 | */ 32 | public class CorruptFileBlockIterator implements RemoteIterator { 33 | private final DFSClient dfs; 34 | private final String path; 35 | 36 | private String[] files = null; 37 | private int fileIdx = 0; 38 | private String cookie = null; 39 | private Path nextPath = null; 40 | 41 | private int callsMade = 0; 42 | 43 | public CorruptFileBlockIterator(DFSClient dfs, Path path) throws IOException { 44 | this.dfs = dfs; 45 | this.path = path2String(path); 46 | loadNext(); 47 | } 48 | 49 | /** 50 | * @return the number of calls made to the DFSClient. 51 | * This is for debugging and testing purposes. 52 | */ 53 | public int getCallsMade() { 54 | return callsMade; 55 | } 56 | 57 | private String path2String(Path path) { 58 | return path.toUri().getPath(); 59 | } 60 | 61 | private Path string2Path(String string) { 62 | return new Path(string); 63 | } 64 | 65 | private void loadNext() throws IOException { 66 | if (files == null || fileIdx >= files.length) { 67 | CorruptFileBlocks cfb = dfs.listCorruptFileBlocks(path, cookie); 68 | files = cfb.getFiles(); 69 | cookie = cfb.getCookie(); 70 | fileIdx = 0; 71 | callsMade++; 72 | } 73 | 74 | if (fileIdx >= files.length) { 75 | // received an empty response 76 | // there are no more corrupt file blocks 77 | nextPath = null; 78 | } else { 79 | nextPath = string2Path(files[fileIdx]); 80 | fileIdx++; 81 | } 82 | } 83 | 84 | @Override 85 | public boolean hasNext() { 86 | return nextPath != null; 87 | } 88 | 89 | @Override 90 | public Path next() throws IOException { 91 | if (!hasNext()) { 92 | throw new NoSuchElementException("No more corrupt file blocks"); 93 | } 94 | 95 | Path result = nextPath; 96 | loadNext(); 97 | 98 | return result; 99 | } 100 | } -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/DataXceiverServer.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.net.URI; 21 | import java.net.URISyntaxException; 22 | import java.util.LinkedList; 23 | import java.util.List; 24 | import java.util.concurrent.ConcurrentLinkedQueue; 25 | import java.util.concurrent.ThreadFactory; 26 | 27 | import org.apache.commons.logging.Log; 28 | import org.apache.commons.logging.LogFactory; 29 | import org.apache.hadoop.hdfs.server.datanode.DataNodeBridge; 30 | import org.accelio.jxio.EventName; 31 | import org.accelio.jxio.EventReason; 32 | import org.accelio.jxio.ServerPortal; 33 | import org.accelio.jxio.WorkerCache; 34 | import org.accelio.jxio.ServerSession.SessionKey; 35 | import org.accelio.jxio.WorkerCache.Worker; 36 | 37 | import com.google.common.util.concurrent.ThreadFactoryBuilder; 38 | 39 | /** 40 | * R4H's parallel class to the original org.apache.hadoop.hdfs.server.datanode.DataXceiverServer 41 | * 42 | * @see org.apache.hadoop.hdfs.server.datanode.DataXceiverServer Implemented with JXIO RDMA library to listen for R4H 43 | * clients connection requests. It opens new sessions for every requests and bind it with JX's Event Queue Handler 44 | * worker thread 45 | */ 46 | class DataXceiverServer extends ServerPortalWorker implements Runnable { 47 | static final Log LOG = LogFactory.getLog(DataXceiverServer.class.getName()); 48 | private final ThreadGroup threadGroup; 49 | final DataNodeBridge dnBridge; 50 | private final boolean isForwardEnable; 51 | private final LinkedList spPool; 52 | private final LinkedList ioExecutorPool; 53 | private final ConcurrentLinkedQueue auxExecutorPool; 54 | private final ThreadFactory auxThreadFactory; 55 | 56 | private class DXSCallbacks implements ServerPortal.Callbacks { 57 | 58 | @Override 59 | public void onSessionEvent(EventName session_event, EventReason reason) { 60 | switch (session_event) { 61 | case SESSION_CLOSED: 62 | LOG.info(String.format("Session event: type=%s reason=%s", session_event, reason)); 63 | break; 64 | case SESSION_ERROR: 65 | case SESSION_REJECT: 66 | LOG.error(String.format("Session event: type=%s reason=%s", session_event, reason)); 67 | break; 68 | default: 69 | break; 70 | } 71 | } 72 | 73 | /** 74 | * @param sesKey 75 | * @param srcIP 76 | */ 77 | @Override 78 | public void onSessionNew(SessionKey sesKey, String srcIP, Worker workerHint) { 79 | LOG.info(String.format("New session request from %s (uri=%s)", srcIP, sesKey.getUri())); 80 | ServerPortalWorker spw; 81 | if (isForwardEnable) { 82 | if ((workerHint != null) && (workerHint instanceof ServerPortalWorker)) { 83 | spw = (ServerPortalWorker) workerHint; 84 | } else { 85 | LOG.error("Got invalid server portal worker cache hint. Getting manually next free worker."); 86 | spw = getNextFreePortalWorker(); 87 | } 88 | } else { 89 | spw = DataXceiverServer.this; 90 | } 91 | 92 | spw.incrementSessionsCounter(); 93 | R4HExecutor ioExecutor = getNextIOExecutor(); 94 | R4HExecutor auxExecutor = getNextFreeAuxillaryExecutor(); 95 | DataXceiverBase dxc = new DataXceiver(DataXceiverServer.this, spw, sesKey, ioExecutor, auxExecutor); 96 | 97 | if (isForwardEnable) { 98 | sp.forward(spw.sp, dxc.getSessionServer()); 99 | if (LOG.isDebugEnabled()) { 100 | LOG.debug("After session forward: spw=" + spw + " , uri=" + dxc.getUri()); 101 | } 102 | } else { 103 | sp.accept(dxc.getSessionServer()); 104 | if (LOG.isDebugEnabled()) { 105 | LOG.debug("After session accept: spw=" + spw + " , uri=" + dxc.getUri()); 106 | } 107 | } 108 | } 109 | 110 | private R4HExecutor getNextFreeAuxillaryExecutor() { 111 | R4HExecutor ans; 112 | if (!auxExecutorPool.isEmpty()) { 113 | ans = auxExecutorPool.poll(); 114 | } else { 115 | LOG.warn("No more auxillary executors available in the pool, allocating a new auxillary executor"); 116 | ans = new R4HExecutor(auxThreadFactory); 117 | } 118 | return ans; 119 | } 120 | 121 | private R4HExecutor getNextIOExecutor() { 122 | R4HExecutor ans = ioExecutorPool.poll(); 123 | ioExecutorPool.add(ans); 124 | return ans; 125 | } 126 | 127 | } 128 | 129 | private ServerPortalWorker getNextFreePortalWorker() { 130 | ServerPortalWorker minSessionsSpw = spPool.getFirst(); 131 | 132 | for (ServerPortalWorker spw : spPool) { 133 | if (spw.isFree()) { 134 | return spw; 135 | } else if (spw.getSessionsCounterValue() < minSessionsSpw.getSessionsCounterValue()) { 136 | minSessionsSpw = spw; 137 | } 138 | } 139 | // if we got here then all spw exceeded the #sessions limit - pick the spw with the lowest #sessions. 140 | return minSessionsSpw; 141 | } 142 | 143 | DataXceiverServer(DataNodeBridge dnBridge) throws URISyntaxException { 144 | super(new URI(String.format("rdma://%s", dnBridge.getDN().getDisplayName())), dnBridge.numOfMsgsToBind, dnBridge.msgInSize, 145 | dnBridge.msgOutSize, dnBridge.numSessionsLimit, 0, dnBridge.dynamicMsgAllocationAmount, !dnBridge.isForwardEnable); 146 | LOG.debug("After EventQueueHandler creation"); 147 | 148 | this.dnBridge = dnBridge; // TODO: remove debug DNexpoable and rename top DataNodeBridge 149 | this.threadGroup = new ThreadGroup("R4H Datanode Threads"); 150 | LOG.info("Creating DataXceiverServer - uri=" + uri); 151 | DataXceiverServer.DXSCallbacks dxsCbs = this.new DXSCallbacks(); 152 | 153 | LOG.trace("writePacketSize=" + dnBridge.getWritePacketSize()); 154 | 155 | URI workerUri = new URI(String.format("rdma://%s:0", this.uri.getHost())); 156 | 157 | this.spPool = new LinkedList(); 158 | 159 | isForwardEnable = dnBridge.isForwardEnable; 160 | 161 | if (isForwardEnable) { 162 | this.sp = new ServerPortal(eqh, uri, dxsCbs, new WorkerCache.WorkerProvider() { 163 | @Override 164 | public Worker getWorker() { 165 | return getNextFreePortalWorker(); 166 | } 167 | }); 168 | 169 | LOG.info("Using forward model"); 170 | int spwAmount = dnBridge.spwAmount; 171 | 172 | LOG.info(String.format("Starting ahead %d server portal worker with sessions limit of %d sessions each", spwAmount, 173 | dnBridge.numSessionsLimit)); 174 | for (int i = 1; i <= spwAmount; i++) { 175 | ServerPortalWorker spw = new ServerPortalWorker(workerUri, dnBridge.numOfMsgsToBind / spwAmount, dnBridge.msgInSize, 176 | dnBridge.msgOutSize, dnBridge.numSessionsLimit, i, dnBridge.dynamicMsgAllocationAmount); 177 | spw.start(); 178 | spPool.add(spw); 179 | LOG.info("Started new server portal worker thread: " + spw); 180 | } 181 | } else { 182 | this.sp = new ServerPortal(eqh, uri, dxsCbs); 183 | 184 | LOG.info("Using accept model"); 185 | LOG.info("Started a new worker thread: " + super.toString()); 186 | } 187 | 188 | auxThreadFactory = new ThreadFactoryBuilder().setNameFormat("r4h-auxillary-thread-%d").build(); 189 | ThreadFactory ioThreadFactory = new ThreadFactoryBuilder().setNameFormat("r4h-io-thread-%d").build(); 190 | 191 | LOG.info(String.format("Allocating ahead %d IO executors", dnBridge.numOfioExecutors)); 192 | this.ioExecutorPool = new LinkedList(); 193 | for (int i = 0; i < dnBridge.numOfioExecutors; i++) { 194 | ioExecutorPool.add(new R4HExecutor(ioThreadFactory)); 195 | } 196 | 197 | LOG.info(String.format("Allocating ahead %d Auxillary executors", dnBridge.numOfAuxExecutors)); 198 | this.auxExecutorPool = new ConcurrentLinkedQueue(); 199 | for (int i = 0; i < dnBridge.numOfAuxExecutors; i++) { 200 | auxExecutorPool.add(new R4HExecutor(auxThreadFactory)); 201 | } 202 | 203 | LOG.trace(this.toString()); 204 | } 205 | 206 | @Override 207 | public void run() { 208 | LOG.debug("Start running event queue handler"); 209 | this.eqh.run(); 210 | LOG.debug("Returned from event queue handler run"); 211 | } 212 | 213 | public void stop() { 214 | LOG.info("Stopping R4H resources"); 215 | 216 | LOG.debug("Closing listener event queue handler"); 217 | eqh.stop(); 218 | 219 | if (isForwardEnable) { 220 | LOG.debug("Closing all workers"); 221 | for (ServerPortalWorker spw : spPool) { 222 | spw.eqh.stop(); 223 | } 224 | } 225 | spPool.clear(); 226 | 227 | LOG.info("Stopped R4H resources"); 228 | } 229 | 230 | @Override 231 | public String toString() { 232 | return String.format("DataXceiverServer{SP='%s', EQH='%s', ThreadPool='%s', URI='%s'}", sp, eqh.toString(), threadGroup.getName(), uri); 233 | } 234 | 235 | void returnAuxillaryExecutortoPool(R4HExecutor auxExecutor, boolean needThreadInit) { 236 | if (LOG.isDebugEnabled()) { 237 | LOG.debug("Returning Auxillary executor to pool :" + auxExecutor); 238 | } 239 | if (needThreadInit) { 240 | LOG.warn("Auxillary thread init requested. Allocating a new executor"); 241 | List remainingTasks = auxExecutor.shutdownNow(); 242 | if (!remainingTasks.isEmpty()) { 243 | LOG.warn(String.format("Shutting down Auxillary thread with %d remaining unexecuted tasks", remainingTasks.size())); 244 | } 245 | auxExecutorPool.add(new R4HExecutor(auxThreadFactory)); 246 | } else { 247 | auxExecutorPool.add(auxExecutor); 248 | } 249 | } 250 | } 251 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/DummyDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.io.DataOutputStream; 21 | import java.io.IOException; 22 | 23 | /* 24 | * This class intended for letting R4H use Vanilla's original BlockReceiver.receivePacket() 25 | * Since we would like to separate networkIO from diskIO+CRCVerify for using multiIO thread pool and the original receivePacket() sends packet to mirror pipeline we need to sends the packet to mirror separatly and let receviePacket() do the diskIO+CRCverify only 26 | * An instance from this class will be provided to original receivePacket to avoid it's internal implementation of sending packets 27 | */ 28 | public class DummyDataOutputStream extends DataOutputStream { 29 | public DummyDataOutputStream() { 30 | super(null); 31 | } 32 | 33 | @Override 34 | public void write(byte[] b, int off, int len) throws IOException { 35 | } 36 | 37 | @Override 38 | public void write(byte[] b) throws IOException { 39 | } 40 | 41 | @Override 42 | public void write(int b) throws IOException { 43 | } 44 | 45 | @Override 46 | public void flush() throws IOException { 47 | } 48 | } -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/IOBufferSupplier.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import java.io.IOException; 4 | import java.nio.ByteBuffer; 5 | import java.util.concurrent.ConcurrentLinkedQueue; 6 | 7 | public class IOBufferSupplier { 8 | private final ConcurrentLinkedQueue ioBufferPool = new ConcurrentLinkedQueue(); 9 | 10 | public IOBufferSupplier(int numOfBuffers, int bufferSize) { 11 | allocateBuffers(numOfBuffers, bufferSize); 12 | } 13 | 14 | public ByteBuffer getBuffer() throws IOException { 15 | ByteBuffer buff = ioBufferPool.poll(); 16 | if (buff == null) { 17 | throw new IOException("No more buffers for async IO"); 18 | } 19 | buff.clear(); 20 | return buff; 21 | } 22 | 23 | public void returnBuffer(ByteBuffer buff) { 24 | if (buff != null) { 25 | buff.clear(); 26 | ioBufferPool.add(buff); 27 | } 28 | } 29 | 30 | public int size() { 31 | return ioBufferPool.size(); 32 | } 33 | 34 | public void allocateBuffers(int numOfBuffers, int bufferSize) { 35 | ByteBuffer buffer = ByteBuffer.allocate(bufferSize * numOfBuffers); 36 | int position = 0; 37 | for (int j = 0; j < numOfBuffers; j++) { 38 | buffer.position(position); 39 | buffer.limit(position + bufferSize); 40 | ioBufferPool.add(buffer.slice()); 41 | position += bufferSize; 42 | } 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/JXIOClientResource.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import java.util.AbstractMap.SimpleEntry; 4 | 5 | import org.accelio.jxio.MsgPool; 6 | import com.mellanox.r4h.R4HEventHandler; 7 | 8 | class JXIOClientResource extends SimpleEntry { 9 | 10 | public JXIOClientResource(R4HEventHandler eqh, MsgPool msgPool) { 11 | super(eqh, msgPool); 12 | } 13 | 14 | R4HEventHandler getEqh() { 15 | return this.getKey(); 16 | } 17 | 18 | MsgPool getMsgPool() { 19 | return this.getValue(); 20 | } 21 | 22 | } 23 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/JXOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.io.IOException; 21 | import java.io.OutputStream; 22 | 23 | import org.apache.commons.logging.LogFactory; 24 | 25 | import org.accelio.jxio.ClientSession; 26 | import org.accelio.jxio.Msg; 27 | import org.accelio.jxio.MsgPool; 28 | 29 | /** 30 | * @todo 31 | * Implement close(). Is used in DFSClient. 32 | * @todo 33 | * Return messages to pool. 34 | */ 35 | public class JXOutputStream extends OutputStream { 36 | Msg msg = null; 37 | private boolean flushed = true; 38 | private final MsgPool msgPool; 39 | private final ClientSession cs; 40 | 41 | /** 42 | * TODO:UPDATE!! Creates a new JXIO output stream to write data and send requests to an underlying Client Session. 43 | * 44 | * @param pool 45 | * An underlying MsgPool to retrieve Msgs to write the data 46 | * @param cs 47 | * An underlying ClinetSession to send requests when current Msg is full or flushed 48 | */ 49 | public JXOutputStream(MsgPool pool, ClientSession cs) { 50 | this.msgPool = pool; 51 | this.cs = cs; 52 | } 53 | 54 | /** 55 | * TODO:UPDATE!! Writes the specified byte to the current underlying Msg's out buffer. 56 | * 57 | * @param b 58 | * the byte to be written. 59 | * @exception IOException 60 | * if an I/O error occurs. 61 | */ 62 | public synchronized void write(int b) throws IOException { 63 | if (this.msg == null) { 64 | this.msg = msgPool.getMsg(); 65 | } 66 | if (!msg.getOut().hasRemaining()) { 67 | throw new IOException("MSG's out buffer is full"); 68 | } 69 | msg.getOut().put((byte) b); 70 | flushed = false; 71 | } 72 | 73 | /** 74 | * TODO:UPDATE!! Writes len bytes from the specified byte array starting at offset off to 75 | * this buffered output stream. 76 | *

77 | * Ordinarily this method stores bytes from the given array into the current underlying Msg's out buffer, flushing the buffer to the underlying 78 | * output by sending a request with the ClientSeesion as needed. If the requested length is at least as large as this stream's buffer, however, 79 | * then this method will flush by sending a request with the ClientSeesion and current Msg. No data will be copied. 80 | * 81 | * @param b 82 | * the data. 83 | * @param off 84 | * the start offset in the data. 85 | * @param len 86 | * the number of bytes to write. 87 | * @exception IOException 88 | * if an I/O error occurs. 89 | */ 90 | public synchronized void write(byte b[], int off, int len) throws IOException { 91 | if (this.msg == null) { 92 | this.msg = msgPool.getMsg(); 93 | } 94 | if (len > msg.getOut().remaining()) { 95 | throw new IOException("No enough space remained on MSG's out buffer"); 96 | // TODO: format msg with numbers 97 | } 98 | 99 | msg.getOut().put(b, off, len); 100 | flushed = false; 101 | } 102 | 103 | /** 104 | * TODO:UPDATE!! Flushes this buffered output stream. This forces any buffered output bytes to be written out to the 105 | * underlying output stream. 106 | * 107 | * @exception IOException 108 | * if an I/O error occurs. 109 | * @see java.io.FilterOutputStream#out 110 | */ 111 | public synchronized void flush() throws IOException { 112 | if (!flushed) { 113 | R4HProtocol.wrappedSendRequest(cs, msg, LogFactory.getLog(JXOutputStream.class)); 114 | msg = null; 115 | flushed = true; 116 | } 117 | } 118 | 119 | /** 120 | * Close the stream. 121 | * Actually it closes the client session. 122 | */ 123 | public synchronized void close() { 124 | cs.close(); 125 | } 126 | 127 | } 128 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/MessageAction.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import org.accelio.jxio.Msg; 4 | 5 | public interface MessageAction { 6 | 7 | public void onMessageAction(Msg msg); 8 | 9 | } 10 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/PacketMessageContext.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import java.nio.ByteBuffer; 4 | import java.util.concurrent.atomic.AtomicInteger; 5 | 6 | import org.accelio.jxio.Msg; 7 | import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; 8 | import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck; 9 | 10 | class PacketMessageContext { 11 | 12 | private final AtomicInteger referenceCounter; 13 | private ByteBuffer ioBuffer; 14 | private PipelineAck messageAck; 15 | private long seqNo; 16 | private long offsetInBlock; 17 | private boolean isLastPkt; 18 | private boolean syncBlock; 19 | 20 | PacketMessageContext() { 21 | this.referenceCounter = new AtomicInteger(0); 22 | } 23 | 24 | ByteBuffer getIOBuffer() { 25 | return this.ioBuffer; 26 | } 27 | 28 | void setPacketData(PacketHeader packetHeader, ByteBuffer ioBuffer) { 29 | this.offsetInBlock = packetHeader.getOffsetInBlock(); 30 | this.seqNo = packetHeader.getSeqno(); 31 | this.syncBlock = packetHeader.getSyncBlock(); 32 | this.isLastPkt = packetHeader.isLastPacketInBlock(); 33 | this.ioBuffer = ioBuffer; 34 | } 35 | 36 | void setMessageAck(PipelineAck messageAck) { 37 | this.messageAck = messageAck; 38 | } 39 | 40 | PipelineAck getMessageAck() { 41 | return this.messageAck; 42 | } 43 | 44 | boolean getSyncBlock() { 45 | return this.syncBlock; 46 | } 47 | 48 | boolean isLastPacketInBlock() { 49 | return this.isLastPkt; 50 | } 51 | 52 | long getSeqno() { 53 | return this.seqNo; 54 | } 55 | 56 | long getUpdatedOffsetInBlock() { 57 | return this.offsetInBlock; 58 | } 59 | 60 | int incrementReferenceCounter() { 61 | return this.referenceCounter.incrementAndGet(); 62 | } 63 | 64 | int decrementReferenceCounter() { 65 | return this.referenceCounter.decrementAndGet(); 66 | } 67 | 68 | static PacketMessageContext getPacketMessageContext(Msg msg) { 69 | return (PacketMessageContext) msg.getUserContext(); 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/R4HDatanodePlugin.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.io.IOException; 21 | import java.net.URISyntaxException; 22 | 23 | import org.accelio.jxio.EventQueueHandler; 24 | import org.apache.commons.logging.Log; 25 | import org.apache.commons.logging.LogFactory; 26 | import org.apache.hadoop.hdfs.server.datanode.DataNode; 27 | import org.apache.hadoop.hdfs.server.datanode.DataNodeBridge; 28 | import org.apache.hadoop.util.Daemon; 29 | import org.apache.hadoop.util.ServicePlugin; 30 | 31 | /** 32 | * R4H implementation for a datanode service plugin to run a R4H serve side by side to HDFS server. Datanode starts 33 | * service plugins during initialization by going through the comma separated list of ServicePlugin implementation 34 | * classes configured on dfs.datanode.plugins parameter 35 | */ 36 | public class R4HDatanodePlugin implements ServicePlugin { 37 | private DataNode dn; 38 | private DataXceiverServer dxs; 39 | private Daemon daemon; 40 | private static Log LOG = LogFactory.getLog(R4HDatanodePlugin.class.getName()); 41 | private DataNodeBridge dnExposer; 42 | 43 | // is not called on datanode 44 | public void close() throws IOException { 45 | stop(); 46 | } 47 | 48 | /* 49 | * Starts a new thread of R4H server. Designed to be called by Datanode as a service. 50 | * 51 | * @param service must be a Datanode instance or Object[] for standalone untitest 52 | * 53 | * @see org.apache.hadoop.util.ServicePlugin#start(java.lang.Object) 54 | */ 55 | public void start(Object service) { 56 | LOG.debug("Starting plugin"); 57 | if (!(service instanceof DataNode)) { 58 | throw new IllegalArgumentException("Unexpected service object type"); 59 | } 60 | LOG.debug("Service object is DataNode"); 61 | dn = (DataNode) service; 62 | dnExposer = new DataNodeBridge(dn); 63 | 64 | try { 65 | DataNodeBridge dnBridge = new DataNodeBridge(dn); 66 | dxs = new DataXceiverServer(dnBridge); 67 | } catch (URISyntaxException e) { 68 | throw new IllegalArgumentException("Failed to create URI for DataXceriverServer", e); 69 | } 70 | daemon = new Daemon(dxs); 71 | daemon.setName(String.format("DataXceiverServer-JXIO-Listener-%d", daemon.getId())); 72 | daemon.start(); 73 | 74 | LOG.info("Started"); 75 | // JXIO version 76 | LOG.info("JXIO version :\t" + EventQueueHandler.class.getPackage().getImplementationVersion()); 77 | // R4H version 78 | LOG.info("R4H verison :\t" + DataXceiver.class.getPackage().getImplementationVersion()); 79 | } 80 | 81 | /* 82 | * Stop R4H server and wait until it closes all it's resources 83 | * 84 | * @see org.apache.hadoop.util.ServicePlugin#stop() 85 | */ 86 | public void stop() { 87 | stop(-1); // Infinite wait for daemon to stop 88 | } 89 | 90 | /** 91 | * Stop R4H server 92 | * 93 | * @param waitForDaemon 94 | * milliseconds to wait for resources to be closed or -1 for infinate wait 95 | */ 96 | public void stop(int waitForDaemon) { 97 | if (waitForDaemon < -1) { 98 | throw new IllegalArgumentException("Illegal (begative) number of milliseconds argument to wait for deamon to stop"); 99 | } 100 | 101 | LOG.debug("Stopping R4H Datanode plugin"); 102 | Daemon dm = new Daemon(new Runnable() { 103 | 104 | @Override 105 | public void run() { 106 | dxs.stop(); 107 | } 108 | }); 109 | dm.start(); 110 | 111 | try { 112 | if (waitForDaemon == -1) { 113 | daemon.join(); 114 | } else { 115 | daemon.join(waitForDaemon); 116 | } 117 | } catch (InterruptedException e) { 118 | LOG.debug("daemon join interrupted. Exception: " + e.toString()); 119 | } 120 | 121 | if (dm.isAlive()) { 122 | LOG.error("timeout waiting for R4H plugin to stop"); 123 | } else { 124 | LOG.info("R4H Datanode plugin stopped"); 125 | } 126 | } 127 | 128 | /** 129 | * @return true if server thread is still alive 130 | */ 131 | public boolean isAlive() { 132 | if (daemon == null) 133 | return false; 134 | 135 | return daemon.isAlive(); 136 | } 137 | 138 | @Override 139 | public String toString() { 140 | if (daemon == null) { 141 | return super.toString() + " is not started yet"; 142 | } else { 143 | return String.format("R4HDatanodePlugin{DN='%s', DXS='%s', isAlive='%s'}", (dn == null) ? "-" : dn.toString(), dxs.toString(), 144 | daemon.isAlive() ? "TRUE" : "FALSE"); 145 | } 146 | } 147 | } 148 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/R4HEventHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.commons.logging.Log; 21 | import org.apache.commons.logging.LogFactory; 22 | import org.apache.hadoop.util.StringUtils; 23 | import org.accelio.jxio.EventQueueHandler; 24 | 25 | public class R4HEventHandler extends EventQueueHandler { 26 | Runnable onBreakEqh; 27 | 28 | /** 29 | * @param callbacks 30 | */ 31 | public R4HEventHandler(Callbacks callbacks, Runnable onBreakEqhFunc) { 32 | super(callbacks); 33 | this.onBreakEqh = onBreakEqhFunc; 34 | } 35 | 36 | public R4HEventHandler(Callbacks callbacks) { 37 | super(callbacks); 38 | } 39 | 40 | private final static Log LOG = LogFactory.getLog(R4HEventHandler.class.getName()); 41 | private boolean stopR4HLoop = false; 42 | private boolean wasEvLoopReturnedAfterStop = false; 43 | 44 | @Override 45 | public void run() { 46 | try { 47 | while (!this.stopR4HLoop) { 48 | int ne = runEventLoop(INFINITE_EVENTS, INFINITE_DURATION); 49 | if (ne == -1) { 50 | LOG.error("JXIO catched exception during event processing"); 51 | if (this.didExceptionOccur()) { 52 | LOG.error(StringUtils.stringifyException(this.getCaughtException())); 53 | } else { 54 | LOG.warn("No exception to retrieve after JXIO indication for exception during event processing"); 55 | } 56 | } 57 | if (stopR4HLoop) { 58 | synchronized (this) { 59 | wasEvLoopReturnedAfterStop = true; 60 | this.notify(); 61 | } 62 | } else { 63 | onBreakEqh(); 64 | } 65 | 66 | } 67 | } catch (Throwable t) { 68 | LOG.fatal("A R4H event handler worker was crashed. " + StringUtils.stringifyException(t)); 69 | } 70 | } 71 | 72 | private void onBreakEqh() { 73 | if (onBreakEqh != null) { 74 | onBreakEqh.run(); 75 | } 76 | } 77 | 78 | @Override 79 | public void breakEventLoop() { 80 | super.breakEventLoop(); 81 | } 82 | 83 | @Override 84 | public void stop() { 85 | 86 | synchronized (this) { 87 | stopR4HLoop = true; 88 | super.stop(); 89 | while (!wasEvLoopReturnedAfterStop) { 90 | try { 91 | wait(); 92 | } catch (InterruptedException e) { 93 | } 94 | } 95 | } 96 | this.close(); 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/R4HExecutor.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import java.util.List; 4 | import java.util.concurrent.ExecutorService; 5 | import java.util.concurrent.Executors; 6 | import java.util.concurrent.ThreadFactory; 7 | 8 | import org.accelio.jxio.Msg; 9 | 10 | public class R4HExecutor { 11 | 12 | private ExecutorService executor; 13 | 14 | R4HExecutor(ThreadFactory threadFactory) { 15 | this.executor = Executors.newSingleThreadExecutor(threadFactory); 16 | } 17 | 18 | public void execute(final Msg msg, final MessageAction action, final Runnable task) { 19 | 20 | final PacketMessageContext pmc = PacketMessageContext.getPacketMessageContext(msg); 21 | 22 | // increment counter 23 | pmc.incrementReferenceCounter(); 24 | 25 | // execute 26 | this.executor.execute(new Runnable() { 27 | 28 | @Override 29 | public void run() { 30 | task.run(); 31 | 32 | // decrement and send 33 | int refCount = pmc.decrementReferenceCounter(); 34 | 35 | if (refCount == 0) { 36 | 37 | action.onMessageAction(msg); 38 | } 39 | } 40 | }); 41 | } 42 | 43 | public void execute(Runnable task) { 44 | this.executor.execute(task); 45 | } 46 | 47 | public List shutdownNow() { 48 | return this.executor.shutdownNow(); 49 | } 50 | 51 | } -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/R4HProtocol.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import java.lang.management.ManagementFactory; 4 | import java.net.InetAddress; 5 | import java.net.URI; 6 | import java.net.URISyntaxException; 7 | import java.net.UnknownHostException; 8 | import java.security.MessageDigest; 9 | import java.security.NoSuchAlgorithmException; 10 | 11 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 12 | 13 | import org.accelio.jxio.exceptions.JxioGeneralException; 14 | import org.accelio.jxio.exceptions.JxioQueueOverflowException; 15 | import org.accelio.jxio.exceptions.JxioSessionClosedException; 16 | import org.accelio.jxio.ServerSession; 17 | import org.accelio.jxio.ClientSession; 18 | import org.accelio.jxio.Msg; 19 | 20 | import org.apache.commons.logging.Log; 21 | 22 | public class R4HProtocol { 23 | public static final int ACK_SIZE = (1024); 24 | public static final int MAX_SEND_PACKETS = 80; 25 | public static final int MAX_DATA_QUEUE_PACKETS = 80; // 80 * 64K = 5MB 26 | public static final int JX_SERVER_SPARE_MSGS = 8; 27 | public static final int JX_BUF_SPARE = 128; 28 | public static final int SERVER_MSG_POOL_SIZE = MAX_SEND_PACKETS + JX_SERVER_SPARE_MSGS; 29 | public static final int MSG_POOLS_GROWTH_FACTOR = 10; 30 | private static final int CLIENT_HASH_LENGTH = 8; // In characters 31 | public static final int CLIENT_MSGPOOL_SPARE = 8; 32 | static final int CLIENT_MSG_POOL_IN_SIZE = ACK_SIZE; 33 | 34 | public static String createSessionHash() { 35 | long tid = Thread.currentThread().getId(); 36 | String pid = ManagementFactory.getRuntimeMXBean().getName(); 37 | String localIP = "127.0.0.0"; 38 | try { 39 | localIP = (InetAddress.getLocalHost()).toString(); 40 | } catch (UnknownHostException e) { 41 | } 42 | long time = System.nanoTime(); 43 | 44 | MessageDigest stringToHash; 45 | String hash = String.format("%d%s%d%s", time, pid, tid, localIP).replaceAll("[^A-Za-z0-9]", ""); 46 | try { 47 | stringToHash = MessageDigest.getInstance("SHA-1"); 48 | String hashInput = String.format("%d%s%d%s", time, pid, tid, localIP); 49 | stringToHash.update(hashInput.getBytes()); 50 | StringBuffer sb = new StringBuffer(); 51 | byte[] mdbytes = stringToHash.digest(); 52 | 53 | for (int i = 0; i < mdbytes.length; i++) { 54 | sb.append(Integer.toString((mdbytes[i] & 0xff) + 0x100, 16).substring(1)); 55 | } 56 | hash = sb.toString().substring(0, CLIENT_HASH_LENGTH); 57 | } catch (NoSuchAlgorithmException e) { 58 | } 59 | 60 | return hash; 61 | } 62 | 63 | public static URI createInitialURI(DatanodeInfo[] targets, String hash) throws URISyntaxException { 64 | String pipeline = createPipelineString(targets); 65 | return new URI(String.format("rdma://%s/?pipeline=%s&clientHash=%s", targets[0].getName(), pipeline, hash)); 66 | } 67 | 68 | public static URI createPipelineURI(DatanodeInfo[] targets, String clientHash) throws URISyntaxException { 69 | String pipeline = createPipelineString(targets); 70 | long tid = Thread.currentThread().getId(); 71 | String pid = ManagementFactory.getRuntimeMXBean().getName(); 72 | 73 | return new URI(String.format("rdma://%s/?pipeline=%s&sourcePID=%s&sourceTID=%d%s", targets[0].getName(), pipeline, pid, tid, clientHash)); 74 | } 75 | 76 | private static String createPipelineString(DatanodeInfo[] targets){ 77 | if (targets == null || targets.length == 0) 78 | return ""; 79 | 80 | StringBuilder b = new StringBuilder(); 81 | int targetsLength = targets.length; 82 | for (int i = 0; i < targetsLength - 1 ; i++) { 83 | b.append(targets[i].getName()).append(":"); 84 | } 85 | return b.append(targets[targetsLength - 1].getName()).toString(); 86 | } 87 | 88 | public static void wrappedSendResponse(ServerSession session, Msg message, Log log) { 89 | try { 90 | session.sendResponse(message); 91 | } catch (JxioSessionClosedException exc) { 92 | log.error("Sending response message failed, session was closed in the middle: " + exc); 93 | } catch (JxioGeneralException exc) { 94 | log.error("Sending response message failed, (general error): " + exc); 95 | } 96 | } 97 | 98 | public static void wrappedSendRequest(ClientSession session, Msg message, Log log) { 99 | try { 100 | session.sendRequest(message); 101 | } catch (JxioSessionClosedException | JxioQueueOverflowException | JxioGeneralException e) { 102 | log.error("Sending request message failed: " + e); 103 | } 104 | } 105 | 106 | } 107 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/ServerPortalWorker.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import java.net.URI; 21 | import java.util.List; 22 | import java.util.concurrent.ConcurrentLinkedQueue; 23 | import java.util.concurrent.atomic.AtomicInteger; 24 | import java.util.concurrent.atomic.AtomicLong; 25 | 26 | import org.apache.commons.logging.Log; 27 | import org.apache.commons.logging.LogFactory; 28 | import org.apache.hadoop.util.StringUtils; 29 | import org.accelio.jxio.ClientSession; 30 | import org.accelio.jxio.EventQueueHandler; 31 | import org.accelio.jxio.Msg; 32 | import org.accelio.jxio.MsgPool; 33 | import org.accelio.jxio.ServerPortal; 34 | import org.accelio.jxio.ServerSession; 35 | import org.accelio.jxio.EventQueueHandler.Callbacks; 36 | import org.accelio.jxio.WorkerCache.Worker; 37 | 38 | public class ServerPortalWorker implements Worker { 39 | private static final Log LOG = LogFactory.getLog(ServerPortalWorker.class.getName()); 40 | private Thread th = null; 41 | protected final EventQueueHandler eqh; 42 | protected ServerPortal sp = null; 43 | protected final URI uri; 44 | private final IOBufferSupplier ioBufferSupplier; 45 | private final ConcurrentLinkedQueue asyncOprQueue; 46 | private final AtomicLong msgCounter = new AtomicLong(0); 47 | private final AtomicInteger sessionsCounter = new AtomicInteger(0); 48 | private final int msgOutSize; 49 | private final int msgInSize; 50 | private final int numSessionsLimit; 51 | private final int spwNumber; 52 | private final int dynamicMsgAllocationAmount; 53 | 54 | private final Callbacks onDynamicMsgPoolAllocation = new EventQueueHandler.Callbacks() { 55 | 56 | @Override 57 | public MsgPool getAdditionalMsgPool(int inSize, int outSize) { 58 | LOG.warn("Potential resources leak - JXIO is missing msg resources, allocating additional buffers. SP=" + ServerPortalWorker.this.sp); 59 | 60 | // allocating new ioBuffers. 61 | ioBufferSupplier.allocateBuffers(dynamicMsgAllocationAmount, msgInSize); 62 | 63 | return new MsgPool(dynamicMsgAllocationAmount, msgInSize, msgOutSize); 64 | } 65 | }; 66 | 67 | private final Runnable onEqhBreak = new Runnable() { 68 | @Override 69 | public void run() { 70 | processReplies(); 71 | } 72 | }; 73 | 74 | private class AsyncReply implements Runnable { 75 | final ServerSession session; 76 | final Msg msg; 77 | final List onFlightMsgs; 78 | 79 | AsyncReply(DataXceiverBase dxc, Msg msg, List onFlightMsgs) { 80 | this.session = dxc.getSessionServer(); 81 | this.msg = msg; 82 | this.onFlightMsgs = onFlightMsgs; 83 | } 84 | 85 | @Override 86 | public void run() { 87 | if (!session.getIsClosing()) { 88 | R4HProtocol.wrappedSendResponse(session, msg, LOG); 89 | if (onFlightMsgs != null) { // TODO: list is not safe!!! 90 | onFlightMsgs.remove(msg); 91 | } 92 | } 93 | } 94 | } 95 | 96 | private class AsyncRequest implements Runnable { 97 | final Msg msg; 98 | final DataXceiverBase dxc; 99 | 100 | AsyncRequest(Msg msg, DataXceiverBase dxc) { 101 | this.msg = msg; 102 | this.dxc = dxc; 103 | } 104 | 105 | @Override 106 | public void run() { 107 | if (!dxc.clientSession.getIsClosing()) { 108 | R4HProtocol.wrappedSendRequest(dxc.clientSession, msg, LOG); 109 | dxc.clientOnFlightNumMsgs++; 110 | } 111 | } 112 | } 113 | 114 | private class AsyncPipelineConnection implements Runnable { 115 | final DataXceiverBase dxc; 116 | final ClientSession.Callbacks callbacks; 117 | final String clientURI; 118 | final WriteOprHeader oprHeader; 119 | 120 | AsyncPipelineConnection(ClientSession.Callbacks callbacks, String clientURI, WriteOprHeader header, DataXceiverBase dxc) { 121 | this.callbacks = callbacks; 122 | this.clientURI = clientURI; 123 | this.oprHeader = header; 124 | this.dxc = dxc; 125 | } 126 | 127 | @Override 128 | public void run() { 129 | try { 130 | /* Open connection */ 131 | int index = clientURI.indexOf("&clientHash="); 132 | URI uri = R4HProtocol.createPipelineURI(oprHeader.getTargets(), clientURI.substring(index)); 133 | LOG.info("Open a proxy client session: " + uri); 134 | dxc.clientSession = new ClientSession(eqh, uri, callbacks); 135 | } catch (Throwable t) { 136 | LOG.error("Failed to create pipeline connection"); 137 | if (dxc.clientSession != null && !dxc.clientSession.getIsClosing()) { 138 | dxc.clientSession.close(); 139 | } 140 | } 141 | } 142 | } 143 | 144 | ServerPortalWorker(URI uri, int numOfMsgsToBind, int msgInSize, int msgOutSize, int numSessionsLimit, int spwNumber, 145 | int dynamicMsgAllocationAmount, boolean allocateResources) { 146 | this.uri = uri; 147 | this.msgInSize = msgInSize; 148 | this.msgOutSize = msgOutSize; 149 | this.numSessionsLimit = numSessionsLimit; 150 | this.spwNumber = spwNumber; 151 | this.dynamicMsgAllocationAmount = dynamicMsgAllocationAmount; 152 | 153 | if (allocateResources) { // dxcs in accept model or worker in forward model 154 | this.eqh = new R4HEventHandler(onDynamicMsgPoolAllocation, onEqhBreak); 155 | MsgPool msgPool = new MsgPool(numOfMsgsToBind, msgInSize, msgOutSize); 156 | this.ioBufferSupplier = new IOBufferSupplier(numOfMsgsToBind, msgInSize); 157 | this.eqh.bindMsgPool(msgPool); 158 | this.asyncOprQueue = new ConcurrentLinkedQueue<>(); 159 | } else { // dxcs in forward model 160 | this.eqh = new R4HEventHandler(null, null); 161 | this.ioBufferSupplier = null; 162 | this.asyncOprQueue = null; 163 | } 164 | } 165 | 166 | // Constructor for workers in forward model 167 | ServerPortalWorker(URI uri, int numOfMsgsToBind, int msgInSize, int msgOutSize, int isFreeLimit, int spwNumber, int dynamicMsgAllocationAmount) { 168 | this(uri, numOfMsgsToBind, msgInSize, msgOutSize, isFreeLimit, spwNumber, dynamicMsgAllocationAmount, true); 169 | this.sp = new ServerPortal(eqh, uri, null); 170 | this.th = new Thread(eqh); 171 | this.th.setName(String.format("ServerPortalWorker-%d", spwNumber)); 172 | } 173 | 174 | void start() { 175 | th.start(); 176 | } 177 | 178 | @Override 179 | public String toString() { 180 | return String.format("ServerPortalWorker{thread='%s', sp='%s', ioBufferSupplier='%s'}", th, sp, ioBufferSupplier.size()); 181 | } 182 | 183 | void processReplies() { 184 | msgCounter.set(0); 185 | Runnable opr = asyncOprQueue.poll(); 186 | while (opr != null) { 187 | try { 188 | opr.run(); 189 | } catch (Throwable t) { 190 | LOG.error(StringUtils.stringifyException(t)); 191 | } 192 | opr = asyncOprQueue.poll(); 193 | } 194 | } 195 | 196 | void queueAsyncReply(DataXceiverBase dxc, Msg msg, List onFlightMsgs, boolean breakEventLoop) { 197 | asyncOprQueue.add(new AsyncReply(dxc, msg, onFlightMsgs)); 198 | if (msgCounter.incrementAndGet() % 10 == 0 || breakEventLoop || onFlightMsgs.size() < 10) { 199 | eqh.breakEventLoop(); 200 | } 201 | } 202 | 203 | void queueAsyncRunnable(Runnable task) { 204 | asyncOprQueue.add(task); 205 | eqh.breakEventLoop(); 206 | } 207 | 208 | void queueAsyncRequest(Msg msg, DataXceiverBase dxc) { 209 | asyncOprQueue.add(new AsyncRequest(msg, dxc)); 210 | eqh.breakEventLoop(); 211 | } 212 | 213 | void queueAsyncPipelineConnection(ClientSession.Callbacks callbacks, String clientURI, WriteOprHeader header, DataXceiverBase dxc) { 214 | asyncOprQueue.add(new AsyncPipelineConnection(callbacks, clientURI, header, dxc)); 215 | eqh.breakEventLoop(); 216 | } 217 | 218 | @Override 219 | public boolean isFree() { 220 | return this.sessionsCounter.get() < this.numSessionsLimit; 221 | } 222 | 223 | void incrementSessionsCounter() { 224 | int value = this.sessionsCounter.incrementAndGet(); 225 | if (LOG.isDebugEnabled()) { 226 | LOG.debug(String.format("incrementing spw's counter - counter after increment = %d , spw number = %d , isFreeLimit = %d", value, 227 | this.spwNumber, this.numSessionsLimit)); 228 | } 229 | } 230 | 231 | int getSessionsCounterValue() { 232 | return this.sessionsCounter.intValue(); 233 | } 234 | 235 | void decermentSessionsCounter() { 236 | int value = this.sessionsCounter.decrementAndGet(); 237 | if (LOG.isDebugEnabled()) { 238 | LOG.debug(String.format("decrementing spw's counter - counter after decerment = %d , spw number = %d , isFreeLimit = %d", value, 239 | this.spwNumber, this.numSessionsLimit)); 240 | } 241 | } 242 | 243 | IOBufferSupplier getIOBufferSupplier() { 244 | return this.ioBufferSupplier; 245 | } 246 | } 247 | -------------------------------------------------------------------------------- /src/common/src/com/mellanox/r4h/WriteOprHeader.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 21 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 22 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 23 | import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; 24 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 25 | import org.apache.hadoop.security.token.Token; 26 | import org.apache.hadoop.util.DataChecksum; 27 | 28 | /** 29 | * Encapsulating HDFS WriteOperationHeader to be read from a DataInput and written to a DataOutput. Additional it 30 | * provides the ability to write itself to a DataOutput with a pipeline modification 31 | */ 32 | public class WriteOprHeader { 33 | 34 | final private ExtendedBlock blk; 35 | final private Token blockToken; 36 | final private String clientName; 37 | final private DatanodeInfo[] targets; 38 | final private DatanodeInfo source; 39 | final private BlockConstructionStage stage; 40 | final private int pipelineSize; 41 | final private long minBytesRcvd; 42 | final private long maxBytesRcvd; 43 | final private long latestGenerationStamp; 44 | final private DataChecksum requestedChecksum; 45 | final private CachingStrategy cachingStrategy; 46 | 47 | final private boolean isDatanode; 48 | final private boolean isClient; 49 | final private boolean isTransfer; 50 | 51 | public WriteOprHeader(ExtendedBlock blk, Token blockToken, String clientName, DatanodeInfo[] targets, DatanodeInfo source, 52 | BlockConstructionStage stage, int pipelineSize, long minBytesRcvd, long maxBytesRcvd, long latestGenerationStamp, 53 | DataChecksum requestedChecksum, CachingStrategy cachingStrategy) { 54 | this.blk = blk; 55 | this.blockToken = blockToken; 56 | this.clientName = clientName; 57 | this.targets = targets; 58 | this.source = source; 59 | this.stage = stage; 60 | this.pipelineSize = pipelineSize; 61 | this.minBytesRcvd = minBytesRcvd; 62 | this.maxBytesRcvd = maxBytesRcvd; 63 | this.latestGenerationStamp = latestGenerationStamp; 64 | this.requestedChecksum = requestedChecksum; 65 | this.cachingStrategy = cachingStrategy; 66 | 67 | isDatanode = clientName.length() == 0; 68 | isClient = !isDatanode; 69 | isTransfer = stage == BlockConstructionStage.TRANSFER_RBW || stage == BlockConstructionStage.TRANSFER_FINALIZED; 70 | 71 | } 72 | 73 | public CachingStrategy getCachingStrategy() { 74 | return cachingStrategy; 75 | } 76 | 77 | public boolean isDatanode() { 78 | return clientName.length() == 0; 79 | } 80 | 81 | public boolean isClient() { 82 | return !isDatanode; 83 | } 84 | 85 | public boolean isTransfer() { 86 | return (stage == BlockConstructionStage.TRANSFER_RBW || stage == BlockConstructionStage.TRANSFER_FINALIZED); 87 | } 88 | 89 | @Override 90 | public String toString() { 91 | StringBuffer sb = new StringBuffer(); 92 | sb.append(String 93 | .format("WriteOprHeader{blk='%s', blockToken='%s', clientName='%s', source='%s', stage='%s', pipelineSize(entire)='%d', minBytesRcvd='%d', maxBytesRcvd='%d', requestedChecksum='%s'", 94 | blk, blockToken, clientName, source, stage, pipelineSize, minBytesRcvd, maxBytesRcvd, requestedChecksum)); 95 | if ((targets) != null && (targets.length > 0)) { 96 | sb.append(String.format("PIPELINE_TARGETS[%d]='[", targets.length)); 97 | for (int i = 0; i < targets.length; i++) { 98 | sb.append(String.format(" --> '%s'", targets[i])); 99 | } 100 | sb.append("]'"); 101 | } else { 102 | sb.append("PIPELINE_TARGETS[0]"); 103 | } 104 | 105 | sb.append(" }"); 106 | 107 | return sb.toString(); 108 | } 109 | 110 | // @Override 111 | // public void write(DataOutput out) throws IOException { 112 | // out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION); 113 | // out.write(DataTransferProtocol.OP_WRITE_BLOCK); 114 | // out.writeLong(getBlkid()); 115 | // out.writeLong(getGenerationStamp()); 116 | // out.writeInt(getPipelineSize()); 117 | // out.writeBoolean(isRecovery()); 118 | // Text.writeString(out, getClient()); 119 | // out.writeBoolean(isHasSrcDataNode()); 120 | // if (isHasSrcDataNode()) { 121 | // getSrcDataNode().write(out); 122 | // } 123 | // int pipelineSize = getNumTargets(); // TODO: VInt 124 | // out.writeInt(pipelineSize); 125 | // if (pipelineSize > 0) { 126 | // for (int i = 0; i < pipelineSize; i++) { 127 | // targets[i].write(out); 128 | // } 129 | // } 130 | // getAccessToken().write(out); 131 | // } 132 | 133 | // @Override 134 | // public void readFields(DataInput in) throws IOException { 135 | // short version = in.readShort(); 136 | // if (version != DataTransferProtocol.DATA_TRANSFER_VERSION) { // TODO: compare to an own R4H version field!! 137 | // throw new IOException(String.format("got unexpected DATA_TRANSFER_VERSION=%d - (support VERESION=%d only)", version, 138 | // DataTransferProtocol.DATA_TRANSFER_VERSION)); 139 | // } 140 | // byte op = in.readByte(); 141 | // if (op != DataTransferProtocol.OP_WRITE_BLOCK) { 142 | // throw new IOException(String.format("got unexpected OP_CODE=%d - expected %d(OP_WRITE_BLOCK", op, DataTransferProtocol.OP_WRITE_BLOCK)); 143 | // } 144 | // setBlkid(in.readLong()); 145 | // setGenerationStamp(in.readLong()); 146 | // setPipelineSize(in.readInt()); 147 | // setRecovery(in.readBoolean()); 148 | // setClient(Text.readString(in)); 149 | // setHasSrcDataNode(in.readBoolean()); 150 | // if (isHasSrcDataNode()) { 151 | // setSrcDataNode(new DatanodeInfo()); 152 | // getSrcDataNode().readFields(in); 153 | // } 154 | // 155 | // int numTargets = in.readInt(); 156 | // if (numTargets < 0) { 157 | // throw new IOException("Mislabelled incoming datastream."); 158 | // } 159 | // DatanodeInfo[] targets = new DatanodeInfo[numTargets]; 160 | // for (int i = 0; i < targets.length; i++) { 161 | // DatanodeInfo tmp = new DatanodeInfo(); 162 | // tmp.readFields(in); 163 | // targets[i] = tmp; 164 | // } 165 | // setTargets(targets); 166 | // 167 | // setAccessToken(new Token()); 168 | // getAccessToken().readFields(in); 169 | // } 170 | 171 | // WriteOprHeader createNextPipeHeader() { 172 | // WriteOprHeader nextHeader = new WriteOprHeader(); 173 | // 174 | // nextHeader.setBlkid(getBlkid()); 175 | // nextHeader.setGenerationStamp(getGenerationStamp()); 176 | // nextHeader.setPipelineSize(getPipelineSize()); 177 | // nextHeader.setRecovery(isRecovery()); 178 | // nextHeader.setClient(getClient()); 179 | // nextHeader.setHasSrcDataNode(isHasSrcDataNode()); 180 | // if (isHasSrcDataNode()) { 181 | // nextHeader.setSrcDataNode(getSrcDataNode()); 182 | // } 183 | // 184 | // int nextPipeSize = getNumTargets() - 1; 185 | // if (nextPipeSize > 0) { 186 | // DatanodeInfo[] nextPipe = new DatanodeInfo[nextPipeSize]; 187 | // nextHeader.setTargets(nextPipe); 188 | // for (int i = 1; i < getNumTargets(); i++) { 189 | // nextPipe[i - 1] = getTargetByIndex(i); 190 | // } 191 | // } 192 | // 193 | // nextHeader.setAccessToken(getAccessToken()); 194 | // 195 | // return nextHeader; 196 | // } 197 | 198 | // void writeNextPipeHeader(DataOutput out) throws IOException { 199 | // out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION); 200 | // out.writeByte(DataTransferProtocol.OP_WRITE_BLOCK); 201 | // out.writeLong(getBlkid()); 202 | // out.writeLong(getGenerationStamp()); 203 | // out.writeInt(getPipelineSize()); 204 | // out.writeBoolean(isRecovery()); 205 | // Text.writeString(out, getClient()); 206 | // out.writeBoolean(isHasSrcDataNode()); 207 | // if (isHasSrcDataNode()) { 208 | // getSrcDataNode().write(out); 209 | // } 210 | // int nextPipeSize = getNumTargets() - 1; 211 | // out.writeInt((nextPipeSize > 0) ? nextPipeSize : 0); 212 | // if (nextPipeSize > 0) { 213 | // for (int i = 0; i < nextPipeSize; i++) { 214 | // targets[i + 1].write(out); 215 | // } 216 | // } 217 | // getAccessToken().write(out); 218 | // } 219 | 220 | public int getPipelineSize() { 221 | return pipelineSize; 222 | } 223 | 224 | public String getClientName() { 225 | return clientName; 226 | } 227 | 228 | public DatanodeInfo getSrcDataNode() { 229 | return source; 230 | } 231 | 232 | int getNumTargets() { 233 | return (targets != null) ? getTargets().length : 0; 234 | } 235 | 236 | public DatanodeInfo[] getTargets() { 237 | return targets; 238 | } 239 | 240 | DatanodeInfo getTargetByIndex(int targetIndex) { 241 | return targets[targetIndex]; 242 | } 243 | 244 | public DataChecksum getRequestedChecksum() { 245 | return requestedChecksum; 246 | } 247 | 248 | public BlockConstructionStage getStage() { 249 | return stage; 250 | } 251 | 252 | public ExtendedBlock getBlock() { 253 | return this.blk; 254 | } 255 | 256 | public long getLatestGenerationStamp() { 257 | return latestGenerationStamp; 258 | } 259 | 260 | public long getMinBytesRcvd() { 261 | return minBytesRcvd; 262 | } 263 | 264 | public long getMaxBytesRcvd() { 265 | return maxBytesRcvd; 266 | } 267 | 268 | public Token getBlockToken() { 269 | return blockToken; 270 | } 271 | 272 | // TODO:toString() 273 | } 274 | -------------------------------------------------------------------------------- /src/common/src/org/apache/hadoop/hdfs/BlockStorageLocationUtilBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import java.io.IOException; 4 | import java.util.List; 5 | import java.util.Map; 6 | 7 | import org.apache.hadoop.conf.Configuration; 8 | import org.apache.hadoop.fs.BlockStorageLocation; 9 | import org.apache.hadoop.fs.VolumeId; 10 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 11 | import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata; 12 | import org.apache.hadoop.hdfs.protocol.LocatedBlock; 13 | import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; 14 | 15 | public class BlockStorageLocationUtilBridge extends BlockStorageLocationUtil { 16 | 17 | public static Map queryDatanodesForHdfsBlocksMetadata(Configuration conf, 18 | Map> datanodeBlocks, int poolsize, int timeoutMs, boolean connectToDnViaHostname) 19 | throws InvalidBlockTokenException { 20 | return BlockStorageLocationUtil.queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks, poolsize, timeoutMs, connectToDnViaHostname); 21 | } 22 | 23 | public static Map> associateVolumeIdsWithBlocks(List blocks, 24 | Map metadatas) { 25 | return BlockStorageLocationUtil.associateVolumeIdsWithBlocks(blocks, metadatas); 26 | } 27 | 28 | public static BlockStorageLocation[] convertToVolumeBlockLocations(List blocks, Map> blockVolumeIds) 29 | throws IOException { 30 | return BlockStorageLocationUtil.convertToVolumeBlockLocations(blocks, blockVolumeIds); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/common/src/org/apache/hadoop/hdfs/ClientBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import org.apache.hadoop.security.UserGroupInformation; 4 | 5 | public class ClientBridge { 6 | 7 | public static String getUgiShortUserName(UserGroupInformation ugi) { 8 | return ugi.getShortUserName(); 9 | } 10 | 11 | } 12 | -------------------------------------------------------------------------------- /src/common/src/org/apache/hadoop/hdfs/DFSClientConfBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import java.io.IOException; 4 | 5 | import org.apache.hadoop.conf.Configuration; 6 | import org.apache.hadoop.fs.Options.ChecksumOpt; 7 | import org.apache.hadoop.fs.permission.FsPermission; 8 | import org.apache.hadoop.hdfs.DFSClient.Conf; 9 | import org.apache.hadoop.util.DataChecksum; 10 | 11 | import com.mellanox.r4h.R4HProtocol; 12 | 13 | /** 14 | * DFSClient configuration 15 | */ 16 | public class DFSClientConfBridge extends Conf { 17 | // Default value for time to wait for header ack before pronouncing failure: 18 | private static final int HEADER_ACK_CLIENT_TIMEOUT_SECONDS_DEFAULT = 30; 19 | // The parameter name of time to wait for header ack before pronouncing failure: 20 | private static final String HEADER_ACK_CLIENT_TIMEOUT_PARAM_NAME = "r4h.client.header.ack.timeout"; 21 | // The time to wait for header ack before pronouncing failure: 22 | private long headerAckTimeoutUsec; 23 | // The parameter name of the number of msgs in client's msgpool 24 | private static final String CLIENT_MSG_POOL_NUM_MSGS_PARAM_NAME = "r4h.client.msgpool.num.msgs"; 25 | private static final int CLIENT_MSG_POOL_NUM_MSGS_DEFAULT = R4HProtocol.MAX_DATA_QUEUE_PACKETS + R4HProtocol.CLIENT_MSGPOOL_SPARE; 26 | // The time to wait for header ack before pronouncing failure: 27 | private int msgPoolNumMsgs; 28 | 29 | public DFSClientConfBridge(Configuration conf) { 30 | super(conf); 31 | this.headerAckTimeoutUsec = 1000000 * conf.getInt(HEADER_ACK_CLIENT_TIMEOUT_PARAM_NAME, HEADER_ACK_CLIENT_TIMEOUT_SECONDS_DEFAULT); 32 | this.msgPoolNumMsgs = conf.getInt(CLIENT_MSG_POOL_NUM_MSGS_PARAM_NAME, CLIENT_MSG_POOL_NUM_MSGS_DEFAULT); 33 | } 34 | 35 | public int getClientMsgPoolNumMsgs() { 36 | return msgPoolNumMsgs; 37 | } 38 | 39 | /** 40 | * 41 | * @return headerAckTimeoutUsec 42 | */ 43 | public long getHeaderAckTimeoutUsec() { 44 | return headerAckTimeoutUsec; 45 | } 46 | 47 | public boolean getShortCircuitMmapEnabled() { 48 | return shortCircuitMmapEnabled; 49 | } 50 | 51 | public boolean getConnectToDnViaHostname() { 52 | return connectToDnViaHostname; 53 | } 54 | 55 | /** 56 | * @return the hdfsTimeout 57 | */ 58 | public int getHdfsTimeout() { 59 | return hdfsTimeout; 60 | } 61 | 62 | /** 63 | * @return the maxFailoverAttempts 64 | */ 65 | public int getMaxFailoverAttempts() { 66 | return maxFailoverAttempts; 67 | } 68 | 69 | /** 70 | * @return the maxRetryAttempts 71 | */ 72 | public int getMaxRetryAttempts() { 73 | return maxRetryAttempts; 74 | } 75 | 76 | /** 77 | * @return the failoverSleepBaseMillis 78 | */ 79 | public int getFailoverSleepBaseMillis() { 80 | return failoverSleepBaseMillis; 81 | } 82 | 83 | /** 84 | * @return the failoverSleepMaxMillis 85 | */ 86 | public int getFailoverSleepMaxMillis() { 87 | return failoverSleepMaxMillis; 88 | } 89 | 90 | /** 91 | * @return the maxBlockAcquireFailures 92 | */ 93 | public int getMaxBlockAcquireFailures() { 94 | return maxBlockAcquireFailures; 95 | } 96 | 97 | /** 98 | * @return the confTime 99 | */ 100 | public int getConfTime() { 101 | return confTime; 102 | } 103 | 104 | /** 105 | * @return the ioBufferSize 106 | */ 107 | public int getIoBufferSize() { 108 | return ioBufferSize; 109 | } 110 | 111 | /** 112 | * @return the defaultChecksumOpt 113 | */ 114 | public ChecksumOpt getDefaultChecksumOpt() { 115 | return defaultChecksumOpt; 116 | } 117 | 118 | /** 119 | * @return the writePacketSize 120 | */ 121 | public int getWritePacketSize() { 122 | return writePacketSize; 123 | } 124 | 125 | /** 126 | * @return the socketTimeout 127 | */ 128 | public int getSocketTimeout() { 129 | return socketTimeout; 130 | } 131 | 132 | /** 133 | * @return the socketCacheCapacity 134 | */ 135 | public int getSocketCacheCapacity() { 136 | return socketCacheCapacity; 137 | } 138 | 139 | /** 140 | * @return the socketCacheExpiry 141 | */ 142 | public long getSocketCacheExpiry() { 143 | return socketCacheExpiry; 144 | } 145 | 146 | /** 147 | * @return the excludedNodesCacheExpiry 148 | */ 149 | public long getExcludedNodesCacheExpiry() { 150 | return excludedNodesCacheExpiry; 151 | } 152 | 153 | /** 154 | * @return the nCachedConnRetry 155 | */ 156 | public int getnCachedConnRetry() { 157 | return nCachedConnRetry; 158 | } 159 | 160 | /** 161 | * @return the nBlockWriteRetry 162 | */ 163 | public int getnBlockWriteRetry() { 164 | return nBlockWriteRetry; 165 | } 166 | 167 | /** 168 | * @return the nBlockWriteLocateFollowingRetry 169 | */ 170 | public int getnBlockWriteLocateFollowingRetry() { 171 | return nBlockWriteLocateFollowingRetry; 172 | } 173 | 174 | /** 175 | * @return the defaultBlockSize 176 | */ 177 | public long getDefaultBlockSize() { 178 | return defaultBlockSize; 179 | } 180 | 181 | /** 182 | * @return the prefetchSize 183 | */ 184 | public long getPrefetchSize() { 185 | return prefetchSize; 186 | } 187 | 188 | /** 189 | * @return the defaultReplication 190 | */ 191 | public short getDefaultReplication() { 192 | return defaultReplication; 193 | } 194 | 195 | /** 196 | * @return the taskId 197 | */ 198 | public String getTaskId() { 199 | return taskId; 200 | } 201 | 202 | /** 203 | * @return the uMask 204 | */ 205 | public FsPermission getuMask() { 206 | return uMask; 207 | } 208 | 209 | /** 210 | * @return the connectToDnViaHostname 211 | */ 212 | public boolean isConnectToDnViaHostname() { 213 | return connectToDnViaHostname; 214 | } 215 | 216 | /** 217 | * @return the getHdfsBlocksMetadataEnabled 218 | */ 219 | public boolean isHdfsBlocksMetadataEnabled() { 220 | return getHdfsBlocksMetadataEnabled; 221 | } 222 | 223 | /** 224 | * @return the getFileBlockStorageLocationsNumThreads 225 | */ 226 | public int getFileBlockStorageLocationsNumThreads() { 227 | return getFileBlockStorageLocationsNumThreads; 228 | } 229 | 230 | /** 231 | * @return the getFileBlockStorageLocationsTimeoutMs 232 | */ 233 | public int getFileBlockStorageLocationsTimeoutMs() { 234 | return getFileBlockStorageLocationsTimeoutMs; 235 | } 236 | 237 | /** 238 | * @return the retryTimesForGetLastBlockLength 239 | */ 240 | public int getRetryTimesForGetLastBlockLength() { 241 | return retryTimesForGetLastBlockLength; 242 | } 243 | 244 | /** 245 | * @return the retryIntervalForGetLastBlockLength 246 | */ 247 | public int getRetryIntervalForGetLastBlockLength() { 248 | return retryIntervalForGetLastBlockLength; 249 | } 250 | 251 | /** 252 | * @return the datanodeRestartTimeout 253 | */ 254 | public long getDatanodeRestartTimeout() { 255 | return datanodeRestartTimeout; 256 | } 257 | 258 | /** 259 | * @return the useLegacyBlockReader 260 | */ 261 | public boolean isUseLegacyBlockReader() { 262 | return useLegacyBlockReader; 263 | } 264 | 265 | /** 266 | * @return the useLegacyBlockReaderLocal 267 | */ 268 | public boolean isUseLegacyBlockReaderLocal() { 269 | return useLegacyBlockReaderLocal; 270 | } 271 | 272 | /** 273 | * @return the domainSocketPath 274 | */ 275 | public String getDomainSocketPath() { 276 | return domainSocketPath; 277 | } 278 | 279 | /** 280 | * @return the skipShortCircuitChecksums 281 | */ 282 | public boolean isSkipShortCircuitChecksums() { 283 | return skipShortCircuitChecksums; 284 | } 285 | 286 | /** 287 | * @return the shortCircuitBufferSize 288 | */ 289 | public int getShortCircuitBufferSize() { 290 | return shortCircuitBufferSize; 291 | } 292 | 293 | /** 294 | * @return the shortCircuitLocalReads 295 | */ 296 | public boolean isShortCircuitLocalReads() { 297 | return shortCircuitLocalReads; 298 | } 299 | 300 | /** 301 | * @return the domainSocketDataTraffic 302 | */ 303 | public boolean isDomainSocketDataTraffic() { 304 | return domainSocketDataTraffic; 305 | } 306 | 307 | /** 308 | * @return the shortCircuitStreamsCacheSize 309 | */ 310 | public int getShortCircuitStreamsCacheSize() { 311 | return shortCircuitStreamsCacheSize; 312 | } 313 | 314 | /** 315 | * @return the shortCircuitStreamsCacheExpiryMs 316 | */ 317 | public long getShortCircuitStreamsCacheExpiryMs() { 318 | return shortCircuitStreamsCacheExpiryMs; 319 | } 320 | 321 | /** 322 | * @return the shortCircuitSharedMemoryWatcherInterruptCheckMs 323 | */ 324 | public int getShortCircuitSharedMemoryWatcherInterruptCheckMs() { 325 | return shortCircuitSharedMemoryWatcherInterruptCheckMs; 326 | } 327 | 328 | /** 329 | * @return the shortCircuitMmapEnabled 330 | */ 331 | public boolean isShortCircuitMmapEnabled() { 332 | return shortCircuitMmapEnabled; 333 | } 334 | 335 | /** 336 | * @return the shortCircuitMmapCacheSize 337 | */ 338 | public int getShortCircuitMmapCacheSize() { 339 | return shortCircuitMmapCacheSize; 340 | } 341 | 342 | /** 343 | * @return the shortCircuitMmapCacheExpiryMs 344 | */ 345 | public long getShortCircuitMmapCacheExpiryMs() { 346 | return shortCircuitMmapCacheExpiryMs; 347 | } 348 | 349 | /** 350 | * @return the shortCircuitMmapCacheRetryTimeout 351 | */ 352 | public long getShortCircuitMmapCacheRetryTimeout() { 353 | return shortCircuitMmapCacheRetryTimeout; 354 | } 355 | 356 | /** 357 | * @return the shortCircuitCacheStaleThresholdMs 358 | */ 359 | public long getShortCircuitCacheStaleThresholdMs() { 360 | return shortCircuitCacheStaleThresholdMs; 361 | } 362 | 363 | public DataChecksum createChecksum(ChecksumOpt userOpt) throws IOException { 364 | // Fill in any missing field with the default. 365 | ChecksumOpt myOpt = ChecksumOpt.processChecksumOpt(defaultChecksumOpt, userOpt); 366 | DataChecksum dataChecksum = DataChecksum.newDataChecksum(myOpt.getChecksumType(), myOpt.getBytesPerChecksum()); 367 | if (dataChecksum == null) { 368 | throw new IOException("Invalid checksum type specified: " + myOpt.getChecksumType().name()); 369 | } 370 | return dataChecksum; 371 | } 372 | 373 | public int getTimeWindow() { 374 | return timeWindow; 375 | } 376 | 377 | public boolean connectToDnViaHostname() { 378 | return connectToDnViaHostname; 379 | } 380 | 381 | public DataChecksum createChecksum() throws IOException { 382 | return createChecksum(null); 383 | } 384 | 385 | public int socketTimeout() { 386 | return socketTimeout; 387 | } 388 | } 389 | -------------------------------------------------------------------------------- /src/common/src/org/apache/hadoop/hdfs/server/datanode/DataNodeBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode; 2 | 3 | import java.io.IOException; 4 | 5 | import org.apache.hadoop.conf.Configuration; 6 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 7 | import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager; 8 | import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; 9 | 10 | import com.mellanox.r4h.R4HProtocol; 11 | 12 | /** 13 | * A Package bridge to the original class org.apache.hadoop.hdfs.server.datanode 14 | * 15 | * @see org.apache.hadoop.hdfs.server.datanode.BlockReceiverPatched 16 | * It exposes the neccessary package access fields of BlockReceiver to R4H 17 | */ 18 | public class DataNodeBridge { 19 | 20 | private static final int NUM_OF_MSG_FOR_BLOCK = R4HProtocol.MAX_SEND_PACKETS + R4HProtocol.JX_SERVER_SPARE_MSGS; 21 | private static final int NUM_OF_SERVER_PORTAL_WORKERS_SESSIONS_LIMIT_DEFAULT = 4; 22 | private static final int NUM_OF_DYNAMIC_MSG_ALLOCATION_DEFAULT = 1; 23 | private static final int NUM_OF_MSG_TO_BIND_IN_BLOCKS_DEFAULT = 50; 24 | private static final String NUM_OF_SERVER_PORTAL_WORKERS_SESSIONS_LIMIT_PARAM_NAME = "r4h.workers.sessions.limit"; 25 | private static final String NUM_OF_MSG_TO_BIND_IN_BLOCKS_PARAM_NAME = "r4h.msg.blocks.bind"; 26 | private static final String NUM_OF_DYNAMIC_MSG_ALLOCATION_NAME = "r4h.msg.dynamic.allocation"; 27 | private static final int NUM_OF_IO_EXECUTORS_DEFAULT = 10; 28 | private static final int NUM_OF_PRE_ALLOC_SERVER_PORTAL_WORKERS_DEFAULT = 20; 29 | private static final String NUM_OF_IO_EXECUTORS_PARAM_NAME = "r4h.io.executors"; 30 | private static final String NUM_OF_PRE_ALLOC_SERVER_PORTAL_WORKERS_PARAM_NAME = "r4h.server.portal.workers"; 31 | 32 | public final int numOfAuxExecutors; 33 | public final int numOfioExecutors; 34 | public final int spwAmount; 35 | public final int numOfMsgsToBind; 36 | public final int msgInSize; 37 | public final int msgOutSize; 38 | public final int numSessionsLimit; 39 | public final int dynamicMsgAllocationAmount; 40 | public final boolean isForwardEnable; 41 | 42 | private DataNode dn; 43 | private DataXceiverServer dxcs; 44 | 45 | public DataNodeBridge(DataNode dn) { 46 | if (dn == null) { 47 | throw new IllegalArgumentException("Trying to exppse a null reference DataNode"); 48 | } 49 | 50 | Configuration dnConf = dn.getConf(); 51 | 52 | this.spwAmount = dnConf.getInt(NUM_OF_PRE_ALLOC_SERVER_PORTAL_WORKERS_PARAM_NAME, NUM_OF_PRE_ALLOC_SERVER_PORTAL_WORKERS_DEFAULT); 53 | if (spwAmount < 0) { 54 | throw new IllegalArgumentException(String.format("Illegal number of Server portal workers ( %d )", this.spwAmount)); 55 | } 56 | 57 | this.numOfioExecutors = dnConf.getInt(NUM_OF_IO_EXECUTORS_PARAM_NAME, NUM_OF_IO_EXECUTORS_DEFAULT); 58 | if (this.numOfioExecutors < 1) { 59 | throw new IllegalArgumentException(String.format("Illegal number of IO executors ( %d )", this.numOfioExecutors)); 60 | } 61 | 62 | int blocksToBind = dnConf.getInt(NUM_OF_MSG_TO_BIND_IN_BLOCKS_PARAM_NAME, NUM_OF_MSG_TO_BIND_IN_BLOCKS_DEFAULT); 63 | if (blocksToBind < 1) { 64 | throw new IllegalArgumentException(String.format("Illegal number of messages to bind in blocks ( %d )", blocksToBind)); 65 | } 66 | 67 | this.numSessionsLimit = dnConf.getInt(NUM_OF_SERVER_PORTAL_WORKERS_SESSIONS_LIMIT_PARAM_NAME, 68 | NUM_OF_SERVER_PORTAL_WORKERS_SESSIONS_LIMIT_DEFAULT); 69 | if (numSessionsLimit < 0) { 70 | throw new IllegalArgumentException(String.format("Illegal number of server portal worker sessions limit ( %d )", numSessionsLimit)); 71 | } 72 | 73 | int dynamicBlocksAllocationAmount = dnConf.getInt(NUM_OF_DYNAMIC_MSG_ALLOCATION_NAME, NUM_OF_DYNAMIC_MSG_ALLOCATION_DEFAULT); 74 | if (dynamicBlocksAllocationAmount < 1) { 75 | throw new IllegalArgumentException(String.format("Illegal number of dynamic messages to allocate ( %d )", dynamicBlocksAllocationAmount)); 76 | } 77 | 78 | this.dn = dn; 79 | this.dxcs = (DataXceiverServer) dn.dataXceiverServer.getRunnable(); 80 | this.isForwardEnable = spwAmount != 0; 81 | this.dynamicMsgAllocationAmount = NUM_OF_MSG_FOR_BLOCK * dynamicBlocksAllocationAmount; 82 | this.numOfMsgsToBind = NUM_OF_MSG_FOR_BLOCK * blocksToBind; 83 | this.msgInSize = getWritePacketSize() + R4HProtocol.JX_BUF_SPARE; 84 | this.msgOutSize = R4HProtocol.ACK_SIZE; 85 | this.numOfAuxExecutors = blocksToBind; 86 | } 87 | 88 | public long getEstimateBlockSize() { 89 | return dxcs.estimateBlockSize; 90 | } 91 | 92 | public boolean isBlockTokenEnabled() { 93 | return dn.isBlockTokenEnabled; 94 | } 95 | 96 | public DataNode getDN() { 97 | return this.dn; 98 | } 99 | 100 | public int getWritePacketSize() { 101 | return dn.getDnConf().writePacketSize; 102 | } 103 | 104 | public BlockPoolTokenSecretManager getBlockPoolTokenSecretManager() { 105 | return dn.blockPoolTokenSecretManager; 106 | } 107 | 108 | public DatanodeRegistration getDNRegistrationForBP(String bpid) throws IOException { 109 | return dn.getDNRegistrationForBP(bpid); 110 | } 111 | 112 | public void recoverClose(ExtendedBlock blk, long newGS, long expectedBlockLen) throws IOException { 113 | dn.data.recoverClose(blk, newGS, expectedBlockLen); 114 | } 115 | 116 | } 117 | -------------------------------------------------------------------------------- /src/common/src/org/apache/hadoop/hdfs/server/datanode/R4HBlockReceiver.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode; 2 | 3 | import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT; 4 | 5 | import java.io.BufferedInputStream; 6 | import java.io.DataInputStream; 7 | import java.io.DataOutputStream; 8 | import java.io.FileDescriptor; 9 | import java.io.FileOutputStream; 10 | import java.io.IOException; 11 | import java.lang.reflect.Field; 12 | import java.lang.reflect.InvocationTargetException; 13 | import java.lang.reflect.Method; 14 | import java.nio.ByteBuffer; 15 | 16 | import org.apache.hadoop.io.WritableUtils; 17 | import org.apache.commons.logging.Log; 18 | import org.apache.commons.logging.LogFactory; 19 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 20 | import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; 21 | import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver; 22 | import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams; 23 | import org.apache.hadoop.util.StringUtils; 24 | import org.accelio.jxio.Msg; 25 | 26 | import com.mellanox.r4h.AsyncFileOutputStream; 27 | import com.mellanox.r4h.BRBInetrface; 28 | import com.mellanox.r4h.ByteBufferInputStream; 29 | import com.mellanox.r4h.IOBufferSupplier; 30 | import com.mellanox.r4h.MessageAction; 31 | import com.mellanox.r4h.R4HExecutor; 32 | import com.mellanox.r4h.WriteOprHeader; 33 | import com.mellanox.r4h.AsyncFileOutputStream.AsyncWriteCompletion; 34 | 35 | /** 36 | * Package bridge to the original class org.apache.hadoop.hdfs.server.datanode.DataNode 37 | * 38 | * @see org.apache.hadoop.hdfs.server.datanode.DataNode It exposes the neccessary package access fields of DataNode to 39 | * R4H 40 | */ 41 | public class R4HBlockReceiver extends R4HBlockReceiverBase implements BRBInetrface { 42 | private final static Log LOG = LogFactory.getLog(R4HBlockReceiver.class); 43 | private final Field inputStreamField; 44 | private final Field mirrorOutField; 45 | private final Field packetReceiverField; 46 | private final Field currPacketBufField; 47 | private final Method receivePacketMethod; 48 | private final WriteOprHeader oprHeader; 49 | private final ReplicaInPipelineInterface replicaInfo; 50 | private final IOBufferSupplier bufSupplier; 51 | private final String ssInfo; 52 | private final long startTime; 53 | private ByteBuffer curCopyBuff; 54 | private AsyncFileOutputStream asyncOut; 55 | private final MessageAction msgCallbacks; 56 | private final R4HExecutor ioExecutor; 57 | private Msg currMsg = null; 58 | 59 | public R4HBlockReceiver(IOBufferSupplier bufSupplier, WriteOprHeader oprHeader, DataInputStream inForHeaderOnly, String sessionInfo, 60 | DataNodeBridge dnEx, R4HExecutor ioExecutor, MessageAction msgCallbacks) throws IOException, NoSuchFieldException, SecurityException, 61 | NoSuchMethodException, IllegalArgumentException, IllegalAccessException { 62 | super(oprHeader, inForHeaderOnly, sessionInfo, dnEx); 63 | this.bufSupplier = bufSupplier; 64 | // TODO: check if it is really a newSingleThreadExecutor 65 | ssInfo = sessionInfo; 66 | inputStreamField = BlockReceiver.class.getDeclaredField("in"); 67 | inputStreamField.setAccessible(true); 68 | mirrorOutField = BlockReceiver.class.getDeclaredField("mirrorOut"); 69 | mirrorOutField.setAccessible(true); 70 | 71 | receivePacketMethod = BlockReceiver.class.getDeclaredMethod("receivePacket"); 72 | receivePacketMethod.setAccessible(true); 73 | Field tmpField = BlockReceiver.class.getDeclaredField("replicaInfo"); 74 | tmpField.setAccessible(true); 75 | replicaInfo = (ReplicaInPipelineInterface) tmpField.get(this); 76 | packetReceiverField = BlockReceiver.class.getDeclaredField("packetReceiver"); 77 | packetReceiverField.setAccessible(true); 78 | 79 | currPacketBufField = PacketReceiver.class.getDeclaredField("curPacketBuf"); 80 | currPacketBufField.setAccessible(true); 81 | 82 | this.oprHeader = oprHeader; 83 | this.ioExecutor = ioExecutor; 84 | this.msgCallbacks = msgCallbacks; 85 | 86 | setAsyncFileOutputStreams(); 87 | 88 | startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; 89 | } 90 | 91 | public PacketHeader getPacketHeader() throws IllegalArgumentException, IllegalAccessException { 92 | return getPacketReceiver().getHeader(); 93 | } 94 | 95 | private PacketReceiver getPacketReceiver() throws IllegalArgumentException, IllegalAccessException { 96 | return (PacketReceiver) packetReceiverField.get(this); 97 | } 98 | 99 | private void updatePacketCopyBuffer() throws IllegalArgumentException, IllegalAccessException, NoSuchFieldException, SecurityException, 100 | IOException { 101 | curCopyBuff = bufSupplier.getBuffer(); 102 | PacketReceiver receiver = getPacketReceiver(); 103 | currPacketBufField.set(receiver, curCopyBuff); 104 | } 105 | 106 | public ByteBuffer getCurCopyBuff() { 107 | return curCopyBuff; 108 | } 109 | 110 | public void setMirrorOut(DataOutputStream dos) throws IllegalArgumentException, IllegalAccessException { 111 | mirrorOutField.set(this, dos); 112 | } 113 | 114 | public ReplicaInPipelineInterface getReplicaInfo() { 115 | return replicaInfo; 116 | } 117 | 118 | public void processPacket(Msg msg) throws IOException { 119 | msg.getIn().clear(); 120 | DataInputStream in = new DataInputStream(new ByteBufferInputStream(msg.getIn())); 121 | int headerStart = WritableUtils.readVInt(in); 122 | msg.getIn().position(headerStart); 123 | in = new DataInputStream(new BufferedInputStream(new ByteBufferInputStream(msg.getIn()), 512)); 124 | 125 | // update currMsg for executing async tasks. 126 | this.currMsg = msg; 127 | asyncOut.setCurrMsg(msg); 128 | 129 | try { 130 | updatePacketCopyBuffer(); 131 | inputStreamField.set(this, in); 132 | receivePacketMethod.invoke(this); 133 | } catch (Exception e) { 134 | if ((e instanceof IllegalAccessException) || (e instanceof IllegalArgumentException) || (e instanceof InvocationTargetException)) { 135 | LOG.fatal("failed during reflection invokation: " + StringUtils.stringifyException(e)); 136 | } 137 | 138 | throw new RuntimeException(e); 139 | 140 | } 141 | } 142 | 143 | public void finalizeBlock() throws IOException { 144 | long offset = 0; 145 | this.close(); 146 | final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0; 147 | oprHeader.getBlock().setNumBytes(replicaInfo.getNumBytes()); 148 | getDataNode().data.finalizeBlock(oprHeader.getBlock()); 149 | closeBlock(); 150 | if (ClientTraceLog.isInfoEnabled() && oprHeader.isClient()) { 151 | LOG.info(String.format(DN_CLIENTTRACE_FORMAT, ssInfo, getDataNode().getXferAddress(), oprHeader.getBlock().getNumBytes(), "HDFS_WRITE", 152 | oprHeader.getClientName(), offset, getStorageID(), oprHeader.getBlock(), endTime - startTime)); 153 | } else { 154 | LOG.info("Received " + oprHeader.getBlock() + " size " + oprHeader.getBlock().getNumBytes() + " from " + inAddr); 155 | } 156 | 157 | if (LOG.isDebugEnabled()) { 158 | LOG.debug("Finalized and clsoed block"); 159 | } 160 | } 161 | 162 | public ExtendedBlock getBlock() { 163 | return this.oprHeader.getBlock(); 164 | } 165 | 166 | @Override 167 | void flushOrSync(final boolean isSync) throws IOException { 168 | if (LOG.isTraceEnabled()) { 169 | LOG.trace("executing async flushOrSync()"); 170 | } 171 | this.ioExecutor.execute(currMsg, msgCallbacks, new Runnable() { 172 | 173 | @Override 174 | public void run() { 175 | try { 176 | if (LOG.isTraceEnabled()) { 177 | LOG.trace("on async call to flushOrSync()"); 178 | } 179 | R4HBlockReceiver.super.flushOrSync(isSync); 180 | if (LOG.isTraceEnabled()) { 181 | LOG.trace("after async call to flushOrSync()"); 182 | } 183 | } catch (Throwable w) { 184 | LOG.error(StringUtils.stringifyException(w)); 185 | } 186 | } 187 | }); 188 | } 189 | 190 | private void setAsyncFileOutputStreams() throws NoSuchFieldException, SecurityException, IllegalArgumentException, IllegalAccessException, 191 | IOException { 192 | Field streamsField = BlockReceiver.class.getDeclaredField("streams"); 193 | streamsField.setAccessible(true); 194 | ReplicaOutputStreams streams = (ReplicaOutputStreams) streamsField.get(this); 195 | Field streamsDataOutField = ReplicaOutputStreams.class.getDeclaredField("dataOut"); 196 | streamsDataOutField.setAccessible(true); 197 | Field streamsChecksumOutField = ReplicaOutputStreams.class.getDeclaredField("checksumOut"); 198 | streamsChecksumOutField.setAccessible(true); 199 | 200 | // Overriding data file output stream to be async - using reflection 201 | Field outField = BlockReceiver.class.getDeclaredField("out"); 202 | outField.setAccessible(true); 203 | FileOutputStream out = (FileOutputStream) outField.get(this); 204 | FileDescriptor fd = out.getFD(); 205 | asyncOut = new AsyncFileOutputStream(fd, ioExecutor, msgCallbacks, new AsyncWriteCompletion() { 206 | 207 | @Override 208 | public void onWriteComplete(Object context, IOException e) { 209 | if (LOG.isTraceEnabled()) { 210 | LOG.trace("on data write completion"); 211 | if (context != null) { 212 | if (context instanceof PacketHeader) { 213 | LOG.trace("data output stream write completion for " + (PacketHeader) context); 214 | } else if (context instanceof AsyncFileOutputStream.AsyncWrite) { 215 | LOG.trace("data output stream write completion for " + (AsyncFileOutputStream.AsyncWrite) context); 216 | } 217 | 218 | } 219 | 220 | if (e != null) { 221 | LOG.error("error during writing packet data to disk: " + StringUtils.stringifyException(e)); 222 | } 223 | } 224 | } 225 | }); 226 | asyncOut.limitAsyncIObyThreadID(Thread.currentThread().getId()); 227 | outField.set(this, asyncOut); 228 | streamsDataOutField.set(streams, asyncOut); 229 | } 230 | 231 | // TODO: toString() 232 | 233 | @Override 234 | public void closeBlock() { 235 | getDataNode().closeBlock(getBlock(), DataNode.EMPTY_DEL_HINT, getReplicaInfo().getStorageUuid()); 236 | } 237 | 238 | @Override 239 | public String getStorageID() throws IOException { 240 | return getDataNode().getDNRegistrationForBP(getBlock().getBlockPoolId()).getDatanodeUuid(); 241 | } 242 | 243 | } 244 | -------------------------------------------------------------------------------- /src/r4h-cdh5.1.2/src/com/mellanox/r4h/client/HdfsDataInputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.IOException; 21 | import java.util.List; 22 | 23 | import org.apache.hadoop.classification.InterfaceAudience; 24 | import org.apache.hadoop.classification.InterfaceStability; 25 | import org.apache.hadoop.fs.FSDataInputStream; 26 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 27 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 28 | import org.apache.hadoop.hdfs.protocol.LocatedBlock; 29 | 30 | import com.mellanox.r4h.DFSInputStream; 31 | 32 | /** 33 | * The Hdfs implementation of {@link FSDataInputStream}. 34 | */ 35 | @InterfaceAudience.Public 36 | @InterfaceStability.Evolving 37 | public class HdfsDataInputStream extends FSDataInputStream { 38 | public HdfsDataInputStream(DFSInputStream in) throws IOException { 39 | super(in); 40 | } 41 | 42 | /** 43 | * Get the datanode from which the stream is currently reading. 44 | */ 45 | public DatanodeInfo getCurrentDatanode() { 46 | return ((DFSInputStream) in).getCurrentDatanode(); 47 | } 48 | 49 | /** 50 | * Get the block containing the target position. 51 | */ 52 | public ExtendedBlock getCurrentBlock() { 53 | return ((DFSInputStream) in).getCurrentBlock(); 54 | } 55 | 56 | /** 57 | * Get the collection of blocks that has already been located. 58 | */ 59 | public synchronized List getAllBlocks() throws IOException { 60 | return ((DFSInputStream) in).getAllBlocks(); 61 | } 62 | 63 | /** 64 | * Get the visible length of the file. It will include the length of the last 65 | * block even if that is in UnderConstruction state. 66 | * 67 | * @return The visible length of the file. 68 | */ 69 | public long getVisibleLength() throws IOException { 70 | return ((DFSInputStream) in).getFileLength(); 71 | } 72 | 73 | /** 74 | * Get statistics about the reads which this DFSInputStream has done. 75 | * Note that because HdfsDataInputStream is buffered, these stats may 76 | * be higher than you would expect just by adding up the number of 77 | * bytes read through HdfsDataInputStream. 78 | */ 79 | public synchronized DFSInputStream.ReadStatistics getReadStatistics() { 80 | return ((DFSInputStream) in).getReadStatistics(); 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/r4h-cdh5.1.2/src/com/mellanox/r4h/client/HdfsDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.IOException; 21 | import java.util.EnumSet; 22 | 23 | import org.apache.hadoop.classification.InterfaceAudience; 24 | import org.apache.hadoop.classification.InterfaceStability; 25 | import org.apache.hadoop.fs.FSDataOutputStream; 26 | import org.apache.hadoop.fs.FileSystem; 27 | 28 | import com.mellanox.r4h.DFSOutputStream; 29 | 30 | /** 31 | * The Hdfs implementation of {@link FSDataOutputStream}. 32 | */ 33 | @InterfaceAudience.Public 34 | @InterfaceStability.Evolving 35 | public class HdfsDataOutputStream extends FSDataOutputStream { 36 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 37 | super(out, stats, startPosition); 38 | } 39 | 40 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats) throws IOException { 41 | this(out, stats, 0L); 42 | } 43 | 44 | /** 45 | * Get the actual number of replicas of the current block. 46 | * 47 | * This can be different from the designated replication factor of the file 48 | * because the namenode does not maintain replication for the blocks which are 49 | * currently being written to. Depending on the configuration, the client may 50 | * continue to write to a block even if a few datanodes in the write pipeline 51 | * have failed, or the client may add a new datanodes once a datanode has 52 | * failed. 53 | * 54 | * @return the number of valid replicas of the current block 55 | */ 56 | public synchronized int getCurrentBlockReplication() throws IOException { 57 | return ((DFSOutputStream) getWrappedStream()).getCurrentBlockReplication(); 58 | } 59 | 60 | /** 61 | * Sync buffered data to DataNodes (flush to disk devices). 62 | * 63 | * @param syncFlags 64 | * Indicate the detailed semantic and actions of the hsync. 65 | * @throws IOException 66 | * @see FSDataOutputStream#hsync() 67 | */ 68 | public void hsync(EnumSet syncFlags) throws IOException { 69 | ((DFSOutputStream) getWrappedStream()).hsync(syncFlags); 70 | } 71 | 72 | public static enum SyncFlag { 73 | /** 74 | * When doing sync to DataNodes, also update the metadata (block 75 | * length) in the NameNode 76 | */ 77 | UPDATE_LENGTH; 78 | } 79 | } -------------------------------------------------------------------------------- /src/r4h-cdh5.1.2/src/org/apache/hadoop/hdfs/server/datanode/R4HBlockReceiverBase.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode; 2 | 3 | import java.io.DataInputStream; 4 | import java.io.IOException; 5 | 6 | import com.mellanox.r4h.WriteOprHeader; 7 | 8 | public abstract class R4HBlockReceiverBase extends BlockReceiver { 9 | 10 | public R4HBlockReceiverBase(WriteOprHeader oprHeader, DataInputStream inForHeaderOnly, String sessionInfo, DataNodeBridge dnEx) 11 | throws IOException { 12 | super(oprHeader.getBlock(), inForHeaderOnly, sessionInfo, sessionInfo, oprHeader.getStage(), oprHeader.getLatestGenerationStamp(), oprHeader 13 | .getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader.getClientName(), oprHeader.getSrcDataNode(), dnEx.getDN(), oprHeader 14 | .getRequestedChecksum(), oprHeader.getCachingStrategy()); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /src/r4h-cdh5.1.2/test/com/mellanox/r4h/MiniDFSClusterManager.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h; 19 | 20 | import java.io.File; 21 | import java.io.FileNotFoundException; 22 | import java.io.FileOutputStream; 23 | import java.io.FileWriter; 24 | import java.io.IOException; 25 | import java.util.Map; 26 | import java.util.TreeMap; 27 | 28 | import org.apache.commons.cli.CommandLine; 29 | import org.apache.commons.cli.CommandLineParser; 30 | import org.apache.commons.cli.GnuParser; 31 | import org.apache.commons.cli.HelpFormatter; 32 | import org.apache.commons.cli.OptionBuilder; 33 | import org.apache.commons.cli.Options; 34 | import org.apache.commons.cli.ParseException; 35 | import org.apache.commons.logging.Log; 36 | import org.apache.commons.logging.LogFactory; 37 | import org.apache.hadoop.conf.Configuration; 38 | import org.apache.hadoop.hdfs.HdfsConfiguration; 39 | import org.apache.hadoop.hdfs.MiniDFSCluster; 40 | import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; 41 | import org.mortbay.util.ajax.JSON; 42 | 43 | /** 44 | * This class drives the creation of a mini-cluster on the local machine. By 45 | * default, a MiniDFSCluster is spawned on the first available ports that are 46 | * found. 47 | * 48 | * A series of command line flags controls the startup cluster options. 49 | * 50 | * This class can dump a Hadoop configuration and some basic metadata (in JSON) 51 | * into a textfile. 52 | * 53 | * To shutdown the cluster, kill the process. 54 | * 55 | * To run this from the command line, do the following (replacing the jar 56 | * version as appropriate): 57 | * 58 | * $HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/share/hadoop/hdfs/hadoop-hdfs-0.24.0-SNAPSHOT-tests.jar org.apache.hadoop.test.MiniDFSClusterManager -options... 59 | */ 60 | public class MiniDFSClusterManager { 61 | private static final Log LOG = 62 | LogFactory.getLog(MiniDFSClusterManager.class); 63 | 64 | private MiniDFSCluster dfs; 65 | private String writeDetails; 66 | private int numDataNodes; 67 | private int nameNodePort; 68 | private StartupOption dfsOpts; 69 | private String writeConfig; 70 | private Configuration conf; 71 | private boolean format; 72 | 73 | private static final long SLEEP_INTERVAL_MS = 1000 * 60; 74 | 75 | /** 76 | * Creates configuration options object. 77 | */ 78 | @SuppressWarnings("static-access") 79 | private Options makeOptions() { 80 | Options options = new Options(); 81 | options 82 | .addOption("datanodes", true, "How many datanodes to start (default 1)") 83 | .addOption("format", false, "Format the DFS (default false)") 84 | .addOption("cmdport", true, 85 | "Which port to listen on for commands (default 0--we choose)") 86 | .addOption("nnport", true, "NameNode port (default 0--we choose)") 87 | .addOption("namenode", true, "URL of the namenode (default " 88 | + "is either the DFS cluster or a temporary dir)") 89 | .addOption(OptionBuilder 90 | .hasArgs() 91 | .withArgName("property=value") 92 | .withDescription("Options to pass into configuration object") 93 | .create("D")) 94 | .addOption(OptionBuilder 95 | .hasArg() 96 | .withArgName("path") 97 | .withDescription("Save configuration to this XML file.") 98 | .create("writeConfig")) 99 | .addOption(OptionBuilder 100 | .hasArg() 101 | .withArgName("path") 102 | .withDescription("Write basic information to this JSON file.") 103 | .create("writeDetails")) 104 | .addOption(OptionBuilder.withDescription("Prints option help.") 105 | .create("help")); 106 | return options; 107 | } 108 | 109 | /** 110 | * Main entry-point. 111 | */ 112 | public void run(String[] args) throws IOException { 113 | if (!parseArguments(args)) { 114 | return; 115 | } 116 | start(); 117 | sleepForever(); 118 | } 119 | 120 | private void sleepForever() { 121 | while (true) { 122 | try { 123 | Thread.sleep(SLEEP_INTERVAL_MS); 124 | if (!dfs.isClusterUp()) { 125 | LOG.info("Cluster is no longer up, exiting"); 126 | return; 127 | } 128 | } catch (InterruptedException e) { 129 | // nothing 130 | } 131 | } 132 | } 133 | 134 | /** 135 | * Starts DFS as specified in member-variable options. Also writes out 136 | * configuration and details, if requested. 137 | */ 138 | public void start() throws IOException, FileNotFoundException { 139 | dfs = new MiniDFSCluster.Builder(conf).nameNodePort(nameNodePort) 140 | .numDataNodes(numDataNodes) 141 | .startupOption(dfsOpts) 142 | .format(format) 143 | .build(); 144 | dfs.waitActive(); 145 | 146 | LOG.info("Started MiniDFSCluster -- namenode on port " 147 | + dfs.getNameNodePort()); 148 | 149 | if (writeConfig != null) { 150 | FileOutputStream fos = new FileOutputStream(new File(writeConfig)); 151 | conf.writeXml(fos); 152 | fos.close(); 153 | } 154 | 155 | if (writeDetails != null) { 156 | Map map = new TreeMap(); 157 | if (dfs != null) { 158 | map.put("namenode_port", dfs.getNameNodePort()); 159 | } 160 | 161 | FileWriter fw = new FileWriter(new File(writeDetails)); 162 | fw.write(new JSON().toJSON(map)); 163 | fw.close(); 164 | } 165 | } 166 | 167 | /** 168 | * Parses arguments and fills out the member variables. 169 | * @param args Command-line arguments. 170 | * @return true on successful parse; false to indicate that the 171 | * program should exit. 172 | */ 173 | private boolean parseArguments(String[] args) { 174 | Options options = makeOptions(); 175 | CommandLine cli; 176 | try { 177 | CommandLineParser parser = new GnuParser(); 178 | cli = parser.parse(options, args); 179 | } catch(ParseException e) { 180 | LOG.warn("options parsing failed: "+e.getMessage()); 181 | new HelpFormatter().printHelp("...", options); 182 | return false; 183 | } 184 | 185 | if (cli.hasOption("help")) { 186 | new HelpFormatter().printHelp("...", options); 187 | return false; 188 | } 189 | 190 | if (cli.getArgs().length > 0) { 191 | for (String arg : cli.getArgs()) { 192 | LOG.error("Unrecognized option: " + arg); 193 | new HelpFormatter().printHelp("...", options); 194 | return false; 195 | } 196 | } 197 | 198 | // HDFS 199 | numDataNodes = intArgument(cli, "datanodes", 1); 200 | nameNodePort = intArgument(cli, "nnport", 0); 201 | if (cli.hasOption("format")) { 202 | dfsOpts = StartupOption.FORMAT; 203 | format = true; 204 | } else { 205 | dfsOpts = StartupOption.REGULAR; 206 | format = false; 207 | } 208 | 209 | // Runner 210 | writeDetails = cli.getOptionValue("writeDetails"); 211 | writeConfig = cli.getOptionValue("writeConfig"); 212 | 213 | // General 214 | conf = new HdfsConfiguration(); 215 | updateConfiguration(conf, cli.getOptionValues("D")); 216 | 217 | return true; 218 | } 219 | 220 | /** 221 | * Updates configuration based on what's given on the command line. 222 | * 223 | * @param conf2 The configuration object 224 | * @param keyvalues An array of interleaved key value pairs. 225 | */ 226 | private void updateConfiguration(Configuration conf2, String[] keyvalues) { 227 | int num_confs_updated = 0; 228 | if (keyvalues != null) { 229 | for (String prop : keyvalues) { 230 | String[] keyval = prop.split("=", 2); 231 | if (keyval.length == 2) { 232 | conf2.set(keyval[0], keyval[1]); 233 | num_confs_updated++; 234 | } else { 235 | LOG.warn("Ignoring -D option " + prop); 236 | } 237 | } 238 | } 239 | LOG.info("Updated " + num_confs_updated + 240 | " configuration settings from command line."); 241 | } 242 | 243 | /** 244 | * Extracts an integer argument with specified default value. 245 | */ 246 | private int intArgument(CommandLine cli, String argName, int defaultValue) { 247 | String o = cli.getOptionValue(argName); 248 | try { 249 | if (o != null) { 250 | return Integer.parseInt(o); 251 | } 252 | } catch (NumberFormatException ex) { 253 | LOG.error("Couldn't parse value (" + o + ") for option " 254 | + argName + ". Using default: " + defaultValue); 255 | } 256 | 257 | return defaultValue; 258 | } 259 | 260 | /** 261 | * Starts a MiniDFSClusterManager with parameters drawn from the command line. 262 | */ 263 | public static void main(String[] args) throws IOException { 264 | new MiniDFSClusterManager().run(args); 265 | } 266 | } 267 | -------------------------------------------------------------------------------- /src/r4h-cdh5.1.2/test/com/mellanox/r4h/TestFSOutputSummer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h; 19 | 20 | import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; 21 | import static org.junit.Assert.assertEquals; 22 | import static org.junit.Assert.assertTrue; 23 | 24 | import java.io.IOException; 25 | import java.util.Random; 26 | 27 | import org.apache.hadoop.conf.Configuration; 28 | import org.apache.hadoop.fs.FSDataInputStream; 29 | import org.apache.hadoop.fs.FSDataOutputStream; 30 | import org.apache.hadoop.fs.FileSystem; 31 | import org.apache.hadoop.fs.Path; 32 | import org.apache.hadoop.hdfs.DFSConfigKeys; 33 | import org.apache.hadoop.hdfs.HdfsConfiguration; 34 | import org.junit.Test; 35 | 36 | /** 37 | * This class tests if FSOutputSummer works correctly. 38 | */ 39 | public class TestFSOutputSummer { 40 | private static final long seed = 0xDEADBEEFL; 41 | private static final int BYTES_PER_CHECKSUM = 10; 42 | private static final int BLOCK_SIZE = 2*BYTES_PER_CHECKSUM; 43 | private static final int HALF_CHUNK_SIZE = BYTES_PER_CHECKSUM/2; 44 | private static final int FILE_SIZE = 2*BLOCK_SIZE-1; 45 | private static final short NUM_OF_DATANODES = 2; 46 | private byte[] expected = new byte[FILE_SIZE]; 47 | private byte[] actual = new byte[FILE_SIZE]; 48 | private FileSystem fileSys; 49 | 50 | /* create a file, write all data at once */ 51 | private void writeFile1(Path name) throws Exception { 52 | FSDataOutputStream stm = fileSys.create(name, true, 53 | fileSys.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096), 54 | NUM_OF_DATANODES, BLOCK_SIZE); 55 | stm.write(expected); 56 | stm.close(); 57 | checkFile(name); 58 | cleanupFile(name); 59 | } 60 | 61 | /* create a file, write data chunk by chunk */ 62 | private void writeFile2(Path name) throws Exception { 63 | FSDataOutputStream stm = fileSys.create(name, true, 64 | fileSys.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096), 65 | NUM_OF_DATANODES, BLOCK_SIZE); 66 | int i=0; 67 | for( ;i() { 112 | @Override 113 | public DistributedFileSystem run() throws Exception { 114 | return (DistributedFileSystem)FileSystem.newInstance(conf); 115 | } 116 | }); 117 | final FSDataOutputStream out = append(dfs, p); 118 | write(out, 0, half); 119 | out.close(); 120 | } 121 | 122 | //d. On M2, open file and read 1 block of data from it. Close file. 123 | checkFile(p, 2*half, conf); 124 | } finally { 125 | cluster.shutdown(); 126 | } 127 | } 128 | 129 | /** Try openning a file for append. */ 130 | private static FSDataOutputStream append(FileSystem fs, Path p) throws Exception { 131 | for(int i = 0; i < 10; i++) { 132 | try { 133 | return fs.append(p); 134 | } catch(RemoteException re) { 135 | if (re.getClassName().equals(RecoveryInProgressException.class.getName())) { 136 | MiniDFSClusterBridge.getAppendTestUtilLOG().info("Will sleep and retry, i=" + i +", p="+p, re); 137 | Thread.sleep(1000); 138 | } 139 | else 140 | throw re; 141 | } 142 | } 143 | throw new IOException("Cannot append to " + p); 144 | } 145 | 146 | static private int userCount = 0; 147 | //check the file 148 | static void checkFile(Path p, int expectedsize, final Configuration conf 149 | ) throws IOException, InterruptedException { 150 | //open the file with another user account 151 | final String username = UserGroupInformation.getCurrentUser().getShortUserName() 152 | + "_" + ++userCount; 153 | 154 | UserGroupInformation ugi = UserGroupInformation.createUserForTesting(username, 155 | new String[] {"supergroup"}); 156 | 157 | final FileSystem fs = DFSTestUtil.getFileSystemAs(ugi, conf); 158 | 159 | final HdfsDataInputStream in = (HdfsDataInputStream)fs.open(p); 160 | 161 | //Check visible length 162 | Assert.assertTrue(in.getVisibleLength() >= expectedsize); 163 | 164 | //Able to read? 165 | for(int i = 0; i < expectedsize; i++) { 166 | Assert.assertEquals((byte)i, (byte)in.read()); 167 | } 168 | 169 | in.close(); 170 | } 171 | 172 | /** Write something to a file */ 173 | private static void write(OutputStream out, int offset, int length 174 | ) throws IOException { 175 | final byte[] bytes = new byte[length]; 176 | for(int i = 0; i < length; i++) { 177 | bytes[i] = (byte)(offset + i); 178 | } 179 | out.write(bytes); 180 | } 181 | } 182 | 183 | -------------------------------------------------------------------------------- /src/r4h-cdh5.1.2/test/hdfs/server/datanode/fsdataset/impl/FsDataSetBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; 2 | 3 | import java.io.File; 4 | import java.io.IOException; 5 | 6 | 7 | public class FsDataSetBridge { 8 | 9 | public static File findMetaFile(File blockFile) throws IOException { 10 | return FsDatasetUtil.findMetaFile(blockFile); 11 | } 12 | 13 | } 14 | -------------------------------------------------------------------------------- /src/r4h-cdh5.1.2/test/org/apache/hadoop/hdfs/MiniDFSClusterBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import java.lang.reflect.Method; 4 | 5 | import org.apache.commons.logging.Log; 6 | import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf; 7 | 8 | public class MiniDFSClusterBridge { 9 | 10 | public static int getAppendTestUtils_FILE_SIZE() { 11 | return AppendTestUtil.FILE_SIZE; 12 | } 13 | 14 | public static int getAppendTestUtil_BLOCK_SIZE() { 15 | return AppendTestUtil.BLOCK_SIZE; 16 | } 17 | 18 | public static int getNNConf_ipcPort(NNConf nnConf) { 19 | return nnConf.getIpcPort(); 20 | } 21 | 22 | public static String getNNConf_nnId(NNConf nnConf) { 23 | return nnConf.getNnId(); 24 | } 25 | 26 | public static String getNNConf_cliusterId(NNConf nnConf) { 27 | return nnConf.getClusterId(); 28 | } 29 | 30 | public static int getNNConf_httpPort(NNConf nnConf) { 31 | return nnConf.getHttpPort(); 32 | } 33 | 34 | public static Log getAppendTestUtilLOG() { 35 | return AppendTestUtil.LOG; 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/src/com/mellanox/r4h/DataXceiver.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto; 4 | import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; 5 | 6 | import java.io.DataInputStream; 7 | import java.io.DataOutputStream; 8 | import java.io.IOException; 9 | 10 | import org.accelio.jxio.ServerSession.SessionKey; 11 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 12 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 13 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 14 | import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; 15 | import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; 16 | import org.apache.hadoop.hdfs.protocolPB.PBHelper; 17 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 18 | 19 | public class DataXceiver extends DataXceiverBase { 20 | 21 | DataXceiver(DataXceiverServer dxcs, ServerPortalWorker spw, SessionKey sKey, R4HExecutor ioExec, R4HExecutor auxExec) { 22 | super(dxcs, spw, sKey, ioExec, auxExec); 23 | } 24 | 25 | @Override 26 | void parseOpWriteBlock(DataInputStream in) throws IOException { 27 | final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in)); 28 | final DatanodeInfo[] targets = PBHelper.convert(proto.getTargetsList()); 29 | oprHeader = new WriteOprHeader2_5(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convertStorageType(proto 30 | .getStorageType()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), targets, 31 | PBHelper.convertStorageTypes(proto.getTargetStorageTypesList(), targets.length), PBHelper.convert(proto.getSource()), 32 | BlockConstructionStage.valueOf(proto.getStage().name()), proto.getPipelineSize(), proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(), 33 | proto.getLatestGenerationStamp(), fromProto(proto.getRequestedChecksum()), 34 | (proto.hasCachingStrategy() ? getCachingStrategy(proto.getCachingStrategy()) : CachingStrategy.newDefaultStrategy())); 35 | } 36 | 37 | @Override 38 | void senderWriteBlock(DataOutputStream out, ExtendedBlock origBlk) throws IOException { 39 | new Sender(out).writeBlock(origBlk, ((WriteOprHeader2_5) oprHeader).getStorageType(), oprHeader.getBlockToken(), oprHeader.getClientName(), 40 | oprHeader.getTargets(), ((WriteOprHeader2_5) oprHeader).getAllStorageTypes(), oprHeader.getSrcDataNode(), oprHeader.getStage(), 41 | oprHeader.getPipelineSize(), oprHeader.getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader.getLatestGenerationStamp(), 42 | oprHeader.getRequestedChecksum(), oprHeader.getCachingStrategy()); 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/src/com/mellanox/r4h/WriteOprHeader2_5.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.hadoop.hdfs.StorageType; 21 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 22 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 23 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 24 | import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; 25 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 26 | import org.apache.hadoop.security.token.Token; 27 | import org.apache.hadoop.util.DataChecksum; 28 | 29 | /** 30 | * Encapsulating HDFS WriteOperationHeader to be read from a DataInput and written to a DataOutput. Additional it 31 | * provides the ability to write itself to a DataOutput with a pipeline modification 32 | * This is the extension for Hadoop 2.5.0 33 | */ 34 | public class WriteOprHeader2_5 extends WriteOprHeader { 35 | 36 | final private StorageType storageType; 37 | final private StorageType[] allStorageTypes; 38 | 39 | public WriteOprHeader2_5(ExtendedBlock blk, StorageType storageType, Token blockToken, String clientName, 40 | DatanodeInfo[] targets, StorageType[] allStorageTypes, DatanodeInfo source, BlockConstructionStage stage, int pipelineSize, 41 | long minBytesRcvd, long maxBytesRcvd, long latestGenerationStamp, DataChecksum requestedChecksum, CachingStrategy cachingStrategy) { 42 | super(blk, blockToken, clientName, targets, source, stage, pipelineSize, minBytesRcvd, maxBytesRcvd, latestGenerationStamp, 43 | requestedChecksum, cachingStrategy); 44 | this.storageType = storageType; 45 | this.allStorageTypes = allStorageTypes; 46 | 47 | } 48 | 49 | public StorageType getStorageType() { 50 | return storageType; 51 | } 52 | 53 | public StorageType[] getAllStorageTypes() { 54 | return allStorageTypes; 55 | } 56 | 57 | } 58 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/src/com/mellanox/r4h/client/HdfsDataInputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.InputStream; 21 | import java.io.IOException; 22 | import java.util.List; 23 | 24 | import org.apache.hadoop.classification.InterfaceAudience; 25 | import org.apache.hadoop.classification.InterfaceStability; 26 | import org.apache.hadoop.fs.FSDataInputStream; 27 | import org.apache.hadoop.crypto.CryptoInputStream; 28 | import com.mellanox.r4h.DFSInputStream; 29 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 30 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 31 | import org.apache.hadoop.hdfs.protocol.LocatedBlock; 32 | 33 | import com.google.common.base.Preconditions; 34 | 35 | /** 36 | * The Hdfs implementation of {@link FSDataInputStream}. 37 | */ 38 | @InterfaceAudience.Public 39 | @InterfaceStability.Evolving 40 | public class HdfsDataInputStream extends FSDataInputStream { 41 | public HdfsDataInputStream(DFSInputStream in) throws IOException { 42 | super(in); 43 | } 44 | 45 | public HdfsDataInputStream(CryptoInputStream in) throws IOException { 46 | super(in); 47 | Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream, "CryptoInputStream should wrap a DFSInputStream"); 48 | } 49 | 50 | private DFSInputStream getDFSInputStream() { 51 | if (in instanceof CryptoInputStream) { 52 | return (DFSInputStream) ((CryptoInputStream) in).getWrappedStream(); 53 | } 54 | return (DFSInputStream) in; 55 | } 56 | 57 | /** 58 | * Get a reference to the wrapped output stream. We always want to return the 59 | * actual underlying InputStream, even when we're using a CryptoStream. e.g. 60 | * in the delegated methods below. 61 | * 62 | * @return the underlying output stream 63 | */ 64 | public InputStream getWrappedStream() { 65 | return in; 66 | } 67 | 68 | /** 69 | * Get the datanode from which the stream is currently reading. 70 | */ 71 | public DatanodeInfo getCurrentDatanode() { 72 | return getDFSInputStream().getCurrentDatanode(); 73 | } 74 | 75 | /** 76 | * Get the block containing the target position. 77 | */ 78 | public ExtendedBlock getCurrentBlock() { 79 | return getDFSInputStream().getCurrentBlock(); 80 | } 81 | 82 | /** 83 | * Get the collection of blocks that has already been located. 84 | */ 85 | public synchronized List getAllBlocks() throws IOException { 86 | return getDFSInputStream().getAllBlocks(); 87 | } 88 | 89 | /** 90 | * Get the visible length of the file. It will include the length of the last 91 | * block even if that is in UnderConstruction state. 92 | * 93 | * @return The visible length of the file. 94 | */ 95 | public long getVisibleLength() throws IOException { 96 | return getDFSInputStream().getFileLength(); 97 | } 98 | 99 | /** 100 | * Get statistics about the reads which this DFSInputStream has done. 101 | * Note that because HdfsDataInputStream is buffered, these stats may 102 | * be higher than you would expect just by adding up the number of 103 | * bytes read through HdfsDataInputStream. 104 | */ 105 | public synchronized DFSInputStream.ReadStatistics getReadStatistics() { 106 | return getDFSInputStream().getReadStatistics(); 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/src/com/mellanox/r4h/client/HdfsDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.IOException; 21 | import java.io.OutputStream; 22 | import java.util.EnumSet; 23 | 24 | import org.apache.hadoop.classification.InterfaceAudience; 25 | import org.apache.hadoop.classification.InterfaceStability; 26 | import org.apache.hadoop.crypto.CryptoOutputStream; 27 | import org.apache.hadoop.fs.FSDataOutputStream; 28 | import org.apache.hadoop.fs.FileSystem; 29 | import com.mellanox.r4h.DFSOutputStream; 30 | 31 | import com.google.common.base.Preconditions; 32 | 33 | /** 34 | * The Hdfs implementation of {@link FSDataOutputStream}. 35 | */ 36 | @InterfaceAudience.Public 37 | @InterfaceStability.Evolving 38 | public class HdfsDataOutputStream extends FSDataOutputStream { 39 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 40 | super(out, stats, startPosition); 41 | } 42 | 43 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats) throws IOException { 44 | this(out, stats, 0L); 45 | } 46 | 47 | public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 48 | super(out, stats, startPosition); 49 | Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream, "CryptoOutputStream should wrap a DFSOutputStream"); 50 | } 51 | 52 | public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats) throws IOException { 53 | this(out, stats, 0L); 54 | } 55 | 56 | /** 57 | * Get the actual number of replicas of the current block. 58 | * 59 | * This can be different from the designated replication factor of the file 60 | * because the namenode does not maintain replication for the blocks which are 61 | * currently being written to. Depending on the configuration, the client may 62 | * continue to write to a block even if a few datanodes in the write pipeline 63 | * have failed, or the client may add a new datanodes once a datanode has 64 | * failed. 65 | * 66 | * @return the number of valid replicas of the current block 67 | */ 68 | public synchronized int getCurrentBlockReplication() throws IOException { 69 | OutputStream wrappedStream = getWrappedStream(); 70 | if (wrappedStream instanceof CryptoOutputStream) { 71 | wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream(); 72 | } 73 | return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication(); 74 | } 75 | 76 | /** 77 | * Sync buffered data to DataNodes (flush to disk devices). 78 | * 79 | * @param syncFlags 80 | * Indicate the detailed semantic and actions of the hsync. 81 | * @throws IOException 82 | * @see FSDataOutputStream#hsync() 83 | */ 84 | public void hsync(EnumSet syncFlags) throws IOException { 85 | OutputStream wrappedStream = getWrappedStream(); 86 | if (wrappedStream instanceof CryptoOutputStream) { 87 | ((CryptoOutputStream) wrappedStream).flush(); 88 | wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream(); 89 | } 90 | ((DFSOutputStream) wrappedStream).hsync(syncFlags); 91 | } 92 | 93 | public static enum SyncFlag { 94 | 95 | /** 96 | * When doing sync to DataNodes, also update the metadata (block length) in 97 | * the NameNode. 98 | */ 99 | UPDATE_LENGTH; 100 | } 101 | } 102 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/src/org/apache/hadoop/hdfs/DFSInotifyEventInputStreamBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import java.io.IOException; 4 | 5 | import org.apache.hadoop.hdfs.protocol.ClientProtocol; 6 | 7 | public class DFSInotifyEventInputStreamBridge extends DFSInotifyEventInputStream { 8 | 9 | public DFSInotifyEventInputStreamBridge(ClientProtocol namenode) throws IOException { 10 | super(namenode); 11 | } 12 | 13 | public DFSInotifyEventInputStreamBridge(ClientProtocol namenode, long lastReadTxid) throws IOException { 14 | super(namenode, lastReadTxid); 15 | } 16 | 17 | } 18 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/src/org/apache/hadoop/hdfs/server/datanode/R4HBlockReceiverBase.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode; 2 | 3 | import java.io.DataInputStream; 4 | import java.io.IOException; 5 | 6 | import com.mellanox.r4h.WriteOprHeader; 7 | import com.mellanox.r4h.WriteOprHeader2_5; 8 | 9 | public abstract class R4HBlockReceiverBase extends BlockReceiver { 10 | 11 | public R4HBlockReceiverBase(WriteOprHeader oprHeader, DataInputStream inForHeaderOnly, String sessionInfo, DataNodeBridge dnEx) 12 | throws IOException { 13 | super(oprHeader.getBlock(), ((WriteOprHeader2_5) oprHeader).getStorageType(), inForHeaderOnly, sessionInfo, sessionInfo, 14 | oprHeader.getStage(), oprHeader.getLatestGenerationStamp(), oprHeader.getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader 15 | .getClientName(), oprHeader.getSrcDataNode(), dnEx.getDN(), oprHeader.getRequestedChecksum(), oprHeader.getCachingStrategy()); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/test/com/mellanox/r4h/MiniDFSClusterManager.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h; 19 | 20 | import java.io.File; 21 | import java.io.FileNotFoundException; 22 | import java.io.FileOutputStream; 23 | import java.io.FileWriter; 24 | import java.io.IOException; 25 | import java.util.Map; 26 | import java.util.TreeMap; 27 | 28 | import org.apache.commons.cli.CommandLine; 29 | import org.apache.commons.cli.CommandLineParser; 30 | import org.apache.commons.cli.GnuParser; 31 | import org.apache.commons.cli.HelpFormatter; 32 | import org.apache.commons.cli.OptionBuilder; 33 | import org.apache.commons.cli.Options; 34 | import org.apache.commons.cli.ParseException; 35 | import org.apache.commons.logging.Log; 36 | import org.apache.commons.logging.LogFactory; 37 | import org.apache.hadoop.conf.Configuration; 38 | import org.apache.hadoop.hdfs.HdfsConfiguration; 39 | import org.apache.hadoop.hdfs.MiniDFSCluster; 40 | import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; 41 | import org.mortbay.util.ajax.JSON; 42 | 43 | /** 44 | * This class drives the creation of a mini-cluster on the local machine. By 45 | * default, a MiniDFSCluster is spawned on the first available ports that are 46 | * found. 47 | * 48 | * A series of command line flags controls the startup cluster options. 49 | * 50 | * This class can dump a Hadoop configuration and some basic metadata (in JSON) 51 | * into a textfile. 52 | * 53 | * To shutdown the cluster, kill the process. 54 | * 55 | * To run this from the command line, do the following (replacing the jar 56 | * version as appropriate): 57 | * 58 | * $HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/share/hadoop/hdfs/hadoop-hdfs-0.24.0-SNAPSHOT-tests.jar org.apache.hadoop.test.MiniDFSClusterManager 59 | * -options... 60 | */ 61 | public class MiniDFSClusterManager { 62 | private static final Log LOG = LogFactory.getLog(MiniDFSClusterManager.class); 63 | 64 | private MiniDFSCluster dfs; 65 | private String writeDetails; 66 | private int numDataNodes; 67 | private int nameNodePort; 68 | private StartupOption dfsOpts; 69 | private String writeConfig; 70 | private Configuration conf; 71 | private boolean format; 72 | 73 | private static final long SLEEP_INTERVAL_MS = 1000 * 60; 74 | 75 | /** 76 | * Creates configuration options object. 77 | */ 78 | @SuppressWarnings("static-access") 79 | private Options makeOptions() { 80 | Options options = new Options(); 81 | options.addOption("datanodes", true, "How many datanodes to start (default 1)") 82 | .addOption("format", false, "Format the DFS (default false)") 83 | .addOption("cmdport", true, "Which port to listen on for commands (default 0--we choose)") 84 | .addOption("nnport", true, "NameNode port (default 0--we choose)") 85 | .addOption("namenode", true, "URL of the namenode (default " + "is either the DFS cluster or a temporary dir)") 86 | .addOption( 87 | OptionBuilder.hasArgs().withArgName("property=value").withDescription("Options to pass into configuration object") 88 | .create("D")) 89 | .addOption(OptionBuilder.hasArg().withArgName("path").withDescription("Save configuration to this XML file.").create("writeConfig")) 90 | .addOption( 91 | OptionBuilder.hasArg().withArgName("path").withDescription("Write basic information to this JSON file.") 92 | .create("writeDetails")).addOption(OptionBuilder.withDescription("Prints option help.").create("help")); 93 | return options; 94 | } 95 | 96 | /** 97 | * Main entry-point. 98 | */ 99 | public void run(String[] args) throws IOException { 100 | if (!parseArguments(args)) { 101 | return; 102 | } 103 | start(); 104 | sleepForever(); 105 | } 106 | 107 | private void sleepForever() { 108 | while (true) { 109 | try { 110 | Thread.sleep(SLEEP_INTERVAL_MS); 111 | if (!dfs.isClusterUp()) { 112 | LOG.info("Cluster is no longer up, exiting"); 113 | return; 114 | } 115 | } catch (InterruptedException e) { 116 | // nothing 117 | } 118 | } 119 | } 120 | 121 | /** 122 | * Starts DFS as specified in member-variable options. Also writes out 123 | * configuration and details, if requested. 124 | */ 125 | public void start() throws IOException, FileNotFoundException { 126 | dfs = new MiniDFSCluster.Builder(conf).nameNodePort(nameNodePort).numDataNodes(numDataNodes).startupOption(dfsOpts).format(format).build(); 127 | dfs.waitActive(); 128 | 129 | LOG.info("Started MiniDFSCluster -- namenode on port " + dfs.getNameNodePort()); 130 | 131 | if (writeConfig != null) { 132 | FileOutputStream fos = new FileOutputStream(new File(writeConfig)); 133 | conf.writeXml(fos); 134 | fos.close(); 135 | } 136 | 137 | if (writeDetails != null) { 138 | Map map = new TreeMap(); 139 | if (dfs != null) { 140 | map.put("namenode_port", dfs.getNameNodePort()); 141 | } 142 | 143 | FileWriter fw = new FileWriter(new File(writeDetails)); 144 | fw.write(new JSON().toJSON(map)); 145 | fw.close(); 146 | } 147 | } 148 | 149 | /** 150 | * Parses arguments and fills out the member variables. 151 | * 152 | * @param args 153 | * Command-line arguments. 154 | * @return true on successful parse; false to indicate that the 155 | * program should exit. 156 | */ 157 | private boolean parseArguments(String[] args) { 158 | Options options = makeOptions(); 159 | CommandLine cli; 160 | try { 161 | CommandLineParser parser = new GnuParser(); 162 | cli = parser.parse(options, args); 163 | } catch (ParseException e) { 164 | LOG.warn("options parsing failed: " + e.getMessage()); 165 | new HelpFormatter().printHelp("...", options); 166 | return false; 167 | } 168 | 169 | if (cli.hasOption("help")) { 170 | new HelpFormatter().printHelp("...", options); 171 | return false; 172 | } 173 | 174 | if (cli.getArgs().length > 0) { 175 | for (String arg : cli.getArgs()) { 176 | LOG.error("Unrecognized option: " + arg); 177 | new HelpFormatter().printHelp("...", options); 178 | return false; 179 | } 180 | } 181 | 182 | // HDFS 183 | numDataNodes = intArgument(cli, "datanodes", 1); 184 | nameNodePort = intArgument(cli, "nnport", 0); 185 | if (cli.hasOption("format")) { 186 | dfsOpts = StartupOption.FORMAT; 187 | format = true; 188 | } else { 189 | dfsOpts = StartupOption.REGULAR; 190 | format = false; 191 | } 192 | 193 | // Runner 194 | writeDetails = cli.getOptionValue("writeDetails"); 195 | writeConfig = cli.getOptionValue("writeConfig"); 196 | 197 | // General 198 | conf = new HdfsConfiguration(); 199 | updateConfiguration(conf, cli.getOptionValues("D")); 200 | 201 | return true; 202 | } 203 | 204 | /** 205 | * Updates configuration based on what's given on the command line. 206 | * 207 | * @param conf2 208 | * The configuration object 209 | * @param keyvalues 210 | * An array of interleaved key value pairs. 211 | */ 212 | private void updateConfiguration(Configuration conf2, String[] keyvalues) { 213 | int num_confs_updated = 0; 214 | if (keyvalues != null) { 215 | for (String prop : keyvalues) { 216 | String[] keyval = prop.split("=", 2); 217 | if (keyval.length == 2) { 218 | conf2.set(keyval[0], keyval[1]); 219 | num_confs_updated++; 220 | } else { 221 | LOG.warn("Ignoring -D option " + prop); 222 | } 223 | } 224 | } 225 | LOG.info("Updated " + num_confs_updated + " configuration settings from command line."); 226 | } 227 | 228 | /** 229 | * Extracts an integer argument with specified default value. 230 | */ 231 | private int intArgument(CommandLine cli, String argName, int defaultValue) { 232 | String o = cli.getOptionValue(argName); 233 | try { 234 | if (o != null) { 235 | return Integer.parseInt(o); 236 | } 237 | } catch (NumberFormatException ex) { 238 | LOG.error("Couldn't parse value (" + o + ") for option " + argName + ". Using default: " + defaultValue); 239 | } 240 | 241 | return defaultValue; 242 | } 243 | 244 | /** 245 | * Starts a MiniDFSClusterManager with parameters drawn from the command line. 246 | */ 247 | public static void main(String[] args) throws IOException { 248 | new MiniDFSClusterManager().run(args); 249 | } 250 | } 251 | -------------------------------------------------------------------------------- /src/r4h-cdh5.3.1/test/org/apache/hadoop/hdfs/MiniDFSClusterBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import java.lang.reflect.Method; 4 | 5 | import org.apache.commons.logging.Log; 6 | import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf; 7 | 8 | public class MiniDFSClusterBridge { 9 | 10 | public static int getAppendTestUtils_FILE_SIZE() { 11 | return AppendTestUtil.FILE_SIZE; 12 | } 13 | 14 | public static int getAppendTestUtil_BLOCK_SIZE() { 15 | return AppendTestUtil.BLOCK_SIZE; 16 | } 17 | 18 | public static int getNNConf_ipcPort(NNConf nnConf) { 19 | return nnConf.getIpcPort(); 20 | } 21 | 22 | public static String getNNConf_nnId(NNConf nnConf) { 23 | return nnConf.getNnId(); 24 | } 25 | 26 | public static String getNNConf_cliusterId(NNConf nnConf) { 27 | return nnConf.getClusterId(); 28 | } 29 | 30 | public static int getNNConf_httpPort(NNConf nnConf) { 31 | return nnConf.getHttpPort(); 32 | } 33 | 34 | public static Log getAppendTestUtilLOG() { 35 | return AppendTestUtil.LOG; 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /src/r4h-hdp2.1.2/src/com/mellanox/r4h/client/HdfsDataInputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.IOException; 21 | import java.util.List; 22 | 23 | import org.apache.hadoop.classification.InterfaceAudience; 24 | import org.apache.hadoop.classification.InterfaceStability; 25 | import org.apache.hadoop.fs.FSDataInputStream; 26 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 27 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 28 | import org.apache.hadoop.hdfs.protocol.LocatedBlock; 29 | 30 | import com.mellanox.r4h.DFSInputStream; 31 | 32 | /** 33 | * The Hdfs implementation of {@link FSDataInputStream}. 34 | */ 35 | @InterfaceAudience.Public 36 | @InterfaceStability.Evolving 37 | public class HdfsDataInputStream extends FSDataInputStream { 38 | public HdfsDataInputStream(DFSInputStream in) throws IOException { 39 | super(in); 40 | } 41 | 42 | /** 43 | * Get the datanode from which the stream is currently reading. 44 | */ 45 | public DatanodeInfo getCurrentDatanode() { 46 | return ((DFSInputStream) in).getCurrentDatanode(); 47 | } 48 | 49 | /** 50 | * Get the block containing the target position. 51 | */ 52 | public ExtendedBlock getCurrentBlock() { 53 | return ((DFSInputStream) in).getCurrentBlock(); 54 | } 55 | 56 | /** 57 | * Get the collection of blocks that has already been located. 58 | */ 59 | public synchronized List getAllBlocks() throws IOException { 60 | return ((DFSInputStream) in).getAllBlocks(); 61 | } 62 | 63 | /** 64 | * Get the visible length of the file. It will include the length of the last 65 | * block even if that is in UnderConstruction state. 66 | * 67 | * @return The visible length of the file. 68 | */ 69 | public long getVisibleLength() throws IOException { 70 | return ((DFSInputStream) in).getFileLength(); 71 | } 72 | 73 | /** 74 | * Get statistics about the reads which this DFSInputStream has done. 75 | * Note that because HdfsDataInputStream is buffered, these stats may 76 | * be higher than you would expect just by adding up the number of 77 | * bytes read through HdfsDataInputStream. 78 | */ 79 | public synchronized DFSInputStream.ReadStatistics getReadStatistics() { 80 | return ((DFSInputStream) in).getReadStatistics(); 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/r4h-hdp2.1.2/src/com/mellanox/r4h/client/HdfsDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.IOException; 21 | import java.util.EnumSet; 22 | 23 | import org.apache.hadoop.classification.InterfaceAudience; 24 | import org.apache.hadoop.classification.InterfaceStability; 25 | import org.apache.hadoop.fs.FSDataOutputStream; 26 | import org.apache.hadoop.fs.FileSystem; 27 | 28 | import com.mellanox.r4h.DFSOutputStream; 29 | 30 | /** 31 | * The Hdfs implementation of {@link FSDataOutputStream}. 32 | */ 33 | @InterfaceAudience.Public 34 | @InterfaceStability.Evolving 35 | public class HdfsDataOutputStream extends FSDataOutputStream { 36 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 37 | super(out, stats, startPosition); 38 | } 39 | 40 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats) throws IOException { 41 | this(out, stats, 0L); 42 | } 43 | 44 | /** 45 | * Get the actual number of replicas of the current block. 46 | * 47 | * This can be different from the designated replication factor of the file 48 | * because the namenode does not maintain replication for the blocks which are 49 | * currently being written to. Depending on the configuration, the client may 50 | * continue to write to a block even if a few datanodes in the write pipeline 51 | * have failed, or the client may add a new datanodes once a datanode has 52 | * failed. 53 | * 54 | * @return the number of valid replicas of the current block 55 | */ 56 | public synchronized int getCurrentBlockReplication() throws IOException { 57 | return ((DFSOutputStream) getWrappedStream()).getCurrentBlockReplication(); 58 | } 59 | 60 | /** 61 | * Sync buffered data to DataNodes (flush to disk devices). 62 | * 63 | * @param syncFlags 64 | * Indicate the detailed semantic and actions of the hsync. 65 | * @throws IOException 66 | * @see FSDataOutputStream#hsync() 67 | */ 68 | public void hsync(EnumSet syncFlags) throws IOException { 69 | ((DFSOutputStream) getWrappedStream()).hsync(syncFlags); 70 | } 71 | 72 | public static enum SyncFlag { 73 | /** 74 | * When doing sync to DataNodes, also update the metadata (block 75 | * length) in the NameNode 76 | */ 77 | UPDATE_LENGTH; 78 | } 79 | } -------------------------------------------------------------------------------- /src/r4h-hdp2.1.2/src/org/apache/hadoop/hdfs/server/datanode/R4HBlockReceiverBase.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode; 2 | 3 | import java.io.DataInputStream; 4 | import java.io.IOException; 5 | import com.mellanox.r4h.WriteOprHeader; 6 | 7 | public abstract class R4HBlockReceiverBase extends BlockReceiver { 8 | public R4HBlockReceiverBase(WriteOprHeader oprHeader, DataInputStream inForHeaderOnly, String sessionInfo, DataNodeBridge dnEx) 9 | throws IOException, SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException, NoSuchMethodException { 10 | super(oprHeader.getBlock(), inForHeaderOnly, sessionInfo, sessionInfo, oprHeader.getStage(), oprHeader.getLatestGenerationStamp(), oprHeader 11 | .getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader.getClientName(), oprHeader.getSrcDataNode(), dnEx.getDN(), oprHeader 12 | .getRequestedChecksum(), oprHeader.getCachingStrategy()); 13 | } 14 | 15 | } 16 | -------------------------------------------------------------------------------- /src/r4h-hdp2.1.2/test/TBD: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Mellanox/R4H/db73b0ebceb55cc73691685a76933eb5b6b707f8/src/r4h-hdp2.1.2/test/TBD -------------------------------------------------------------------------------- /src/r4h-hdp2.2.0/src/com/mellanox/r4h/DataXceiver.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto; 4 | import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; 5 | 6 | import java.io.DataInputStream; 7 | import java.io.DataOutputStream; 8 | import java.io.IOException; 9 | 10 | import org.accelio.jxio.ServerSession.SessionKey; 11 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 12 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 13 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 14 | import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; 15 | import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; 16 | import org.apache.hadoop.hdfs.protocolPB.PBHelper; 17 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 18 | 19 | public class DataXceiver extends DataXceiverBase { 20 | 21 | DataXceiver(DataXceiverServer dxcs, ServerPortalWorker spw, SessionKey sKey, R4HExecutor ioExec, R4HExecutor auxExec) { 22 | super(dxcs, spw, sKey, ioExec, auxExec); 23 | } 24 | 25 | @Override 26 | void parseOpWriteBlock(DataInputStream in) throws IOException { 27 | final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in)); 28 | final DatanodeInfo[] targets = PBHelper.convert(proto.getTargetsList()); 29 | oprHeader = new WriteOprHeader2_6(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convertStorageType(proto 30 | .getStorageType()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), targets, 31 | PBHelper.convertStorageTypes(proto.getTargetStorageTypesList(), targets.length), PBHelper.convert(proto.getSource()), 32 | BlockConstructionStage.valueOf(proto.getStage().name()), proto.getPipelineSize(), proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(), 33 | proto.getLatestGenerationStamp(), fromProto(proto.getRequestedChecksum()), 34 | (proto.hasCachingStrategy() ? getCachingStrategy(proto.getCachingStrategy()) : CachingStrategy.newDefaultStrategy()), 35 | (proto.hasAllowLazyPersist() ? proto.getAllowLazyPersist() : false)); 36 | } 37 | 38 | @Override 39 | void senderWriteBlock(DataOutputStream out, ExtendedBlock origBlk) throws IOException { 40 | new Sender(out).writeBlock(origBlk, ((WriteOprHeader2_6) oprHeader).getStorageType(), oprHeader.getBlockToken(), oprHeader.getClientName(), 41 | oprHeader.getTargets(), ((WriteOprHeader2_6) oprHeader).getAllStorageTypes(), oprHeader.getSrcDataNode(), oprHeader.getStage(), 42 | oprHeader.getPipelineSize(), oprHeader.getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader.getLatestGenerationStamp(), 43 | oprHeader.getRequestedChecksum(), oprHeader.getCachingStrategy(), ((WriteOprHeader2_6) oprHeader).getAllowLazyPersist()); 44 | } 45 | 46 | } 47 | -------------------------------------------------------------------------------- /src/r4h-hdp2.2.0/src/com/mellanox/r4h/WriteOprHeader2_6.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.hadoop.hdfs.StorageType; 21 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 22 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 23 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 24 | import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; 25 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 26 | import org.apache.hadoop.security.token.Token; 27 | import org.apache.hadoop.util.DataChecksum; 28 | 29 | /** 30 | * Encapsulating HDFS WriteOperationHeader to be read from a DataInput and written to a DataOutput. Additional it 31 | * provides the ability to write itself to a DataOutput with a pipeline modification 32 | * This is the extension for Hadoop 2.6.0 33 | */ 34 | public class WriteOprHeader2_6 extends WriteOprHeader { 35 | 36 | final private StorageType storageType; 37 | final private StorageType[] allStorageTypes; 38 | final private boolean allowLazyPersist; 39 | 40 | public WriteOprHeader2_6(ExtendedBlock blk, StorageType storageType, Token blockToken, String clientName, 41 | DatanodeInfo[] targets, StorageType[] allStorageTypes, DatanodeInfo source, BlockConstructionStage stage, int pipelineSize, 42 | long minBytesRcvd, long maxBytesRcvd, long latestGenerationStamp, DataChecksum requestedChecksum, CachingStrategy cachingStrategy, 43 | boolean allowLazyPersist) { 44 | super(blk, blockToken, clientName, targets, source, stage, pipelineSize, minBytesRcvd, maxBytesRcvd, latestGenerationStamp, 45 | requestedChecksum, cachingStrategy); 46 | this.storageType = storageType; 47 | this.allStorageTypes = allStorageTypes; 48 | this.allowLazyPersist = allowLazyPersist; 49 | 50 | } 51 | 52 | public StorageType getStorageType() { 53 | return storageType; 54 | } 55 | 56 | public StorageType[] getAllStorageTypes() { 57 | return allStorageTypes; 58 | } 59 | 60 | public boolean getAllowLazyPersist() { 61 | return allowLazyPersist; 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/r4h-hdp2.2.0/src/com/mellanox/r4h/client/HdfsDataInputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.InputStream; 21 | import java.io.IOException; 22 | import java.util.List; 23 | 24 | import org.apache.hadoop.classification.InterfaceAudience; 25 | import org.apache.hadoop.classification.InterfaceStability; 26 | import org.apache.hadoop.fs.FSDataInputStream; 27 | import org.apache.hadoop.crypto.CryptoInputStream; 28 | import com.mellanox.r4h.DFSInputStream; 29 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 30 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 31 | import org.apache.hadoop.hdfs.protocol.LocatedBlock; 32 | 33 | import com.google.common.base.Preconditions; 34 | 35 | /** 36 | * The Hdfs implementation of {@link FSDataInputStream}. 37 | */ 38 | @InterfaceAudience.Public 39 | @InterfaceStability.Evolving 40 | public class HdfsDataInputStream extends FSDataInputStream { 41 | public HdfsDataInputStream(DFSInputStream in) throws IOException { 42 | super(in); 43 | } 44 | 45 | public HdfsDataInputStream(CryptoInputStream in) throws IOException { 46 | super(in); 47 | Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream, "CryptoInputStream should wrap a DFSInputStream"); 48 | } 49 | 50 | private DFSInputStream getDFSInputStream() { 51 | if (in instanceof CryptoInputStream) { 52 | return (DFSInputStream) ((CryptoInputStream) in).getWrappedStream(); 53 | } 54 | return (DFSInputStream) in; 55 | } 56 | 57 | /** 58 | * Get a reference to the wrapped output stream. We always want to return the 59 | * actual underlying InputStream, even when we're using a CryptoStream. e.g. 60 | * in the delegated methods below. 61 | * 62 | * @return the underlying output stream 63 | */ 64 | public InputStream getWrappedStream() { 65 | return in; 66 | } 67 | 68 | /** 69 | * Get the datanode from which the stream is currently reading. 70 | */ 71 | public DatanodeInfo getCurrentDatanode() { 72 | return getDFSInputStream().getCurrentDatanode(); 73 | } 74 | 75 | /** 76 | * Get the block containing the target position. 77 | */ 78 | public ExtendedBlock getCurrentBlock() { 79 | return getDFSInputStream().getCurrentBlock(); 80 | } 81 | 82 | /** 83 | * Get the collection of blocks that has already been located. 84 | */ 85 | public synchronized List getAllBlocks() throws IOException { 86 | return getDFSInputStream().getAllBlocks(); 87 | } 88 | 89 | /** 90 | * Get the visible length of the file. It will include the length of the last 91 | * block even if that is in UnderConstruction state. 92 | * 93 | * @return The visible length of the file. 94 | */ 95 | public long getVisibleLength() throws IOException { 96 | return getDFSInputStream().getFileLength(); 97 | } 98 | 99 | /** 100 | * Get statistics about the reads which this DFSInputStream has done. 101 | * Note that because HdfsDataInputStream is buffered, these stats may 102 | * be higher than you would expect just by adding up the number of 103 | * bytes read through HdfsDataInputStream. 104 | */ 105 | public synchronized DFSInputStream.ReadStatistics getReadStatistics() { 106 | return getDFSInputStream().getReadStatistics(); 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /src/r4h-hdp2.2.0/src/com/mellanox/r4h/client/HdfsDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.IOException; 21 | import java.io.OutputStream; 22 | import java.util.EnumSet; 23 | 24 | import org.apache.hadoop.classification.InterfaceAudience; 25 | import org.apache.hadoop.classification.InterfaceStability; 26 | import org.apache.hadoop.crypto.CryptoOutputStream; 27 | import org.apache.hadoop.fs.FSDataOutputStream; 28 | import org.apache.hadoop.fs.FileSystem; 29 | import com.mellanox.r4h.DFSOutputStream; 30 | 31 | import com.google.common.base.Preconditions; 32 | 33 | /** 34 | * The Hdfs implementation of {@link FSDataOutputStream}. 35 | */ 36 | @InterfaceAudience.Public 37 | @InterfaceStability.Evolving 38 | public class HdfsDataOutputStream extends FSDataOutputStream { 39 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 40 | super(out, stats, startPosition); 41 | } 42 | 43 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats) throws IOException { 44 | this(out, stats, 0L); 45 | } 46 | 47 | public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 48 | super(out, stats, startPosition); 49 | Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream, "CryptoOutputStream should wrap a DFSOutputStream"); 50 | } 51 | 52 | public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats) throws IOException { 53 | this(out, stats, 0L); 54 | } 55 | 56 | /** 57 | * Get the actual number of replicas of the current block. 58 | * 59 | * This can be different from the designated replication factor of the file 60 | * because the namenode does not maintain replication for the blocks which are 61 | * currently being written to. Depending on the configuration, the client may 62 | * continue to write to a block even if a few datanodes in the write pipeline 63 | * have failed, or the client may add a new datanodes once a datanode has 64 | * failed. 65 | * 66 | * @return the number of valid replicas of the current block 67 | */ 68 | public synchronized int getCurrentBlockReplication() throws IOException { 69 | OutputStream wrappedStream = getWrappedStream(); 70 | if (wrappedStream instanceof CryptoOutputStream) { 71 | wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream(); 72 | } 73 | return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication(); 74 | } 75 | 76 | /** 77 | * Sync buffered data to DataNodes (flush to disk devices). 78 | * 79 | * @param syncFlags 80 | * Indicate the detailed semantic and actions of the hsync. 81 | * @throws IOException 82 | * @see FSDataOutputStream#hsync() 83 | */ 84 | public void hsync(EnumSet syncFlags) throws IOException { 85 | OutputStream wrappedStream = getWrappedStream(); 86 | if (wrappedStream instanceof CryptoOutputStream) { 87 | ((CryptoOutputStream) wrappedStream).flush(); 88 | wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream(); 89 | } 90 | ((DFSOutputStream) wrappedStream).hsync(syncFlags); 91 | } 92 | 93 | public static enum SyncFlag { 94 | 95 | /** 96 | * When doing sync to DataNodes, also update the metadata (block length) in 97 | * the NameNode. 98 | */ 99 | UPDATE_LENGTH; 100 | } 101 | } 102 | -------------------------------------------------------------------------------- /src/r4h-hdp2.2.0/src/org/apache/hadoop/hdfs/DFSInotifyEventInputStreamBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import java.io.IOException; 4 | 5 | import org.apache.hadoop.hdfs.protocol.ClientProtocol; 6 | 7 | public class DFSInotifyEventInputStreamBridge extends DFSInotifyEventInputStream { 8 | 9 | public DFSInotifyEventInputStreamBridge(ClientProtocol namenode) throws IOException { 10 | super(namenode); 11 | } 12 | 13 | public DFSInotifyEventInputStreamBridge(ClientProtocol namenode, long lastReadTxid) throws IOException { 14 | super(namenode, lastReadTxid); 15 | } 16 | 17 | } 18 | -------------------------------------------------------------------------------- /src/r4h-hdp2.2.0/src/org/apache/hadoop/hdfs/server/datanode/R4HBlockReceiverBase.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode; 2 | 3 | import java.io.DataInputStream; 4 | import java.io.IOException; 5 | 6 | import com.mellanox.r4h.WriteOprHeader; 7 | import com.mellanox.r4h.WriteOprHeader2_6; 8 | 9 | public abstract class R4HBlockReceiverBase extends BlockReceiver { 10 | public R4HBlockReceiverBase(WriteOprHeader oprHeader, DataInputStream inForHeaderOnly, String sessionInfo, DataNodeBridge dnEx) 11 | throws IOException { 12 | super(oprHeader.getBlock(), ((WriteOprHeader2_6) oprHeader).getStorageType(), inForHeaderOnly, sessionInfo, sessionInfo, 13 | oprHeader.getStage(), oprHeader.getLatestGenerationStamp(), oprHeader.getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader 14 | .getClientName(), oprHeader.getSrcDataNode(), dnEx.getDN(), oprHeader.getRequestedChecksum(), oprHeader.getCachingStrategy(), 15 | ((WriteOprHeader2_6) oprHeader).getAllowLazyPersist()); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/r4h-hdp2.2.0/test/TBD: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Mellanox/R4H/db73b0ebceb55cc73691685a76933eb5b6b707f8/src/r4h-hdp2.2.0/test/TBD -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/com/mellanox/r4h/DataXceiver.java: -------------------------------------------------------------------------------- 1 | package com.mellanox.r4h; 2 | 3 | import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto; 4 | import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed; 5 | 6 | import java.io.DataInputStream; 7 | import java.io.DataOutputStream; 8 | import java.io.IOException; 9 | 10 | import org.accelio.jxio.ServerSession.SessionKey; 11 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 12 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 13 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 14 | import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; 15 | import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; 16 | import org.apache.hadoop.hdfs.protocolPB.PBHelper; 17 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 18 | 19 | public class DataXceiver extends DataXceiverBase { 20 | 21 | DataXceiver(DataXceiverServer dxcs, ServerPortalWorker spw, SessionKey sKey, R4HExecutor ioExec, R4HExecutor auxExec) { 22 | super(dxcs, spw, sKey, ioExec, auxExec); 23 | } 24 | 25 | @Override 26 | void parseOpWriteBlock(DataInputStream in) throws IOException { 27 | final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in)); 28 | final DatanodeInfo[] targets = PBHelper.convert(proto.getTargetsList()); 29 | oprHeader = new WriteOprHeader2_7(PBHelper.convert(proto.getHeader().getBaseHeader().getBlock()), PBHelper.convertStorageType(proto 30 | .getStorageType()), PBHelper.convert(proto.getHeader().getBaseHeader().getToken()), proto.getHeader().getClientName(), targets, 31 | PBHelper.convertStorageTypes(proto.getTargetStorageTypesList(), targets.length), PBHelper.convert(proto.getSource()), 32 | BlockConstructionStage.valueOf(proto.getStage().name()), proto.getPipelineSize(), proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(), 33 | proto.getLatestGenerationStamp(), fromProto(proto.getRequestedChecksum()), 34 | (proto.hasCachingStrategy() ? getCachingStrategy(proto.getCachingStrategy()) : CachingStrategy.newDefaultStrategy()), 35 | (proto.hasAllowLazyPersist() ? proto.getAllowLazyPersist() : false), (proto.hasPinning() ? proto.getPinning() : false), 36 | PBHelper.convertBooleanList(proto.getTargetPinningsList())); 37 | } 38 | 39 | @Override 40 | void senderWriteBlock(DataOutputStream out, ExtendedBlock origBlk) throws IOException { 41 | new Sender(out).writeBlock(origBlk, ((WriteOprHeader2_7) oprHeader).getStorageType(), oprHeader.getBlockToken(), oprHeader.getClientName(), 42 | oprHeader.getTargets(), ((WriteOprHeader2_7) oprHeader).getAllStorageTypes(), oprHeader.getSrcDataNode(), oprHeader.getStage(), 43 | oprHeader.getPipelineSize(), oprHeader.getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader.getLatestGenerationStamp(), 44 | oprHeader.getRequestedChecksum(), oprHeader.getCachingStrategy(), ((WriteOprHeader2_7) oprHeader).getAllowLazyPersist(), 45 | ((WriteOprHeader2_7) oprHeader).getAllowPinning(), ((WriteOprHeader2_7) oprHeader).getTargetPinnings()); 46 | } 47 | 48 | } 49 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/com/mellanox/r4h/R4HPipelineAck.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck; 21 | import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; 22 | 23 | public class R4HPipelineAck extends PipelineAck { 24 | 25 | /** 26 | * Constructor assuming no next DN in pipeline 27 | * @param seqno sequence number 28 | * @param replies an array of replies 29 | */ 30 | public R4HPipelineAck(long seqno, Status[] replies) { 31 | super(seqno, toIntArray(replies)); 32 | } 33 | 34 | /** 35 | * Constructor 36 | * @param seqno sequence number 37 | * @param replies an array of replies 38 | * @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline 39 | */ 40 | public R4HPipelineAck(long seqno, Status[] replies, long downstreamAckTimeNanos) { 41 | super(seqno, toIntArray(replies), downstreamAckTimeNanos); 42 | } 43 | 44 | /** default constructor **/ 45 | public R4HPipelineAck() { 46 | super(); 47 | } 48 | 49 | /** 50 | * get the ith reply - The api has been changed since hadoop 2.6 51 | * @return the the ith reply 52 | */ 53 | public Status getReply(int i) { 54 | return PipelineAck.getStatusFromHeader(this.getHeaderFlag(i)); 55 | } 56 | 57 | private static int[] toIntArray(Status[] statusArray) { 58 | int[] intArray = new int[statusArray.length]; 59 | for (int i = 0 ; i < statusArray.length ; i++) { 60 | intArray[i] = statusArray[i].getNumber(); 61 | } 62 | return intArray; 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/com/mellanox/r4h/WriteOprHeader2_7.java: -------------------------------------------------------------------------------- 1 | /* 2 | ** Copyright (C) 2014 Mellanox Technologies 3 | ** 4 | ** Licensed under the Apache License, Version 2.0 (the "License"); 5 | ** you may not use this file except in compliance with the License. 6 | ** You may obtain a copy of the License at: 7 | ** 8 | ** http://www.apache.org/licenses/LICENSE-2.0 9 | ** 10 | ** Unless required by applicable law or agreed to in writing, software 11 | ** distributed under the License is distributed on an "AS IS" BASIS, 12 | ** WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 13 | ** either express or implied. See the License for the specific language 14 | ** governing permissions and limitations under the License. 15 | ** 16 | */ 17 | 18 | package com.mellanox.r4h; 19 | 20 | import org.apache.hadoop.fs.StorageType; 21 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 22 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 23 | import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; 24 | import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; 25 | import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; 26 | import org.apache.hadoop.security.token.Token; 27 | import org.apache.hadoop.util.DataChecksum; 28 | 29 | /** 30 | * Encapsulating HDFS WriteOperationHeader to be read from a DataInput and written to a DataOutput. Additional it 31 | * provides the ability to write itself to a DataOutput with a pipeline modification 32 | * This is the extension for Hadoop 2.6.0 33 | */ 34 | public class WriteOprHeader2_7 extends WriteOprHeader { 35 | 36 | final private StorageType storageType; 37 | final private StorageType[] allStorageTypes; 38 | final private boolean allowLazyPersist; 39 | final private boolean allowPinning; 40 | final private boolean[] targetPinnings; 41 | 42 | public WriteOprHeader2_7(ExtendedBlock blk, StorageType storageType, Token blockToken, String clientName, 43 | DatanodeInfo[] targets, StorageType[] allStorageTypes, DatanodeInfo source, BlockConstructionStage stage, int pipelineSize, 44 | long minBytesRcvd, long maxBytesRcvd, long latestGenerationStamp, DataChecksum requestedChecksum, CachingStrategy cachingStrategy, 45 | boolean allowLazyPersist, boolean allowPinning, boolean[] targetPinnings) { 46 | super(blk, blockToken, clientName, targets, source, stage, pipelineSize, minBytesRcvd, maxBytesRcvd, latestGenerationStamp, 47 | requestedChecksum, cachingStrategy); 48 | this.storageType = storageType; 49 | this.allStorageTypes = allStorageTypes; 50 | this.allowLazyPersist = allowLazyPersist; 51 | this.allowPinning = allowPinning; 52 | this.targetPinnings = targetPinnings; 53 | 54 | } 55 | 56 | public StorageType getStorageType() { 57 | return storageType; 58 | } 59 | 60 | public StorageType[] getAllStorageTypes() { 61 | return allStorageTypes; 62 | } 63 | 64 | public boolean getAllowLazyPersist() { 65 | return allowLazyPersist; 66 | } 67 | 68 | public boolean getAllowPinning() { 69 | return allowPinning; 70 | } 71 | 72 | public boolean[] getTargetPinnings(){ 73 | return targetPinnings; 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/com/mellanox/r4h/client/HdfsDataInputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.InputStream; 21 | import java.io.IOException; 22 | import java.util.List; 23 | 24 | import org.apache.hadoop.classification.InterfaceAudience; 25 | import org.apache.hadoop.classification.InterfaceStability; 26 | import org.apache.hadoop.fs.FSDataInputStream; 27 | import org.apache.hadoop.crypto.CryptoInputStream; 28 | import com.mellanox.r4h.DFSInputStream; 29 | import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 30 | import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 31 | import org.apache.hadoop.hdfs.protocol.LocatedBlock; 32 | 33 | import com.google.common.base.Preconditions; 34 | 35 | /** 36 | * The Hdfs implementation of {@link FSDataInputStream}. 37 | */ 38 | @InterfaceAudience.Public 39 | @InterfaceStability.Evolving 40 | public class HdfsDataInputStream extends FSDataInputStream { 41 | public HdfsDataInputStream(DFSInputStream in) throws IOException { 42 | super(in); 43 | } 44 | 45 | public HdfsDataInputStream(CryptoInputStream in) throws IOException { 46 | super(in); 47 | Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream, "CryptoInputStream should wrap a DFSInputStream"); 48 | } 49 | 50 | private DFSInputStream getDFSInputStream() { 51 | if (in instanceof CryptoInputStream) { 52 | return (DFSInputStream) ((CryptoInputStream) in).getWrappedStream(); 53 | } 54 | return (DFSInputStream) in; 55 | } 56 | 57 | /** 58 | * Get a reference to the wrapped output stream. We always want to return the 59 | * actual underlying InputStream, even when we're using a CryptoStream. e.g. 60 | * in the delegated methods below. 61 | * 62 | * @return the underlying output stream 63 | */ 64 | public InputStream getWrappedStream() { 65 | return in; 66 | } 67 | 68 | /** 69 | * Get the datanode from which the stream is currently reading. 70 | */ 71 | public DatanodeInfo getCurrentDatanode() { 72 | return getDFSInputStream().getCurrentDatanode(); 73 | } 74 | 75 | /** 76 | * Get the block containing the target position. 77 | */ 78 | public ExtendedBlock getCurrentBlock() { 79 | return getDFSInputStream().getCurrentBlock(); 80 | } 81 | 82 | /** 83 | * Get the collection of blocks that has already been located. 84 | */ 85 | public synchronized List getAllBlocks() throws IOException { 86 | return getDFSInputStream().getAllBlocks(); 87 | } 88 | 89 | /** 90 | * Get the visible length of the file. It will include the length of the last 91 | * block even if that is in UnderConstruction state. 92 | * 93 | * @return The visible length of the file. 94 | */ 95 | public long getVisibleLength() throws IOException { 96 | return getDFSInputStream().getFileLength(); 97 | } 98 | 99 | /** 100 | * Get statistics about the reads which this DFSInputStream has done. 101 | * Note that because HdfsDataInputStream is buffered, these stats may 102 | * be higher than you would expect just by adding up the number of 103 | * bytes read through HdfsDataInputStream. 104 | */ 105 | public synchronized DFSInputStream.ReadStatistics getReadStatistics() { 106 | return getDFSInputStream().getReadStatistics(); 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/com/mellanox/r4h/client/HdfsDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.mellanox.r4h.client; 19 | 20 | import java.io.IOException; 21 | import java.io.OutputStream; 22 | import java.util.EnumSet; 23 | 24 | import org.apache.hadoop.classification.InterfaceAudience; 25 | import org.apache.hadoop.classification.InterfaceStability; 26 | import org.apache.hadoop.crypto.CryptoOutputStream; 27 | import org.apache.hadoop.fs.FSDataOutputStream; 28 | import org.apache.hadoop.fs.FileSystem; 29 | import com.mellanox.r4h.DFSOutputStream; 30 | 31 | import com.google.common.base.Preconditions; 32 | 33 | /** 34 | * The Hdfs implementation of {@link FSDataOutputStream}. 35 | */ 36 | @InterfaceAudience.Public 37 | @InterfaceStability.Evolving 38 | public class HdfsDataOutputStream extends FSDataOutputStream { 39 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 40 | super(out, stats, startPosition); 41 | } 42 | 43 | public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats) throws IOException { 44 | this(out, stats, 0L); 45 | } 46 | 47 | public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats, long startPosition) throws IOException { 48 | super(out, stats, startPosition); 49 | Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream, "CryptoOutputStream should wrap a DFSOutputStream"); 50 | } 51 | 52 | public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats) throws IOException { 53 | this(out, stats, 0L); 54 | } 55 | 56 | /** 57 | * Get the actual number of replicas of the current block. 58 | * 59 | * This can be different from the designated replication factor of the file 60 | * because the namenode does not maintain replication for the blocks which are 61 | * currently being written to. Depending on the configuration, the client may 62 | * continue to write to a block even if a few datanodes in the write pipeline 63 | * have failed, or the client may add a new datanodes once a datanode has 64 | * failed. 65 | * 66 | * @return the number of valid replicas of the current block 67 | */ 68 | public synchronized int getCurrentBlockReplication() throws IOException { 69 | OutputStream wrappedStream = getWrappedStream(); 70 | if (wrappedStream instanceof CryptoOutputStream) { 71 | wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream(); 72 | } 73 | return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication(); 74 | } 75 | 76 | /** 77 | * Sync buffered data to DataNodes (flush to disk devices). 78 | * 79 | * @param syncFlags 80 | * Indicate the detailed semantic and actions of the hsync. 81 | * @throws IOException 82 | * @see FSDataOutputStream#hsync() 83 | */ 84 | public void hsync(EnumSet syncFlags) throws IOException { 85 | OutputStream wrappedStream = getWrappedStream(); 86 | if (wrappedStream instanceof CryptoOutputStream) { 87 | ((CryptoOutputStream) wrappedStream).flush(); 88 | wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream(); 89 | } 90 | ((DFSOutputStream) wrappedStream).hsync(syncFlags); 91 | } 92 | 93 | public static enum SyncFlag { 94 | 95 | /** 96 | * When doing sync to DataNodes, also update the metadata (block length) in 97 | * the NameNode. 98 | */ 99 | UPDATE_LENGTH, 100 | 101 | /** 102 | * Sync the data to DataNode, close the current block, and allocate a new 103 | * block 104 | */ 105 | END_BLOCK; 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/org/apache/hadoop/hdfs/DFSClientConfBridge2_7.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import org.apache.hadoop.conf.Configuration; 4 | 5 | public class DFSClientConfBridge2_7 extends DFSClientConfBridge { 6 | public DFSClientConfBridge2_7(Configuration conf) { 7 | super(conf); 8 | } 9 | 10 | public long getDfsclientSlowIoWarningThresholdMs() { 11 | return dfsclientSlowIoWarningThresholdMs; 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/org/apache/hadoop/hdfs/DFSInotifyEventInputStreamBridge.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs; 2 | 3 | import java.io.IOException; 4 | 5 | import org.apache.hadoop.hdfs.protocol.ClientProtocol; 6 | import org.apache.htrace.Sampler; 7 | 8 | public class DFSInotifyEventInputStreamBridge extends DFSInotifyEventInputStream { 9 | 10 | public DFSInotifyEventInputStreamBridge(Sampler traceSampler, ClientProtocol namenode) throws IOException { 11 | super(traceSampler, namenode); 12 | } 13 | 14 | public DFSInotifyEventInputStreamBridge(Sampler traceSampler, ClientProtocol namenode, long lastReadTxid) throws IOException { 15 | super(traceSampler, namenode, lastReadTxid); 16 | } 17 | 18 | } 19 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/src/org/apache/hadoop/hdfs/server/datanode/R4HBlockReceiverBase.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hdfs.server.datanode; 2 | 3 | import java.io.DataInputStream; 4 | import java.io.IOException; 5 | 6 | import com.mellanox.r4h.WriteOprHeader; 7 | import com.mellanox.r4h.WriteOprHeader2_7; 8 | 9 | public abstract class R4HBlockReceiverBase extends BlockReceiver { 10 | public R4HBlockReceiverBase(WriteOprHeader oprHeader, DataInputStream inForHeaderOnly, String sessionInfo, DataNodeBridge dnEx) 11 | throws IOException { 12 | super(oprHeader.getBlock(), ((WriteOprHeader2_7) oprHeader).getStorageType(), inForHeaderOnly, sessionInfo, sessionInfo, 13 | oprHeader.getStage(), oprHeader.getLatestGenerationStamp(), oprHeader.getMinBytesRcvd(), oprHeader.getMaxBytesRcvd(), oprHeader 14 | .getClientName(), oprHeader.getSrcDataNode(), dnEx.getDN(), oprHeader.getRequestedChecksum(), oprHeader.getCachingStrategy(), 15 | ((WriteOprHeader2_7) oprHeader).getAllowLazyPersist(), ((WriteOprHeader2_7) oprHeader).getAllowPinning()); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/r4h-hdp2.3.0/test/TBD: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Mellanox/R4H/db73b0ebceb55cc73691685a76933eb5b6b707f8/src/r4h-hdp2.3.0/test/TBD --------------------------------------------------------------------------------