You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by xv...@apache.org on 2021/12/07 06:46:59 UTC

[druid] 01/01: Migrade Druid HttpClient from Netty 3 to 4

This is an automated email from the ASF dual-hosted git repository.

xvrl pushed a commit to branch remove-netty-3
in repository https://gitbox.apache.org/repos/asf/druid.git

commit 0fd7ad25cdd67de0e62cca8b3eb18ba61bd96796
Author: Xavier Léauté <xv...@apache.org>
AuthorDate: Sun Dec 5 13:11:43 2021 -0800

    Migrade Druid HttpClient from Netty 3 to 4
    
    - HttpResponse no longer includes request content, this changes some
      HttpReponseHandler implementations to no longer assume content is
      generated as part of the handleResponse call, and instead expect all
      request content as part of the handleChunk
    
    - HttpResponseHandler changes uncovered a long-standing bug in
      AppendableByteArrayInputStream where calling exceptionCaught before
      appending any bytes would not throw the exception to the reader
    
    - Netty HTTP decoder now stores decoder errors in the message instead of
      triggering exceptionCaught, so we need to check the statuf of the
      decoder.
    
    - Netty Messages and underlying buffers are reference counted, so they
      need to be explicitly released after handling. As part of this
      ChannelBuffer became ByteBuf and are reference counted instead of
      garbage collected.
    
    - To support back-pressure, Channels need to disable AUTO_READ and call
      read() explicitly when needed instead of using setReadable(true/false)
    
    - ClientBootstrap no longer uses separate bosspool and workerpool sizes,
      instead relying on the workerpool.
    
    - Netty HTTP codec now supports resuming handling of HTTP messages after
      handling CONNECT, removing the need to manually add the codec back to
      the pipeline.
    
    - Remove deprecated Netty method calls and rename classes to align with
      Netty class name changes
    
    - Remove deprecated use of ExpectedException in favor of
      Assert.assertThrows for all tests that had to be touched.
---
 .../java/util/http/client/HttpClientConfig.java    |  13 -
 .../java/util/http/client/HttpClientInit.java      |  85 ++-----
 .../java/util/http/client/NettyHttpClient.java     | 190 ++++++++-------
 .../druid/java/util/http/client/Request.java       |  82 +++----
 .../client/io/AppendableByteArrayInputStream.java  |   4 +
 ...lineFactory.java => HttpClientInitializer.java} |  30 ++-
 .../http/client/pool/ChannelResourceFactory.java   | 263 +++++++++------------
 .../client/response/BytesFullResponseHandler.java  |  14 +-
 .../client/response/BytesFullResponseHolder.java   |   2 +-
 .../http/client/response/FullResponseHolder.java   |   6 +-
 .../http/client/response/HttpResponseHandler.java  |   8 +-
 .../response/InputStreamFullResponseHandler.java   |  13 +-
 .../response/InputStreamFullResponseHolder.java    |   2 +-
 .../response/InputStreamResponseHandler.java       |  13 +-
 .../response/ObjectOrErrorResponseHandler.java     |   8 +-
 .../SequenceInputStreamResponseHandler.java        |  28 +--
 .../client/response/StatusResponseHandler.java     |  17 +-
 .../http/client/response/StatusResponseHolder.java |   3 +-
 .../client/response/StringFullResponseHandler.java |  10 +-
 .../client/response/StringFullResponseHolder.java  |   9 +-
 .../java/util/http/client/FriendlyServersTest.java | 192 +++++++--------
 .../java/util/http/client/JankyServersTest.java    | 224 +++++++-----------
 .../io/AppendableByteArrayInputStreamTest.java     |  11 +
 .../InputStreamFullResponseHandlerTest.java        |  24 +-
 .../response/ObjectOrErrorResponseHandlerTest.java |  42 ++--
 .../SequenceInputStreamResponseHandlerTest.java    |  68 +++---
 docs/development/modules.md                        |   3 +-
 .../druid/security/basic/CommonCacheNotifier.java  |  10 +-
 ...natorPollingBasicAuthenticatorCacheManager.java |   2 +-
 ...rdinatorPollingBasicAuthorizerCacheManager.java |   4 +-
 .../security/kerberos/KerberosHttpClient.java      |   4 +-
 .../security/kerberos/ResponseCookieHandler.java   |   8 +-
 .../RetryIfUnauthorizedResponseHandler.java        |  16 +-
 .../indexing/kafka/KafkaIndexTaskClientTest.java   |   8 +-
 .../kinesis/KinesisIndexTaskClientTest.java        |   8 +-
 .../druid/indexing/common/IndexTaskClient.java     |  13 +-
 .../common/actions/RemoteTaskActionClient.java     |   6 +-
 .../task/batch/parallel/HttpShuffleClient.java     |   2 +-
 .../ParallelIndexSupervisorTaskClient.java         |   8 +-
 .../druid/indexing/overlord/RemoteTaskRunner.java  |   4 +-
 .../overlord/WorkerTaskRunnerQueryAdapter.java     |   8 +-
 .../overlord/hrtr/HttpRemoteTaskRunner.java        |   2 +-
 .../druid/indexing/overlord/hrtr/WorkerHolder.java |  12 +-
 .../SeekableStreamIndexTaskClient.java             |   4 +-
 .../druid/indexing/worker/WorkerTaskManager.java   |  14 +-
 .../druid/indexing/common/IndexTaskClientTest.java |  34 +--
 .../common/actions/RemoteTaskActionClientTest.java |  40 ++--
 .../overlord/WorkerTaskRunnerQueryAdpaterTest.java |   4 +-
 .../overlord/http/OverlordResourceTest.java        |  10 +-
 .../clients/AbstractQueryResourceTestClient.java   |  10 +-
 .../clients/ClientInfoResourceTestClient.java      |   4 +-
 .../clients/CompactionResourceTestClient.java      |   4 +-
 .../clients/CoordinatorResourceTestClient.java     |  12 +-
 .../clients/EventReceiverFirehoseTestClient.java   |   4 +-
 .../clients/OverlordResourceTestClient.java        |   4 +-
 .../testing/utils/DruidClusterAdminClient.java     |   6 +-
 .../org/apache/druid/testing/utils/HttpUtil.java   |   4 +-
 .../tests/leadership/ITHighAvailabilityTest.java   |   6 +-
 .../query/ITQueryRetryTestOnMissingSegments.java   |   6 +-
 .../apache/druid/tests/query/ITSqlCancelTest.java  |   2 +-
 .../apache/druid/tests/query/ITUnionQueryTest.java |   4 +-
 .../druid/tests/query/ITWikipediaQueryTest.java    |  14 +-
 .../security/AbstractAuthConfigurationTest.java    |   4 +-
 .../security/ITBasicAuthConfigurationTest.java     |   2 +-
 .../security/ITBasicAuthLdapConfigurationTest.java |   2 +-
 .../ITCoordinatorOverlordProxyAuthTest.java        |   2 +-
 .../org/apache/druid/tests/security/ITTLSTest.java |   4 +-
 owasp-dependency-check-suppressions.xml            |  12 -
 pom.xml                                            |  12 -
 .../org/apache/druid/client/DirectDruidClient.java |  55 +++--
 .../org/apache/druid/client/InputStreamHolder.java |  13 +-
 .../client/coordinator/CoordinatorClient.java      |   4 +-
 .../client/indexing/HttpIndexingServiceClient.java |   4 +-
 .../apache/druid/discovery/DruidLeaderClient.java  |  29 +--
 .../query/lookup/LookupReferencesManager.java      |   6 +-
 .../coordination/ChangeRequestHttpSyncer.java      |   8 +-
 .../BytesAccumulatingResponseHandler.java          |   6 +-
 .../server/coordinator/HttpLoadQueuePeon.java      |   8 +-
 .../lookup/cache/LookupCoordinatorManager.java     |  16 +-
 .../apache/druid/client/DirectDruidClientTest.java |   8 +-
 .../druid/client/HttpServerInventoryViewTest.java  |  13 +-
 .../org/apache/druid/client/TestHttpClient.java    |  26 +-
 .../indexing/HttpIndexingServiceClientTest.java    |  45 ++--
 .../druid/discovery/DruidLeaderClientTest.java     |  40 ++--
 .../query/lookup/LookupReferencesManagerTest.java  |  50 ++--
 .../coordination/ChangeRequestHttpSyncerTest.java  |  13 +-
 .../server/coordinator/HttpLoadQueuePeonTest.java  |  11 +-
 .../coordinator/duty/CompactSegmentsTest.java      |  14 +-
 .../druid/server/initialization/JettyQosTest.java  |   2 +-
 .../druid/server/initialization/JettyTest.java     |   2 +-
 .../org/apache/druid/cli/PullDependencies.java     |   2 +-
 .../server/router/CoordinatorRuleManager.java      |   4 +-
 .../server/router/CoordinatorRuleManagerTest.java  |   2 +-
 .../druid/sql/calcite/schema/SystemSchema.java     |   8 +-
 .../druid/sql/calcite/schema/SystemSchemaTest.java |  14 +-
 95 files changed, 923 insertions(+), 1183 deletions(-)

diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java b/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java
index 08e5dac..99a20d2 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java
@@ -67,9 +67,6 @@ public class HttpClientConfig
 
   public static final CompressionCodec DEFAULT_COMPRESSION_CODEC = CompressionCodec.GZIP;
 
-  // Default from NioClientSocketChannelFactory.DEFAULT_BOSS_COUNT, which is private:
-  private static final int DEFAULT_BOSS_COUNT = 1;
-
   // Default from SelectorUtil.DEFAULT_IO_THREADS, which is private:
   private static final int DEFAULT_WORKER_COUNT = JvmUtils.getRuntimeInfo().getAvailableProcessors() * 2;
 
@@ -85,7 +82,6 @@ public class HttpClientConfig
   private final HttpClientProxyConfig proxyConfig;
   private final Duration readTimeout;
   private final Duration sslHandshakeTimeout;
-  private final int bossPoolSize;
   private final int workerPoolSize;
   private final CompressionCodec compressionCodec;
   private final Duration unusedConnectionTimeoutDuration;
@@ -96,7 +92,6 @@ public class HttpClientConfig
       HttpClientProxyConfig proxyConfig,
       Duration readTimeout,
       Duration sslHandshakeTimeout,
-      int bossPoolSize,
       int workerPoolSize,
       CompressionCodec compressionCodec,
       Duration unusedConnectionTimeoutDuration
@@ -107,7 +102,6 @@ public class HttpClientConfig
     this.proxyConfig = proxyConfig;
     this.readTimeout = readTimeout;
     this.sslHandshakeTimeout = sslHandshakeTimeout;
-    this.bossPoolSize = bossPoolSize;
     this.workerPoolSize = workerPoolSize;
     this.compressionCodec = compressionCodec;
     this.unusedConnectionTimeoutDuration = unusedConnectionTimeoutDuration;
@@ -138,11 +132,6 @@ public class HttpClientConfig
     return sslHandshakeTimeout;
   }
 
-  public int getBossPoolSize()
-  {
-    return bossPoolSize;
-  }
-
   public int getWorkerPoolSize()
   {
     return workerPoolSize;
@@ -165,7 +154,6 @@ public class HttpClientConfig
     private HttpClientProxyConfig proxyConfig = null;
     private Duration readTimeout = null;
     private Duration sslHandshakeTimeout = null;
-    private int bossCount = DEFAULT_BOSS_COUNT;
     private int workerCount = DEFAULT_WORKER_COUNT;
     private CompressionCodec compressionCodec = DEFAULT_COMPRESSION_CODEC;
     private Duration unusedConnectionTimeoutDuration = DEFAULT_UNUSED_CONNECTION_TIMEOUT_DURATION;
@@ -230,7 +218,6 @@ public class HttpClientConfig
           proxyConfig,
           readTimeout,
           sslHandshakeTimeout,
-          bossCount,
           workerCount,
           compressionCodec,
           unusedConnectionTimeoutDuration
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java b/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java
index fd3ee80..bd211cf 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java
@@ -21,27 +21,24 @@ package org.apache.druid.java.util.http.client;
 
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.internal.logging.InternalLoggerFactory;
+import io.netty.util.internal.logging.Slf4JLoggerFactory;
 import org.apache.druid.java.util.common.lifecycle.Lifecycle;
-import org.apache.druid.java.util.http.client.netty.HttpClientPipelineFactory;
+import org.apache.druid.java.util.http.client.netty.HttpClientInitializer;
 import org.apache.druid.java.util.http.client.pool.ChannelResourceFactory;
 import org.apache.druid.java.util.http.client.pool.ResourcePool;
 import org.apache.druid.java.util.http.client.pool.ResourcePoolConfig;
-import org.jboss.netty.bootstrap.ClientBootstrap;
-import org.jboss.netty.channel.socket.nio.NioClientBossPool;
-import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
-import org.jboss.netty.channel.socket.nio.NioWorkerPool;
-import org.jboss.netty.logging.InternalLoggerFactory;
-import org.jboss.netty.logging.Slf4JLoggerFactory;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.ThreadNameDeterminer;
-import org.jboss.netty.util.Timer;
 
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.TrustManagerFactory;
 import java.io.FileInputStream;
 import java.security.KeyStore;
 import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 
 /**
  *
@@ -53,39 +50,13 @@ public class HttpClientInit
     try {
       // We need to use the full constructor in order to set a ThreadNameDeterminer. The other parameters are taken
       // from the defaults in HashedWheelTimer's other constructors.
-      final HashedWheelTimer timer = new HashedWheelTimer(
-          new ThreadFactoryBuilder().setDaemon(true)
-                                    .setNameFormat("HttpClient-Timer-%s")
-                                    .build(),
-          ThreadNameDeterminer.CURRENT,
-          100,
-          TimeUnit.MILLISECONDS,
-          512
-      );
-      lifecycle.addMaybeStartHandler(
-          new Lifecycle.Handler()
-          {
-            @Override
-            public void start()
-            {
-              timer.start();
-            }
-
-            @Override
-            public void stop()
-            {
-              timer.stop();
-            }
-          }
-      );
       return lifecycle.addMaybeStartManagedInstance(
           new NettyHttpClient(
               new ResourcePool<>(
                   new ChannelResourceFactory(
-                      createBootstrap(lifecycle, timer, config.getBossPoolSize(), config.getWorkerPoolSize()),
+                      createBootstrap(lifecycle, config.getWorkerPoolSize()),
                       config.getSslContext(),
                       config.getProxyConfig(),
-                      timer,
                       config.getSslHandshakeTimeout() == null ? -1 : config.getSslHandshakeTimeout().getMillis()
                   ),
                   new ResourcePoolConfig(
@@ -94,8 +65,7 @@ public class HttpClientInit
                   )
               ),
               config.getReadTimeout(),
-              config.getCompressionCodec(),
-              timer
+              config.getCompressionCodec()
           )
       );
     }
@@ -124,37 +94,28 @@ public class HttpClientInit
     }
   }
 
-  private static ClientBootstrap createBootstrap(Lifecycle lifecycle, Timer timer, int bossPoolSize, int workerPoolSize)
+  private static Bootstrap createBootstrap(Lifecycle lifecycle, int workerPoolSize)
   {
-    final NioClientBossPool bossPool = new NioClientBossPool(
-        Executors.newCachedThreadPool(
-            new ThreadFactoryBuilder()
-                .setDaemon(true)
-                .setNameFormat("HttpClient-Netty-Boss-%s")
-                .build()
-        ),
-        bossPoolSize,
-        timer,
-        ThreadNameDeterminer.CURRENT
-    );
 
-    final NioWorkerPool workerPool = new NioWorkerPool(
+    final EventLoopGroup workerGroup = new NioEventLoopGroup(
+        workerPoolSize,
         Executors.newCachedThreadPool(
             new ThreadFactoryBuilder()
                 .setDaemon(true)
                 .setNameFormat("HttpClient-Netty-Worker-%s")
                 .build()
-        ),
-        workerPoolSize,
-        ThreadNameDeterminer.CURRENT
+        )
     );
 
-    final ClientBootstrap bootstrap = new ClientBootstrap(new NioClientSocketChannelFactory(bossPool, workerPool));
-
-    bootstrap.setOption("keepAlive", true);
-    bootstrap.setPipelineFactory(new HttpClientPipelineFactory());
+    final Bootstrap bootstrap = new Bootstrap()
+        .group(workerGroup)
+        .channel(NioSocketChannel.class)
+        .option(ChannelOption.SO_KEEPALIVE, true)
+        .option(ChannelOption.AUTO_READ, false)
+        .handler(new HttpClientInitializer());
 
-    InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
+    // TODO figure out logging
+    InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE);
 
     try {
       lifecycle.addMaybeStartHandler(
@@ -168,7 +129,7 @@ public class HttpClientInit
             @Override
             public void stop()
             {
-              bootstrap.releaseExternalResources();
+              workerGroup.shutdownGracefully();
             }
           }
       );
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java b/core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
index 25adef4..5c16e92 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
@@ -24,6 +24,25 @@ import com.google.common.collect.Multimap;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelException;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.handler.codec.DecoderResult;
+import io.netty.handler.codec.http.DefaultHttpContent;
+import io.netty.handler.codec.http.DefaultHttpRequest;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpObject;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpVersion;
+import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.handler.timeout.ReadTimeoutHandler;
+import io.netty.util.ReferenceCountUtil;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
@@ -34,24 +53,6 @@ import org.apache.druid.java.util.http.client.pool.ResourceContainer;
 import org.apache.druid.java.util.http.client.pool.ResourcePool;
 import org.apache.druid.java.util.http.client.response.ClientResponse;
 import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelStateEvent;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.handler.codec.http.DefaultHttpRequest;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpVersion;
-import org.jboss.netty.handler.timeout.ReadTimeoutHandler;
-import org.jboss.netty.util.Timer;
 import org.joda.time.Duration;
 
 import java.net.URL;
@@ -68,7 +69,6 @@ public class NettyHttpClient extends AbstractHttpClient
   private static final String READ_TIMEOUT_HANDLER_NAME = "read-timeout";
   private static final String LAST_HANDLER_NAME = "last-handler";
 
-  private final Timer timer;
   private final ResourcePool<String, ChannelFuture> pool;
   private final HttpClientConfig.CompressionCodec compressionCodec;
   private final Duration defaultReadTimeout;
@@ -77,18 +77,12 @@ public class NettyHttpClient extends AbstractHttpClient
   NettyHttpClient(
       ResourcePool<String, ChannelFuture> pool,
       Duration defaultReadTimeout,
-      HttpClientConfig.CompressionCodec compressionCodec,
-      Timer timer
+      HttpClientConfig.CompressionCodec compressionCodec
   )
   {
     this.pool = Preconditions.checkNotNull(pool, "pool");
     this.defaultReadTimeout = defaultReadTimeout;
     this.compressionCodec = Preconditions.checkNotNull(compressionCodec);
-    this.timer = timer;
-
-    if (defaultReadTimeout != null && defaultReadTimeout.getMillis() > 0) {
-      Preconditions.checkNotNull(timer, "timer");
-    }
   }
 
   @LifecycleStart
@@ -128,14 +122,11 @@ public class NettyHttpClient extends AbstractHttpClient
       return Futures.immediateFailedFuture(
           new ChannelException(
               "Faulty channel in resource pool",
-              channelFuture.getCause()
+              channelFuture.cause()
           )
       );
     } else {
-      channel = channelFuture.getChannel();
-
-      // In case we get a channel that never had its readability turned back on.
-      channel.setReadable(true);
+      channel = channelFuture.channel();
     }
     final String urlFile = StringUtils.nullToEmptyNonDruidDataString(url.getFile());
     final HttpRequest httpRequest = new DefaultHttpRequest(
@@ -144,13 +135,13 @@ public class NettyHttpClient extends AbstractHttpClient
         urlFile.isEmpty() ? "/" : urlFile
     );
 
-    if (!headers.containsKey(HttpHeaders.Names.HOST)) {
-      httpRequest.headers().add(HttpHeaders.Names.HOST, getHost(url));
+    if (!headers.containsKey(HttpHeaderNames.HOST.toString())) {
+      httpRequest.headers().add(HttpHeaderNames.HOST, getHost(url));
     }
 
     // If Accept-Encoding is set in the Request, use that. Otherwise use the default from "compressionCodec".
-    if (!headers.containsKey(HttpHeaders.Names.ACCEPT_ENCODING)) {
-      httpRequest.headers().set(HttpHeaders.Names.ACCEPT_ENCODING, compressionCodec.getEncodingString());
+    if (!headers.containsKey(HttpHeaderNames.ACCEPT_ENCODING.toString())) {
+      httpRequest.headers().set(HttpHeaderNames.ACCEPT_ENCODING, compressionCodec.getEncodingString());
     }
 
     for (Map.Entry<String, Collection<String>> entry : headers.asMap().entrySet()) {
@@ -161,23 +152,19 @@ public class NettyHttpClient extends AbstractHttpClient
       }
     }
 
-    if (request.hasContent()) {
-      httpRequest.setContent(request.getContent());
-    }
-
     final long readTimeout = getReadTimeout(requestReadTimeout);
     final SettableFuture<Final> retVal = SettableFuture.create();
 
     if (readTimeout > 0) {
-      channel.getPipeline().addLast(
+      channel.pipeline().addLast(
           READ_TIMEOUT_HANDLER_NAME,
-          new ReadTimeoutHandler(timer, readTimeout, TimeUnit.MILLISECONDS)
+          new ReadTimeoutHandler(readTimeout, TimeUnit.MILLISECONDS)
       );
     }
 
-    channel.getPipeline().addLast(
+    channel.pipeline().addLast(
         LAST_HANDLER_NAME,
-        new SimpleChannelUpstreamHandler()
+        new ChannelInboundHandlerAdapter()
         {
           private volatile ClientResponse<Intermediate> response = null;
 
@@ -192,18 +179,25 @@ public class NettyHttpClient extends AbstractHttpClient
           private long resumeWatermark = -1;
 
           @Override
-          public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+          public void channelRead(ChannelHandlerContext ctx, Object msg)
           {
             if (log.isDebugEnabled()) {
-              log.debug("[%s] messageReceived: %s", requestDesc, e.getMessage());
+              log.debug("[%s] messageReceived: %s", requestDesc, msg);
             }
             try {
-              Object msg = e.getMessage();
+              if (msg instanceof HttpObject) {
+                HttpObject httpObject = (HttpObject) msg;
+                final DecoderResult decoderResult = httpObject.decoderResult();
+                if (decoderResult.isFailure()) {
+                  sendError(decoderResult.cause());
+                  return;
+                }
+              }
 
               if (msg instanceof HttpResponse) {
                 HttpResponse httpResponse = (HttpResponse) msg;
                 if (log.isDebugEnabled()) {
-                  log.debug("[%s] Got response: %s", requestDesc, httpResponse.getStatus());
+                  log.debug("[%s] Got response: %s", requestDesc, httpResponse.status());
                 }
 
                 HttpResponseHandler.TrafficCop trafficCop = resumeChunkNum -> {
@@ -212,9 +206,9 @@ public class NettyHttpClient extends AbstractHttpClient
 
                     if (suspendWatermark >= 0 && resumeWatermark >= suspendWatermark) {
                       suspendWatermark = -1;
-                      channel.setReadable(true);
                       long backPressureDuration = System.nanoTime() - backPressureStartTimeNs;
                       log.debug("[%s] Resumed reads from channel (chunkNum = %,d).", requestDesc, resumeChunkNum);
+                      channel.read();
                       return backPressureDuration;
                     }
                   }
@@ -227,30 +221,27 @@ public class NettyHttpClient extends AbstractHttpClient
                 }
 
                 assert currentChunkNum == 0;
-                possiblySuspendReads(response);
-
-                if (!httpResponse.isChunked()) {
-                  finishRequest();
-                }
-              } else if (msg instanceof HttpChunk) {
-                HttpChunk httpChunk = (HttpChunk) msg;
+                possiblyRead(response);
+              } else if (msg instanceof HttpContent) {
+                HttpContent httpChunk = (HttpContent) msg;
                 if (log.isDebugEnabled()) {
                   log.debug(
                       "[%s] Got chunk: %sB, last=%s",
                       requestDesc,
-                      httpChunk.getContent().readableBytes(),
-                      httpChunk.isLast()
+                      httpChunk.content().readableBytes(),
+                      httpChunk instanceof LastHttpContent
                   );
                 }
 
-                if (httpChunk.isLast()) {
+                response = handler.handleChunk(response, httpChunk, ++currentChunkNum);
+                if (response.isFinished() && !retVal.isDone()) {
+                  retVal.set((Final) response.getObj());
+                }
+
+                if (httpChunk instanceof LastHttpContent) {
                   finishRequest();
                 } else {
-                  response = handler.handleChunk(response, httpChunk, ++currentChunkNum);
-                  if (response.isFinished() && !retVal.isDone()) {
-                    retVal.set((Final) response.getObj());
-                  }
-                  possiblySuspendReads(response);
+                  possiblyRead(response);
                 }
               } else {
                 throw new ISE("Unknown message type[%s]", msg.getClass());
@@ -267,17 +258,23 @@ public class NettyHttpClient extends AbstractHttpClient
 
               throw ex;
             }
+            finally {
+              ReferenceCountUtil.release(msg);
+            }
           }
 
-          private void possiblySuspendReads(ClientResponse<?> response)
+          private void possiblyRead(ClientResponse<?> response)
           {
-            if (!response.isContinueReading()) {
+            if (response.isContinueReading()) {
+              channel.read();
+            } else {
               synchronized (watermarkLock) {
                 suspendWatermark = Math.max(suspendWatermark, currentChunkNum);
                 if (suspendWatermark > resumeWatermark) {
-                  channel.setReadable(false);
                   backPressureStartTimeNs = System.nanoTime();
                   log.debug("[%s] Suspended reads from channel (chunkNum = %,d).", requestDesc, currentChunkNum);
+                } else {
+                  channel.read();
                 }
               }
             }
@@ -300,15 +297,13 @@ public class NettyHttpClient extends AbstractHttpClient
               retVal.set(finalResponse.getObj());
             }
             removeHandlers();
-            channel.setReadable(true);
             channelResourceContainer.returnResource();
           }
 
           @Override
-          public void exceptionCaught(ChannelHandlerContext context, ExceptionEvent event)
+          public void exceptionCaught(ChannelHandlerContext context, Throwable cause)
           {
             if (log.isDebugEnabled()) {
-              final Throwable cause = event.getCause();
               if (cause == null) {
                 log.debug("[%s] Caught exception", requestDesc);
               } else {
@@ -316,10 +311,15 @@ public class NettyHttpClient extends AbstractHttpClient
               }
             }
 
-            retVal.setException(event.getCause());
+            sendError(cause);
+          }
+
+          private void sendError(Throwable cause)
+          {
+            retVal.setException(cause);
             // response is non-null if we received initial chunk and then exception occurs
             if (response != null) {
-              handler.exceptionCaught(response, event.getCause());
+              handler.exceptionCaught(response, cause);
             }
             try {
               if (channel.isOpen()) {
@@ -335,7 +335,7 @@ public class NettyHttpClient extends AbstractHttpClient
           }
 
           @Override
-          public void channelDisconnected(ChannelHandlerContext context, ChannelStateEvent event)
+          public void channelInactive(ChannelHandlerContext context)
           {
             if (log.isDebugEnabled()) {
               log.debug("[%s] Channel disconnected", requestDesc);
@@ -355,30 +355,31 @@ public class NettyHttpClient extends AbstractHttpClient
           private void removeHandlers()
           {
             if (readTimeout > 0) {
-              channel.getPipeline().remove(READ_TIMEOUT_HANDLER_NAME);
+              channel.pipeline().remove(READ_TIMEOUT_HANDLER_NAME);
             }
-            channel.getPipeline().remove(LAST_HANDLER_NAME);
+            channel.pipeline().remove(LAST_HANDLER_NAME);
           }
         }
     );
 
-    channel.write(httpRequest).addListener(
-        new ChannelFutureListener()
-        {
-          @Override
-          public void operationComplete(ChannelFuture future)
-          {
-            if (!future.isSuccess()) {
-              channel.close();
-              channelResourceContainer.returnResource();
-              if (!retVal.isDone()) {
-                retVal.setException(
-                    new ChannelException(
-                        StringUtils.format("[%s] Failed to write request to channel", requestDesc),
-                        future.getCause()
-                    )
-                );
-              }
+    channel.write(httpRequest);
+    if (request.hasContent()) {
+      channel.write(new DefaultHttpContent(request.getContent()));
+    }
+    channel.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT).addListener(
+        (ChannelFutureListener) future -> {
+          if (future.isSuccess()) {
+            channel.read();
+          } else {
+            channel.close();
+            channelResourceContainer.returnResource();
+            if (!retVal.isDone()) {
+              retVal.setException(
+                  new ChannelException(
+                      StringUtils.format("[%s] Failed to write request to channel", requestDesc),
+                      future.cause()
+                  )
+              );
             }
           }
         }
@@ -398,12 +399,7 @@ public class NettyHttpClient extends AbstractHttpClient
       timeout = 0;
     }
 
-    if (timeout > 0 && timer == null) {
-      log.warn("Cannot time out requests without a timer! Disabling timeout for this request.");
-      return 0;
-    } else {
-      return timeout;
-    }
+    return timeout;
   }
 
   private String getHost(URL url)
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/Request.java b/core/src/main/java/org/apache/druid/java/util/http/client/Request.java
index 4794680..dff54ba 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/Request.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/Request.java
@@ -19,47 +19,33 @@
 
 package org.apache.druid.java.util.http.client;
 
-import com.google.common.base.Supplier;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBufferFactory;
-import org.jboss.netty.buffer.HeapChannelBufferFactory;
-import org.jboss.netty.handler.codec.base64.Base64;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import org.apache.druid.java.util.common.StringUtils;
 
 import java.net.URL;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 /**
  */
 public class Request
 {
-  private static final ChannelBufferFactory FACTORY = HeapChannelBufferFactory.getInstance();
-
   private final HttpMethod method;
   private final URL url;
   private final Multimap<String, String> headers = Multimaps.newListMultimap(
       new HashMap<>(),
-      new Supplier<List<String>>()
-      {
-        @Override
-        public List<String> get()
-        {
-          return new ArrayList<>();
-        }
-      }
+      ArrayList::new
   );
 
-  private ChannelBuffer content;
+  private ByteBuf content;
 
   public Request(
       HttpMethod method,
@@ -90,19 +76,38 @@ public class Request
     return content != null;
   }
 
-  public ChannelBuffer getContent()
+  public ByteBuf getContent()
   {
-    return content;
+    // return a duplicate buffer since with increased reference count
+    // this ensures Netty does not free the underlying array after it gets handled,
+    // since we sometimes read the buffer after it has been dispatched to Netty
+    // (e.g. when alling withUrl or copy, which migh happen after Netty has handled it already)
+    //
+    // Since we always create unbooled heap buffers they shouldn't impact existing pools and
+    // will get garbage collected with the request object itself.
+    return content.retainedDuplicate();
   }
 
   public Request copy()
   {
     Request retVal = new Request(method, url);
-    retVal.headers.putAll(this.headers);
-    retVal.content = content == null ? null : content.copy();
+    retVal.headers.putAll(headers);
+    if (hasContent()) {
+      retVal.content = content.retainedDuplicate();
+    }
     return retVal;
   }
 
+  public Request withUrl(URL url)
+  {
+    Request req = new Request(method, url);
+    req.headers.putAll(headers);
+    if (hasContent()) {
+      req.content = content.retainedDuplicate();
+    }
+    return req;
+  }
+
   public Request setHeader(String header, String value)
   {
     headers.replaceValues(header, Collections.singletonList(value));
@@ -134,11 +139,6 @@ public class Request
     return setContent(null, bytes);
   }
 
-  public Request setContent(ChannelBuffer content)
-  {
-    return setContent(null, content);
-  }
-
   public Request setContent(String contentType, byte[] bytes)
   {
     return setContent(contentType, bytes, 0, bytes.length);
@@ -146,39 +146,31 @@ public class Request
 
   public Request setContent(String contentType, byte[] bytes, int offset, int length)
   {
-    return setContent(contentType, FACTORY.getBuffer(bytes, offset, length));
+    // see getContent for why we create unpooled wrapped buffers
+    return setContent(contentType, Unpooled.wrappedBuffer(bytes, offset, length));
   }
 
-  public Request setContent(String contentType, ChannelBuffer content)
+  private Request setContent(String contentType, ByteBuf content)
   {
     if (contentType != null) {
-      setHeader(HttpHeaders.Names.CONTENT_TYPE, contentType);
+      setHeader(HttpHeaderNames.CONTENT_TYPE.toString(), contentType);
     }
 
     this.content = content;
 
-    setHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(content.writerIndex()));
+    setHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(content.writerIndex()));
 
     return this;
   }
 
   public Request setBasicAuthentication(String username, String password)
   {
-    setHeader(HttpHeaders.Names.AUTHORIZATION, makeBasicAuthenticationString(username, password));
+    setHeader(HttpHeaderNames.AUTHORIZATION.toString(), makeBasicAuthenticationString(username, password));
     return this;
   }
 
   public static String makeBasicAuthenticationString(String username, String password)
   {
-    return "Basic " + base64Encode(username + ":" + password);
-  }
-
-  private static String base64Encode(final String value)
-  {
-    final ChannelBufferFactory bufferFactory = HeapChannelBufferFactory.getInstance();
-
-    return Base64
-        .encode(bufferFactory.getBuffer(ByteBuffer.wrap(value.getBytes(StandardCharsets.UTF_8))), false)
-        .toString(StandardCharsets.UTF_8);
+    return "Basic " + StringUtils.utf8Base64(username + ":" + password);
   }
 }
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStream.java b/core/src/main/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStream.java
index 6789faf..2c1cd24 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStream.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStream.java
@@ -132,6 +132,10 @@ public class AppendableByteArrayInputStream extends InputStream
     while (numToScan > numScanned) {
       if (currIndex >= curr.length) {
         synchronized (singleByteReaderDoer) {
+          if (throwable != null) {
+            throw new IOException(throwable);
+          }
+
           if (bytes.isEmpty()) {
             if (done) {
               break;
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/netty/HttpClientPipelineFactory.java b/core/src/main/java/org/apache/druid/java/util/http/client/netty/HttpClientInitializer.java
similarity index 50%
rename from core/src/main/java/org/apache/druid/java/util/http/client/netty/HttpClientPipelineFactory.java
rename to core/src/main/java/org/apache/druid/java/util/http/client/netty/HttpClientInitializer.java
index 72d9ebd..7cc00e8 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/netty/HttpClientPipelineFactory.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/netty/HttpClientInitializer.java
@@ -19,24 +19,28 @@
 
 package org.apache.druid.java.util.http.client.netty;
 
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.DefaultChannelPipeline;
-import org.jboss.netty.handler.codec.http.HttpClientCodec;
-import org.jboss.netty.handler.codec.http.HttpContentDecompressor;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelInitializer;
+import io.netty.handler.codec.http.HttpClientCodec;
+import io.netty.handler.codec.http.HttpContentDecompressor;
+import io.netty.handler.codec.http.HttpObjectDecoder;
 
 /**
  */
-public class HttpClientPipelineFactory implements ChannelPipelineFactory
+public class HttpClientInitializer extends ChannelInitializer<Channel>
 {
   @Override
-  public ChannelPipeline getPipeline()
+  public void initChannel(Channel channel)
   {
-    ChannelPipeline pipeline = new DefaultChannelPipeline();
-
-    pipeline.addLast("codec", new HttpClientCodec());
-    pipeline.addLast("inflater", new HttpContentDecompressor());
-
-    return pipeline;
+    channel.pipeline()
+           .addLast("codec", new HttpClientCodec(
+               HttpObjectDecoder.DEFAULT_MAX_INITIAL_LINE_LENGTH,
+               HttpObjectDecoder.DEFAULT_MAX_HEADER_SIZE,
+               HttpObjectDecoder.DEFAULT_MAX_CHUNK_SIZE,
+               HttpClientCodec.DEFAULT_FAIL_ON_MISSING_RESPONSE,
+               HttpObjectDecoder.DEFAULT_VALIDATE_HEADERS,
+               true // continue parsing requests after HTTP CONNECT
+           ))
+           .addLast("inflater", new HttpContentDecompressor());
   }
 }
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/pool/ChannelResourceFactory.java b/core/src/main/java/org/apache/druid/java/util/http/client/pool/ChannelResourceFactory.java
index d6465be..69be9c6 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/pool/ChannelResourceFactory.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/pool/ChannelResourceFactory.java
@@ -20,29 +20,28 @@
 package org.apache.druid.java.util.http.client.pool;
 
 import com.google.common.base.Preconditions;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelException;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.ChannelPromise;
+import io.netty.handler.codec.http.DefaultHttpRequest;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
+import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.util.ReferenceCountUtil;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.HttpClientProxyConfig;
 import org.apache.druid.java.util.http.client.Request;
-import org.jboss.netty.bootstrap.ClientBootstrap;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.handler.codec.http.DefaultHttpRequest;
-import org.jboss.netty.handler.codec.http.HttpClientCodec;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.util.Timer;
 
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLEngine;
@@ -61,29 +60,22 @@ public class ChannelResourceFactory implements ResourceFactory<String, ChannelFu
   private static final long DEFAULT_SSL_HANDSHAKE_TIMEOUT_MILLIS = TimeUnit.SECONDS.toMillis(10);
   private static final String DRUID_PROXY_HANDLER = "druid_proxyHandler";
 
-  private final ClientBootstrap bootstrap;
+  private final Bootstrap bootstrap;
   private final SSLContext sslContext;
   private final HttpClientProxyConfig proxyConfig;
-  private final Timer timer;
   private final long sslHandshakeTimeout;
 
   public ChannelResourceFactory(
-      ClientBootstrap bootstrap,
+      Bootstrap bootstrap,
       SSLContext sslContext,
       HttpClientProxyConfig proxyConfig,
-      Timer timer,
       long sslHandshakeTimeout
   )
   {
     this.bootstrap = Preconditions.checkNotNull(bootstrap, "bootstrap");
     this.sslContext = sslContext;
     this.proxyConfig = proxyConfig;
-    this.timer = timer;
     this.sslHandshakeTimeout = sslHandshakeTimeout >= 0 ? sslHandshakeTimeout : DEFAULT_SSL_HANDSHAKE_TIMEOUT_MILLIS;
-
-    if (sslContext != null) {
-      Preconditions.checkNotNull(timer, "timer is required when sslContext is present");
-    }
   }
 
   @Override
@@ -107,88 +99,84 @@ public class ChannelResourceFactory implements ResourceFactory<String, ChannelFu
       final ChannelFuture proxyFuture = bootstrap.connect(
           new InetSocketAddress(proxyConfig.getHost(), proxyConfig.getPort())
       );
-      connectFuture = Channels.future(proxyFuture.getChannel());
+      ChannelPromise connectPromise = proxyFuture.channel().newPromise();
+      connectFuture = connectPromise;
 
-      final String proxyUri = StringUtils.format("%s:%d", host, port);
+      final String proxyUri = host + ":" + port;
       DefaultHttpRequest connectRequest = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.CONNECT, proxyUri);
 
       if (proxyConfig.getUser() != null) {
         connectRequest.headers().add(
-            "Proxy-Authorization", Request.makeBasicAuthenticationString(
+            HttpHeaderNames.PROXY_AUTHORIZATION.toString(),
+            Request.makeBasicAuthenticationString(
                 proxyConfig.getUser(), proxyConfig.getPassword()
             )
         );
       }
 
-      proxyFuture.addListener(new ChannelFutureListener()
-      {
-        @Override
-        public void operationComplete(ChannelFuture f1)
-        {
-          if (f1.isSuccess()) {
-            final Channel channel = f1.getChannel();
-            channel.getPipeline().addLast(
-                DRUID_PROXY_HANDLER,
-                new SimpleChannelUpstreamHandler()
+      proxyFuture.addListener((ChannelFutureListener) f1 -> {
+        if (f1.isSuccess()) {
+          final Channel channel = f1.channel();
+          channel.pipeline().addLast(
+              DRUID_PROXY_HANDLER,
+              new ChannelInboundHandlerAdapter()
+              {
+                @Override
+                public void channelRead(ChannelHandlerContext ctx, Object msg)
                 {
-                  @Override
-                  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
-                  {
-                    Object msg = e.getMessage();
+                  final ChannelPipeline pipeline = ctx.pipeline();
+                  pipeline.remove(DRUID_PROXY_HANDLER);
 
-                    final ChannelPipeline pipeline = ctx.getPipeline();
-                    pipeline.remove(DRUID_PROXY_HANDLER);
-
-                    if (msg instanceof HttpResponse) {
-                      HttpResponse httpResponse = (HttpResponse) msg;
-                      if (HttpResponseStatus.OK.equals(httpResponse.getStatus())) {
-                        // When the HttpClientCodec sees the CONNECT response complete, it goes into a "done"
-                        // mode which makes it just do nothing.  Swap it with a new instance that will cover
-                        // subsequent requests
-                        pipeline.replace("codec", "codec", new HttpClientCodec());
-                        connectFuture.setSuccess();
-                      } else {
-                        connectFuture.setFailure(
-                            new ChannelException(
-                                StringUtils.format(
-                                    "Got status[%s] from CONNECT request to proxy[%s]",
-                                    httpResponse.getStatus(),
-                                    proxyUri
-                                )
-                            )
-                        );
-                      }
+                  if (msg instanceof HttpResponse) {
+                    HttpResponse httpResponse = (HttpResponse) msg;
+                    if (HttpResponseStatus.OK.equals(httpResponse.status())) {
+                      connectPromise.setSuccess();
                     } else {
-                      connectFuture.setFailure(new ChannelException(StringUtils.format(
-                          "Got message of type[%s], don't know what to do.", msg.getClass()
-                      )));
-                    }
-                  }
-                }
-            );
-            channel.write(connectRequest).addListener(
-                new ChannelFutureListener()
-                {
-                  @Override
-                  public void operationComplete(ChannelFuture f2)
-                  {
-                    if (!f2.isSuccess()) {
-                      connectFuture.setFailure(
+                      connectPromise.setFailure(
                           new ChannelException(
-                              StringUtils.format("Problem with CONNECT request to proxy[%s]", proxyUri), f2.getCause()
+                              StringUtils.format(
+                                  "Got status[%s] from CONNECT request to proxy[%s]",
+                                  httpResponse.status(),
+                                  proxyUri
+                              )
                           )
                       );
                     }
+                  } else {
+                    connectPromise.setFailure(new ChannelException(StringUtils.format(
+                        "Got message of type[%s], don't know what to do.", msg.getClass()
+                    )));
                   }
+                  ReferenceCountUtil.release(msg);
                 }
-            );
-          } else {
-            connectFuture.setFailure(
-                new ChannelException(
-                    StringUtils.format("Problem connecting to proxy[%s]", proxyUri), f1.getCause()
-                )
-            );
-          }
+
+                @Override
+                public void channelReadComplete(ChannelHandlerContext ctx)
+                {
+                  ctx.channel().read();
+                }
+              }
+          );
+          channel.write(connectRequest);
+          channel.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT).addListener(
+              (ChannelFutureListener) f2 -> {
+                if (f2.isSuccess()) {
+                  channel.read();
+                } else {
+                  connectPromise.setFailure(
+                      new ChannelException(
+                          StringUtils.format("Problem with CONNECT request to proxy[%s]", proxyUri), f2.cause()
+                      )
+                  );
+                }
+              }
+          );
+        } else {
+          connectPromise.setFailure(
+              new ChannelException(
+                  StringUtils.format("Problem connecting to proxy[%s]", proxyUri), f1.cause()
+              )
+          );
         }
       });
     } else {
@@ -205,33 +193,25 @@ public class ChannelResourceFactory implements ResourceFactory<String, ChannelFu
       sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
       sslEngine.setSSLParameters(sslParameters);
       sslEngine.setUseClientMode(true);
-      final SslHandler sslHandler = new SslHandler(
-          sslEngine,
-          SslHandler.getDefaultBufferPool(),
-          false,
-          timer,
-          sslHandshakeTimeout
-      );
+      final SslHandler sslHandler = new SslHandler(sslEngine);
+      sslHandler.setHandshakeTimeoutMillis(sslHandshakeTimeout);
 
-      // https://github.com/netty/netty/issues/160
-      sslHandler.setCloseOnSSLException(true);
-
-      final ChannelFuture handshakeFuture = Channels.future(connectFuture.getChannel());
-      connectFuture.getChannel().getPipeline().addLast(
-          "connectionErrorHandler", new SimpleChannelUpstreamHandler()
+      final ChannelPromise handshakePromise = connectFuture.channel().newPromise();
+      connectFuture.channel().pipeline().addLast(
+          "connectionErrorHandler", new ChannelInboundHandlerAdapter()
           {
             @Override
-            public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+            public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
             {
-              final Channel channel = ctx.getChannel();
+              final Channel channel = ctx.channel();
               if (channel == null) {
                 // For the case where this pipeline is not attached yet.
-                handshakeFuture.setFailure(new ChannelException(
-                    StringUtils.format("Channel is null. The context name is [%s]", ctx.getName())
+                handshakePromise.setFailure(new ChannelException(
+                    StringUtils.format("Channel is null. The context name is [%s]", ctx.name())
                 ));
                 return;
               }
-              handshakeFuture.setFailure(e.getCause());
+              handshakePromise.setFailure(cause);
               if (channel.isOpen()) {
                 channel.close();
               }
@@ -239,46 +219,35 @@ public class ChannelResourceFactory implements ResourceFactory<String, ChannelFu
           }
       );
       connectFuture.addListener(
-          new ChannelFutureListener()
-          {
-            @Override
-            public void operationComplete(ChannelFuture f)
-            {
-              if (f.isSuccess()) {
-                final ChannelPipeline pipeline = f.getChannel().getPipeline();
-                pipeline.addFirst("ssl", sslHandler);
-                sslHandler.handshake().addListener(
-                    new ChannelFutureListener()
-                    {
-                      @Override
-                      public void operationComplete(ChannelFuture f2)
-                      {
-                        if (f2.isSuccess()) {
-                          handshakeFuture.setSuccess();
-                        } else {
-                          handshakeFuture.setFailure(
-                              new ChannelException(
-                                  StringUtils.format("Failed to handshake with host[%s]", hostname),
-                                  f2.getCause()
-                              )
-                          );
-                        }
-                      }
+          (ChannelFutureListener) f -> {
+            if (f.isSuccess()) {
+              final ChannelPipeline pipeline = f.channel().pipeline();
+              pipeline.addFirst("ssl", sslHandler);
+              sslHandler.handshakeFuture().addListener(f2 -> {
+                    if (f2.isSuccess()) {
+                      handshakePromise.setSuccess();
+                    } else {
+                      handshakePromise.setFailure(
+                          new ChannelException(
+                              StringUtils.format("Failed to handshake with host[%s]", hostname),
+                              f2.cause()
+                          )
+                      );
                     }
-                );
-              } else {
-                handshakeFuture.setFailure(
-                    new ChannelException(
-                        StringUtils.format("Failed to connect to host[%s]", hostname),
-                        f.getCause()
-                    )
-                );
-              }
+                  }
+              );
+            } else {
+              handshakePromise.setFailure(
+                  new ChannelException(
+                      StringUtils.format("Failed to connect to host[%s]", hostname),
+                      f.cause()
+                  )
+              );
             }
           }
       );
 
-      retVal = handshakeFuture;
+      retVal = handshakePromise;
     } else {
       retVal = connectFuture;
     }
@@ -289,10 +258,10 @@ public class ChannelResourceFactory implements ResourceFactory<String, ChannelFu
   @Override
   public boolean isGood(ChannelFuture resource)
   {
-    Channel channel = resource.awaitUninterruptibly().getChannel();
+    Channel channel = resource.awaitUninterruptibly().channel();
 
     boolean isSuccess = resource.isSuccess();
-    boolean isConnected = channel.isConnected();
+    boolean isConnected = channel.isActive();
     boolean isOpen = channel.isOpen();
 
     if (log.isTraceEnabled()) {
@@ -306,6 +275,6 @@ public class ChannelResourceFactory implements ResourceFactory<String, ChannelFu
   public void close(ChannelFuture resource)
   {
     log.trace("Closing");
-    resource.awaitUninterruptibly().getChannel().close();
+    resource.awaitUninterruptibly().channel().close();
   }
 }
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHandler.java
index 665f496..4a37f59 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHandler.java
@@ -19,9 +19,9 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
+import io.netty.buffer.ByteBuf;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 
 /**
  * {@link HttpResponseHandler} for stream data of byte array type.
@@ -34,8 +34,6 @@ public class BytesFullResponseHandler implements HttpResponseHandler<BytesFullRe
   {
     BytesFullResponseHolder holder = new BytesFullResponseHolder(response);
 
-    holder.addChunk(getContentBytes(response.getContent()));
-
     return ClientResponse.unfinished(
         holder
     );
@@ -44,7 +42,7 @@ public class BytesFullResponseHandler implements HttpResponseHandler<BytesFullRe
   @Override
   public ClientResponse<BytesFullResponseHolder> handleChunk(
       ClientResponse<BytesFullResponseHolder> response,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
@@ -54,7 +52,7 @@ public class BytesFullResponseHandler implements HttpResponseHandler<BytesFullRe
       return ClientResponse.finished(null);
     }
 
-    holder.addChunk(getContentBytes(chunk.getContent()));
+    holder.addChunk(getContentBytes(chunk.content()));
     return response;
   }
 
@@ -70,7 +68,7 @@ public class BytesFullResponseHandler implements HttpResponseHandler<BytesFullRe
     // Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
   }
 
-  private byte[] getContentBytes(ChannelBuffer content)
+  private byte[] getContentBytes(ByteBuf content)
   {
     byte[] contentBytes = new byte[content.readableBytes()];
     content.readBytes(contentBytes);
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHolder.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHolder.java
index b428031..9fe505e 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHolder.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/BytesFullResponseHolder.java
@@ -19,7 +19,7 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponse;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/FullResponseHolder.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/FullResponseHolder.java
index 27c2ed2..cd2a2c5 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/FullResponseHolder.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/FullResponseHolder.java
@@ -19,8 +19,8 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
 
 /**
  * This class is to hold data while receiving stream data via HTTP. Used with {@link HttpResponseHandler}.
@@ -38,7 +38,7 @@ public abstract class FullResponseHolder<T>
 
   public HttpResponseStatus getStatus()
   {
-    return response.getStatus();
+    return response.status();
   }
 
   public HttpResponse getResponse()
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/HttpResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/HttpResponseHandler.java
index 43bca7a..7cd1874 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/HttpResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/HttpResponseHandler.java
@@ -19,8 +19,8 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 
 /**
  * A handler for an HTTP request.
@@ -59,7 +59,7 @@ public interface HttpResponseHandler<IntermediateType, FinalType>
   ClientResponse<IntermediateType> handleResponse(HttpResponse response, TrafficCop trafficCop);
 
   /**
-   * Called for chunked responses, indicating another HttpChunk has arrived.
+   * Called for chunked responses, indicating another HttpContent has arrived.
    *
    * @param clientResponse last response returned by the prior handleResponse() or handleChunk()
    * @param chunk          the new chunk of data
@@ -69,7 +69,7 @@ public interface HttpResponseHandler<IntermediateType, FinalType>
    */
   ClientResponse<IntermediateType> handleChunk(
       ClientResponse<IntermediateType> clientResponse,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   );
 
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandler.java
index 71f1a85..93d0e26 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandler.java
@@ -19,9 +19,9 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
+import io.netty.buffer.ByteBuf;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 
 /**
  * This is a clone of {@link InputStreamResponseHandler} except that it retains HTTP status/response object in the
@@ -33,18 +33,17 @@ public class InputStreamFullResponseHandler implements HttpResponseHandler<Input
   public ClientResponse<InputStreamFullResponseHolder> handleResponse(HttpResponse response, TrafficCop trafficCop)
   {
     InputStreamFullResponseHolder holder = new InputStreamFullResponseHolder(response);
-    holder.addChunk(getContentBytes(response.getContent()));
     return ClientResponse.finished(holder);
   }
 
   @Override
   public ClientResponse<InputStreamFullResponseHolder> handleChunk(
       ClientResponse<InputStreamFullResponseHolder> clientResponse,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
-    clientResponse.getObj().addChunk(getContentBytes(chunk.getContent()));
+    clientResponse.getObj().addChunk(getContentBytes(chunk.content()));
     return clientResponse;
   }
 
@@ -65,7 +64,7 @@ public class InputStreamFullResponseHandler implements HttpResponseHandler<Input
     clientResponse.getObj().exceptionCaught(e);
   }
 
-  private byte[] getContentBytes(ChannelBuffer content)
+  private byte[] getContentBytes(ByteBuf content)
   {
     byte[] contentBytes = new byte[content.readableBytes()];
     content.readBytes(contentBytes);
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHolder.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHolder.java
index 2660908..9cfff6c 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHolder.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHolder.java
@@ -19,8 +19,8 @@
 
 package org.apache.druid.java.util.http.client.response;
 
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.java.util.http.client.io.AppendableByteArrayInputStream;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 
 import java.io.InputStream;
 
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamResponseHandler.java
index 66855d2..47bf267 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/InputStreamResponseHandler.java
@@ -19,10 +19,10 @@
 
 package org.apache.druid.java.util.http.client.response;
 
+import io.netty.buffer.ByteBuf;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.java.util.http.client.io.AppendableByteArrayInputStream;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 
 import java.io.InputStream;
 
@@ -34,18 +34,17 @@ public class InputStreamResponseHandler implements HttpResponseHandler<Appendabl
   public ClientResponse<AppendableByteArrayInputStream> handleResponse(HttpResponse response, TrafficCop trafficCop)
   {
     AppendableByteArrayInputStream in = new AppendableByteArrayInputStream();
-    in.add(getContentBytes(response.getContent()));
     return ClientResponse.finished(in);
   }
 
   @Override
   public ClientResponse<AppendableByteArrayInputStream> handleChunk(
       ClientResponse<AppendableByteArrayInputStream> clientResponse,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
-    clientResponse.getObj().add(getContentBytes(chunk.getContent()));
+    clientResponse.getObj().add(getContentBytes(chunk.content()));
     return clientResponse;
   }
 
@@ -67,7 +66,7 @@ public class InputStreamResponseHandler implements HttpResponseHandler<Appendabl
     obj.exceptionCaught(e);
   }
 
-  private byte[] getContentBytes(ChannelBuffer content)
+  private byte[] getContentBytes(ByteBuf content)
   {
     byte[] contentBytes = new byte[content.readableBytes()];
     content.readBytes(contentBytes);
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandler.java
index c884e7f..6fa1c82 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandler.java
@@ -19,9 +19,9 @@
 
 package org.apache.druid.java.util.http.client.response;
 
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.java.util.common.Either;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 
 import java.nio.charset.StandardCharsets;
 
@@ -47,7 +47,7 @@ public class ObjectOrErrorResponseHandler<IntermediateType, FinalType>
       final TrafficCop trafficCop
   )
   {
-    if (response.getStatus().getCode() / 100 == 2) {
+    if (response.status().code() / 100 == 2) {
       final ClientResponse<IntermediateType> delegateResponse = okHandler.handleResponse(response, trafficCop);
 
       return new ClientResponse<>(
@@ -70,7 +70,7 @@ public class ObjectOrErrorResponseHandler<IntermediateType, FinalType>
   @Override
   public ClientResponse<Either<StringFullResponseHolder, IntermediateType>> handleChunk(
       final ClientResponse<Either<StringFullResponseHolder, IntermediateType>> clientResponse,
-      final HttpChunk chunk,
+      final HttpContent chunk,
       final long chunkNum
   )
   {
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
index e1094ba..f816281 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
@@ -20,11 +20,12 @@
 package org.apache.druid.java.util.http.client.response;
 
 import com.google.common.io.ByteSource;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.java.util.common.logger.Logger;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBufferInputStream;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -56,18 +57,15 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
   @Override
   public ClientResponse<InputStream> handleResponse(HttpResponse response, TrafficCop trafficCop)
   {
-    try (ChannelBufferInputStream channelStream = new ChannelBufferInputStream(response.getContent())) {
-      queue.put(channelStream);
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
+    try {
+      // add empty initial buffer since SequenceInputStream will peek the first element right away
+      queue.put(new ByteBufInputStream(Unpooled.EMPTY_BUFFER));
     }
     catch (InterruptedException e) {
-      log.error(e, "Queue appending interrupted");
+      log.warn(e, "Thread interrupted while taking from queue");
       Thread.currentThread().interrupt();
       throw new RuntimeException(e);
     }
-    byteCount.addAndGet(response.getContent().readableBytes());
     return ClientResponse.finished(
         new SequenceInputStream(
             new Enumeration<InputStream>()
@@ -102,14 +100,14 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
   @Override
   public ClientResponse<InputStream> handleChunk(
       ClientResponse<InputStream> clientResponse,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
-    final ChannelBuffer channelBuffer = chunk.getContent();
-    final int bytes = channelBuffer.readableBytes();
+    final ByteBuf byteBuf = chunk.content();
+    final int bytes = byteBuf.readableBytes();
     if (bytes > 0) {
-      try (ChannelBufferInputStream channelStream = new ChannelBufferInputStream(channelBuffer)) {
+      try (ByteBufInputStream channelStream = new ByteBufInputStream(byteBuf)) {
         queue.put(channelStream);
         // Queue.size() can be expensive in some implementations, but LinkedBlockingQueue.size is just an AtomicLong
         log.debug("Added stream. Queue length %d", queue.size());
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java
index 0774352..9aa2179 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java
@@ -19,8 +19,8 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 
 import java.nio.charset.StandardCharsets;
 
@@ -43,8 +43,8 @@ public class StatusResponseHandler implements HttpResponseHandler<StatusResponse
   {
     return ClientResponse.unfinished(
         new StatusResponseHolder(
-            response.getStatus(),
-            new StringBuilder(response.getContent().toString(StandardCharsets.UTF_8))
+            response.status(),
+            new StringBuilder()
         )
     );
   }
@@ -52,17 +52,12 @@ public class StatusResponseHandler implements HttpResponseHandler<StatusResponse
   @Override
   public ClientResponse<StatusResponseHolder> handleChunk(
       ClientResponse<StatusResponseHolder> response,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
     final StringBuilder builder = response.getObj().getBuilder();
-
-    if (builder == null) {
-      return ClientResponse.finished(null);
-    }
-
-    builder.append(chunk.getContent().toString(StandardCharsets.UTF_8));
+    builder.append(chunk.content().toString(StandardCharsets.UTF_8));
     return response;
   }
 
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHolder.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHolder.java
index a8bf8a3..ce7db0f 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHolder.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHolder.java
@@ -19,7 +19,8 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
 
 /**
  */
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHandler.java
index 57af8e1..d0d4788 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHandler.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHandler.java
@@ -19,8 +19,8 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
 
 import java.nio.charset.Charset;
 
@@ -41,13 +41,13 @@ public class StringFullResponseHandler
   @Override
   public ClientResponse<StringFullResponseHolder> handleResponse(HttpResponse response, TrafficCop trafficCop)
   {
-    return ClientResponse.unfinished(new StringFullResponseHolder(response, charset));
+    return ClientResponse.unfinished(new StringFullResponseHolder(response));
   }
 
   @Override
   public ClientResponse<StringFullResponseHolder> handleChunk(
       ClientResponse<StringFullResponseHolder> response,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
@@ -57,7 +57,7 @@ public class StringFullResponseHandler
       return ClientResponse.finished(null);
     }
 
-    holder.addChunk(chunk.getContent().toString(charset));
+    holder.addChunk(chunk.content().toString(charset));
     return response;
   }
 
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHolder.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHolder.java
index 457c6e2..555963b 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHolder.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/StringFullResponseHolder.java
@@ -19,21 +19,18 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.handler.codec.http.HttpResponse;
-
-import java.nio.charset.Charset;
+import io.netty.handler.codec.http.HttpResponse;
 
 public class StringFullResponseHolder extends FullResponseHolder<String>
 {
   private final StringBuilder builder;
 
   public StringFullResponseHolder(
-      HttpResponse response,
-      Charset charset
+      HttpResponse response
   )
   {
     super(response);
-    this.builder = new StringBuilder(response.getContent().toString(charset));
+    this.builder = new StringBuilder();
   }
 
   public StringFullResponseHolder addChunk(String chunk)
diff --git a/core/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java b/core/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
index 60b770c..e8181c9 100644
--- a/core/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
@@ -19,7 +19,9 @@
 
 package org.apache.druid.java.util.http.client;
 
-import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.channel.ChannelException;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.lifecycle.Lifecycle;
 import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
@@ -32,9 +34,6 @@ import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.server.SslConnectionFactory;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -65,27 +64,22 @@ public class FriendlyServersTest
     final ExecutorService exec = Executors.newSingleThreadExecutor();
     final ServerSocket serverSocket = new ServerSocket(0);
     exec.submit(
-        new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            while (!Thread.currentThread().isInterrupted()) {
-              try (
-                  Socket clientSocket = serverSocket.accept();
-                  BufferedReader in = new BufferedReader(
-                      new InputStreamReader(clientSocket.getInputStream(), StandardCharsets.UTF_8)
-                  );
-                  OutputStream out = clientSocket.getOutputStream()
-              ) {
-                while (!in.readLine().equals("")) {
-                  // skip lines
-                }
-                out.write("HTTP/1.1 200 OK\r\nContent-Length: 6\r\n\r\nhello!".getBytes(StandardCharsets.UTF_8));
-              }
-              catch (Exception e) {
-                // Suppress
+        () -> {
+          while (!Thread.currentThread().isInterrupted()) {
+            try (
+                Socket clientSocket = serverSocket.accept();
+                BufferedReader in = new BufferedReader(
+                    new InputStreamReader(clientSocket.getInputStream(), StandardCharsets.UTF_8)
+                );
+                OutputStream out = clientSocket.getOutputStream()
+            ) {
+              while (!in.readLine().equals("")) {
+                // skip lines
               }
+              out.write("HTTP/1.1 200 OK\r\nContent-Length: 6\r\n\r\nhello!".getBytes(StandardCharsets.UTF_8));
+            }
+            catch (Exception e) {
+              // Suppress
             }
           }
         }
@@ -104,7 +98,7 @@ public class FriendlyServersTest
               StatusResponseHandler.getInstance()
           ).get();
 
-      Assert.assertEquals(200, response.getStatus().getCode());
+      Assert.assertEquals(200, response.getStatus().code());
       Assert.assertEquals("hello!", response.getContent());
     }
     finally {
@@ -122,35 +116,30 @@ public class FriendlyServersTest
     final ExecutorService exec = Executors.newSingleThreadExecutor();
     final ServerSocket serverSocket = new ServerSocket(0);
     exec.submit(
-        new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            while (!Thread.currentThread().isInterrupted()) {
-              try (
-                  Socket clientSocket = serverSocket.accept();
-                  BufferedReader in = new BufferedReader(
-                      new InputStreamReader(clientSocket.getInputStream(), StandardCharsets.UTF_8)
-                  );
-                  OutputStream out = clientSocket.getOutputStream()
-              ) {
-                StringBuilder request = new StringBuilder();
-                String line;
-                while (!"".equals((line = in.readLine()))) {
-                  request.append(line).append("\r\n");
-                }
-                requestContent.set(request.toString());
-                out.write("HTTP/1.1 200 OK\r\n\r\n".getBytes(StandardCharsets.UTF_8));
-
-                while (!in.readLine().equals("")) {
-                  // skip lines
-                }
-                out.write("HTTP/1.1 200 OK\r\nContent-Length: 6\r\n\r\nhello!".getBytes(StandardCharsets.UTF_8));
+        () -> {
+          while (!Thread.currentThread().isInterrupted()) {
+            try (
+                Socket clientSocket = serverSocket.accept();
+                BufferedReader in = new BufferedReader(
+                    new InputStreamReader(clientSocket.getInputStream(), StandardCharsets.UTF_8)
+                );
+                OutputStream out = clientSocket.getOutputStream()
+            ) {
+              StringBuilder request = new StringBuilder();
+              String line;
+              while (!"".equals((line = in.readLine()))) {
+                request.append(line).append("\r\n");
               }
-              catch (Exception e) {
-                Assert.fail(e.toString());
+              requestContent.set(request.toString());
+              out.write("HTTP/1.1 200 OK\r\n\r\n".getBytes(StandardCharsets.UTF_8));
+
+              while (!in.readLine().equals("")) {
+                // skip lines
               }
+              out.write("HTTP/1.1 200 OK\r\nContent-Length: 6\r\n\r\nhello!".getBytes(StandardCharsets.UTF_8));
+            }
+            catch (Exception e) {
+              Assert.fail(e.toString());
             }
           }
         }
@@ -174,11 +163,11 @@ public class FriendlyServersTest
               StatusResponseHandler.getInstance()
           ).get();
 
-      Assert.assertEquals(200, response.getStatus().getCode());
+      Assert.assertEquals(200, response.getStatus().code());
       Assert.assertEquals("hello!", response.getContent());
 
       Assert.assertEquals(
-          "CONNECT anotherHost:8080 HTTP/1.1\r\nProxy-Authorization: Basic Ym9iOnNhbGx5\r\n",
+          "CONNECT anotherHost:8080 HTTP/1.1\r\nproxy-authorization: Basic Ym9iOnNhbGx5\r\n",
           requestContent.get()
       );
     }
@@ -196,31 +185,26 @@ public class FriendlyServersTest
     final ServerSocket serverSocket = new ServerSocket(0);
     final AtomicBoolean foundAcceptEncoding = new AtomicBoolean();
     exec.submit(
-        new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            while (!Thread.currentThread().isInterrupted()) {
-              try (
-                  Socket clientSocket = serverSocket.accept();
-                  BufferedReader in = new BufferedReader(
-                      new InputStreamReader(clientSocket.getInputStream(), StandardCharsets.UTF_8)
-                  );
-                  OutputStream out = clientSocket.getOutputStream()
-              ) {
-                // Read headers
-                String header;
-                while (!(header = in.readLine()).equals("")) {
-                  if ("Accept-Encoding: identity".equals(header)) {
-                    foundAcceptEncoding.set(true);
-                  }
+        () -> {
+          while (!Thread.currentThread().isInterrupted()) {
+            try (
+                Socket clientSocket = serverSocket.accept();
+                BufferedReader in = new BufferedReader(
+                    new InputStreamReader(clientSocket.getInputStream(), StandardCharsets.UTF_8)
+                );
+                OutputStream out = clientSocket.getOutputStream()
+            ) {
+              // Read headers
+              String header;
+              while (!(header = in.readLine()).equals("")) {
+                if ("accept-encoding: identity".equals(header)) {
+                  foundAcceptEncoding.set(true);
                 }
-                out.write("HTTP/1.1 200 OK\r\nContent-Length: 6\r\n\r\nhello!".getBytes(StandardCharsets.UTF_8));
-              }
-              catch (Exception e) {
-                // Suppress
               }
+              out.write("HTTP/1.1 200 OK\r\nContent-Length: 6\r\n\r\nhello!".getBytes(StandardCharsets.UTF_8));
+            }
+            catch (Exception ignored) {
+              // Suppress
             }
           }
         }
@@ -241,7 +225,7 @@ public class FriendlyServersTest
               StatusResponseHandler.getInstance()
           ).get();
 
-      Assert.assertEquals(200, response.getStatus().getCode());
+      Assert.assertEquals(200, response.getStatus().code());
       Assert.assertEquals("hello!", response.getContent());
       Assert.assertTrue(foundAcceptEncoding.get());
     }
@@ -297,56 +281,42 @@ public class FriendlyServersTest
                 ),
                 StatusResponseHandler.getInstance()
             ).get().getStatus();
-        Assert.assertEquals(404, status.getCode());
+        Assert.assertEquals(404, status.code());
       }
 
       // Incorrect name ("127.0.0.1")
-      {
-        final ListenableFuture<StatusResponseHolder> response1 = trustingClient
+      final Throwable cause = Assert.assertThrows(ExecutionException.class, () -> {
+        trustingClient
             .go(
                 new Request(
                     HttpMethod.GET,
                     new URL(StringUtils.format("https://127.0.0.1:%d/", sslConnector.getLocalPort()))
                 ),
                 StatusResponseHandler.getInstance()
-            );
+            ).get();
 
-        Throwable ea = null;
-        try {
-          response1.get();
-        }
-        catch (ExecutionException e) {
-          ea = e.getCause();
-        }
 
-        Assert.assertTrue("ChannelException thrown by 'get'", ea instanceof ChannelException);
-        Assert.assertTrue("Expected error message", ea.getCause().getMessage().contains("Failed to handshake"));
-      }
+      }).getCause();
+      Assert.assertTrue("ChannelException thrown by 'get'", cause instanceof ChannelException);
+      Assert.assertTrue("Expected error message", cause.getCause().getMessage().contains("Failed to handshake"));
 
-      {
+      final Throwable untrustedCause = Assert.assertThrows(ExecutionException.class, () -> {
         // Untrusting client
-        final ListenableFuture<StatusResponseHolder> response2 = skepticalClient
+        skepticalClient
             .go(
                 new Request(
                     HttpMethod.GET,
                     new URL(StringUtils.format("https://localhost:%d/", sslConnector.getLocalPort()))
                 ),
                 StatusResponseHandler.getInstance()
-            );
-
-        Throwable eb = null;
-        try {
-          response2.get();
-        }
-        catch (ExecutionException e) {
-          eb = e.getCause();
-        }
-        Assert.assertNotNull("ChannelException thrown by 'get'", eb);
-        Assert.assertTrue(
-            "Root cause is SSLHandshakeException",
-            eb.getCause().getCause() instanceof SSLHandshakeException
-        );
-      }
+            )
+            .get();
+      }).getCause();
+      Assert.assertTrue("ChannelException thrown by 'get'", untrustedCause instanceof ChannelException);
+      Assert.assertTrue(
+          "Root cause is SSLHandshakeException",
+          untrustedCause.getCause().getCause() instanceof SSLHandshakeException
+      );
     }
     finally {
       lifecycle.stop();
@@ -370,7 +340,7 @@ public class FriendlyServersTest
                 StatusResponseHandler.getInstance()
             ).get().getStatus();
 
-        Assert.assertEquals(200, status.getCode());
+        Assert.assertEquals(200, status.code());
       }
 
       {
@@ -381,7 +351,7 @@ public class FriendlyServersTest
                 StatusResponseHandler.getInstance()
             ).get().getStatus();
 
-        Assert.assertEquals(200, status.getCode());
+        Assert.assertEquals(200, status.code());
       }
     }
     finally {
diff --git a/core/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java b/core/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java
index 3d12cf7..dd96873 100644
--- a/core/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java
@@ -20,13 +20,13 @@
 package org.apache.druid.java.util.http.client;
 
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.channel.ChannelException;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.timeout.ReadTimeoutException;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.lifecycle.Lifecycle;
 import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.timeout.ReadTimeoutException;
 import org.joda.time.Duration;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -70,71 +70,56 @@ public class JankyServersTest
     closingServerSocket = new ServerSocket(0);
 
     exec.submit(
-        new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            while (!Thread.currentThread().isInterrupted()) {
-              try (
-                  Socket clientSocket = silentServerSocket.accept();
-                  InputStream in = clientSocket.getInputStream()
-              ) {
-                while (in.read() != -1) {
-                  /* Do nothing. Read bytes till the end of the stream. */
-                }
-              }
-              catch (Exception e) {
-                // Suppress
+        () -> {
+          while (!Thread.currentThread().isInterrupted()) {
+            try (
+                Socket clientSocket = silentServerSocket.accept();
+                InputStream in = clientSocket.getInputStream()
+            ) {
+              while (in.read() != -1) {
+                /* Do nothing. Read bytes till the end of the stream. */
               }
             }
+            catch (Exception e) {
+              // Suppress
+            }
           }
         }
     );
 
     exec.submit(
-        new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            while (!Thread.currentThread().isInterrupted()) {
-              try (
-                  Socket clientSocket = closingServerSocket.accept();
-                  InputStream in = clientSocket.getInputStream()
-              ) {
-                in.read();
-                clientSocket.close();
-              }
-              catch (Exception e) {
-                // Suppress
-              }
+        () -> {
+          while (!Thread.currentThread().isInterrupted()) {
+            try (
+                Socket clientSocket = closingServerSocket.accept();
+                InputStream in = clientSocket.getInputStream()
+            ) {
+              in.read();
+              clientSocket.close();
+            }
+            catch (Exception e) {
+              // Suppress
             }
           }
         }
     );
 
     exec.submit(
-        new Runnable()
-        {
-          @Override
-          public void run()
-          {
-            while (!Thread.currentThread().isInterrupted()) {
-              try (
-                  Socket clientSocket = echoServerSocket.accept();
-                  OutputStream out = clientSocket.getOutputStream();
-                  InputStream in = clientSocket.getInputStream()
-              ) {
-                int b;
-                while ((b = in.read()) != -1) {
-                  out.write(b);
-                }
-              }
-              catch (Exception e) {
-                // Suppress
+        () -> {
+          while (!Thread.currentThread().isInterrupted()) {
+            try (
+                Socket clientSocket = echoServerSocket.accept();
+                OutputStream out = clientSocket.getOutputStream();
+                InputStream in = clientSocket.getInputStream()
+            ) {
+              int b;
+              while ((b = in.read()) != -1) {
+                out.write(b);
               }
             }
+            catch (Exception e) {
+              // Suppress
+            }
           }
         }
     );
@@ -162,15 +147,8 @@ public class JankyServersTest
               StatusResponseHandler.getInstance()
           );
 
-      Throwable e = null;
-      try {
-        future.get();
-      }
-      catch (ExecutionException e1) {
-        e = e1.getCause();
-      }
-
-      Assert.assertTrue("ReadTimeoutException thrown by 'get'", e instanceof ReadTimeoutException);
+      final ExecutionException e = Assert.assertThrows(ExecutionException.class, future::get);
+      Assert.assertTrue("ReadTimeoutException thrown by 'get'", e.getCause() instanceof ReadTimeoutException);
     }
     finally {
       lifecycle.stop();
@@ -191,15 +169,8 @@ public class JankyServersTest
               new Duration(100L)
           );
 
-      Throwable e = null;
-      try {
-        future.get();
-      }
-      catch (ExecutionException e1) {
-        e = e1.getCause();
-      }
-
-      Assert.assertTrue("ReadTimeoutException thrown by 'get'", e instanceof ReadTimeoutException);
+      final ExecutionException e = Assert.assertThrows(ExecutionException.class, future::get);
+      Assert.assertTrue("ReadTimeoutException thrown by 'get'", e.getCause() instanceof ReadTimeoutException);
     }
     finally {
       lifecycle.stop();
@@ -223,15 +194,8 @@ public class JankyServersTest
               StatusResponseHandler.getInstance()
           );
 
-      Throwable e = null;
-      try {
-        response.get();
-      }
-      catch (ExecutionException e1) {
-        e = e1.getCause();
-      }
-
-      Assert.assertTrue("ChannelException thrown by 'get'", e instanceof ChannelException);
+      final ExecutionException e = Assert.assertThrows(ExecutionException.class, response::get);
+      Assert.assertTrue("ChannelException thrown by 'get'", e.getCause() instanceof ChannelException);
     }
     finally {
       lifecycle.stop();
@@ -250,16 +214,9 @@ public class JankyServersTest
               new Request(HttpMethod.GET, new URL(StringUtils.format("http://localhost:%d/", closingServerSocket.getLocalPort()))),
               StatusResponseHandler.getInstance()
           );
-      Throwable e = null;
-      try {
-        response.get();
-      }
-      catch (ExecutionException e1) {
-        e = e1.getCause();
-        e1.printStackTrace();
-      }
 
-      Assert.assertTrue("ChannelException thrown by 'get'", isChannelClosedException(e));
+      final ExecutionException e = Assert.assertThrows(ExecutionException.class, response::get);
+      Assert.assertTrue("ChannelException thrown by 'get'", isChannelClosedException(e.getCause()));
     }
     finally {
       lifecycle.stop();
@@ -280,16 +237,8 @@ public class JankyServersTest
               StatusResponseHandler.getInstance()
           );
 
-      Throwable e = null;
-      try {
-        response.get();
-      }
-      catch (ExecutionException e1) {
-        e = e1.getCause();
-        e1.printStackTrace();
-      }
-
-      Assert.assertTrue("ChannelException thrown by 'get'", isChannelClosedException(e));
+      final ExecutionException e = Assert.assertThrows(ExecutionException.class, response::get);
+      Assert.assertTrue("ChannelException thrown by 'get'", isChannelClosedException(e.getCause()));
     }
     finally {
       lifecycle.stop();
@@ -304,49 +253,52 @@ public class JankyServersTest
   }
 
   @Test
-  public void testHttpEchoServer() throws Throwable
+  public void testHttpEchoServer()
   {
-    final Lifecycle lifecycle = new Lifecycle();
-    try {
-      final HttpClientConfig config = HttpClientConfig.builder().build();
-      final HttpClient client = HttpClientInit.createClient(config, lifecycle);
-      final ListenableFuture<StatusResponseHolder> response = client
-          .go(
-              new Request(HttpMethod.GET, new URL(StringUtils.format("http://localhost:%d/", echoServerSocket.getLocalPort()))),
-              StatusResponseHandler.getInstance()
-          );
-
-      expectedException.expect(ExecutionException.class);
-      expectedException.expectMessage("java.lang.IllegalArgumentException: invalid version format: GET");
+    ExecutionException exception = Assert.assertThrows(ExecutionException.class, () -> {
+      final Lifecycle lifecycle = new Lifecycle();
+      try {
+        final HttpClientConfig config = HttpClientConfig.builder().build();
+        HttpClientInit.createClient(config, lifecycle)
+                      .go(
+                          new Request(
+                              HttpMethod.GET,
+                              new URL(StringUtils.format("http://localhost:%d/", echoServerSocket.getLocalPort()))
+                          ),
+                          StatusResponseHandler.getInstance()
+                      )
+                      .get();
+      }
+      finally {
+        lifecycle.stop();
+      }
+    });
 
-      response.get();
-    }
-    finally {
-      lifecycle.stop();
-    }
+    Assert.assertEquals("java.lang.IllegalArgumentException: invalid version format: GET", exception.getMessage());
   }
 
   @Test
-  public void testHttpsEchoServer() throws Throwable
+  public void testHttpsEchoServer()
   {
-    final Lifecycle lifecycle = new Lifecycle();
-    try {
-      final HttpClientConfig config = HttpClientConfig.builder().withSslContext(SSLContext.getDefault()).build();
-      final HttpClient client = HttpClientInit.createClient(config, lifecycle);
-
-      final ListenableFuture<StatusResponseHolder> response = client
-          .go(
-              new Request(HttpMethod.GET, new URL(StringUtils.format("https://localhost:%d/", echoServerSocket.getLocalPort()))),
-              StatusResponseHandler.getInstance()
-          );
-
-      expectedException.expect(ExecutionException.class);
-      expectedException.expectMessage("org.jboss.netty.channel.ChannelException: Faulty channel in resource pool");
+    final ExecutionException exception = Assert.assertThrows(ExecutionException.class, () -> {
+      final Lifecycle lifecycle = new Lifecycle();
+      try {
+        final HttpClientConfig config = HttpClientConfig.builder().withSslContext(SSLContext.getDefault()).build();
+        HttpClientInit.createClient(config, lifecycle)
+                      .go(
+                          new Request(
+                              HttpMethod.GET,
+                              new URL(StringUtils.format("https://localhost:%d/", echoServerSocket.getLocalPort()))
+                          ),
+                          StatusResponseHandler.getInstance()
+                      )
+                      .get();
+      }
+      finally {
+        lifecycle.stop();
+      }
+    });
 
-      response.get();
-    }
-    finally {
-      lifecycle.stop();
-    }
+    Assert.assertEquals("io.netty.channel.ChannelException: Faulty channel in resource pool", exception.getMessage());
   }
 }
diff --git a/core/src/test/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStreamTest.java b/core/src/test/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStreamTest.java
index 780cd12..ab5d86c 100644
--- a/core/src/test/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStreamTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/http/client/io/AppendableByteArrayInputStreamTest.java
@@ -212,6 +212,17 @@ public class AppendableByteArrayInputStreamTest
   }
 
   @Test
+  public void testExceptionBeforeFirstAdd()
+  {
+    AppendableByteArrayInputStream in = new AppendableByteArrayInputStream();
+
+    in.exceptionCaught(new RuntimeException("exception before add"));
+
+    IOException e = Assert.assertThrows(IOException.class, in::read);
+    Assert.assertEquals("exception before add", e.getCause().getMessage());
+  }
+
+  @Test
   public void testExceptionUnblocks() throws InterruptedException
   {
     final AppendableByteArrayInputStream in = new AppendableByteArrayInputStream();
diff --git a/core/src/test/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandlerTest.java b/core/src/test/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandlerTest.java
index 9c40047..cb41553 100644
--- a/core/src/test/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandlerTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/http/client/response/InputStreamFullResponseHandlerTest.java
@@ -19,14 +19,15 @@
 
 package org.apache.druid.java.util.http.client.response;
 
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.DefaultHttpContent;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.apache.commons.io.IOUtils;
 import org.apache.druid.java.util.common.StringUtils;
-import org.jboss.netty.buffer.BigEndianHeapChannelBuffer;
-import org.jboss.netty.handler.codec.http.DefaultHttpChunk;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -38,14 +39,15 @@ public class InputStreamFullResponseHandlerTest
   public void testSimple() throws Exception
   {
     HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(false);
-    response.setContent(new BigEndianHeapChannelBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
 
     InputStreamFullResponseHandler responseHandler = new InputStreamFullResponseHandler();
     ClientResponse<InputStreamFullResponseHolder> clientResp = responseHandler.handleResponse(response, null);
 
-    DefaultHttpChunk chunk = new DefaultHttpChunk(new BigEndianHeapChannelBuffer("efg".getBytes(StringUtils.UTF8_STRING)));
-    clientResp = responseHandler.handleChunk(clientResp, chunk, 0);
+    HttpContent chunk0 = new DefaultHttpContent(Unpooled.wrappedBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
+    clientResp = responseHandler.handleChunk(clientResp, chunk0, 0);
+
+    HttpContent chunk1 = new DefaultHttpContent(Unpooled.wrappedBuffer("efg".getBytes(StringUtils.UTF8_STRING)));
+    clientResp = responseHandler.handleChunk(clientResp, chunk1, 1);
 
     clientResp = responseHandler.done(clientResp);
 
@@ -57,8 +59,6 @@ public class InputStreamFullResponseHandlerTest
   public void testException() throws Exception
   {
     HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(false);
-    response.setContent(new BigEndianHeapChannelBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
 
     InputStreamFullResponseHandler responseHandler = new InputStreamFullResponseHandler();
     ClientResponse<InputStreamFullResponseHolder> clientResp = responseHandler.handleResponse(response, null);
diff --git a/core/src/test/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandlerTest.java b/core/src/test/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandlerTest.java
index 4e2247c..6000340 100644
--- a/core/src/test/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandlerTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/http/client/response/ObjectOrErrorResponseHandlerTest.java
@@ -19,15 +19,16 @@
 
 package org.apache.druid.java.util.http.client.response;
 
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.DefaultHttpContent;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.apache.commons.io.IOUtils;
 import org.apache.druid.java.util.common.Either;
 import org.apache.druid.java.util.common.StringUtils;
-import org.jboss.netty.buffer.BigEndianHeapChannelBuffer;
-import org.jboss.netty.handler.codec.http.DefaultHttpChunk;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -41,18 +42,16 @@ public class ObjectOrErrorResponseHandlerTest
   public void testOk() throws Exception
   {
     HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(false);
-    response.setContent(new BigEndianHeapChannelBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
-
     final ObjectOrErrorResponseHandler<InputStreamFullResponseHolder, InputStreamFullResponseHolder> responseHandler =
         new ObjectOrErrorResponseHandler<>(new InputStreamFullResponseHandler());
 
     ClientResponse<Either<StringFullResponseHolder, InputStreamFullResponseHolder>> clientResp =
         responseHandler.handleResponse(response, null);
 
-    DefaultHttpChunk chunk =
-        new DefaultHttpChunk(new BigEndianHeapChannelBuffer("efg".getBytes(StringUtils.UTF8_STRING)));
-    clientResp = responseHandler.handleChunk(clientResp, chunk, 0);
+    HttpContent chunk0 = new DefaultHttpContent(Unpooled.wrappedBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
+    HttpContent chunk1 = new DefaultHttpContent(Unpooled.wrappedBuffer("efg".getBytes(StringUtils.UTF8_STRING)));
+    clientResp = responseHandler.handleChunk(clientResp, chunk0, 0);
+    clientResp = responseHandler.handleChunk(clientResp, chunk1, 1);
     clientResp = responseHandler.done(clientResp);
 
     Assert.assertTrue(clientResp.isFinished());
@@ -63,12 +62,9 @@ public class ObjectOrErrorResponseHandlerTest
   }
 
   @Test
-  public void testExceptionAfterOk() throws Exception
+  public void testExceptionAfterOk()
   {
     HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(false);
-    response.setContent(new BigEndianHeapChannelBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
-
     final ObjectOrErrorResponseHandler<InputStreamFullResponseHolder, InputStreamFullResponseHolder> responseHandler =
         new ObjectOrErrorResponseHandler<>(new InputStreamFullResponseHandler());
 
@@ -95,8 +91,6 @@ public class ObjectOrErrorResponseHandlerTest
   public void testServerError() throws Exception
   {
     HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR);
-    response.setChunked(false);
-    response.setContent(new BigEndianHeapChannelBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
 
     final ObjectOrErrorResponseHandler<InputStreamFullResponseHolder, InputStreamFullResponseHolder> responseHandler =
         new ObjectOrErrorResponseHandler<>(new InputStreamFullResponseHandler());
@@ -104,17 +98,19 @@ public class ObjectOrErrorResponseHandlerTest
     ClientResponse<Either<StringFullResponseHolder, InputStreamFullResponseHolder>> clientResp =
         responseHandler.handleResponse(response, null);
 
-    DefaultHttpChunk chunk =
-        new DefaultHttpChunk(new BigEndianHeapChannelBuffer("efg".getBytes(StringUtils.UTF8_STRING)));
-    clientResp = responseHandler.handleChunk(clientResp, chunk, 0);
+    HttpContent chunk0 = new DefaultHttpContent(Unpooled.wrappedBuffer("abcd".getBytes(StringUtils.UTF8_STRING)));
+    HttpContent chunk1 = new DefaultHttpContent(Unpooled.wrappedBuffer("efg".getBytes(StringUtils.UTF8_STRING)));
+
+    clientResp = responseHandler.handleChunk(clientResp, chunk0, 0);
+    clientResp = responseHandler.handleChunk(clientResp, chunk1, 1);
     clientResp = responseHandler.done(clientResp);
 
     // 5xx HTTP code is handled by the error handler.
     Assert.assertTrue(clientResp.isFinished());
     Assert.assertTrue(clientResp.getObj().isError());
     Assert.assertEquals(
-        HttpResponseStatus.INTERNAL_SERVER_ERROR.getCode(),
-        clientResp.getObj().error().getResponse().getStatus().getCode()
+        HttpResponseStatus.INTERNAL_SERVER_ERROR.code(),
+        clientResp.getObj().error().getResponse().status().code()
     );
     Assert.assertEquals("abcdefg", clientResp.getObj().error().getContent());
   }
diff --git a/core/src/test/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandlerTest.java b/core/src/test/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandlerTest.java
index 2552a2f..556ef83 100644
--- a/core/src/test/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandlerTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandlerTest.java
@@ -19,12 +19,15 @@
 
 package org.apache.druid.java.util.http.client.response;
 
-import org.jboss.netty.buffer.BigEndianHeapChannelBuffer;
-import org.jboss.netty.handler.codec.http.DefaultHttpChunk;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.Unpooled;
+import io.netty.buffer.UnpooledHeapByteBuf;
+import io.netty.handler.codec.http.DefaultHttpContent;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -83,25 +86,23 @@ public class SequenceInputStreamResponseHandlerTest
 
     SequenceInputStreamResponseHandler responseHandler = new SequenceInputStreamResponseHandler();
     final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(true);
+
     ClientResponse<InputStream> clientResponse = responseHandler.handleResponse(response, null);
     final int failAt = RANDOM.nextInt(allBytes.length);
     long chunkNum = 0;
     while (it.hasNext()) {
-      final DefaultHttpChunk chunk = new DefaultHttpChunk(
-          new BigEndianHeapChannelBuffer(it.next())
-          {
-            @Override
-            public void getBytes(int index, byte[] dst, int dstIndex, int length)
-            {
-              if (dstIndex + length >= failAt) {
-                throw new TesterException();
-              }
-              super.getBytes(index, dst, dstIndex, length);
-            }
+      final byte[] bytes = it.next();
+      ByteBuf data = new UnpooledHeapByteBuf(ByteBufAllocator.DEFAULT, bytes, bytes.length) {
+        @Override
+        public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length)
+        {
+          if (dstIndex + length >= failAt) {
+            throw new TesterException();
           }
-      );
-      clientResponse = responseHandler.handleChunk(clientResponse, chunk, ++chunkNum);
+          return super.getBytes(index, dst, dstIndex, length);
+        }
+      };
+      clientResponse = responseHandler.handleChunk(clientResponse, new DefaultHttpContent(data), ++chunkNum);
     }
     clientResponse = responseHandler.done(clientResponse);
 
@@ -119,21 +120,20 @@ public class SequenceInputStreamResponseHandlerTest
   {
     SequenceInputStreamResponseHandler responseHandler = new SequenceInputStreamResponseHandler();
     final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(false);
-    response.setContent(
-        new BigEndianHeapChannelBuffer(allBytes)
+
+    ClientResponse<InputStream> clientResponse = responseHandler.handleResponse(response, null);
+    clientResponse = responseHandler.handleChunk(clientResponse, new DefaultHttpContent(
+        new UnpooledHeapByteBuf(ByteBufAllocator.DEFAULT, allBytes, allBytes.length)
         {
           @Override
-          public void getBytes(int index, byte[] dst, int dstIndex, int length)
+          public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length)
           {
             if (dstIndex + length >= allBytes.length) {
               throw new TesterException();
             }
-            super.getBytes(index, dst, dstIndex, length);
+            return super.getBytes(index, dst, dstIndex, length);
           }
-        }
-    );
-    ClientResponse<InputStream> clientResponse = responseHandler.handleResponse(response, null);
+        }), 0);
     clientResponse = responseHandler.done(clientResponse);
 
     final InputStream stream = clientResponse.getObj();
@@ -148,11 +148,11 @@ public class SequenceInputStreamResponseHandlerTest
 
     SequenceInputStreamResponseHandler responseHandler = new SequenceInputStreamResponseHandler();
     final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(true);
+
     ClientResponse<InputStream> clientResponse = responseHandler.handleResponse(response, null);
     long chunkNum = 0;
     while (it.hasNext()) {
-      final DefaultHttpChunk chunk = new DefaultHttpChunk(new BigEndianHeapChannelBuffer(it.next()));
+      final DefaultHttpContent chunk = new DefaultHttpContent(Unpooled.wrappedBuffer(it.next()));
       clientResponse = responseHandler.handleChunk(clientResponse, chunk, ++chunkNum);
     }
     clientResponse = responseHandler.done(clientResponse);
@@ -179,11 +179,10 @@ public class SequenceInputStreamResponseHandlerTest
 
     SequenceInputStreamResponseHandler responseHandler = new SequenceInputStreamResponseHandler();
     final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(true);
     ClientResponse<InputStream> clientResponse = responseHandler.handleResponse(response, null);
     long chunkNum = 0;
     while (it.hasNext()) {
-      final DefaultHttpChunk chunk = new DefaultHttpChunk(new BigEndianHeapChannelBuffer(it.next()));
+      final DefaultHttpContent chunk = new DefaultHttpContent(Unpooled.wrappedBuffer(it.next()));
       clientResponse = responseHandler.handleChunk(clientResponse, chunk, ++chunkNum);
     }
     clientResponse = responseHandler.done(clientResponse);
@@ -207,9 +206,10 @@ public class SequenceInputStreamResponseHandlerTest
   {
     SequenceInputStreamResponseHandler responseHandler = new SequenceInputStreamResponseHandler();
     final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-    response.setChunked(false);
-    response.setContent(new BigEndianHeapChannelBuffer(allBytes));
+
     ClientResponse<InputStream> clientResponse = responseHandler.handleResponse(response, null);
+    clientResponse = responseHandler.handleChunk(clientResponse,
+                                                 new DefaultHttpContent(Unpooled.wrappedBuffer(allBytes)), 0);
     clientResponse = responseHandler.done(clientResponse);
 
     final InputStream stream = clientResponse.getObj();
diff --git a/docs/development/modules.md b/docs/development/modules.md
index 50e2a97..f889ece 100644
--- a/docs/development/modules.md
+++ b/docs/development/modules.md
@@ -1,3 +1,4 @@
+
 ---
 id: modules
 title: "Creating extensions"
@@ -390,7 +391,7 @@ Managing library collisions can be daunting for extensions which draw in commonl
 "com.metamx.druid",
 "asm",
 "org.ow2.asm",
-"org.jboss.netty",
+"io.netty",
 "com.google.guava",
 "com.google.code.findbugs",
 "com.google.protobuf",
diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java
index 358ac10..5d6a4c2 100644
--- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java
+++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java
@@ -21,6 +21,9 @@ package org.apache.druid.security.basic;
 
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.discovery.DiscoveryDruidNode;
 import org.apache.druid.discovery.DruidNodeDiscovery;
 import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
@@ -36,9 +39,6 @@ import org.apache.druid.java.util.http.client.response.ClientResponse;
 import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.server.DruidNode;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 import org.joda.time.Duration;
 
 import javax.ws.rs.core.MediaType;
@@ -219,7 +219,7 @@ public class CommonCacheNotifier
     {
       return ClientResponse.unfinished(
           new StatusResponseHolder(
-              response.getStatus(),
+              response.status(),
               null
           )
       );
@@ -228,7 +228,7 @@ public class CommonCacheNotifier
     @Override
     public ClientResponse<StatusResponseHolder> handleChunk(
         ClientResponse<StatusResponseHolder> response,
-        HttpChunk chunk,
+        HttpContent chunk,
         long chunkNum
     )
     {
diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java
index 5c87fa5..55e7bcc 100644
--- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java
+++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.druid.client.coordinator.Coordinator;
 import org.apache.druid.concurrent.LifecycleLock;
@@ -47,7 +48,6 @@ import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator;
 import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser;
 import org.apache.druid.server.security.Authenticator;
 import org.apache.druid.server.security.AuthenticatorMapper;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.Duration;
 
 import javax.annotation.Nullable;
diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java
index 32e69d9..7d1d276 100644
--- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java
+++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java
@@ -23,6 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.client.coordinator.Coordinator;
 import org.apache.druid.concurrent.LifecycleLock;
 import org.apache.druid.discovery.DruidLeaderClient;
@@ -50,8 +52,6 @@ import org.apache.druid.security.basic.authorization.entity.GroupMappingAndRoleM
 import org.apache.druid.security.basic.authorization.entity.UserAndRoleMap;
 import org.apache.druid.server.security.Authorizer;
 import org.apache.druid.server.security.AuthorizerMapper;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Duration;
 
 import javax.annotation.Nullable;
diff --git a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosHttpClient.java b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosHttpClient.java
index eee32ac..2ac0c11 100644
--- a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosHttpClient.java
+++ b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosHttpClient.java
@@ -23,6 +23,7 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import io.netty.handler.codec.http.HttpHeaderNames;
 import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.AbstractHttpClient;
@@ -30,7 +31,6 @@ import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
 import org.joda.time.Duration;
 
 import java.net.CookieManager;
@@ -107,7 +107,7 @@ public class KerberosHttpClient extends AbstractHttpClient
             return DruidKerberosUtil.kerberosChallenge(host);
           }
         });
-        request.setHeader(HttpHeaders.Names.AUTHORIZATION, "Negotiate " + challenge);
+        request.setHeader(HttpHeaderNames.AUTHORIZATION.toString(), "Negotiate " + challenge);
         should_retry_on_unauthorized_response = false;
       } else {
         should_retry_on_unauthorized_response = true;
diff --git a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/ResponseCookieHandler.java b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/ResponseCookieHandler.java
index 5097344..0025ac9 100644
--- a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/ResponseCookieHandler.java
+++ b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/ResponseCookieHandler.java
@@ -21,12 +21,12 @@ package org.apache.druid.security.kerberos;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Maps;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.response.ClientResponse;
 import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 
 import java.io.IOException;
 import java.net.CookieManager;
@@ -73,7 +73,7 @@ public class ResponseCookieHandler<Intermediate, Final> implements HttpResponseH
   @Override
   public ClientResponse<Intermediate> handleChunk(
       ClientResponse<Intermediate> clientResponse,
-      HttpChunk httpChunk,
+      HttpContent httpChunk,
       long chunkNum
   )
   {
diff --git a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java
index c71fa71..a421c26 100644
--- a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java
+++ b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java
@@ -19,12 +19,12 @@
 
 package org.apache.druid.security.kerberos;
 
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.response.ClientResponse;
 import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 public class RetryIfUnauthorizedResponseHandler<Intermediate, Final>
     implements HttpResponseHandler<RetryResponseHolder<Intermediate>, RetryResponseHolder<Final>>
@@ -45,10 +45,8 @@ public class RetryIfUnauthorizedResponseHandler<Intermediate, Final>
       TrafficCop trafficCop
   )
   {
-    log.debug("UnauthorizedResponseHandler - Got response status [%s]", httpResponse.getStatus());
-    if (httpResponse.getStatus().equals(HttpResponseStatus.UNAUTHORIZED)) {
-      // Drain the buffer
-      httpResponse.getContent().toString();
+    log.debug("UnauthorizedResponseHandler - Got response status [%s]", httpResponse.status());
+    if (httpResponse.status().equals(HttpResponseStatus.UNAUTHORIZED)) {
       return ClientResponse.unfinished(RetryResponseHolder.retry());
     } else {
       return wrap(httpResponseHandler.handleResponse(httpResponse, trafficCop));
@@ -59,12 +57,12 @@ public class RetryIfUnauthorizedResponseHandler<Intermediate, Final>
   @SuppressWarnings("ReturnValueIgnored")
   public ClientResponse<RetryResponseHolder<Intermediate>> handleChunk(
       ClientResponse<RetryResponseHolder<Intermediate>> clientResponse,
-      HttpChunk httpChunk,
+      HttpContent httpChunk,
       long chunkNum
   )
   {
     if (clientResponse.getObj().shouldRetry()) {
-      httpChunk.getContent().toString();
+      httpChunk.content().toString();
       return clientResponse;
     } else {
       return wrap(httpResponseHandler.handleChunk(unwrap(clientResponse), httpChunk, chunkNum));
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java
index 3646b4a..6c38fcb 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java
@@ -26,6 +26,10 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.indexer.TaskLocation;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexing.common.IndexTaskClient;
@@ -44,10 +48,6 @@ import org.easymock.CaptureType;
 import org.easymock.EasyMock;
 import org.easymock.EasyMockSupport;
 import org.hamcrest.CoreMatchers;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.After;
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java
index 1463b7a..58af6ce 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java
@@ -26,6 +26,10 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.indexer.TaskLocation;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexing.common.IndexTaskClient;
@@ -44,10 +48,6 @@ import org.easymock.CaptureType;
 import org.easymock.EasyMock;
 import org.easymock.EasyMockSupport;
 import org.hamcrest.CoreMatchers;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.After;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java
index db1692a..543a6cb 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java
@@ -33,6 +33,9 @@ import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
+import io.netty.channel.ChannelException;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.indexer.TaskLocation;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.java.util.common.Either;
@@ -48,9 +51,6 @@ import org.apache.druid.java.util.http.client.response.ObjectOrErrorResponseHand
 import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.segment.realtime.firehose.ChatHandlerResource;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Duration;
 import org.joda.time.Period;
 
@@ -204,7 +204,7 @@ public abstract class IndexTaskClient implements AutoCloseable
 
   protected boolean isSuccess(StringFullResponseHolder responseHolder)
   {
-    return responseHolder.getStatus().getCode() / 100 == 2;
+    return responseHolder.getStatus().code() / 100 == 2;
   }
 
   @VisibleForTesting
@@ -359,6 +359,7 @@ public abstract class IndexTaskClient implements AutoCloseable
 
         if (response.isValue()) {
           return response.valueOrThrow();
+
         } else {
           final StringBuilder exceptionMessage = new StringBuilder();
           final HttpResponseStatus httpResponseStatus = response.error().getStatus();
@@ -375,7 +376,7 @@ public abstract class IndexTaskClient implements AutoCloseable
             exceptionMessage.append("; first 1KB of body: ").append(choppedMessage);
           }
 
-          if (httpResponseStatus.getCode() == 400) {
+          if (httpResponseStatus.code() == 400) {
             // don't bother retrying if it's a bad request
             throw new IAE(exceptionMessage.toString());
           } else {
@@ -466,7 +467,7 @@ public abstract class IndexTaskClient implements AutoCloseable
         new ObjectOrErrorResponseHandler<>(responseHandler);
 
     try {
-      log.debug("HTTP %s: %s", request.getMethod().getName(), request.getUrl().toString());
+      log.debug("HTTP %s: %s", request.getMethod().name(), request.getUrl().toString());
       return httpClient.go(request, wrappedHandler, httpTimeout).get();
     }
     catch (Exception e) {
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java
index eb1d62d..6a1fd7f 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClient.java
@@ -20,6 +20,8 @@
 package org.apache.druid.indexing.common.actions;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.channel.ChannelException;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.indexing.common.RetryPolicy;
 import org.apache.druid.indexing.common.RetryPolicyFactory;
@@ -28,8 +30,6 @@ import org.apache.druid.java.util.common.IOE;
 import org.apache.druid.java.util.common.jackson.JacksonUtils;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.Duration;
 
 import javax.ws.rs.core.MediaType;
@@ -84,7 +84,7 @@ public class RemoteTaskActionClient implements TaskActionClient
                              .setContent(MediaType.APPLICATION_JSON, dataToSend)
         );
 
-        if (fullResponseHolder.getStatus().getCode() / 100 == 2) {
+        if (fullResponseHolder.getStatus().code() / 100 == 2) {
           final Map<String, Object> responseDict = jsonMapper.readValue(
               fullResponseHolder.getContent(),
               JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClient.java
index 1946978..951ce4d 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClient.java
@@ -21,6 +21,7 @@ package org.apache.druid.indexing.common.task.batch.parallel;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.guice.annotations.EscalatedClient;
 import org.apache.druid.java.util.common.FileUtils;
 import org.apache.druid.java.util.common.StringUtils;
@@ -29,7 +30,6 @@ import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler;
 import org.apache.druid.utils.CompressionUtils;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java
index f12342e..4fcd6d9 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskClient.java
@@ -21,13 +21,13 @@ package org.apache.druid.indexing.common.task.batch.parallel;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.indexing.common.IndexTaskClient;
 import org.apache.druid.indexing.common.TaskInfoProvider;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 
@@ -66,7 +66,7 @@ public class ParallelIndexSupervisorTaskClient extends IndexTaskClient
       throw new ISE(
           "task[%s] failed to allocate a new segment identifier with the HTTP code[%d] and content[%s]",
           supervisorTaskId,
-          response.getStatus().getCode(),
+          response.getStatus().code(),
           response.getContent()
       );
     } else {
@@ -101,7 +101,7 @@ public class ParallelIndexSupervisorTaskClient extends IndexTaskClient
       throw new ISE(
           "task[%s] failed to allocate a new segment identifier with the HTTP code[%d] and content[%s]",
           supervisorTaskId,
-          response.getStatus().getCode(),
+          response.getStatus().code(),
           response.getContent()
       );
     } else {
@@ -129,7 +129,7 @@ public class ParallelIndexSupervisorTaskClient extends IndexTaskClient
         throw new ISE(
             "Failed to send taskReports to task[%s] with the HTTP code [%d]",
             supervisorTaskId,
-            response.getStatus().getCode()
+            response.getStatus().code()
         );
       }
     }
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java
index 3c1cbb6..7b5aadf 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java
@@ -42,6 +42,8 @@ import com.google.common.util.concurrent.ListenableScheduledFuture;
 import com.google.common.util.concurrent.ListeningScheduledExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.SettableFuture;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.commons.lang.mutable.MutableInt;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.cache.PathChildrenCache;
@@ -83,8 +85,6 @@ import org.apache.druid.server.initialization.IndexerZkConfig;
 import org.apache.druid.tasklogs.TaskLogStreamer;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Duration;
 import org.joda.time.Period;
 
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdapter.java
index 362a832..c736aec 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdapter.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdapter.java
@@ -22,6 +22,8 @@ package org.apache.druid.indexing.overlord;
 import com.google.common.base.Optional;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.timeout.TimeoutException;
 import org.apache.druid.guice.annotations.EscalatedGlobal;
 import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner;
@@ -31,8 +33,6 @@ import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import javax.inject.Inject;
 import java.net.URL;
@@ -110,8 +110,8 @@ public class WorkerTaskRunnerQueryAdapter
             "Action [%s] failed for worker [%s] with status %s(%s)",
             action,
             workerHost,
-            response.getStatus().getCode(),
-            response.getStatus().getReasonPhrase()
+            response.getStatus().code(),
+            response.getStatus().reasonPhrase()
         );
       }
     }
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
index 45af215..98c0411 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java
@@ -39,6 +39,7 @@ import com.google.common.util.concurrent.ListenableScheduledFuture;
 import com.google.common.util.concurrent.ListeningScheduledExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.errorprone.annotations.concurrent.GuardedBy;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.druid.concurrent.LifecycleLock;
 import org.apache.druid.discovery.DiscoveryDruidNode;
@@ -81,7 +82,6 @@ import org.apache.druid.java.util.http.client.response.InputStreamResponseHandle
 import org.apache.druid.server.initialization.IndexerZkConfig;
 import org.apache.druid.tasklogs.TaskLogStreamer;
 import org.apache.zookeeper.KeeperException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.Period;
 
 import javax.annotation.Nullable;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java
index ff9aab1..33adc25 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java
@@ -22,6 +22,8 @@ package org.apache.druid.indexing.overlord.hrtr;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexing.common.task.Task;
 import org.apache.druid.indexing.overlord.ImmutableWorkerInfo;
@@ -40,8 +42,6 @@ import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.server.coordination.ChangeRequestHttpSyncer;
 import org.apache.druid.server.coordination.ChangeRequestsSnapshot;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.DateTime;
 
 import java.net.URL;
@@ -235,20 +235,20 @@ public class WorkerHolder
             try {
               final StatusResponseHolder response = httpClient.go(
                   new Request(HttpMethod.POST, url)
-                      .addHeader(HttpHeaders.Names.CONTENT_TYPE, SmileMediaTypes.APPLICATION_JACKSON_SMILE)
+                      .addHeader(HttpHeaderNames.CONTENT_TYPE.toString(), SmileMediaTypes.APPLICATION_JACKSON_SMILE)
                       .setContent(smileMapper.writeValueAsBytes(task)),
                   StatusResponseHandler.getInstance(),
                   config.getAssignRequestHttpTimeout().toStandardDuration()
               ).get();
 
-              if (response.getStatus().getCode() == 200) {
+              if (response.getStatus().code() == 200) {
                 return true;
               } else {
                 throw new RE(
                     "Failed to assign task[%s] to worker[%s]. Response Code[%s] and Message[%s]. Retrying...",
                     task.getId(),
                     worker.getHost(),
-                    response.getStatus().getCode(),
+                    response.getStatus().code(),
                     response.getContent()
                 );
               }
@@ -286,7 +286,7 @@ public class WorkerHolder
                   config.getShutdownRequestHttpTimeout().toStandardDuration()
               ).get();
 
-              if (response.getStatus().getCode() == 200) {
+              if (response.getStatus().code() == 200) {
                 log.info(
                     "Sent shutdown message to worker: %s, status %s, response: %s",
                     worker.getHost(),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java
index 8ea6321..9387c27 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java
@@ -22,6 +22,8 @@ package org.apache.druid.indexing.seekablestream;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.indexing.common.IndexTaskClient;
 import org.apache.druid.indexing.common.TaskInfoProvider;
 import org.apache.druid.java.util.common.ISE;
@@ -31,8 +33,6 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
index b2b65ec..92be132 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
@@ -30,6 +30,8 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.client.indexing.IndexingService;
 import org.apache.druid.concurrent.LifecycleLock;
 import org.apache.druid.discovery.DruidLeaderClient;
@@ -50,8 +52,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.server.coordination.ChangeRequestHistory;
 import org.apache.druid.server.coordination.ChangeRequestsSnapshot;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 
 import javax.ws.rs.core.MediaType;
 import java.io.File;
@@ -508,11 +508,11 @@ public class WorkerTaskManager
               StringFullResponseHolder fullResponseHolder = overlordClient.go(
                   overlordClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/taskStatus")
                                 .setContent(jsonMapper.writeValueAsBytes(taskIds))
-                                .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON)
-                                .addHeader(HttpHeaders.Names.CONTENT_TYPE, MediaType.APPLICATION_JSON)
+                                .addHeader(HttpHeaderNames.ACCEPT.toString(), MediaType.APPLICATION_JSON)
+                                .addHeader(HttpHeaderNames.CONTENT_TYPE.toString(), MediaType.APPLICATION_JSON)
 
               );
-              if (fullResponseHolder.getStatus().getCode() == 200) {
+              if (fullResponseHolder.getStatus().code() == 200) {
                 String responseContent = fullResponseHolder.getContent();
                 taskStatusesFromOverlord = jsonMapper.readValue(
                     responseContent,
@@ -521,7 +521,7 @@ public class WorkerTaskManager
                     }
                 );
                 log.debug("Received completed task status response [%s].", responseContent);
-              } else if (fullResponseHolder.getStatus().getCode() == 404) {
+              } else if (fullResponseHolder.getStatus().code() == 404) {
                 // NOTE: this is to support backward compatibility, when overlord doesn't have "activeTasks" endpoint.
                 // this if clause should be removed in a future release.
                 log.debug("Deleting all completed tasks. Overlord appears to be running on older version.");
@@ -529,7 +529,7 @@ public class WorkerTaskManager
               } else {
                 log.info(
                     "Got non-success code[%s] from overlord while getting active tasks. will retry on next scheduled run.",
-                    fullResponseHolder.getStatus().getCode()
+                    fullResponseHolder.getStatus().code()
                 );
               }
             }
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/IndexTaskClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/IndexTaskClientTest.java
index 128fdb3..1d6bd90 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/IndexTaskClientTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/IndexTaskClientTest.java
@@ -22,6 +22,11 @@ package org.apache.druid.indexing.common;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.Futures;
+import io.netty.channel.ChannelException;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.apache.druid.indexer.TaskLocation;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.jackson.DefaultObjectMapper;
@@ -30,11 +35,6 @@ import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.segment.realtime.firehose.ChatHandlerResource;
 import org.easymock.EasyMock;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.joda.time.Duration;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -43,7 +43,6 @@ import org.junit.rules.ExpectedException;
 
 import java.io.IOException;
 import java.net.MalformedURLException;
-import java.nio.charset.StandardCharsets;
 import java.util.function.Function;
 
 public class IndexTaskClientTest
@@ -86,8 +85,7 @@ public class IndexTaskClientTest
                 Futures.immediateFuture(
                     Either.value(
                         new StringFullResponseHolder(
-                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK),
-                            StandardCharsets.UTF_8
+                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK)
                         )
                     )
                 )
@@ -115,8 +113,7 @@ public class IndexTaskClientTest
                 Futures.immediateFuture(
                     Either.error(
                         new StringFullResponseHolder(
-                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR),
-                            StandardCharsets.UTF_8
+                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR)
                         ).addChunk("Error")
                     )
                 )
@@ -127,8 +124,7 @@ public class IndexTaskClientTest
                 Futures.immediateFuture(
                     Either.value(
                         new StringFullResponseHolder(
-                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK),
-                            StandardCharsets.UTF_8
+                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK)
                         )
                     )
                 )
@@ -166,8 +162,7 @@ public class IndexTaskClientTest
                 Futures.immediateFuture(
                     Either.error(
                         new StringFullResponseHolder(
-                            incorrectResponse,
-                            StandardCharsets.UTF_8
+                            incorrectResponse
                         )
                     )
                 )
@@ -178,8 +173,7 @@ public class IndexTaskClientTest
                 Futures.immediateFuture(
                     Either.value(
                         new StringFullResponseHolder(
-                            correctResponse,
-                            StandardCharsets.UTF_8
+                            correctResponse
                         )
                     )
                 )
@@ -208,8 +202,7 @@ public class IndexTaskClientTest
                 Futures.immediateFuture(
                     Either.error(
                         new StringFullResponseHolder(
-                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.BAD_REQUEST),
-                            StandardCharsets.UTF_8
+                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.BAD_REQUEST)
                         ).addChunk("Error")
                     )
                 )
@@ -240,8 +233,7 @@ public class IndexTaskClientTest
                 Futures.immediateFuture(
                     Either.error(
                         new StringFullResponseHolder(
-                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR),
-                            StandardCharsets.UTF_8
+                            new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR)
                         ).addChunk("Error")
                     )
                 )
@@ -274,7 +266,7 @@ public class IndexTaskClientTest
             .andReturn(
                 Futures.immediateFuture(
                     Either.error(
-                        new StringFullResponseHolder(response, StandardCharsets.UTF_8).addChunk("Error")
+                        new StringFullResponseHolder(response).addChunk("Error")
                     )
                 )
             )
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java
index c3528c7..d53ff14 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java
@@ -20,6 +20,9 @@
 package org.apache.druid.indexing.common.actions;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.indexing.common.RetryPolicyConfig;
 import org.apache.druid.indexing.common.RetryPolicyFactory;
@@ -33,19 +36,12 @@ import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.easymock.EasyMock;
-import org.jboss.netty.buffer.BigEndianHeapChannelBuffer;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
 
 import java.io.IOException;
 import java.net.URL;
-import java.nio.charset.StandardCharsets;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -53,9 +49,6 @@ import java.util.Map;
 
 public class RemoteTaskActionClientTest
 {
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
   private DruidLeaderClient druidLeaderClient;
   private final ObjectMapper objectMapper = new DefaultObjectMapper();
 
@@ -84,13 +77,11 @@ public class RemoteTaskActionClientTest
     ));
     responseBody.put("result", expectedLocks);
     String strResult = objectMapper.writeValueAsString(responseBody);
-    final HttpResponse response = EasyMock.createNiceMock(HttpResponse.class);
-    EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0));
+    final HttpResponse response = EasyMock.createMock(HttpResponse.class);
+    EasyMock.expect(response.status()).andReturn(HttpResponseStatus.OK).anyTimes();
     EasyMock.replay(response);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        response,
-        StandardCharsets.UTF_8
+        response
     ).addChunk(strResult);
 
     // set up mocks
@@ -119,13 +110,11 @@ public class RemoteTaskActionClientTest
             .andReturn(request);
 
     // return status code 200 and a list with size equals 1
-    final HttpResponse response = EasyMock.createNiceMock(HttpResponse.class);
-    EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0));
+    final HttpResponse response = EasyMock.createMock(HttpResponse.class);
+    EasyMock.expect(response.status()).andReturn(HttpResponseStatus.BAD_REQUEST).anyTimes();
     EasyMock.replay(response);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        response,
-        StandardCharsets.UTF_8
+        response
     ).addChunk("testSubmitWithIllegalStatusCode");
 
     // set up mocks
@@ -139,11 +128,14 @@ public class RemoteTaskActionClientTest
         new RetryPolicyFactory(objectMapper.readValue("{\"maxRetryCount\":0}", RetryPolicyConfig.class)),
         objectMapper
     );
-    expectedException.expect(IOException.class);
-    expectedException.expectMessage(
+    final IOException exception = Assert.assertThrows(
+        IOException.class,
+        () -> client.submit(new LockListAction())
+    );
+    Assert.assertEquals(
         "Error with status[400 Bad Request] and message[testSubmitWithIllegalStatusCode]. "
-        + "Check overlord logs for details."
+        + "Check overlord logs for details.",
+        exception.getMessage()
     );
-    client.submit(new LockListAction());
   }
 }
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdpaterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdpaterTest.java
index 4bb1c9b..735ce94 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdpaterTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/WorkerTaskRunnerQueryAdpaterTest.java
@@ -23,6 +23,8 @@ import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.util.concurrent.SettableFuture;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.indexing.worker.Worker;
 import org.apache.druid.indexing.worker.config.WorkerConfig;
 import org.apache.druid.java.util.common.DateTimes;
@@ -33,8 +35,6 @@ import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.easymock.Capture;
 import org.easymock.EasyMock;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
index b919706..70bb8fb 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
@@ -25,6 +25,7 @@ import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.indexer.RunnerTaskState;
 import org.apache.druid.indexer.TaskInfo;
 import org.apache.druid.indexer.TaskLocation;
@@ -57,7 +58,6 @@ import org.apache.druid.server.security.AuthorizerMapper;
 import org.apache.druid.server.security.ForbiddenException;
 import org.apache.druid.server.security.Resource;
 import org.easymock.EasyMock;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.joda.time.Interval;
@@ -1374,7 +1374,7 @@ public class OverlordResourceTest
 
     final Response response = overlordResource.enableWorker(host);
 
-    Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus());
+    Assert.assertEquals(HttpResponseStatus.OK.code(), response.getStatus());
     Assert.assertEquals(ImmutableMap.of(host, "enabled"), response.getEntity());
   }
 
@@ -1397,7 +1397,7 @@ public class OverlordResourceTest
 
     final Response response = overlordResource.disableWorker(host);
 
-    Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus());
+    Assert.assertEquals(HttpResponseStatus.OK.code(), response.getStatus());
     Assert.assertEquals(ImmutableMap.of(host, "disabled"), response.getEntity());
   }
 
@@ -1420,7 +1420,7 @@ public class OverlordResourceTest
 
     final Response response = overlordResource.enableWorker(host);
 
-    Assert.assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.getCode(), response.getStatus());
+    Assert.assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.code(), response.getStatus());
     Assert.assertEquals(ImmutableMap.of("error", "Worker API returns error!"), response.getEntity());
   }
 
@@ -1443,7 +1443,7 @@ public class OverlordResourceTest
 
     final Response response = overlordResource.disableWorker(host);
 
-    Assert.assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.getCode(), response.getStatus());
+    Assert.assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.code(), response.getStatus());
     Assert.assertEquals(ImmutableMap.of("error", "Worker API returns error!"), response.getEntity());
   }
 
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java
index 907ac8d..316cd05 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java
@@ -23,6 +23,9 @@ import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.guice.annotations.Smile;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
@@ -33,9 +36,6 @@ import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder;
 import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.testing.guice.TestClient;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import javax.annotation.Nullable;
 import javax.ws.rs.core.MediaType;
@@ -142,7 +142,7 @@ public abstract class AbstractQueryResourceTestClient<QueryType>
       request.setContent(this.contentTypeHeader, encoderDecoderMap.get(this.contentTypeHeader).encode(query));
       if (this.acceptHeader != null) {
         expectedResponseType = this.acceptHeader;
-        request.addHeader(HttpHeaders.Names.ACCEPT, this.acceptHeader);
+        request.addHeader(HttpHeaderNames.ACCEPT.toString(), this.acceptHeader);
       }
 
       BytesFullResponseHolder response = httpClient.go(
@@ -159,7 +159,7 @@ public abstract class AbstractQueryResourceTestClient<QueryType>
         );
       }
 
-      String responseType = response.getResponse().headers().get(HttpHeaders.Names.CONTENT_TYPE);
+      String responseType = response.getResponse().headers().get(HttpHeaderNames.CONTENT_TYPE);
       if (!expectedResponseType.equals(responseType)) {
         throw new ISE(
             "Content-Type[%s] in HTTP response does not match the expected[%s]",
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/ClientInfoResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/ClientInfoResourceTestClient.java
index 09dfff0..6dfbb26 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/clients/ClientInfoResourceTestClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/ClientInfoResourceTestClient.java
@@ -22,6 +22,8 @@ package org.apache.druid.testing.clients;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.http.client.HttpClient;
@@ -30,8 +32,6 @@ import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.testing.IntegrationTestingConfig;
 import org.apache.druid.testing.guice.TestClient;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import java.net.URL;
 import java.util.List;
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java
index 2fd1a5e..38f9920 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java
@@ -22,6 +22,8 @@ package org.apache.druid.testing.clients;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.http.client.HttpClient;
@@ -32,8 +34,6 @@ import org.apache.druid.server.coordinator.CoordinatorCompactionConfig;
 import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
 import org.apache.druid.testing.IntegrationTestingConfig;
 import org.apache.druid.testing.guice.TestClient;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import java.net.URL;
 import java.util.List;
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java
index 1b96f3f..371c47b 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java
@@ -24,6 +24,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.net.HostAndPort;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.StringUtils;
@@ -38,8 +40,6 @@ import org.apache.druid.server.lookup.cache.LookupExtractorFactoryMapContainer;
 import org.apache.druid.testing.IntegrationTestingConfig;
 import org.apache.druid.testing.guice.TestClient;
 import org.apache.druid.timeline.DataSegment;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
@@ -183,10 +183,10 @@ public class CoordinatorResourceTestClient
           responseHandler
       ).get();
 
-      if (response.getStatus().getCode() == HttpResponseStatus.NO_CONTENT.getCode()) {
+      if (response.getStatus().code() == HttpResponseStatus.NO_CONTENT.code()) {
         return null;
       }
-      if (response.getStatus().getCode() != HttpResponseStatus.OK.getCode()) {
+      if (response.getStatus().code() != HttpResponseStatus.OK.code()) {
         throw new ISE(
             "Error while making request to url[%s] status[%s] content[%s]",
             url,
@@ -315,10 +315,10 @@ public class CoordinatorResourceTestClient
           responseHandler
       ).get();
 
-      if (response.getStatus().getCode() == HttpResponseStatus.NOT_FOUND.getCode()) {
+      if (response.getStatus().code() == HttpResponseStatus.NOT_FOUND.code()) {
         return null;
       }
-      if (response.getStatus().getCode() != HttpResponseStatus.OK.getCode()) {
+      if (response.getStatus().code() != HttpResponseStatus.OK.code()) {
         throw new ISE(
             "Error while making request to url[%s] status[%s] content[%s]",
             url,
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java
index 761d9f5..6a77992a 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java
@@ -22,6 +22,8 @@ package org.apache.druid.testing.clients;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.jackson.JacksonUtils;
@@ -31,8 +33,6 @@ import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.testing.guice.TestClient;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import javax.ws.rs.core.MediaType;
 import java.io.BufferedReader;
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java
index a4932d1..b67de3d 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java
@@ -23,6 +23,8 @@ import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Predicates;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.client.indexing.TaskStatusResponse;
 import org.apache.druid.indexer.TaskState;
 import org.apache.druid.indexer.TaskStatusPlus;
@@ -43,8 +45,6 @@ import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
 import org.apache.druid.testing.IntegrationTestingConfig;
 import org.apache.druid.testing.guice.TestClient;
 import org.apache.druid.testing.utils.ITRetryUtil;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Interval;
 
 import java.net.URL;
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java
index d0f2bbe..4fd0edc 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java
@@ -27,6 +27,9 @@ import com.github.dockerjava.core.DockerClientBuilder;
 import com.github.dockerjava.core.command.ExecStartResultCallback;
 import com.github.dockerjava.netty.NettyDockerCmdExecFactory;
 import com.google.inject.Inject;
+import io.netty.channel.ChannelException;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.StringUtils;
@@ -38,9 +41,6 @@ import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
 import org.apache.druid.testing.IntegrationTestingConfig;
 import org.apache.druid.testing.guice.TestClient;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import java.io.ByteArrayOutputStream;
 import java.net.URL;
diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java
index 5e9ad4d..0062b18 100644
--- a/integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java
+++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java
@@ -19,6 +19,8 @@
 
 package org.apache.druid.testing.utils;
 
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.logger.Logger;
@@ -27,8 +29,6 @@ import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.testing.clients.AbstractQueryResourceTestClient;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import javax.annotation.Nullable;
 import javax.ws.rs.core.MediaType;
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/leadership/ITHighAvailabilityTest.java b/integration-tests/src/test/java/org/apache/druid/tests/leadership/ITHighAvailabilityTest.java
index 78e071b..b60c2c2 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/leadership/ITHighAvailabilityTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/leadership/ITHighAvailabilityTest.java
@@ -22,6 +22,8 @@ package org.apache.druid.tests.leadership;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.cli.CliCustomNodeRole;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.curator.discovery.ServerDiscoveryFactory;
@@ -45,8 +47,6 @@ import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.testing.utils.SqlTestQueryHelper;
 import org.apache.druid.tests.TestNGGroup;
 import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.testng.Assert;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
@@ -196,7 +196,7 @@ public class ITHighAvailabilityTest
           new Request(HttpMethod.GET, new URL(location)),
           StatusResponseHandler.getInstance()
       ).get();
-      LOG.info("%s responded with %s", location, response.getStatus().getCode());
+      LOG.info("%s responded with %s", location, response.getStatus().code());
       Assert.assertEquals(response.getStatus(), HttpResponseStatus.OK);
       count++;
     }
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/query/ITQueryRetryTestOnMissingSegments.java b/integration-tests/src/test/java/org/apache/druid/tests/query/ITQueryRetryTestOnMissingSegments.java
index 73394c3..db93eea 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/query/ITQueryRetryTestOnMissingSegments.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/query/ITQueryRetryTestOnMissingSegments.java
@@ -22,6 +22,7 @@ package org.apache.druid.tests.query;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
@@ -37,7 +38,6 @@ import org.apache.druid.testing.utils.QueryWithResults;
 import org.apache.druid.testing.utils.TestQueryHelper;
 import org.apache.druid.tests.TestNGGroup;
 import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.testng.Assert;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Guice;
@@ -153,7 +153,7 @@ public class ITQueryRetryTestOnMissingSegments
             .queryAsync(queryHelper.getQueryURL(config.getBrokerUrl()), queryWithResult.getQuery())
             .get();
 
-        if (responseHolder.getStatus().getCode() == HttpResponseStatus.OK.getCode()) {
+        if (responseHolder.getStatus().code() == HttpResponseStatus.OK.code()) {
           querySuccess++;
 
           List<Map<String, Object>> result = jsonMapper.readValue(
@@ -180,7 +180,7 @@ public class ITQueryRetryTestOnMissingSegments
           } else {
             resultMatches++;
           }
-        } else if (responseHolder.getStatus().getCode() == HttpResponseStatus.INTERNAL_SERVER_ERROR.getCode() &&
+        } else if (responseHolder.getStatus().code() == HttpResponseStatus.INTERNAL_SERVER_ERROR.code() &&
                    expectation == Expectation.QUERY_FAILURE) {
           final Map<String, Object> response = jsonMapper.readValue(responseHolder.getContent(), Map.class);
           final String errorMessage = (String) response.get("errorMessage");
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/query/ITSqlCancelTest.java b/integration-tests/src/test/java/org/apache/druid/tests/query/ITSqlCancelTest.java
index 905dcfe..f379c0d 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/query/ITSqlCancelTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/query/ITSqlCancelTest.java
@@ -22,6 +22,7 @@ package org.apache.druid.tests.query;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
@@ -36,7 +37,6 @@ import org.apache.druid.testing.guice.DruidTestModuleFactory;
 import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.testing.utils.SqlTestQueryHelper;
 import org.apache.druid.tests.TestNGGroup;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java b/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java
index 1182183..5cdec10 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java
@@ -20,6 +20,8 @@
 package org.apache.druid.tests.query;
 
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.commons.io.IOUtils;
 import org.apache.druid.curator.discovery.ServerDiscoveryFactory;
 import org.apache.druid.curator.discovery.ServerDiscoverySelector;
@@ -43,8 +45,6 @@ import org.apache.druid.testing.utils.ServerDiscoveryUtil;
 import org.apache.druid.tests.TestNGGroup;
 import org.apache.druid.tests.indexer.AbstractITBatchIndexTest;
 import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
 import org.testng.annotations.BeforeSuite;
 import org.testng.annotations.Guice;
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/query/ITWikipediaQueryTest.java b/integration-tests/src/test/java/org/apache/druid/tests/query/ITWikipediaQueryTest.java
index ee53ab5..d07fe95 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/query/ITWikipediaQueryTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/query/ITWikipediaQueryTest.java
@@ -22,6 +22,7 @@ package org.apache.druid.tests.query;
 import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
 import org.apache.druid.query.Druids;
@@ -34,7 +35,6 @@ import org.apache.druid.testing.guice.DruidTestModuleFactory;
 import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.testing.utils.TestQueryHelper;
 import org.apache.druid.tests.TestNGGroup;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.testng.Assert;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.DataProvider;
@@ -139,10 +139,10 @@ public class ITWikipediaQueryTest
 
           for (Future<StatusResponseHolder> future : futures) {
             StatusResponseHolder status = future.get();
-            if (status.getStatus().getCode() == QueryCapacityExceededException.STATUS_CODE) {
+            if (status.getStatus().code() == QueryCapacityExceededException.STATUS_CODE) {
               limited++;
               Assert.assertTrue(status.getContent().contains(QueryCapacityExceededException.makeLaneErrorMessage("one", 1)));
-            } else if (status.getStatus().getCode() == HttpResponseStatus.OK.getCode()) {
+            } else if (status.getStatus().code() == HttpResponseStatus.OK.code()) {
               success++;
             }
           }
@@ -169,14 +169,14 @@ public class ITWikipediaQueryTest
         getQueryBuilder().build()
     ).get();
 
-    Assert.assertEquals(followUp.getStatus().getCode(), HttpResponseStatus.OK.getCode());
+    Assert.assertEquals(followUp.getStatus().code(), HttpResponseStatus.OK.code());
 
     StatusResponseHolder andAnother = queryClient.queryAsync(
         queryHelper.getQueryURL(config.getBrokerUrl()),
         getQueryBuilder().build()
     ).get();
 
-    Assert.assertEquals(andAnother.getStatus().getCode(), HttpResponseStatus.OK.getCode());
+    Assert.assertEquals(andAnother.getStatus().code(), HttpResponseStatus.OK.code());
   }
 
   @Test
@@ -202,9 +202,9 @@ public class ITWikipediaQueryTest
 
     for (Future<StatusResponseHolder> future : futures) {
       StatusResponseHolder status = future.get();
-      if (status.getStatus().getCode() == QueryCapacityExceededException.STATUS_CODE) {
+      if (status.getStatus().code() == QueryCapacityExceededException.STATUS_CODE) {
         limited++;
-      } else if (status.getStatus().getCode() == HttpResponseStatus.OK.getCode()) {
+      } else if (status.getStatus().code() == HttpResponseStatus.OK.code()) {
         success++;
       }
     }
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java
index 42556ad..2b2308e 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java
@@ -25,6 +25,8 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.calcite.avatica.AvaticaSqlException;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.guice.annotations.Client;
@@ -45,8 +47,6 @@ import org.apache.druid.testing.clients.CoordinatorResourceTestClient;
 import org.apache.druid.testing.utils.HttpUtil;
 import org.apache.druid.testing.utils.TestQueryHelper;
 import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java
index 8c2227e..23b776a 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.tests.security;
 
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.CredentialedHttpClient;
@@ -30,7 +31,6 @@ import org.apache.druid.testing.guice.DruidTestModuleFactory;
 import org.apache.druid.testing.utils.HttpUtil;
 import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.tests.TestNGGroup;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthLdapConfigurationTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthLdapConfigurationTest.java
index 5adc9bb..a9b81b6 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthLdapConfigurationTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthLdapConfigurationTest.java
@@ -22,6 +22,7 @@ package org.apache.druid.tests.security;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.CredentialedHttpClient;
@@ -34,7 +35,6 @@ import org.apache.druid.testing.guice.DruidTestModuleFactory;
 import org.apache.druid.testing.utils.HttpUtil;
 import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.tests.TestNGGroup;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/ITCoordinatorOverlordProxyAuthTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/ITCoordinatorOverlordProxyAuthTest.java
index f6cc85a..d0ac949 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/security/ITCoordinatorOverlordProxyAuthTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/security/ITCoordinatorOverlordProxyAuthTest.java
@@ -20,10 +20,10 @@
 package org.apache.druid.tests.security;
 
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.testing.clients.CoordinatorResourceTestClient;
 import org.apache.druid.testing.guice.DruidTestModuleFactory;
 import org.apache.druid.tests.TestNGGroup;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.testng.Assert;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java
index 09167c0..f445024 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java
@@ -22,6 +22,8 @@ package org.apache.druid.tests.security;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.guice.annotations.Client;
 import org.apache.druid.guice.http.DruidHttpClientConfig;
 import org.apache.druid.https.SSLClientConfig;
@@ -43,8 +45,6 @@ import org.apache.druid.testing.IntegrationTestingConfig;
 import org.apache.druid.testing.guice.DruidTestModuleFactory;
 import org.apache.druid.testing.utils.ITTLSCertificateChecker;
 import org.apache.druid.tests.TestNGGroup;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Duration;
 import org.testng.Assert;
 import org.testng.annotations.Guice;
diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml
index 842ebb8..99d6816 100644
--- a/owasp-dependency-check-suppressions.xml
+++ b/owasp-dependency-check-suppressions.xml
@@ -154,18 +154,6 @@
     <cvssBelow>10</cvssBelow>  <!-- suppress all CVEs for jackson-mapper-asl:1.9.13 ince it is via curator-x-discovery -->
   </suppress>
   <suppress>
-    <!-- TODO: Fix by updating org.apache.druid.java.util.http.client.NettyHttpClient to use netty 4 -->
-    <notes><![CDATA[
-   file name: netty-3.10.6.Final.jar
-   ]]></notes>
-    <packageUrl regex="true">^pkg:maven/io\.netty/netty@3.10.6.Final$</packageUrl>
-    <cve>CVE-2019-16869</cve>
-    <cve>CVE-2019-20444</cve>
-    <cve>CVE-2019-20445</cve>
-    <cve>CVE-2021-37136</cve>
-    <cve>CVE-2021-37137</cve>
-  </suppress>
-  <suppress>
     <!-- TODO: Fix by upgrading hadoop-auth version -->
     <notes><![CDATA[
    file name: nimbus-jose-jwt-4.41.1.jar
diff --git a/pom.xml b/pom.xml
index 134f8b6..99ed192 100644
--- a/pom.xml
+++ b/pom.xml
@@ -338,12 +338,6 @@
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-framework</artifactId>
                 <version>${apache.curator.version}</version>
-                <exclusions>
-                    <exclusion>
-                        <groupId>org.jboss.netty</groupId>
-                        <artifactId>netty</artifactId>
-                    </exclusion>
-                </exclusions>
             </dependency>
             <dependency>
                 <groupId>org.apache.curator</groupId>
@@ -972,12 +966,6 @@
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-test</artifactId>
                 <version>${apache.curator.test.version}</version>
-                <exclusions>
-                    <exclusion>
-                        <groupId>org.jboss.netty</groupId>
-                        <artifactId>netty</artifactId>
-                    </exclusion>
-                </exclusions>
                 <scope>test</scope>
             </dependency>
             <dependency>
diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
index c7556b0..0a9790b 100644
--- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
+++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
@@ -27,6 +27,12 @@ import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.concurrent.Execs;
@@ -56,12 +62,6 @@ import org.apache.druid.query.context.ConcurrentResponseContext;
 import org.apache.druid.query.context.ResponseContext;
 import org.apache.druid.server.QueryResource;
 import org.apache.druid.utils.CloseableUtils;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.handler.codec.http.HttpChunk;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 import org.joda.time.Duration;
 
 import javax.ws.rs.core.MediaType;
@@ -194,11 +194,11 @@ public class DirectDruidClient<T> implements QueryRunner<T>
         /**
          * Queue a buffer. Returns true if we should keep reading, false otherwise.
          */
-        private boolean enqueue(ChannelBuffer buffer, long chunkNum) throws InterruptedException
+        private boolean enqueue(ByteBuf buffer, long chunkNum) throws InterruptedException
         {
           // Increment queuedByteCount before queueing the object, so queuedByteCount is at least as high as
           // the actual number of queued bytes at any particular time.
-          final InputStreamHolder holder = InputStreamHolder.fromChannelBuffer(buffer, chunkNum);
+          final InputStreamHolder holder = InputStreamHolder.fromByteBuf(buffer, chunkNum);
           final long currentQueuedByteCount = queuedByteCount.addAndGet(holder.getLength());
           queue.put(holder);
 
@@ -228,13 +228,11 @@ public class DirectDruidClient<T> implements QueryRunner<T>
         {
           trafficCopRef.set(trafficCop);
           checkQueryTimeout();
-          checkTotalBytesLimit(response.getContent().readableBytes());
 
           log.debug("Initial response from url[%s] for queryId[%s]", url, query.getId());
           responseStartTimeNs = System.nanoTime();
           acquireResponseMetrics().reportNodeTimeToFirstByte(responseStartTimeNs - requestStartTimeNs).emit(emitter);
 
-          final boolean continueReading;
           try {
             log.trace(
                 "Got a response from [%s] for query ID[%s], subquery ID[%s]",
@@ -248,7 +246,13 @@ public class DirectDruidClient<T> implements QueryRunner<T>
             if (responseContext != null) {
               context.merge(ResponseContext.deserialize(responseContext, objectMapper));
             }
-            continueReading = enqueue(response.getContent(), 0L);
+            // add empty initial buffer since SequenceInputStream will peek the first element right away
+            enqueue(Unpooled.EMPTY_BUFFER, 0L);
+          }
+          catch (InterruptedException e) {
+            log.error(e, "Queue appending interrupted");
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(e);
           }
           catch (final IOException e) {
             log.error(e, "Error parsing response context from url [%s]", url);
@@ -263,12 +267,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
                 }
             );
           }
-          catch (InterruptedException e) {
-            log.error(e, "Queue appending interrupted");
-            Thread.currentThread().interrupt();
-            throw new RuntimeException(e);
-          }
-          totalByteCount.addAndGet(response.getContent().readableBytes());
+
           return ClientResponse.finished(
               new SequenceInputStream(
                   new Enumeration<InputStream>()
@@ -305,28 +304,28 @@ public class DirectDruidClient<T> implements QueryRunner<T>
                     }
                   }
               ),
-              continueReading
+              true
           );
         }
 
         @Override
         public ClientResponse<InputStream> handleChunk(
             ClientResponse<InputStream> clientResponse,
-            HttpChunk chunk,
+            HttpContent chunk,
             long chunkNum
         )
         {
           checkQueryTimeout();
 
-          final ChannelBuffer channelBuffer = chunk.getContent();
-          final int bytes = channelBuffer.readableBytes();
+          final ByteBuf byteBuf = chunk.content();
+          final int bytes = byteBuf.readableBytes();
 
           checkTotalBytesLimit(bytes);
 
           boolean continueReading = true;
           if (bytes > 0) {
             try {
-              continueReading = enqueue(channelBuffer, chunkNum);
+              continueReading = enqueue(byteBuf, chunkNum);
             }
             catch (InterruptedException e) {
               log.error(e, "Unable to put finalizing input stream into Sequence queue for url [%s]", url);
@@ -367,7 +366,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
             try {
               // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out
               // after done is set to true, regardless of the rest of the stream's state.
-              queue.put(InputStreamHolder.fromChannelBuffer(ChannelBuffers.EMPTY_BUFFER, Long.MAX_VALUE));
+              queue.put(InputStreamHolder.fromByteBuf(Unpooled.EMPTY_BUFFER, Long.MAX_VALUE));
             }
             catch (InterruptedException e) {
               log.error(e, "Unable to put finalizing input stream into Sequence queue for url [%s]", url);
@@ -456,7 +455,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
               new URL(url)
           ).setContent(objectMapper.writeValueAsBytes(QueryContexts.withTimeout(query, timeLeft)))
            .setHeader(
-               HttpHeaders.Names.CONTENT_TYPE,
+               HttpHeaderNames.CONTENT_TYPE.toString(),
                isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON
            ),
           responseHandler,
@@ -539,7 +538,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
         Future<StatusResponseHolder> responseFuture = httpClient.go(
             new Request(HttpMethod.DELETE, new URL(cancelUrl))
             .setContent(objectMapper.writeValueAsBytes(query))
-            .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON),
+            .setHeader(HttpHeaderNames.CONTENT_TYPE.toString(), isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON),
             StatusResponseHandler.getInstance(),
             Duration.standardSeconds(1));
 
@@ -549,11 +548,11 @@ public class DirectDruidClient<T> implements QueryRunner<T>
               log.error("Error cancelling query[%s]", query);
             }
             StatusResponseHolder response = responseFuture.get();
-            if (response.getStatus().getCode() >= 500) {
+            if (response.getStatus().code() >= 500) {
               log.error("Error cancelling query[%s]: queriable node returned status[%d] [%s].",
                   query,
-                  response.getStatus().getCode(),
-                  response.getStatus().getReasonPhrase());
+                  response.getStatus().code(),
+                  response.getStatus().reasonPhrase());
             }
           }
           catch (ExecutionException | InterruptedException e) {
diff --git a/server/src/main/java/org/apache/druid/client/InputStreamHolder.java b/server/src/main/java/org/apache/druid/client/InputStreamHolder.java
index a0def4b..c29ea78 100644
--- a/server/src/main/java/org/apache/druid/client/InputStreamHolder.java
+++ b/server/src/main/java/org/apache/druid/client/InputStreamHolder.java
@@ -19,8 +19,8 @@
 
 package org.apache.druid.client;
 
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBufferInputStream;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
 
 import java.io.InputStream;
 
@@ -42,10 +42,15 @@ public class InputStreamHolder
     return new InputStreamHolder(stream, chunkNum, length);
   }
 
-  public static InputStreamHolder fromChannelBuffer(final ChannelBuffer buffer, final long chunkNum)
+  public static InputStreamHolder fromByteBuf(final ByteBuf buffer, final long chunkNum)
   {
     final int length = buffer.readableBytes();
-    return new InputStreamHolder(new ChannelBufferInputStream(buffer), chunkNum, length);
+    final ByteBufInputStream stream = new ByteBufInputStream(
+        buffer.retain(), // increment reference count, so it doesn't get released until we are done reading
+        length,
+        true // release ByteBuf on close
+    );
+    return new InputStreamHolder(stream, chunkNum, length);
   }
 
   public InputStream getStream()
diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java
index 36dba89..94782a6 100644
--- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java
+++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java
@@ -22,6 +22,8 @@ package org.apache.druid.client.coordinator;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.client.ImmutableSegmentLoadInfo;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.java.util.common.ISE;
@@ -29,8 +31,6 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.query.SegmentDescriptor;
 import org.apache.druid.timeline.DataSegment;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
diff --git a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java
index 248e5ea..6cfcf2c 100644
--- a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java
+++ b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java
@@ -25,6 +25,8 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Iterables;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.common.utils.IdUtils;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.indexer.TaskStatusPlus;
@@ -35,8 +37,6 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.timeline.DataSegment;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.DateTime;
 import org.joda.time.Interval;
 
diff --git a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java
index 0679ac3..18ccc62 100644
--- a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java
+++ b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java
@@ -21,6 +21,9 @@ package org.apache.druid.discovery;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
+import io.netty.channel.ChannelException;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.concurrent.LifecycleLock;
 import org.apache.druid.java.util.common.IOE;
 import org.apache.druid.java.util.common.ISE;
@@ -35,9 +38,6 @@ import org.apache.druid.java.util.http.client.response.FullResponseHolder;
 import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHandler;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
@@ -155,13 +155,11 @@ public class DruidLeaderClient
 
         try {
           if (request.getUrl().getQuery() == null) {
-            request = withUrl(
-                request,
+            request = request.withUrl(
                 new URL(StringUtils.format("%s%s", getCurrentKnownLeader(false), request.getUrl().getPath()))
             );
           } else {
-            request = withUrl(
-                request,
+            request = request.withUrl(
                 new URL(StringUtils.format(
                     "%s%s?%s",
                     getCurrentKnownLeader(false),
@@ -183,7 +181,7 @@ public class DruidLeaderClient
         }
       }
 
-      if (HttpResponseStatus.TEMPORARY_REDIRECT.equals(fullResponseHolder.getResponse().getStatus())) {
+      if (HttpResponseStatus.TEMPORARY_REDIRECT.equals(fullResponseHolder.getResponse().status())) {
         String redirectUrlStr = fullResponseHolder.getResponse().headers().get("Location");
         if (redirectUrlStr == null) {
           throw new IOE("No redirect location is found in response from url[%s].", request.getUrl());
@@ -212,7 +210,7 @@ public class DruidLeaderClient
             redirectUrl.getPort()
         ));
 
-        request = withUrl(request, redirectUrl);
+        request = request.withUrl(redirectUrl);
       } else {
         return fullResponseHolder;
       }
@@ -232,7 +230,7 @@ public class DruidLeaderClient
       throw new ISE(ex, "Couldn't find leader.");
     }
 
-    if (responseHolder.getStatus().getCode() == 200) {
+    if (responseHolder.getStatus().code() == 200) {
       String leaderUrl = responseHolder.getContent();
 
       //verify this is valid url
@@ -251,7 +249,7 @@ public class DruidLeaderClient
 
     throw new ISE(
         "Couldn't find leader, failed response status is [%s] and content [%s].",
-        responseHolder.getStatus().getCode(),
+        responseHolder.getStatus().code(),
         responseHolder.getContent()
     );
   }
@@ -286,13 +284,4 @@ public class DruidLeaderClient
     return null;
   }
 
-  private Request withUrl(Request old, URL url)
-  {
-    Request req = new Request(old.getMethod(), url);
-    req.addHeaderValues(old.getHeaders());
-    if (old.hasContent()) {
-      req.setContent(old.getContent());
-    }
-    return req;
-  }
 }
diff --git a/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java b/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java
index 0682a06..3e52750 100644
--- a/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java
+++ b/server/src/main/java/org/apache/druid/query/lookup/LookupReferencesManager.java
@@ -28,6 +28,8 @@ import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.druid.client.coordinator.Coordinator;
 import org.apache.druid.concurrent.LifecycleLock;
@@ -45,8 +47,6 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
 import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
 import javax.annotation.Nullable;
 import java.io.File;
@@ -71,7 +71,7 @@ import java.util.function.Function;
 
 /**
  * This class provide a basic {@link LookupExtractorFactory} references manager. It allows basic operations fetching,
- * listing, adding and deleting of {@link LookupExtractor} objects, and can take periodic snap shot of the loaded lookup
+ * listing, adding and deleting of {@link LookupExtractor} objects, and can take periodic snapshot of the loaded lookup
  * extractor specifications in order to bootstrap nodes after restart.
  *
  * It also implements {@link LookupExtractorFactoryContainerProvider}, to supply queries and indexing transformations
diff --git a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java
index 7f33d1e..ed6288e 100644
--- a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java
+++ b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java
@@ -26,6 +26,8 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.concurrent.LifecycleLock;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.RE;
@@ -35,8 +37,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.Duration;
 
 import javax.annotation.Nullable;
@@ -204,8 +204,8 @@ public class ChangeRequestHttpSyncer<T>
 
       ListenableFuture<InputStream> syncRequestFuture = httpClient.go(
           new Request(HttpMethod.GET, new URL(baseServerURL, req))
-              .addHeader(HttpHeaders.Names.ACCEPT, SmileMediaTypes.APPLICATION_JACKSON_SMILE)
-              .addHeader(HttpHeaders.Names.CONTENT_TYPE, SmileMediaTypes.APPLICATION_JACKSON_SMILE),
+              .addHeader(HttpHeaderNames.ACCEPT.toString(), SmileMediaTypes.APPLICATION_JACKSON_SMILE)
+              .addHeader(HttpHeaderNames.CONTENT_TYPE.toString(), SmileMediaTypes.APPLICATION_JACKSON_SMILE),
           responseHandler,
           Duration.millis(serverHttpTimeout)
       );
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java b/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java
index 88d8226..228f516 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java
@@ -19,10 +19,10 @@
 
 package org.apache.druid.server.coordinator;
 
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.java.util.http.client.io.AppendableByteArrayInputStream;
 import org.apache.druid.java.util.http.client.response.ClientResponse;
 import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 
 /**
  * An async BytesAccumulatingResponseHandler which returns unfinished response
@@ -35,8 +35,8 @@ public class BytesAccumulatingResponseHandler extends InputStreamResponseHandler
   @Override
   public ClientResponse<AppendableByteArrayInputStream> handleResponse(HttpResponse response, TrafficCop trafficCop)
   {
-    status = response.getStatus().getCode();
-    description = response.getStatus().getReasonPhrase();
+    status = response.status().code();
+    description = response.status().reasonPhrase();
     return ClientResponse.unfinished(super.handleResponse(response, trafficCop).getObj());
   }
 
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java
index 4a171e5..b59cfde 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java
@@ -26,6 +26,8 @@ import com.google.common.collect.Iterators;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.StringUtils;
@@ -40,8 +42,6 @@ import org.apache.druid.server.coordination.SegmentChangeRequestDrop;
 import org.apache.druid.server.coordination.SegmentChangeRequestLoad;
 import org.apache.druid.server.coordination.SegmentLoadDropHandler;
 import org.apache.druid.timeline.DataSegment;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.Duration;
 
 import javax.servlet.http.HttpServletResponse;
@@ -186,8 +186,8 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
       BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler();
       ListenableFuture<InputStream> future = httpClient.go(
           new Request(HttpMethod.POST, changeRequestURL)
-              .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON)
-              .addHeader(HttpHeaders.Names.CONTENT_TYPE, MediaType.APPLICATION_JSON)
+              .addHeader(HttpHeaderNames.ACCEPT.toString(), MediaType.APPLICATION_JSON)
+              .addHeader(HttpHeaderNames.CONTENT_TYPE.toString(), MediaType.APPLICATION_JSON)
               .setContent(requestBodyWriter.writeValueAsBytes(newRequests)),
           responseHandler,
           new Duration(config.getHttpLoadQueuePeonHostTimeout().getMillis() + 5000)
diff --git a/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java
index 7526ecb..885f2b2 100644
--- a/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java
+++ b/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java
@@ -36,6 +36,9 @@ import com.google.common.util.concurrent.ListenableScheduledFuture;
 import com.google.common.util.concurrent.ListeningScheduledExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
 import org.apache.druid.audit.AuditInfo;
 import org.apache.druid.common.config.JacksonConfigManager;
 import org.apache.druid.concurrent.LifecycleLock;
@@ -57,9 +60,6 @@ import org.apache.druid.java.util.http.client.response.SequenceInputStreamRespon
 import org.apache.druid.query.lookup.LookupsState;
 import org.apache.druid.server.http.HostAndPortWithScheme;
 import org.apache.druid.server.listener.resource.ListenerResource;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
 
 import javax.annotation.Nullable;
 import java.io.ByteArrayOutputStream;
@@ -796,8 +796,8 @@ public class LookupCoordinatorManager
 
       try (final InputStream result = httpClient.go(
           new Request(HttpMethod.POST, url)
-              .addHeader(HttpHeaders.Names.ACCEPT, SmileMediaTypes.APPLICATION_JACKSON_SMILE)
-              .addHeader(HttpHeaders.Names.CONTENT_TYPE, SmileMediaTypes.APPLICATION_JACKSON_SMILE)
+              .addHeader(HttpHeaderNames.ACCEPT.toString(), SmileMediaTypes.APPLICATION_JACKSON_SMILE)
+              .addHeader(HttpHeaderNames.CONTENT_TYPE.toString(), SmileMediaTypes.APPLICATION_JACKSON_SMILE)
               .setContent(smileMapper.writeValueAsBytes(lookupsUpdate)),
           makeResponseHandler(returnCode, reasonString),
           lookupCoordinatorManagerConfig.getHostTimeout()
@@ -849,7 +849,7 @@ public class LookupCoordinatorManager
 
       try (final InputStream result = httpClient.go(
           new Request(HttpMethod.GET, url)
-              .addHeader(HttpHeaders.Names.ACCEPT, SmileMediaTypes.APPLICATION_JACKSON_SMILE),
+              .addHeader(HttpHeaderNames.ACCEPT.toString(), SmileMediaTypes.APPLICATION_JACKSON_SMILE),
           makeResponseHandler(returnCode, reasonString),
           lookupCoordinatorManagerConfig.getHostTimeout()
       ).get()) {
@@ -902,8 +902,8 @@ public class LookupCoordinatorManager
         @Override
         public ClientResponse<InputStream> handleResponse(HttpResponse response, TrafficCop trafficCop)
         {
-          returnCode.set(response.getStatus().getCode());
-          reasonString.set(response.getStatus().getReasonPhrase());
+          returnCode.set(response.status().code());
+          reasonString.set(response.status().reasonPhrase());
           return super.handleResponse(response, trafficCop);
         }
       };
diff --git a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java
index 6322f73..ab94036 100644
--- a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java
+++ b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java
@@ -23,6 +23,9 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.timeout.ReadTimeoutException;
 import org.apache.druid.client.selector.ConnectionCountServerSelectorStrategy;
 import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy;
 import org.apache.druid.client.selector.QueryableDruidServer;
@@ -50,9 +53,6 @@ import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.NoneShardSpec;
 import org.easymock.Capture;
 import org.easymock.EasyMock;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.timeout.ReadTimeoutException;
 import org.joda.time.Duration;
 import org.junit.Assert;
 import org.junit.Before;
@@ -197,7 +197,7 @@ public class DirectDruidClientTest
     // simulate read timeout
     client.run(QueryPlus.wrap(query));
     Assert.assertEquals(2, client.getNumOpenConnections());
-    futureException.setException(new ReadTimeoutException());
+    futureException.setException(ReadTimeoutException.INSTANCE);
     Assert.assertEquals(1, client.getNumOpenConnections());
 
     // subsequent connections should work
diff --git a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java
index 896f90d..174bfaf 100644
--- a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java
+++ b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java
@@ -25,6 +25,10 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.apache.druid.discovery.DataNodeService;
 import org.apache.druid.discovery.DiscoveryDruidNode;
 import org.apache.druid.discovery.DruidNodeDiscovery;
@@ -47,11 +51,6 @@ import org.apache.druid.server.coordination.ServerType;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 import org.easymock.EasyMock;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.joda.time.Duration;
 import org.junit.Assert;
 import org.junit.Test;
@@ -330,13 +329,13 @@ public class HttpServerInventoryViewTest
             HttpVersion.HTTP_1_1,
             HttpResponseStatus.INTERNAL_SERVER_ERROR
         );
-        httpResponse.setContent(ChannelBuffers.buffer(0));
+
         httpResponseHandler.handleResponse(httpResponse, null);
         return Futures.immediateFailedFuture(new RuntimeException("server error"));
       }
 
       HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-      httpResponse.setContent(ChannelBuffers.buffer(0));
+
       httpResponseHandler.handleResponse(httpResponse, null);
       try {
         return results.take();
diff --git a/server/src/test/java/org/apache/druid/client/TestHttpClient.java b/server/src/test/java/org/apache/druid/client/TestHttpClient.java
index 9bef9be..9dd6b1a 100644
--- a/server/src/test/java/org/apache/druid/client/TestHttpClient.java
+++ b/server/src/test/java/org/apache/druid/client/TestHttpClient.java
@@ -22,6 +22,14 @@ package org.apache.druid.client;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.DefaultLastHttpContent;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.NonnullPair;
 import org.apache.druid.java.util.common.StringUtils;
@@ -41,11 +49,6 @@ import org.apache.druid.query.context.ResponseContext;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.server.QueryResource;
 import org.apache.druid.timeline.DataSegment;
-import org.jboss.netty.buffer.HeapChannelBufferFactory;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.joda.time.Duration;
 
 import javax.annotation.Nullable;
@@ -133,11 +136,14 @@ public class TestHttpClient implements HttpClient
       );
       final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
       response.headers().add(QueryResource.HEADER_RESPONSE_CONTEXT, serializationResult.getResult());
-      response.setContent(
-          HeapChannelBufferFactory.getInstance().getBuffer(serializedContent, 0, serializedContent.length)
-      );
-      final ClientResponse<Intermediate> intermClientResponse = handler.handleResponse(response, NOOP_TRAFFIC_COP);
-      final ClientResponse<Final> finalClientResponse = handler.done(intermClientResponse);
+
+      final ByteBuf byteBuf = Unpooled.wrappedBuffer(serializedContent);
+      final HttpContent content = new DefaultLastHttpContent(byteBuf);
+
+      final ClientResponse<Intermediate> intermClientResponse0 = handler.handleResponse(response, NOOP_TRAFFIC_COP);
+      final ClientResponse<Intermediate> intermClientResponse1 = handler.handleChunk(intermClientResponse0, content, 0);
+      final ClientResponse<Final> finalClientResponse = handler.done(intermClientResponse1);
+
       return Futures.immediateFuture(finalClientResponse.getObj());
     }
     catch (IOException e) {
diff --git a/server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java b/server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java
index b2fed85..f75494c 100644
--- a/server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java
+++ b/server/src/test/java/org/apache/druid/client/indexing/HttpIndexingServiceClientTest.java
@@ -22,17 +22,14 @@ package org.apache.druid.client.indexing;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.easymock.EasyMock;
-import org.jboss.netty.buffer.BigEndianHeapChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -40,7 +37,6 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
 import java.net.URL;
-import java.nio.charset.StandardCharsets;
 import java.util.Map;
 
 public class HttpIndexingServiceClientTest
@@ -89,13 +85,11 @@ public class HttpIndexingServiceClientTest
     };
 
     HttpResponse response = EasyMock.createMock(HttpResponse.class);
-    EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0));
+    EasyMock.expect(response.status()).andReturn(HttpResponseStatus.OK).anyTimes();
     EasyMock.replay(response);
 
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        response,
-        StandardCharsets.UTF_8
+        response
     ).addChunk(jsonMapper.writeValueAsString(samplerResponse));
 
     EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class)))
@@ -142,13 +136,11 @@ public class HttpIndexingServiceClientTest
       }
     };
     HttpResponse response = EasyMock.createMock(HttpResponse.class);
-    EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0));
+    EasyMock.expect(response.status()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).anyTimes();
     EasyMock.replay(response);
 
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        response,
-        StandardCharsets.UTF_8
+        response
     ).addChunk("");
 
     EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class)))
@@ -170,15 +162,13 @@ public class HttpIndexingServiceClientTest
   {
     String taskId = "testTaskId";
     HttpResponse response = EasyMock.createMock(HttpResponse.class);
-    EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0));
+    EasyMock.expect(response.status()).andReturn(HttpResponseStatus.OK).anyTimes();
     EasyMock.replay(response);
 
     Map<String, Object> dummyResponse = ImmutableMap.of("test", "value");
 
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        response,
-        StandardCharsets.UTF_8
+        response
     ).addChunk(jsonMapper.writeValueAsString(dummyResponse));
 
     EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class)))
@@ -206,17 +196,12 @@ public class HttpIndexingServiceClientTest
     HttpResponse response = EasyMock.createMock(HttpResponse.class);
     String errorMsg = "No task reports were found for this task. "
                       + "The task may not exist, or it may not have completed yet.";
-    ChannelBuffer buf = ChannelBuffers.buffer(errorMsg.length());
-    buf.writeBytes(errorMsg.getBytes(StandardCharsets.UTF_8));
-
-    EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(buf);
+    EasyMock.expect(response.status()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes();
     EasyMock.replay(response);
 
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        response,
-        StandardCharsets.UTF_8
-    ).addChunk("");
+        response
+    ).addChunk(errorMsg);
 
     EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class)))
             .andReturn(responseHolder)
@@ -241,13 +226,11 @@ public class HttpIndexingServiceClientTest
   {
     String taskId = "testTaskId";
     HttpResponse response = EasyMock.createMock(HttpResponse.class);
-    EasyMock.expect(response.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0));
+    EasyMock.expect(response.status()).andReturn(HttpResponseStatus.OK).anyTimes();
     EasyMock.replay(response);
 
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        response,
-        StandardCharsets.UTF_8
+        response
     ).addChunk("");
 
     EasyMock.expect(druidLeaderClient.go(EasyMock.anyObject(Request.class)))
diff --git a/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java b/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java
index 570720d..a6d4717 100644
--- a/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java
+++ b/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java
@@ -21,7 +21,6 @@ package org.apache.druid.discovery;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.inject.Binder;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
 import com.google.inject.Key;
@@ -29,6 +28,7 @@ import com.google.inject.Module;
 import com.google.inject.name.Named;
 import com.google.inject.name.Names;
 import com.google.inject.servlet.GuiceFilter;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.guice.GuiceInjectors;
 import org.apache.druid.guice.Jerseys;
 import org.apache.druid.guice.JsonConfigProvider;
@@ -49,11 +49,8 @@ import org.eclipse.jetty.server.handler.HandlerList;
 import org.eclipse.jetty.servlet.DefaultServlet;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.junit.Assert;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
 
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
@@ -69,9 +66,6 @@ import java.nio.charset.StandardCharsets;
  */
 public class DruidLeaderClientTest extends BaseJettyTest
 {
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
   private DiscoveryDruidNode discoveryDruidNode;
   private HttpClient httpClient;
 
@@ -83,21 +77,16 @@ public class DruidLeaderClientTest extends BaseJettyTest
 
     Injector injector = Initialization.makeInjectorWithModules(
         GuiceInjectors.makeStartupInjector(), ImmutableList.<Module>of(
-            new Module()
-            {
-              @Override
-              public void configure(Binder binder)
-              {
-                JsonConfigProvider.bindInstance(
-                    binder,
-                    Key.get(DruidNode.class, Self.class),
-                    node
-                );
-                binder.bind(Integer.class).annotatedWith(Names.named("port")).toInstance(node.getPlaintextPort());
-                binder.bind(JettyServerInitializer.class).to(TestJettyServerInitializer.class).in(LazySingleton.class);
-                Jerseys.addResource(binder, SimpleResource.class);
-                LifecycleModule.register(binder, Server.class);
-              }
+            binder -> {
+              JsonConfigProvider.bindInstance(
+                  binder,
+                  Key.get(DruidNode.class, Self.class),
+                  node
+              );
+              binder.bind(Integer.class).annotatedWith(Names.named("port")).toInstance(node.getPlaintextPort());
+              binder.bind(JettyServerInitializer.class).to(TestJettyServerInitializer.class).in(LazySingleton.class);
+              Jerseys.addResource(binder, SimpleResource.class);
+              LifecycleModule.register(binder, Server.class);
             }
         )
     );
@@ -150,9 +139,10 @@ public class DruidLeaderClientTest extends BaseJettyTest
     );
     druidLeaderClient.start();
 
-    expectedException.expect(IOException.class);
-    expectedException.expectMessage("No known server");
-    druidLeaderClient.makeRequest(HttpMethod.POST, "/simple/direct");
+    final IOException exception = Assert.assertThrows(IOException.class, () -> {
+      druidLeaderClient.makeRequest(HttpMethod.POST, "/simple/direct");
+    });
+    Assert.assertEquals("No known server", exception.getMessage());
   }
 
   @Test
diff --git a/server/src/test/java/org/apache/druid/query/lookup/LookupReferencesManagerTest.java b/server/src/test/java/org/apache/druid/query/lookup/LookupReferencesManagerTest.java
index 8cc6956..64ed808 100644
--- a/server/src/test/java/org/apache/druid/query/lookup/LookupReferencesManagerTest.java
+++ b/server/src/test/java/org/apache/druid/query/lookup/LookupReferencesManagerTest.java
@@ -22,6 +22,9 @@ package org.apache.druid.query.lookup;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.emitter.EmittingLogger;
@@ -29,10 +32,6 @@ import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.server.metrics.NoopServiceEmitter;
 import org.easymock.EasyMock;
-import org.jboss.netty.buffer.BigEndianHeapChannelBuffer;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -41,7 +40,6 @@ import org.junit.rules.TemporaryFolder;
 
 import java.io.IOException;
 import java.net.URL;
-import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
@@ -89,8 +87,7 @@ public class LookupReferencesManagerTest
   private static HttpResponse newEmptyResponse(final HttpResponseStatus status)
   {
     final HttpResponse response = EasyMock.createNiceMock(HttpResponse.class);
-    EasyMock.expect(response.getStatus()).andReturn(status).anyTimes();
-    EasyMock.expect(response.getContent()).andReturn(new BigEndianHeapChannelBuffer(0));
+    EasyMock.expect(response.status()).andReturn(status).anyTimes();
     EasyMock.replay(response);
     return response;
   }
@@ -115,8 +112,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -178,8 +174,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -218,8 +213,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -251,8 +245,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -281,8 +274,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -313,8 +305,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -349,8 +340,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -379,8 +369,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -410,8 +399,7 @@ public class LookupReferencesManagerTest
         druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/coordinator/v1/lookups/config/lookupTier?detailed=true")
     ).andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -471,8 +459,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -515,8 +502,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -603,8 +589,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
     EasyMock.replay(druidLeaderClient);
@@ -714,8 +699,7 @@ public class LookupReferencesManagerTest
     ))
             .andReturn(request);
     StringFullResponseHolder responseHolder = new StringFullResponseHolder(
-        newEmptyResponse(HttpResponseStatus.OK),
-        StandardCharsets.UTF_8
+        newEmptyResponse(HttpResponseStatus.OK)
     ).addChunk(strResult);
     EasyMock.expect(druidLeaderClient.go(request)).andReturn(responseHolder);
 
diff --git a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncerTest.java
index f4dd564..741be5b 100644
--- a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncerTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncerTest.java
@@ -24,6 +24,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.java.util.http.client.HttpClient;
@@ -31,11 +35,6 @@ import org.apache.druid.java.util.http.client.Request;
 import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
 import org.apache.druid.segment.TestHelper;
 import org.easymock.EasyMock;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.joda.time.Duration;
 import org.junit.Test;
 
@@ -192,13 +191,13 @@ public class ChangeRequestHttpSyncerTest
             HttpVersion.HTTP_1_1,
             HttpResponseStatus.INTERNAL_SERVER_ERROR
         );
-        httpResponse.setContent(ChannelBuffers.buffer(0));
+
         httpResponseHandler.handleResponse(httpResponse, null);
         return Futures.immediateFailedFuture(new RuntimeException("server error"));
       }
 
       HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-      httpResponse.setContent(ChannelBuffers.buffer(0));
+
       httpResponseHandler.handleResponse(httpResponse, null);
       try {
         return results.take();
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java
index f83c155..d0a95d6 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java
@@ -23,6 +23,10 @@ import com.fasterxml.jackson.core.type.TypeReference;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.concurrent.Execs;
@@ -34,11 +38,6 @@ import org.apache.druid.server.coordination.DataSegmentChangeRequest;
 import org.apache.druid.server.coordination.SegmentLoadDropHandler;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.joda.time.Duration;
 import org.junit.Test;
 
@@ -193,7 +192,7 @@ public class HttpLoadQueuePeonTest
     )
     {
       HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
-      httpResponse.setContent(ChannelBuffers.buffer(0));
+
       httpResponseHandler.handleResponse(httpResponse, null);
       try {
         List<DataSegmentChangeRequest> changeRequests = ServerTestHelper.MAPPER.readValue(
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java
index 68b9eb2..f2ca77d 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java
@@ -26,6 +26,11 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import junitparams.converters.Nullable;
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.druid.client.DataSourcesSnapshot;
@@ -89,11 +94,6 @@ import org.apache.druid.timeline.partition.ShardSpec;
 import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
 import org.apache.druid.utils.Streams;
 import org.easymock.EasyMock;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.joda.time.DateTime;
 import org.joda.time.Interval;
 import org.joda.time.Period;
@@ -108,7 +108,6 @@ import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.net.URL;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -1726,8 +1725,7 @@ public class CompactSegmentsTest
     {
       final HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK);
       final StringFullResponseHolder holder = new StringFullResponseHolder(
-          httpResponse,
-          StandardCharsets.UTF_8
+          httpResponse
       );
       holder.addChunk(content);
       return holder;
diff --git a/server/src/test/java/org/apache/druid/server/initialization/JettyQosTest.java b/server/src/test/java/org/apache/druid/server/initialization/JettyQosTest.java
index e9f7fc0..a884327 100644
--- a/server/src/test/java/org/apache/druid/server/initialization/JettyQosTest.java
+++ b/server/src/test/java/org/apache/druid/server/initialization/JettyQosTest.java
@@ -27,6 +27,7 @@ import com.google.inject.Binder;
 import com.google.inject.Injector;
 import com.google.inject.Key;
 import com.google.inject.Module;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.druid.guice.GuiceInjectors;
 import org.apache.druid.guice.Jerseys;
 import org.apache.druid.guice.JsonConfigProvider;
@@ -46,7 +47,6 @@ import org.apache.druid.server.security.AuthTestUtils;
 import org.apache.druid.server.security.AuthorizerMapper;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.util.thread.QueuedThreadPool;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.junit.Assert;
 import org.junit.Test;
 
diff --git a/server/src/test/java/org/apache/druid/server/initialization/JettyTest.java b/server/src/test/java/org/apache/druid/server/initialization/JettyTest.java
index ff8ae56..ab6a38b 100644
--- a/server/src/test/java/org/apache/druid/server/initialization/JettyTest.java
+++ b/server/src/test/java/org/apache/druid/server/initialization/JettyTest.java
@@ -26,6 +26,7 @@ import com.google.inject.Injector;
 import com.google.inject.Key;
 import com.google.inject.Module;
 import com.google.inject.multibindings.Multibinder;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.commons.io.IOUtils;
 import org.apache.druid.guice.GuiceInjectors;
 import org.apache.druid.guice.Jerseys;
@@ -52,7 +53,6 @@ import org.apache.druid.server.security.CustomCheckX509TrustManager;
 import org.apache.druid.server.security.TLSCertificateChecker;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.Duration;
 import org.junit.Assert;
 import org.junit.Ignore;
diff --git a/services/src/main/java/org/apache/druid/cli/PullDependencies.java b/services/src/main/java/org/apache/druid/cli/PullDependencies.java
index 936ff5b..efe5b04 100644
--- a/services/src/main/java/org/apache/druid/cli/PullDependencies.java
+++ b/services/src/main/java/org/apache/druid/cli/PullDependencies.java
@@ -126,7 +126,7 @@ public class PullDependencies implements Runnable
       "com.metamx.druid",
       "asm",
       "org.ow2.asm",
-      "org.jboss.netty",
+      "io.netty",
       "com.google.guava",
       "com.google.code.findbugs",
       "com.google.protobuf",
diff --git a/services/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java b/services/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java
index 44f4605..9459386 100644
--- a/services/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java
+++ b/services/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java
@@ -26,6 +26,8 @@ import com.google.common.base.Supplier;
 import com.google.common.collect.Maps;
 import com.google.errorprone.annotations.concurrent.GuardedBy;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.client.coordinator.Coordinator;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.guice.ManageLifecycle;
@@ -39,8 +41,6 @@ import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
 import org.apache.druid.server.coordinator.rules.Rule;
 import org.apache.druid.server.http.RulesResource;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Duration;
 
 import java.util.ArrayList;
diff --git a/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java b/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java
index 89741c0..e879e6f 100644
--- a/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java
+++ b/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java
@@ -22,6 +22,7 @@ package org.apache.druid.server.router;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.druid.discovery.DruidLeaderClient;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.Intervals;
@@ -32,7 +33,6 @@ import org.apache.druid.server.coordinator.rules.IntervalDropRule;
 import org.apache.druid.server.coordinator.rules.PeriodLoadRule;
 import org.apache.druid.server.coordinator.rules.Rule;
 import org.easymock.EasyMock;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.joda.time.Period;
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
index 42b707e..f8e61b9 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
@@ -33,6 +33,7 @@ import com.google.common.collect.Sets;
 import com.google.common.net.HostAndPort;
 import com.google.common.util.concurrent.Futures;
 import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpMethod;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.linq4j.DefaultEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
@@ -83,7 +84,6 @@ import org.apache.druid.sql.calcite.table.RowSignatures;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 import org.apache.druid.timeline.SegmentWithOvershadowedStatus;
-import org.jboss.netty.handler.codec.http.HttpMethod;
 
 import javax.annotation.Nullable;
 import javax.servlet.http.HttpServletResponse;
@@ -1064,12 +1064,12 @@ public class SystemSchema extends AbstractSchema
           new InputStreamFullResponseHandler()
       );
 
-      if (responseHolder.getStatus().getCode() != HttpServletResponse.SC_OK) {
+      if (responseHolder.getStatus().code() != HttpServletResponse.SC_OK) {
         throw new RE(
             "Failed to talk to leader node at [%s]. Error code[%d], description[%s].",
             query,
-            responseHolder.getStatus().getCode(),
-            responseHolder.getStatus().getReasonPhrase()
+            responseHolder.getStatus().code(),
+            responseHolder.getStatus().reasonPhrase()
         );
       }
     }
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
index 9ce4852..05102cd 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
@@ -24,6 +24,11 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
 import junitparams.converters.Nullable;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
@@ -100,11 +105,6 @@ import org.apache.druid.timeline.SegmentId;
 import org.apache.druid.timeline.SegmentWithOvershadowedStatus;
 import org.apache.druid.timeline.partition.NumberedShardSpec;
 import org.easymock.EasyMock;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpVersion;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -1286,7 +1286,7 @@ public class SystemSchemaTest extends CalciteTestBase
 
     EasyMock.expect(client.go(EasyMock.eq(request), EasyMock.anyObject(InputStreamFullResponseHandler.class))).andReturn(responseHolder).once();
 
-    EasyMock.expect(responseHandler.getStatus()).andReturn(httpResp.getStatus().getCode()).anyTimes();
+    EasyMock.expect(responseHandler.getStatus()).andReturn(httpResp.status().code()).anyTimes();
     EasyMock.expect(request.getUrl())
             .andReturn(new URL("http://test-host:1234/druid/indexer/v1/supervisor?system"))
             .anyTimes();
@@ -1357,7 +1357,7 @@ public class SystemSchemaTest extends CalciteTestBase
             .andReturn(createFullResponseHolder(httpResponse, json));
 
     EasyMock.expect(responseHandler.getStatus())
-            .andReturn(httpResponse.getStatus().getCode())
+            .andReturn(httpResponse.status().code())
             .anyTimes();
     EasyMock.expect(request.getUrl())
             .andReturn(new URL("http://test-host:1234/druid/indexer/v1/supervisor?system"))

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org