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 org.apache.commons.io.Charsets;
033import org.apache.commons.logging.Log;
034import org.apache.commons.logging.LogFactory;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.fs.CreateFlag;
037import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
038import org.apache.hadoop.fs.permission.FsPermission;
039import org.apache.hadoop.hdfs.DFSClient;
040import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
041import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
042import org.apache.hadoop.hdfs.web.JsonUtil;
043import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
044import org.apache.hadoop.hdfs.web.resources.GetOpParam;
045import org.apache.hadoop.hdfs.web.resources.PostOpParam;
046import org.apache.hadoop.hdfs.web.resources.PutOpParam;
047import org.apache.hadoop.io.IOUtils;
048import org.apache.hadoop.security.UserGroupInformation;
049import org.apache.hadoop.security.token.Token;
050import org.apache.hadoop.util.LimitInputStream;
051
052import java.io.IOException;
053import java.io.InputStream;
054import java.io.OutputStream;
055import java.net.URI;
056import java.net.URISyntaxException;
057import java.security.PrivilegedExceptionAction;
058import java.util.EnumSet;
059
060import static io.netty.handler.codec.http.HttpHeaders.Names.ACCESS_CONTROL_ALLOW_METHODS;
061import static io.netty.handler.codec.http.HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN;
062import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
063import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH;
064import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
065import static io.netty.handler.codec.http.HttpHeaders.Names.LOCATION;
066import static io.netty.handler.codec.http.HttpHeaders.Values.CLOSE;
067import static io.netty.handler.codec.http.HttpMethod.GET;
068import static io.netty.handler.codec.http.HttpMethod.POST;
069import static io.netty.handler.codec.http.HttpMethod.PUT;
070import static io.netty.handler.codec.http.HttpResponseStatus.CONTINUE;
071import static io.netty.handler.codec.http.HttpResponseStatus.CREATED;
072import static io.netty.handler.codec.http.HttpResponseStatus.OK;
073import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
074import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HDFS_URI_SCHEME;
075import static org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier.HDFS_DELEGATION_KIND;
076
077public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
078  static final Log LOG = LogFactory.getLog(WebHdfsHandler.class);
079  public static final String WEBHDFS_PREFIX = WebHdfsFileSystem.PATH_PREFIX;
080  public static final int WEBHDFS_PREFIX_LENGTH = WEBHDFS_PREFIX.length();
081  public static final String APPLICATION_OCTET_STREAM =
082    "application/octet-stream";
083  public static final String APPLICATION_JSON_UTF8 =
084      "application/json; charset=utf-8";
085
086  private final Configuration conf;
087  private final Configuration confForCreate;
088
089  private String path;
090  private ParameterParser params;
091  private UserGroupInformation ugi;
092
093  public WebHdfsHandler(Configuration conf, Configuration confForCreate)
094    throws IOException {
095    this.conf = conf;
096    this.confForCreate = confForCreate;
097  }
098
099  @Override
100  public void channelRead0(final ChannelHandlerContext ctx,
101                           final HttpRequest req) throws Exception {
102    Preconditions.checkArgument(req.getUri().startsWith(WEBHDFS_PREFIX));
103    QueryStringDecoder queryString = new QueryStringDecoder(req.getUri());
104    params = new ParameterParser(queryString, conf);
105    DataNodeUGIProvider ugiProvider = new DataNodeUGIProvider(params);
106    ugi = ugiProvider.ugi();
107    path = params.path();
108
109    injectToken();
110    ugi.doAs(new PrivilegedExceptionAction<Void>() {
111      @Override
112      public Void run() throws Exception {
113        handle(ctx, req);
114        return null;
115      }
116    });
117  }
118
119  public void handle(ChannelHandlerContext ctx, HttpRequest req)
120    throws IOException, URISyntaxException {
121    String op = params.op();
122    HttpMethod method = req.getMethod();
123    if (PutOpParam.Op.CREATE.name().equalsIgnoreCase(op)
124      && method == PUT) {
125      onCreate(ctx);
126    } else if (PostOpParam.Op.APPEND.name().equalsIgnoreCase(op)
127      && method == POST) {
128      onAppend(ctx);
129    } else if (GetOpParam.Op.OPEN.name().equalsIgnoreCase(op)
130      && method == GET) {
131      onOpen(ctx);
132    } else if(GetOpParam.Op.GETFILECHECKSUM.name().equalsIgnoreCase(op)
133      && method == GET) {
134      onGetFileChecksum(ctx);
135    } else {
136      throw new IllegalArgumentException("Invalid operation " + op);
137    }
138  }
139
140  @Override
141  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
142    LOG.debug("Error ", cause);
143    DefaultHttpResponse resp = ExceptionHandler.exceptionCaught(cause);
144    resp.headers().set(CONNECTION, CLOSE);
145    ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE);
146  }
147
148  private void onCreate(ChannelHandlerContext ctx)
149    throws IOException, URISyntaxException {
150    writeContinueHeader(ctx);
151
152    final String nnId = params.namenodeId();
153    final int bufferSize = params.bufferSize();
154    final short replication = params.replication();
155    final long blockSize = params.blockSize();
156    final FsPermission permission = params.permission();
157
158    EnumSet<CreateFlag> flags = params.overwrite() ?
159      EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
160        : EnumSet.of(CreateFlag.CREATE);
161
162    final DFSClient dfsClient = newDfsClient(nnId, confForCreate);
163    OutputStream out = dfsClient.createWrappedOutputStream(dfsClient.create(
164      path, permission, flags, replication,
165      blockSize, null, bufferSize, null), null);
166    DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, CREATED);
167
168    final URI uri = new URI(HDFS_URI_SCHEME, nnId, path, null, null);
169    resp.headers().set(LOCATION, uri.toString());
170    resp.headers().set(CONTENT_LENGTH, 0);
171    ctx.pipeline().replace(this, HdfsWriter.class.getSimpleName(),
172      new HdfsWriter(dfsClient, out, resp));
173  }
174
175  private void onAppend(ChannelHandlerContext ctx) throws IOException {
176    writeContinueHeader(ctx);
177    final String nnId = params.namenodeId();
178    final int bufferSize = params.bufferSize();
179
180    DFSClient dfsClient = newDfsClient(nnId, conf);
181    OutputStream out = dfsClient.append(path, bufferSize,
182        EnumSet.of(CreateFlag.APPEND), null, null);
183    DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, OK);
184    resp.headers().set(CONTENT_LENGTH, 0);
185    ctx.pipeline().replace(this, HdfsWriter.class.getSimpleName(),
186      new HdfsWriter(dfsClient, out, resp));
187  }
188
189  private void onOpen(ChannelHandlerContext ctx) throws IOException {
190    final String nnId = params.namenodeId();
191    final int bufferSize = params.bufferSize();
192    final long offset = params.offset();
193    final long length = params.length();
194
195    DefaultHttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
196    HttpHeaders headers = response.headers();
197    // Allow the UI to access the file
198    headers.set(ACCESS_CONTROL_ALLOW_METHODS, GET);
199    headers.set(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
200    headers.set(CONTENT_TYPE, APPLICATION_OCTET_STREAM);
201    headers.set(CONNECTION, CLOSE);
202
203    final DFSClient dfsclient = newDfsClient(nnId, conf);
204    HdfsDataInputStream in = dfsclient.createWrappedInputStream(
205      dfsclient.open(path, bufferSize, true));
206    in.seek(offset);
207
208    long contentLength = in.getVisibleLength() - offset;
209    if (length >= 0) {
210      contentLength = Math.min(contentLength, length);
211    }
212    final InputStream data;
213    if (contentLength >= 0) {
214      headers.set(CONTENT_LENGTH, contentLength);
215      data = new LimitInputStream(in, contentLength);
216    } else {
217      data = in;
218    }
219
220    ctx.write(response);
221    ctx.writeAndFlush(new ChunkedStream(data) {
222      @Override
223      public void close() throws Exception {
224        super.close();
225        dfsclient.close();
226      }
227    }).addListener(ChannelFutureListener.CLOSE);
228  }
229
230  private void onGetFileChecksum(ChannelHandlerContext ctx) throws IOException {
231    MD5MD5CRC32FileChecksum checksum = null;
232    final String nnId = params.namenodeId();
233    DFSClient dfsclient = newDfsClient(nnId, conf);
234    try {
235      checksum = dfsclient.getFileChecksum(path, Long.MAX_VALUE);
236      dfsclient.close();
237      dfsclient = null;
238    } finally {
239      IOUtils.cleanup(LOG, dfsclient);
240    }
241    final byte[] js = JsonUtil.toJsonString(checksum).getBytes(Charsets.UTF_8);
242    DefaultFullHttpResponse resp =
243      new DefaultFullHttpResponse(HTTP_1_1, OK, Unpooled.wrappedBuffer(js));
244
245    resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);
246    resp.headers().set(CONTENT_LENGTH, js.length);
247    resp.headers().set(CONNECTION, CLOSE);
248    ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE);
249  }
250
251  private static void writeContinueHeader(ChannelHandlerContext ctx) {
252    DefaultHttpResponse r = new DefaultFullHttpResponse(HTTP_1_1, CONTINUE,
253      Unpooled.EMPTY_BUFFER);
254    ctx.writeAndFlush(r);
255  }
256
257  private static DFSClient newDfsClient
258    (String nnId, Configuration conf) throws IOException {
259    URI uri = URI.create(HDFS_URI_SCHEME + "://" + nnId);
260    return new DFSClient(uri, conf);
261  }
262
263  private void injectToken() throws IOException {
264    if (UserGroupInformation.isSecurityEnabled()) {
265      Token<DelegationTokenIdentifier> token = params.delegationToken();
266      token.setKind(HDFS_DELEGATION_KIND);
267      ugi.addToken(token);
268    }
269  }
270}