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