|
@@ -22,105 +22,118 @@ import java.io.IOException;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
|
|
|
|
+import io.netty.buffer.ByteBuf;
|
|
|
|
+import io.netty.buffer.Unpooled;
|
|
|
|
+import io.netty.channel.ChannelFutureListener;
|
|
|
|
+import io.netty.channel.ChannelHandlerContext;
|
|
|
|
+import io.netty.channel.SimpleChannelInboundHandler;
|
|
|
|
+import io.netty.channel.group.ChannelGroup;
|
|
|
|
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
|
|
|
|
+import io.netty.handler.codec.http.DefaultHttpResponse;
|
|
|
|
+import static io.netty.handler.codec.http.HttpResponseStatus.*;
|
|
|
|
+
|
|
|
|
+import io.netty.handler.codec.http.HttpMethod;
|
|
|
|
+import io.netty.handler.codec.http.HttpRequest;
|
|
|
|
+import io.netty.handler.codec.http.HttpResponseStatus;
|
|
|
|
+import static io.netty.handler.codec.http.HttpVersion.*;
|
|
|
|
+import io.netty.handler.codec.http.QueryStringDecoder;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.hdfs.web.JsonUtil;
|
|
import org.apache.hadoop.hdfs.web.JsonUtil;
|
|
-import org.apache.hadoop.ipc.RemoteException;
|
|
|
|
-import org.jboss.netty.channel.ChannelFuture;
|
|
|
|
-import org.jboss.netty.channel.ChannelFutureListener;
|
|
|
|
-import org.jboss.netty.channel.ChannelHandlerContext;
|
|
|
|
-import org.jboss.netty.channel.MessageEvent;
|
|
|
|
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
|
|
|
|
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
|
|
|
|
-import org.jboss.netty.handler.codec.http.HttpHeaders;
|
|
|
|
-import org.jboss.netty.handler.codec.http.HttpMethod;
|
|
|
|
-import org.jboss.netty.handler.codec.http.HttpRequest;
|
|
|
|
-import org.jboss.netty.handler.codec.http.HttpResponse;
|
|
|
|
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
|
|
|
-import org.jboss.netty.handler.codec.http.HttpVersion;
|
|
|
|
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
|
|
|
|
-
|
|
|
|
-import javax.management.Query;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
|
|
|
|
+
|
|
|
|
|
|
/**
|
|
/**
|
|
* Implement the read-only WebHDFS API for fsimage.
|
|
* Implement the read-only WebHDFS API for fsimage.
|
|
*/
|
|
*/
|
|
-class FSImageHandler extends SimpleChannelUpstreamHandler {
|
|
|
|
|
|
+class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
|
|
public static final Log LOG = LogFactory.getLog(FSImageHandler.class);
|
|
public static final Log LOG = LogFactory.getLog(FSImageHandler.class);
|
|
private final FSImageLoader image;
|
|
private final FSImageLoader image;
|
|
-
|
|
|
|
- FSImageHandler(FSImageLoader image) throws IOException {
|
|
|
|
- this.image = image;
|
|
|
|
- }
|
|
|
|
|
|
+ private final ChannelGroup activeChannels;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void messageReceived(
|
|
|
|
- ChannelHandlerContext ctx, MessageEvent e) throws Exception {
|
|
|
|
- ChannelFuture future = e.getFuture();
|
|
|
|
- try {
|
|
|
|
- future = handleOperation(e);
|
|
|
|
- } finally {
|
|
|
|
- future.addListener(ChannelFutureListener.CLOSE);
|
|
|
|
- }
|
|
|
|
|
|
+ public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
|
|
|
+ activeChannels.add(ctx.channel());
|
|
}
|
|
}
|
|
|
|
|
|
- private ChannelFuture handleOperation(MessageEvent e)
|
|
|
|
- throws IOException {
|
|
|
|
- HttpRequest request = (HttpRequest) e.getMessage();
|
|
|
|
- HttpResponse response = new DefaultHttpResponse(
|
|
|
|
- HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
|
|
|
|
- response.setHeader(HttpHeaders.Names.CONTENT_TYPE, "application/json");
|
|
|
|
|
|
+ FSImageHandler(FSImageLoader image, ChannelGroup activeChannels) throws IOException {
|
|
|
|
+ this.image = image;
|
|
|
|
+ this.activeChannels = activeChannels;
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void channelRead0(ChannelHandlerContext ctx, HttpRequest request)
|
|
|
|
+ throws Exception {
|
|
if (request.getMethod() != HttpMethod.GET) {
|
|
if (request.getMethod() != HttpMethod.GET) {
|
|
- response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
|
|
|
|
- return e.getChannel().write(response);
|
|
|
|
|
|
+ DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1,
|
|
|
|
+ METHOD_NOT_ALLOWED);
|
|
|
|
+ resp.headers().set("Connection", "close");
|
|
|
|
+ ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
|
|
|
|
+ return;
|
|
}
|
|
}
|
|
|
|
|
|
QueryStringDecoder decoder = new QueryStringDecoder(request.getUri());
|
|
QueryStringDecoder decoder = new QueryStringDecoder(request.getUri());
|
|
final String op = getOp(decoder);
|
|
final String op = getOp(decoder);
|
|
|
|
|
|
- String content;
|
|
|
|
- String path = null;
|
|
|
|
- try {
|
|
|
|
- path = getPath(decoder);
|
|
|
|
- if ("GETFILESTATUS".equals(op)) {
|
|
|
|
- content = image.getFileStatus(path);
|
|
|
|
- } else if ("LISTSTATUS".equals(op)) {
|
|
|
|
- content = image.listStatus(path);
|
|
|
|
- } else if ("GETACLSTATUS".equals(op)) {
|
|
|
|
- content = image.getAclStatus(path);
|
|
|
|
- } else {
|
|
|
|
- throw new IllegalArgumentException("Invalid value for webhdfs parameter" + " \"op\"");
|
|
|
|
- }
|
|
|
|
- } catch (IllegalArgumentException ex) {
|
|
|
|
- response.setStatus(HttpResponseStatus.BAD_REQUEST);
|
|
|
|
- content = JsonUtil.toJsonString(ex);
|
|
|
|
- } catch (FileNotFoundException ex) {
|
|
|
|
- response.setStatus(HttpResponseStatus.NOT_FOUND);
|
|
|
|
- content = JsonUtil.toJsonString(ex);
|
|
|
|
- } catch (Exception ex) {
|
|
|
|
- content = JsonUtil.toJsonString(ex);
|
|
|
|
|
|
+ final String content;
|
|
|
|
+ String path = getPath(decoder);
|
|
|
|
+ if ("GETFILESTATUS".equals(op)) {
|
|
|
|
+ content = image.getFileStatus(path);
|
|
|
|
+ } else if ("LISTSTATUS".equals(op)) {
|
|
|
|
+ content = image.listStatus(path);
|
|
|
|
+ } else if ("GETACLSTATUS".equals(op)) {
|
|
|
|
+ content = image.getAclStatus(path);
|
|
|
|
+ } else {
|
|
|
|
+ throw new IllegalArgumentException("Invalid value for webhdfs parameter" + " \"op\"");
|
|
}
|
|
}
|
|
|
|
|
|
- HttpHeaders.setContentLength(response, content.length());
|
|
|
|
- e.getChannel().write(response);
|
|
|
|
- ChannelFuture future = e.getChannel().write(content);
|
|
|
|
|
|
+ LOG.info("op=" + op + " target=" + path);
|
|
|
|
+
|
|
|
|
+ DefaultFullHttpResponse resp = new DefaultFullHttpResponse(
|
|
|
|
+ HTTP_1_1, HttpResponseStatus.OK,
|
|
|
|
+ Unpooled.wrappedBuffer(content.getBytes()));
|
|
|
|
+ resp.headers().set("Content-Type", "application/json");
|
|
|
|
+ resp.headers().set("Content-Length", resp.content().readableBytes());
|
|
|
|
+ resp.headers().set("Connection", "close");
|
|
|
|
+ ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
|
|
|
|
+ }
|
|
|
|
|
|
- LOG.info(response.getStatus().getCode() + " method="
|
|
|
|
- + request.getMethod().getName() + " op=" + op + " target=" + path);
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
|
|
|
|
+ ctx.flush();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
|
|
|
|
+ throws Exception {
|
|
|
|
+ Exception e = cause instanceof Exception ? (Exception) cause : new
|
|
|
|
+ Exception(cause);
|
|
|
|
+ final String output = JsonUtil.toJsonString(e);
|
|
|
|
+ ByteBuf content = Unpooled.wrappedBuffer(output.getBytes());
|
|
|
|
+ final DefaultFullHttpResponse resp = new DefaultFullHttpResponse(
|
|
|
|
+ HTTP_1_1, INTERNAL_SERVER_ERROR, content);
|
|
|
|
+
|
|
|
|
+ resp.headers().set("Content-Type", "application/json");
|
|
|
|
+ if (e instanceof IllegalArgumentException) {
|
|
|
|
+ resp.setStatus(BAD_REQUEST);
|
|
|
|
+ } else if (e instanceof FileNotFoundException) {
|
|
|
|
+ resp.setStatus(NOT_FOUND);
|
|
|
|
+ }
|
|
|
|
|
|
- return future;
|
|
|
|
|
|
+ resp.headers().set("Content-Length", resp.content().readableBytes());
|
|
|
|
+ resp.headers().set("Connection", "close");
|
|
|
|
+ ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
|
|
}
|
|
}
|
|
|
|
|
|
private static String getOp(QueryStringDecoder decoder) {
|
|
private static String getOp(QueryStringDecoder decoder) {
|
|
- Map<String, List<String>> parameters = decoder.getParameters();
|
|
|
|
|
|
+ Map<String, List<String>> parameters = decoder.parameters();
|
|
return parameters.containsKey("op")
|
|
return parameters.containsKey("op")
|
|
? parameters.get("op").get(0).toUpperCase() : null;
|
|
? parameters.get("op").get(0).toUpperCase() : null;
|
|
}
|
|
}
|
|
|
|
|
|
private static String getPath(QueryStringDecoder decoder)
|
|
private static String getPath(QueryStringDecoder decoder)
|
|
throws FileNotFoundException {
|
|
throws FileNotFoundException {
|
|
- String path = decoder.getPath();
|
|
|
|
|
|
+ String path = decoder.path();
|
|
if (path.startsWith("/webhdfs/v1/")) {
|
|
if (path.startsWith("/webhdfs/v1/")) {
|
|
return path.substring(11);
|
|
return path.substring(11);
|
|
} else {
|
|
} else {
|