Skip to content

Commit

Permalink
HADOOP-9509. Implement ONCRPC and XDR. Contributed by Brandon Li
Browse files Browse the repository at this point in the history
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1490845 13f79535-47bb-0310-9956-ffa450edef68
  • Loading branch information
Brandon Li committed Jun 7, 2013
1 parent af8514e commit 8ef140d
Showing 40 changed files with 3,401 additions and 0 deletions.
2 changes: 2 additions & 0 deletions hadoop-common-project/hadoop-common/CHANGES.txt
Original file line number Diff line number Diff line change
@@ -12,6 +12,8 @@ Trunk (Unreleased)
HADOOP-8561. Introduce HADOOP_PROXY_USER for secure impersonation in child
hadoop client processes. (Yu Gao via llu)

HADOOP-9509. Implement ONCRPC and XDR. (brandonli)

IMPROVEMENTS

HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
10 changes: 10 additions & 0 deletions hadoop-common-project/hadoop-nfs/README.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
Hadoop NFS

Hadoop NFS is a Java library for building NFS gateway. It has
the following components:

- ONCRPC: This a implementation of ONCRPC(RFC-5531) and XDR(RFC-4506).
- Mount: This an interface implementation of MOUNT protocol (RFC-1813).
- Portmap: This is a implementation of Binding protocol(RFC-1833).
- NFSv3: This is an interface implementation of NFSv3 protocol(RFC-1813).

98 changes: 98 additions & 0 deletions hadoop-common-project/hadoop-nfs/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. See accompanying LICENSE file.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-project</artifactId>
<version>3.0.0-SNAPSHOT</version>
<relativePath>../../hadoop-project</relativePath>
</parent>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-nfs</artifactId>
<version>3.0.0-SNAPSHOT</version>
<packaging>jar</packaging>

<name>Apache Hadoop NFS</name>
<description>Apache Hadoop NFS library</description>

<properties>
<maven.build.timestamp.format>yyyyMMdd</maven.build.timestamp.format>
<kerberos.realm>LOCALHOST</kerberos.realm>
</properties>

<dependencies>
<dependency>
<!-- Used, even though 'mvn dependency:analyze' doesn't find it -->
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-annotations</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.8.2</version>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<version>1.8.5</version>
</dependency>
<dependency>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
<version>3.6.2.Final</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<version>11.0.2</version>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.oncrpc;

import java.util.Arrays;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.oncrpc.RpcAcceptedReply.AcceptState;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.channel.ChannelHandlerContext;
import org.jboss.netty.channel.MessageEvent;

/**
* A simple client that registers an RPC program with portmap.
*/
public class RegistrationClient extends SimpleTcpClient {
public static final Log LOG = LogFactory.getLog(RegistrationClient.class);

public RegistrationClient(String host, int port, XDR request) {
super(host, port, request);
}

/**
* Handler to handle response from the server.
*/
static class RegistrationClientHandler extends SimpleTcpClientHandler {
public RegistrationClientHandler(XDR request) {
super(request);
}

private boolean validMessageLength(int len) {
// 28 bytes is the minimal success response size (portmapV2)
if (len < 28) {
if (LOG.isDebugEnabled()) {
LOG.debug("Portmap mapping registration failed,"
+ " the response size is less than 28 bytes:" + len);
}
return false;
}
return true;
}

@Override
public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
ChannelBuffer buf = (ChannelBuffer) e.getMessage(); // Read reply
if (!validMessageLength(buf.readableBytes())) {
e.getChannel().close();
return;
}

// handling fragment header for TCP, 4 bytes.
byte[] fragmentHeader = Arrays.copyOfRange(buf.array(), 0, 4);
int fragmentSize = XDR.fragmentSize(fragmentHeader);
boolean isLast = XDR.isLastFragment(fragmentHeader);
assert (fragmentSize == 28 && isLast == true);

XDR xdr = new XDR();
xdr.writeFixedOpaque(Arrays.copyOfRange(buf.array(), 4,
buf.readableBytes()));

RpcReply reply = RpcReply.read(xdr);
if (reply.getState() == RpcReply.ReplyState.MSG_ACCEPTED) {
RpcAcceptedReply acceptedReply = (RpcAcceptedReply) reply;
handle(acceptedReply, xdr);
} else {
RpcDeniedReply deniedReply = (RpcDeniedReply) reply;
handle(deniedReply);
}
e.getChannel().close(); // shutdown now that request is complete
}

private void handle(RpcDeniedReply deniedReply) {
LOG.warn("Portmap mapping registration request was denied , " +
deniedReply);
}

private void handle(RpcAcceptedReply acceptedReply, XDR xdr) {
AcceptState acceptState = acceptedReply.getAcceptState();
assert (acceptState == AcceptState.SUCCESS);
boolean answer = xdr.readBoolean();
if (answer != true) {
LOG.warn("Portmap mapping registration failed, accept state:"
+ acceptState);
}
LOG.info("Portmap mapping registration succeeded");
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.oncrpc;

import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;

/**
* Represents RPC message MSG_ACCEPTED reply body. See RFC 1831 for details.
* This response is sent to a request to indicate success of the request.
*/
public class RpcAcceptedReply extends RpcReply {
public enum AcceptState {
SUCCESS(0), /* RPC executed successfully */
PROG_UNAVAIL(1), /* remote hasn't exported program */
PROG_MISMATCH(2), /* remote can't support version # */
PROC_UNAVAIL(3), /* program can't support procedure */
GARBAGE_ARGS(4), /* procedure can't decode params */
SYSTEM_ERR(5); /* e.g. memory allocation failure */

private final int value;

AcceptState(int value) {
this.value = value;
}

public static AcceptState fromValue(int value) {
return values()[value];
}

public int getValue() {
return value;
}
};

private final RpcAuthInfo verifier;
private final AcceptState acceptState;

RpcAcceptedReply(int xid, int messageType, ReplyState state,
RpcAuthInfo verifier, AcceptState acceptState) {
super(xid, messageType, state);
this.verifier = verifier;
this.acceptState = acceptState;
}

public static RpcAcceptedReply read(int xid, int messageType,
ReplyState replyState, XDR xdr) {
RpcAuthInfo verifier = RpcAuthInfo.read(xdr);
AcceptState acceptState = AcceptState.fromValue(xdr.readInt());
return new RpcAcceptedReply(xid, messageType, replyState, verifier,
acceptState);
}

public RpcAuthInfo getVerifier() {
return verifier;
}

public AcceptState getAcceptState() {
return acceptState;
}

public static XDR voidReply(XDR xdr, int xid) {
return voidReply(xdr, xid, AcceptState.SUCCESS);
}

public static XDR voidReply(XDR xdr, int xid, AcceptState acceptState) {
xdr.writeInt(xid);
xdr.writeInt(RpcMessage.RPC_REPLY);
xdr.writeInt(ReplyState.MSG_ACCEPTED.getValue());
xdr.writeInt(AuthFlavor.AUTH_NONE.getValue());
xdr.writeVariableOpaque(new byte[0]);
xdr.writeInt(acceptState.getValue());
return xdr;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.oncrpc;

import java.util.Arrays;

/**
* Authentication Info as defined in RFC 1831
*/
public class RpcAuthInfo {
/** Different types of authentication as defined in RFC 1831 */
public enum AuthFlavor {
AUTH_NONE(0),
AUTH_SYS(1),
AUTH_SHORT(2),
AUTH_DH(3),
RPCSEC_GSS(6);

private int value;

AuthFlavor(int value) {
this.value = value;
}

public int getValue() {
return value;
}

static AuthFlavor fromValue(int value) {
for (AuthFlavor v : values()) {
if (v.value == value) {
return v;
}
}
throw new IllegalArgumentException("Invalid AuthFlavor value " + value);
}
}

private final AuthFlavor flavor;
private final byte[] body;

protected RpcAuthInfo(AuthFlavor flavor, byte[] body) {
this.flavor = flavor;
this.body = body;
}

public static RpcAuthInfo read(XDR xdr) {
int type = xdr.readInt();
AuthFlavor flavor = AuthFlavor.fromValue(type);
byte[] body = xdr.readVariableOpaque();
return new RpcAuthInfo(flavor, body);
}

public AuthFlavor getFlavor() {
return flavor;
}

public byte[] getBody() {
return Arrays.copyOf(body, body.length);
}

@Override
public String toString() {
return "(AuthFlavor:" + flavor + ")";
}
}
Loading
Oops, something went wrong.

0 comments on commit 8ef140d

Please sign in to comment.