merge trunk to HDFS-7240

This commit is contained in:
Anu Engineer 2017-03-08 15:32:58 -08:00
parent 6b6c61ee74
commit cc63937e31
6 changed files with 21 additions and 20 deletions

View File

@ -102,6 +102,6 @@ class URLDispatcher extends SimpleChannelInboundHandler<HttpRequest> {
* @return true if the request is to be handled by WebHDFS
*/
private boolean isWebHdfsRequest(HttpRequest req) {
return req.uri().startsWith(WEBHDFS_PREFIX);
return req.getUri().startsWith(WEBHDFS_PREFIX);
}
}

View File

@ -26,9 +26,9 @@ import io.netty.handler.codec.http.HttpResponseStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static io.netty.handler.codec.http.HttpHeaderNames.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH;
import static io.netty.handler.codec.http.HttpHeaderValues.CLOSE;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH;
import static io.netty.handler.codec.http.HttpHeaders.Values.CLOSE;
import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;

View File

@ -17,12 +17,8 @@
*/
package org.apache.hadoop.ozone.web.netty;
import static io.netty.handler.codec.http.HttpHeaderNames.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH;
import static io.netty.handler.codec.http.HttpHeaderNames.HOST;
import static io.netty.handler.codec.http.HttpHeaderNames.TRANSFER_ENCODING;
import static io.netty.handler.codec.http.HttpHeaderValues.CLOSE;
import static io.netty.handler.codec.http.HttpHeaderValues.KEEP_ALIVE;
import static io.netty.handler.codec.http.HttpHeaders.Names.*;
import static io.netty.handler.codec.http.HttpHeaders.Values.*;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import java.io.IOException;
@ -46,7 +42,7 @@ import com.sun.jersey.spi.container.ContainerResponseWriter;
import com.sun.jersey.spi.container.WebApplication;
import io.netty.handler.codec.http.DefaultHttpResponse;
import io.netty.handler.codec.http.HttpHeaderUtil;
//import io.netty.handler.codec.http.HttpUtil;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpResponse;
@ -262,7 +258,7 @@ public final class ObjectStoreJerseyContainer {
this.nettyResp = jerseyResponseToNettyResponse(jerseyResp);
this.nettyResp.headers().set(CONTENT_LENGTH, Math.max(0, contentLength));
this.nettyResp.headers().set(CONNECTION,
HttpHeaderUtil.isKeepAlive(this.nettyReq) ? KEEP_ALIVE : CLOSE);
HttpHeaders.isKeepAlive(this.nettyReq) ? KEEP_ALIVE : CLOSE);
this.latch.countDown();
LOG.trace(
"end writeStatusAndHeaders, contentLength = {}, jerseyResp = {}.",
@ -339,9 +335,9 @@ public final class ObjectStoreJerseyContainer {
String host = nettyHeaders.get(HOST);
String scheme = host.startsWith("https") ? "https://" : "http://";
String baseUri = scheme + host + "/";
String reqUri = scheme + host + nettyReq.uri();
String reqUri = scheme + host + nettyReq.getUri();
LOG.trace("baseUri = {}, reqUri = {}", baseUri, reqUri);
return new ContainerRequest(webapp, nettyReq.method().name(),
return new ContainerRequest(webapp, nettyReq.getMethod().name(),
new URI(baseUri), new URI(reqUri), jerseyHeaders, reqIn);
}
}

View File

@ -21,7 +21,7 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.http.HttpContent;
import io.netty.handler.codec.http.HttpHeaderUtil;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpResponse;
import io.netty.handler.codec.http.LastHttpContent;
@ -81,7 +81,7 @@ public final class RequestContentObjectStoreChannelHandler
ChannelFuture respFuture = ctx.writeAndFlush(new ChunkedStream(
this.respIn));
respFuture.addListener(new CloseableCleanupListener(this.respIn));
if (!HttpHeaderUtil.isKeepAlive(this.nettyReq)) {
if (!HttpHeaders.isKeepAlive(this.nettyReq)) {
respFuture.addListener(ChannelFutureListener.CLOSE);
}
}

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.web.netty;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.HttpHeaderUtil;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpResponse;
import org.apache.hadoop.io.IOUtils;
@ -67,7 +67,7 @@ public final class RequestDispatchObjectStoreChannelHandler
throws Exception {
LOG.trace("begin RequestDispatchObjectStoreChannelHandler channelRead0, " +
"ctx = {}, nettyReq = {}", ctx, nettyReq);
if (!nettyReq.decoderResult().isSuccess()) {
if (!nettyReq.getDecoderResult().isSuccess()) {
sendErrorResponse(ctx, BAD_REQUEST);
return;
}
@ -77,7 +77,7 @@ public final class RequestDispatchObjectStoreChannelHandler
this.respIn = new PipedInputStream();
this.respOut = new PipedOutputStream(respIn);
if (HttpHeaderUtil.is100ContinueExpected(nettyReq)) {
if (HttpHeaders.is100ContinueExpected(nettyReq)) {
LOG.trace("Sending continue response.");
ctx.writeAndFlush(new DefaultFullHttpResponse(HTTP_1_1, CONTINUE));
}

View File

@ -57,7 +57,12 @@ public class TestContainerServer {
ContainerTestHelper.getCreateContainerRequest(containerName);
channel.writeInbound(request);
Assert.assertTrue(channel.finish());
ContainerCommandResponseProto response = channel.readOutbound();
Object responseObject = channel.readOutbound();
Assert.assertTrue(responseObject instanceof
ContainerCommandResponseProto);
ContainerCommandResponseProto response =
(ContainerCommandResponseProto) responseObject;
Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
} finally {
if (channel != null) {