001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hdfs.server.datanode.web.webhdfs;
019
020import com.google.common.base.Preconditions;
021import io.netty.buffer.Unpooled;
022import io.netty.channel.ChannelFutureListener;
023import io.netty.channel.ChannelHandlerContext;
024import io.netty.channel.SimpleChannelInboundHandler;
025import io.netty.handler.codec.http.DefaultFullHttpResponse;
026import io.netty.handler.codec.http.DefaultHttpResponse;
027import io.netty.handler.codec.http.HttpHeaders;
028import io.netty.handler.codec.http.HttpMethod;
029import io.netty.handler.codec.http.HttpRequest;
030import io.netty.handler.codec.http.QueryStringDecoder;
031import io.netty.handler.stream.ChunkedStream;
032import java.net.InetSocketAddress;
033import org.apache.commons.io.Charsets;
034import org.apache.commons.logging.Log;
035import org.apache.commons.logging.LogFactory;
036import org.apache.hadoop.conf.Configuration;
037import org.apache.hadoop.fs.CreateFlag;
038import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
039import org.apache.hadoop.fs.permission.FsPermission;
040import org.apache.hadoop.hdfs.DFSClient;
041import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
042import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
043import org.apache.hadoop.hdfs.web.JsonUtil;
044import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
045import org.apache.hadoop.hdfs.web.resources.GetOpParam;
046import org.apache.hadoop.hdfs.web.resources.PostOpParam;
047import org.apache.hadoop.hdfs.web.resources.PutOpParam;
048import org.apache.hadoop.io.IOUtils;
049import org.apache.hadoop.security.UserGroupInformation;
050import org.apache.hadoop.security.token.Token;
051import org.apache.hadoop.util.LimitInputStream;
052
053import java.io.IOException;
054import java.io.InputStream;
055import java.io.OutputStream;
056import java.net.URI;
057import java.net.URISyntaxException;
058import java.security.PrivilegedExceptionAction;
059import java.util.EnumSet;
060
061import static io.netty.handler.codec.http.HttpHeaders.Names.ACCESS_CONTROL_ALLOW_METHODS;
062import static io.netty.handler.codec.http.HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN;
063import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
064import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH;
065import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
066import static io.netty.handler.codec.http.HttpHeaders.Names.LOCATION;
067import static io.netty.handler.codec.http.HttpHeaders.Values.CLOSE;
068import static io.netty.handler.codec.http.HttpMethod.GET;
069import static io.netty.handler.codec.http.HttpMethod.POST;
070import static io.netty.handler.codec.http.HttpMethod.PUT;
071import static io.netty.handler.codec.http.HttpResponseStatus.CONTINUE;
072import static io.netty.handler.codec.http.HttpResponseStatus.CREATED;
073import static io.netty.handler.codec.http.HttpResponseStatus.OK;
074import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
075import static io.netty.handler.codec.rtsp.RtspResponseStatuses.INTERNAL_SERVER_ERROR;
076import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HDFS_URI_SCHEME;
077import static org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier.HDFS_DELEGATION_KIND;
078
079public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
080  static final Log LOG = LogFactory.getLog(WebHdfsHandler.class);
081  static final Log REQLOG = LogFactory.getLog("datanode.webhdfs");
082  public static final String WEBHDFS_PREFIX = WebHdfsFileSystem.PATH_PREFIX;
083  public static final int WEBHDFS_PREFIX_LENGTH = WEBHDFS_PREFIX.length();
084  public static final String APPLICATION_OCTET_STREAM =
085    "application/octet-stream";
086  public static final String APPLICATION_JSON_UTF8 =
087      "application/json; charset=utf-8";
088
089  private final Configuration conf;
090  private final Configuration confForCreate;
091
092  private String path;
093  private ParameterParser params;
094  private UserGroupInformation ugi;
095  private DefaultHttpResponse resp = null;
096
097  public WebHdfsHandler(Configuration conf, Configuration confForCreate)
098    throws IOException {
099    this.conf = conf;
100    this.confForCreate = confForCreate;
101  }
102
103  @Override
104  public void channelRead0(final ChannelHandlerContext ctx,
105                           final HttpRequest req) throws Exception {
106    Preconditions.checkArgument(req.getUri().startsWith(WEBHDFS_PREFIX));
107    QueryStringDecoder queryString = new QueryStringDecoder(req.getUri());
108    params = new ParameterParser(queryString, conf);
109    DataNodeUGIProvider ugiProvider = new DataNodeUGIProvider(params);
110    ugi = ugiProvider.ugi();
111    path = params.path();
112
113    injectToken();
114    ugi.doAs(new PrivilegedExceptionAction<Void>() {
115      @Override
116      public Void run() throws Exception {
117        try {
118          handle(ctx, req);
119        } finally {
120          String host = null;
121          try {
122            host = ((InetSocketAddress)ctx.channel().remoteAddress()).
123                getAddress().getHostAddress();
124          } catch (Exception e) {
125            LOG.warn("Error retrieving hostname: ", e);
126            host = "unknown";
127          }
128          REQLOG.info(host + " " + req.getMethod() + " "  + req.getUri() + " " +
129              getResponseCode());
130        }
131        return null;
132      }
133    });
134  }
135
136  int getResponseCode() {
137    return (resp == null) ? INTERNAL_SERVER_ERROR.code() :
138        resp.getStatus().code();
139  }
140
141  public void handle(ChannelHandlerContext ctx, HttpRequest req)
142    throws IOException, URISyntaxException {
143    String op = params.op();
144    HttpMethod method = req.getMethod();
145    if (PutOpParam.Op.CREATE.name().equalsIgnoreCase(op)
146      && method == PUT) {
147      onCreate(ctx);
148    } else if (PostOpParam.Op.APPEND.name().equalsIgnoreCase(op)
149      && method == POST) {
150      onAppend(ctx);
151    } else if (GetOpParam.Op.OPEN.name().equalsIgnoreCase(op)
152      && method == GET) {
153      onOpen(ctx);
154    } else if(GetOpParam.Op.GETFILECHECKSUM.name().equalsIgnoreCase(op)
155      && method == GET) {
156      onGetFileChecksum(ctx);
157    } else {
158      throw new IllegalArgumentException("Invalid operation " + op);
159    }
160  }
161
162  @Override
163  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
164    LOG.debug("Error ", cause);
165    resp = ExceptionHandler.exceptionCaught(cause);
166    resp.headers().set(CONNECTION, CLOSE);
167    ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE);
168  }
169
170  private void onCreate(ChannelHandlerContext ctx)
171    throws IOException, URISyntaxException {
172    writeContinueHeader(ctx);
173
174    final String nnId = params.namenodeId();
175    final int bufferSize = params.bufferSize();
176    final short replication = params.replication();
177    final long blockSize = params.blockSize();
178    final FsPermission permission = params.permission();
179
180    EnumSet<CreateFlag> flags = params.overwrite() ?
181      EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
182        : EnumSet.of(CreateFlag.CREATE);
183
184    final DFSClient dfsClient = newDfsClient(nnId, confForCreate);
185    OutputStream out = dfsClient.createWrappedOutputStream(dfsClient.create(
186      path, permission, flags, replication,
187      blockSize, null, bufferSize, null), null);
188    resp = new DefaultHttpResponse(HTTP_1_1, CREATED);
189
190    final URI uri = new URI(HDFS_URI_SCHEME, nnId, path, null, null);
191    resp.headers().set(LOCATION, uri.toString());
192    resp.headers().set(CONTENT_LENGTH, 0);
193    ctx.pipeline().replace(this, HdfsWriter.class.getSimpleName(),
194      new HdfsWriter(dfsClient, out, resp));
195  }
196
197  private void onAppend(ChannelHandlerContext ctx) throws IOException {
198    writeContinueHeader(ctx);
199    final String nnId = params.namenodeId();
200    final int bufferSize = params.bufferSize();
201
202    DFSClient dfsClient = newDfsClient(nnId, conf);
203    OutputStream out = dfsClient.append(path, bufferSize,
204        EnumSet.of(CreateFlag.APPEND), null, null);
205    resp = new DefaultHttpResponse(HTTP_1_1, OK);
206    resp.headers().set(CONTENT_LENGTH, 0);
207    ctx.pipeline().replace(this, HdfsWriter.class.getSimpleName(),
208      new HdfsWriter(dfsClient, out, resp));
209  }
210
211  private void onOpen(ChannelHandlerContext ctx) throws IOException {
212    final String nnId = params.namenodeId();
213    final int bufferSize = params.bufferSize();
214    final long offset = params.offset();
215    final long length = params.length();
216
217    resp = new DefaultHttpResponse(HTTP_1_1, OK);
218    HttpHeaders headers = resp.headers();
219    // Allow the UI to access the file
220    headers.set(ACCESS_CONTROL_ALLOW_METHODS, GET);
221    headers.set(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
222    headers.set(CONTENT_TYPE, APPLICATION_OCTET_STREAM);
223    headers.set(CONNECTION, CLOSE);
224
225    final DFSClient dfsclient = newDfsClient(nnId, conf);
226    HdfsDataInputStream in = dfsclient.createWrappedInputStream(
227      dfsclient.open(path, bufferSize, true));
228    in.seek(offset);
229
230    long contentLength = in.getVisibleLength() - offset;
231    if (length >= 0) {
232      contentLength = Math.min(contentLength, length);
233    }
234    final InputStream data;
235    if (contentLength >= 0) {
236      headers.set(CONTENT_LENGTH, contentLength);
237      data = new LimitInputStream(in, contentLength);
238    } else {
239      data = in;
240    }
241
242    ctx.write(resp);
243    ctx.writeAndFlush(new ChunkedStream(data) {
244      @Override
245      public void close() throws Exception {
246        super.close();
247        dfsclient.close();
248      }
249    }).addListener(ChannelFutureListener.CLOSE);
250  }
251
252  private void onGetFileChecksum(ChannelHandlerContext ctx) throws IOException {
253    MD5MD5CRC32FileChecksum checksum = null;
254    final String nnId = params.namenodeId();
255    DFSClient dfsclient = newDfsClient(nnId, conf);
256    try {
257      checksum = dfsclient.getFileChecksum(path, Long.MAX_VALUE);
258      dfsclient.close();
259      dfsclient = null;
260    } finally {
261      IOUtils.cleanup(LOG, dfsclient);
262    }
263    final byte[] js = JsonUtil.toJsonString(checksum).getBytes(Charsets.UTF_8);
264    resp =
265      new DefaultFullHttpResponse(HTTP_1_1, OK, Unpooled.wrappedBuffer(js));
266
267    resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);
268    resp.headers().set(CONTENT_LENGTH, js.length);
269    resp.headers().set(CONNECTION, CLOSE);
270    ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE);
271  }
272
273  private static void writeContinueHeader(ChannelHandlerContext ctx) {
274    DefaultHttpResponse r = new DefaultFullHttpResponse(HTTP_1_1, CONTINUE,
275      Unpooled.EMPTY_BUFFER);
276    ctx.writeAndFlush(r);
277  }
278
279  private static DFSClient newDfsClient
280    (String nnId, Configuration conf) throws IOException {
281    URI uri = URI.create(HDFS_URI_SCHEME + "://" + nnId);
282    return new DFSClient(uri, conf);
283  }
284
285  private void injectToken() throws IOException {
286    if (UserGroupInformation.isSecurityEnabled()) {
287      Token<DelegationTokenIdentifier> token = params.delegationToken();
288      token.setKind(HDFS_DELEGATION_KIND);
289      ugi.addToken(token);
290    }
291  }
292}