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.hbase.client.example;
019
020import java.io.IOException;
021import java.net.InetSocketAddress;
022import java.util.Optional;
023import java.util.concurrent.ExecutionException;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.hbase.HBaseConfiguration;
026import org.apache.hadoop.hbase.TableName;
027import org.apache.hadoop.hbase.client.AsyncConnection;
028import org.apache.hadoop.hbase.client.ConnectionFactory;
029import org.apache.hadoop.hbase.client.Get;
030import org.apache.hadoop.hbase.client.Put;
031import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
032import org.apache.hadoop.hbase.util.Bytes;
033import org.apache.yetus.audience.InterfaceAudience;
034
035import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
036import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
037import org.apache.hbase.thirdparty.io.netty.bootstrap.ServerBootstrap;
038import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
039import org.apache.hbase.thirdparty.io.netty.channel.Channel;
040import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
041import org.apache.hbase.thirdparty.io.netty.channel.ChannelInitializer;
042import org.apache.hbase.thirdparty.io.netty.channel.ChannelOption;
043import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
044import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
045import org.apache.hbase.thirdparty.io.netty.channel.group.ChannelGroup;
046import org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroup;
047import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
048import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioServerSocketChannel;
049import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
050import org.apache.hbase.thirdparty.io.netty.handler.codec.http.DefaultFullHttpResponse;
051import org.apache.hbase.thirdparty.io.netty.handler.codec.http.FullHttpRequest;
052import org.apache.hbase.thirdparty.io.netty.handler.codec.http.HttpHeaderNames;
053import org.apache.hbase.thirdparty.io.netty.handler.codec.http.HttpObjectAggregator;
054import org.apache.hbase.thirdparty.io.netty.handler.codec.http.HttpResponseStatus;
055import org.apache.hbase.thirdparty.io.netty.handler.codec.http.HttpServerCodec;
056import org.apache.hbase.thirdparty.io.netty.handler.codec.http.HttpVersion;
057import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringDecoder;
058import org.apache.hbase.thirdparty.io.netty.util.concurrent.GlobalEventExecutor;
059
060/**
061 * A simple example on how to use {@link org.apache.hadoop.hbase.client.AsyncTable} to write a fully
062 * asynchronous HTTP proxy server. The {@link AsyncConnection} will share the same event loop with
063 * the HTTP server.
064 * <p>
065 * The request URL is:
066 *
067 * <pre>
068 * http://&lt;host&gt;:&lt;port&gt;/&lt;table&gt;/&lt;rowgt;/&lt;family&gt;:&lt;qualifier&gt;
069 * </pre>
070 *
071 * Use HTTP GET to fetch data, and use HTTP PUT to put data. Encode the value as the request content
072 * when doing PUT.
073 */
074@InterfaceAudience.Private
075public class HttpProxyExample {
076
077  private final EventLoopGroup bossGroup = new NioEventLoopGroup(1);
078
079  private final EventLoopGroup workerGroup = new NioEventLoopGroup();
080
081  private final Configuration conf;
082
083  private final int port;
084
085  private AsyncConnection conn;
086
087  private Channel serverChannel;
088
089  private ChannelGroup channelGroup;
090
091  public HttpProxyExample(Configuration conf, int port) {
092    this.conf = conf;
093    this.port = port;
094  }
095
096  private static final class Params {
097    public final String table;
098
099    public final String row;
100
101    public final String family;
102
103    public final String qualifier;
104
105    public Params(String table, String row, String family, String qualifier) {
106      this.table = table;
107      this.row = row;
108      this.family = family;
109      this.qualifier = qualifier;
110    }
111  }
112
113  private static final class RequestHandler extends SimpleChannelInboundHandler<FullHttpRequest> {
114
115    private final AsyncConnection conn;
116
117    private final ChannelGroup channelGroup;
118
119    public RequestHandler(AsyncConnection conn, ChannelGroup channelGroup) {
120      this.conn = conn;
121      this.channelGroup = channelGroup;
122    }
123
124    @Override
125    public void channelActive(ChannelHandlerContext ctx) {
126      channelGroup.add(ctx.channel());
127      ctx.fireChannelActive();
128    }
129
130    @Override
131    public void channelInactive(ChannelHandlerContext ctx) {
132      channelGroup.remove(ctx.channel());
133      ctx.fireChannelInactive();
134    }
135
136    private void write(ChannelHandlerContext ctx, HttpResponseStatus status,
137        Optional<String> content) {
138      DefaultFullHttpResponse resp;
139      if (content.isPresent()) {
140        ByteBuf buf =
141            ctx.alloc().buffer().writeBytes(Bytes.toBytes(content.get()));
142        resp = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, status, buf);
143        resp.headers().set(HttpHeaderNames.CONTENT_LENGTH, buf.readableBytes());
144      } else {
145        resp = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, status);
146      }
147      resp.headers().set(HttpHeaderNames.CONTENT_TYPE, "text-plain; charset=UTF-8");
148      ctx.writeAndFlush(resp);
149    }
150
151    private Params parse(FullHttpRequest req) {
152      String[] components = new QueryStringDecoder(req.uri()).path().split("/");
153      Preconditions.checkArgument(components.length == 4, "Unrecognized uri: %s", req.uri());
154      // path is start with '/' so split will give an empty component
155      String[] cfAndCq = components[3].split(":");
156      Preconditions.checkArgument(cfAndCq.length == 2, "Unrecognized uri: %s", req.uri());
157      return new Params(components[1], components[2], cfAndCq[0], cfAndCq[1]);
158    }
159
160    private void get(ChannelHandlerContext ctx, FullHttpRequest req) {
161      Params params = parse(req);
162      conn.getTable(TableName.valueOf(params.table)).get(new Get(Bytes.toBytes(params.row))
163          .addColumn(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier)))
164          .whenComplete((r, e) -> {
165            if (e != null) {
166              exceptionCaught(ctx, e);
167            } else {
168              byte[] value =
169                  r.getValue(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier));
170              if (value != null) {
171                write(ctx, HttpResponseStatus.OK, Optional.of(Bytes.toStringBinary(value)));
172              } else {
173                write(ctx, HttpResponseStatus.NOT_FOUND, Optional.empty());
174              }
175            }
176          });
177    }
178
179    private void put(ChannelHandlerContext ctx, FullHttpRequest req) {
180      Params params = parse(req);
181      byte[] value = new byte[req.content().readableBytes()];
182      req.content().readBytes(value);
183      conn.getTable(TableName.valueOf(params.table)).put(new Put(Bytes.toBytes(params.row))
184          .addColumn(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier), value))
185          .whenComplete((r, e) -> {
186            if (e != null) {
187              exceptionCaught(ctx, e);
188            } else {
189              write(ctx, HttpResponseStatus.OK, Optional.empty());
190            }
191          });
192    }
193
194    @Override
195    protected void channelRead0(ChannelHandlerContext ctx, FullHttpRequest req) {
196      switch (req.method().name()) {
197        case "GET":
198          get(ctx, req);
199          break;
200        case "PUT":
201          put(ctx, req);
202          break;
203        default:
204          write(ctx, HttpResponseStatus.METHOD_NOT_ALLOWED, Optional.empty());
205          break;
206      }
207    }
208
209    @Override
210    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
211      if (cause instanceof IllegalArgumentException) {
212        write(ctx, HttpResponseStatus.BAD_REQUEST, Optional.of(cause.getMessage()));
213      } else {
214        write(ctx, HttpResponseStatus.INTERNAL_SERVER_ERROR,
215          Optional.of(Throwables.getStackTraceAsString(cause)));
216      }
217    }
218  }
219
220  public void start() throws InterruptedException, ExecutionException {
221    NettyRpcClientConfigHelper.setEventLoopConfig(conf, workerGroup, NioSocketChannel.class);
222    conn = ConnectionFactory.createAsyncConnection(conf).get();
223    channelGroup = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
224    serverChannel = new ServerBootstrap().group(bossGroup, workerGroup)
225        .channel(NioServerSocketChannel.class).childOption(ChannelOption.TCP_NODELAY, true)
226        .childHandler(new ChannelInitializer<Channel>() {
227
228          @Override
229          protected void initChannel(Channel ch) throws Exception {
230            ch.pipeline().addFirst(new HttpServerCodec(), new HttpObjectAggregator(4 * 1024 * 1024),
231              new RequestHandler(conn, channelGroup));
232          }
233        }).bind(port).syncUninterruptibly().channel();
234  }
235
236  public void join() {
237    serverChannel.closeFuture().awaitUninterruptibly();
238  }
239
240  public int port() {
241    if (serverChannel == null) {
242      return port;
243    } else {
244      return ((InetSocketAddress) serverChannel.localAddress()).getPort();
245    }
246  }
247
248  public void stop() throws IOException {
249    serverChannel.close().syncUninterruptibly();
250    serverChannel = null;
251    channelGroup.close().syncUninterruptibly();
252    channelGroup = null;
253    conn.close();
254    conn = null;
255  }
256
257  public static void main(String[] args) throws InterruptedException, ExecutionException {
258    int port = Integer.parseInt(args[0]);
259    HttpProxyExample proxy = new HttpProxyExample(HBaseConfiguration.create(), port);
260    proxy.start();
261    proxy.join();
262  }
263}