HBASE-18271 Shade netty Purge mention of netty-all.

Add in hbase-thirdparty hbase-shaded-netty instead.
s/io.netty/org.apache.hadoop.hbase.shaded.io.netty/ everywhere in hbase.

Also set a system property when running tests and when starting
hbase; required by netty so can find the relocation files in the
bundled .so.
This commit is contained in:
Michael Stack 2017-08-02 14:47:51 -07:00
parent 923195c39e
commit c6ac04ab39
62 changed files with 330 additions and 318 deletions

View File

@ -473,8 +473,16 @@ fi
HEAP_SETTINGS="$JAVA_HEAP_MAX $JAVA_OFFHEAP_MAX"
# Exec unless HBASE_NOEXEC is set.
export CLASSPATH
# Netty is shaded in hbase. The shaded netty is pulled in with org.apache.hbase.thirdparty:hbase-shaded-netty.
# It has a .so in it. Shading requires rename of the .so and then passing a system property so netty finds the
# renamed .so and associates it w/ the relocated netty files.
# So we can find the relocated .so, we need to add a system property pointing at new location. Trick is from:
# https://stackoverflow.com/questions/33825743/rename-files-inside-a-jar-using-some-maven-plugin
# See toward the end of this issue for how to pass config: https://github.com/netty/netty/issues/6665
#
NETTY_PROP="-Dorg.apache.hadoop.hbase.shaded.io.netty.packagePrefix=org.apache.hadoop.hbase.shaded."
if [ "${HBASE_NOEXEC}" != "" ]; then
"$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
"$JAVA" -Dproc_$COMMAND "$NETTY_PROP" -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
else
exec "$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
exec "$JAVA" -Dproc_$COMMAND "$NETTY_PROP" -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
fi

View File

@ -160,8 +160,8 @@
<artifactId>protobuf-java</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<groupId>org.apache.hbase.thirdparty</groupId>
<artifactId>hbase-shaded-netty</artifactId>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;

View File

@ -24,7 +24,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.io.IOException;
import java.util.ArrayList;

View File

@ -23,7 +23,7 @@ import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLE
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;

View File

@ -30,8 +30,8 @@ import java.util.stream.Collectors;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import io.netty.util.Timeout;
import io.netty.util.TimerTask;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.client;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.util.concurrent.CompletableFuture;

View File

@ -21,8 +21,8 @@ import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findException;
import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
import io.netty.util.HashedWheelTimer;
import io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;

View File

@ -23,7 +23,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.SLEEP_DELTA_NS;
import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.util.ArrayList;
import java.util.List;

View File

@ -21,7 +21,7 @@ import static org.apache.hadoop.hbase.shaded.com.google.common.base.Precondition
import static org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.util.List;
import java.util.concurrent.CompletableFuture;

View File

@ -30,8 +30,8 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.updateServerSideMet
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import io.netty.util.HashedWheelTimer;
import io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import java.io.IOException;
import java.util.ArrayList;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.client;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.client;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;

View File

@ -20,16 +20,16 @@
package org.apache.hadoop.hbase.client;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.ByteBufInputStream;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.DatagramChannel;
import io.netty.channel.socket.DatagramPacket;
import io.netty.channel.socket.nio.NioDatagramChannel;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufInputStream;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramPacket;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioDatagramChannel;
import java.io.Closeable;
import java.io.IOException;

View File

@ -42,8 +42,8 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Message;
import com.google.protobuf.RpcChannel;
import io.netty.util.Timeout;
import io.netty.util.TimerTask;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
import java.util.stream.Stream;

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import java.io.IOException;
import java.net.InetSocketAddress;

View File

@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.channel.ChannelDuplexHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelDuplexHandler;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
import java.io.IOException;
import java.util.HashMap;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;

View File

@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.ByteBufOutputStream;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufAllocator;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
import java.io.ByteArrayInputStream;
import java.io.IOException;

View File

@ -17,11 +17,11 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Pair;

View File

@ -17,11 +17,11 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
import java.io.IOException;
import java.net.SocketAddress;

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import java.util.HashMap;
import java.util.Map;

View File

@ -22,26 +22,26 @@ import static org.apache.hadoop.hbase.ipc.CallEvent.Type.TIMEOUT;
import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
import io.netty.handler.timeout.ReadTimeoutHandler;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.ReadTimeoutHandler;
import org.apache.hadoop.hbase.security.NettyHBaseRpcConnectionHeaderHandler;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufOutputStream;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import io.netty.handler.timeout.IdleStateHandler;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import io.netty.util.concurrent.Promise;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFuture;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFutureListener;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandler;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
import org.apache.hadoop.hbase.shaded.io.netty.util.ReferenceCountUtil;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Future;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.FutureListener;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
import java.io.IOException;
import java.util.concurrent.Executors;

View File

@ -21,14 +21,14 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message.Builder;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufInputStream;
import io.netty.buffer.ByteBufOutputStream;
import io.netty.channel.ChannelDuplexHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.handler.timeout.IdleStateEvent;
import io.netty.util.concurrent.PromiseCombiner;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufInputStream;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelDuplexHandler;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
import java.io.IOException;
import java.util.HashMap;

View File

@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.util.HashedWheelTimer;
import io.netty.util.Timeout;
import io.netty.util.TimerTask;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
import java.io.IOException;
import java.net.UnknownHostException;

View File

@ -18,16 +18,16 @@
package org.apache.hadoop.hbase.security;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;
/**
* Unwrap messages with Crypto AES. Should be placed after a
* {@link io.netty.handler.codec.LengthFieldBasedFrameDecoder}
* io.netty.handler.codec.LengthFieldBasedFrameDecoder
*/
@InterfaceAudience.Private
public class CryptoAESUnwrapHandler extends SimpleChannelInboundHandler<ByteBuf> {

View File

@ -18,14 +18,14 @@
package org.apache.hadoop.hbase.security;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOutboundHandlerAdapter;
import io.netty.channel.ChannelPromise;
import io.netty.channel.CoalescingBufferQueue;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.PromiseCombiner;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
import org.apache.hadoop.hbase.shaded.io.netty.channel.CoalescingBufferQueue;
import org.apache.hadoop.hbase.shaded.io.netty.util.ReferenceCountUtil;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;

View File

@ -18,12 +18,12 @@
package org.apache.hadoop.hbase.security;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import io.netty.util.concurrent.Promise;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hbase.security;
import io.netty.channel.ChannelPipeline;
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import java.io.IOException;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.security;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.util.concurrent.Promise;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;

View File

@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase.security;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.ByteToMessageDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.ByteToMessageDecoder;
import java.io.IOException;
import java.util.List;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.security;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import javax.security.sasl.SaslClient;
@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Unwrap sasl messages. Should be placed after a
* {@link io.netty.handler.codec.LengthFieldBasedFrameDecoder}
* io.netty.handler.codec.LengthFieldBasedFrameDecoder
*/
@InterfaceAudience.Private
public class SaslUnwrapHandler extends SimpleChannelInboundHandler<ByteBuf> {

View File

@ -17,14 +17,14 @@
*/
package org.apache.hadoop.hbase.security;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOutboundHandlerAdapter;
import io.netty.channel.ChannelPromise;
import io.netty.channel.CoalescingBufferQueue;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.PromiseCombiner;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
import org.apache.hadoop.hbase.shaded.io.netty.channel.CoalescingBufferQueue;
import org.apache.hadoop.hbase.shaded.io.netty.util.ReferenceCountUtil;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
import java.io.IOException;

View File

@ -20,28 +20,28 @@ package org.apache.hadoop.hbase.client.example;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.group.ChannelGroup;
import io.netty.channel.group.DefaultChannelGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.FullHttpRequest;
import io.netty.handler.codec.http.HttpHeaderNames;
import io.netty.handler.codec.http.HttpObjectAggregator;
import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpServerCodec;
import io.netty.handler.codec.http.HttpVersion;
import io.netty.handler.codec.http.QueryStringDecoder;
import io.netty.util.concurrent.GlobalEventExecutor;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ServerBootstrap;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInitializer;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.channel.group.ChannelGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.group.DefaultChannelGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioServerSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.DefaultFullHttpResponse;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.FullHttpRequest;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpHeaderNames;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpObjectAggregator;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpResponseStatus;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpServerCodec;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpVersion;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.QueryStringDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.GlobalEventExecutor;
import java.io.IOException;
import java.net.InetSocketAddress;

View File

@ -116,12 +116,12 @@
<artifactId>hbase-shaded-miscellaneous</artifactId>
</dependency>
<dependency>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
<groupId>org.apache.hbase.thirdparty</groupId>
<artifactId>hbase-shaded-netty</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</dependency>
</dependencies>

View File

@ -164,8 +164,8 @@
<artifactId>protobuf-java</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<groupId>org.apache.hbase.thirdparty</groupId>
<artifactId>hbase-shaded-netty</artifactId>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>

View File

@ -264,6 +264,7 @@
</properties>
<systemPropertyVariables>
<test.build.webapps>target/test-classes/webapps</test.build.webapps>
<org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>org.apache.hadoop.hbase.shaded.</org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>
</systemPropertyVariables>
</configuration>
</plugin>
@ -521,10 +522,6 @@
<groupId>org.jamon</groupId>
<artifactId>jamon-runtime</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
</dependency>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>javax.ws.rs-api</artifactId>
@ -832,7 +829,7 @@
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
<version>3.6.2.Final</version>
<version>${netty.hadoop.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

View File

@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.io.asyncfs;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
import io.netty.channel.Channel;
import io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import java.io.IOException;
import java.io.InterruptedIOException;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hbase.io.asyncfs;
import static io.netty.handler.timeout.IdleState.READER_IDLE;
import static io.netty.handler.timeout.IdleState.WRITER_IDLE;
import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.READER_IDLE;
import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.WRITER_IDLE;
import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.HEART_BEAT_SEQNO;
import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.READ_TIMEOUT;
import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.completeFile;
@ -27,20 +27,20 @@ import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHel
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandler.Sharable;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.EventLoop;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.protobuf.ProtobufDecoder;
import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
import io.netty.handler.timeout.IdleStateEvent;
import io.netty.handler.timeout.IdleStateHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.netty.util.concurrent.PromiseCombiner;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufAllocator;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandler.Sharable;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Future;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.PromiseCombiner;
import java.io.IOException;
import java.nio.ByteBuffer;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hbase.io.asyncfs;
import static io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;
import static io.netty.handler.timeout.IdleState.READER_IDLE;
import static org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;
import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.READER_IDLE;
import static org.apache.hadoop.fs.CreateFlag.CREATE;
import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.createEncryptor;
@ -32,27 +32,27 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
import com.google.protobuf.CodedOutputStream;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.ByteBufOutputStream;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoop;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.protobuf.ProtobufDecoder;
import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
import io.netty.handler.timeout.IdleStateEvent;
import io.netty.handler.timeout.IdleStateHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import io.netty.util.concurrent.Promise;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufAllocator;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.PooledByteBufAllocator;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFuture;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelFutureListener;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandler;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInitializer;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Future;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.FutureListener;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.io.asyncfs;
import static io.netty.handler.timeout.IdleState.READER_IDLE;
import static org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleState.READER_IDLE;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Charsets;
@ -26,24 +26,24 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import com.google.protobuf.CodedOutputStream;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufOutputStream;
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelDuplexHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOutboundHandlerAdapter;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.ChannelPromise;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import io.netty.handler.codec.MessageToByteEncoder;
import io.netty.handler.codec.protobuf.ProtobufDecoder;
import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
import io.netty.handler.timeout.IdleStateEvent;
import io.netty.handler.timeout.IdleStateHandler;
import io.netty.util.concurrent.Promise;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBufOutputStream;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.CompositeByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelDuplexHandler;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.MessageToByteEncoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateEvent;
import org.apache.hadoop.hbase.shaded.io.netty.handler.timeout.IdleStateHandler;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.Promise;
import java.io.IOException;
import java.lang.reflect.Field;

View File

@ -17,21 +17,21 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.ServerChannel;
import io.netty.channel.group.ChannelGroup;
import io.netty.channel.group.DefaultChannelGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.codec.FixedLengthFrameDecoder;
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import io.netty.util.concurrent.DefaultThreadFactory;
import io.netty.util.concurrent.GlobalEventExecutor;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ServerBootstrap;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInitializer;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ServerChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.group.ChannelGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.group.DefaultChannelGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioServerSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.FixedLengthFrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.GlobalEventExecutor;
import java.io.IOException;
import java.io.InterruptedIOException;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.SimpleChannelInboundHandler;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPipeline;
import org.apache.hadoop.hbase.shaded.io.netty.channel.SimpleChannelInboundHandler;
import java.nio.ByteBuffer;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.group.ChannelGroup;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelInboundHandlerAdapter;
import org.apache.hadoop.hbase.shaded.io.netty.channel.group.ChannelGroup;
import org.apache.hadoop.hbase.classification.InterfaceAudience;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOutboundHandlerAdapter;
import io.netty.channel.ChannelPromise;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOutboundHandlerAdapter;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelPromise;
import org.apache.hadoop.hbase.classification.InterfaceAudience;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.buffer.ByteBuf;
import io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import java.io.IOException;
import java.net.InetAddress;

View File

@ -792,7 +792,7 @@ public class TableMapReduceUtil {
prefixTreeCodecClass, // hbase-prefix-tree (if null will be skipped)
// pull necessary dependencies
org.apache.zookeeper.ZooKeeper.class,
io.netty.channel.Channel.class,
org.apache.hadoop.hbase.shaded.io.netty.channel.Channel.class,
com.google.protobuf.Message.class,
org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists.class,
org.apache.htrace.Trace.class,

View File

@ -21,21 +21,21 @@
package org.apache.hadoop.hbase.master;
import io.netty.bootstrap.Bootstrap;
import io.netty.bootstrap.ChannelFactory;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelException;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.DatagramChannel;
import io.netty.channel.socket.DatagramPacket;
import io.netty.channel.socket.InternetProtocolFamily;
import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.handler.codec.MessageToMessageEncoder;
import io.netty.util.internal.StringUtil;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.Bootstrap;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ChannelFactory;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.Unpooled;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelException;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ChannelOption;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.DatagramPacket;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.InternetProtocolFamily;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioDatagramChannel;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.MessageToMessageEncoder;
import org.apache.hadoop.hbase.shaded.io.netty.util.internal.StringUtil;
import java.io.Closeable;
import java.io.IOException;

View File

@ -24,9 +24,9 @@ import com.lmax.disruptor.RingBuffer;
import com.lmax.disruptor.Sequence;
import com.lmax.disruptor.Sequencer;
import io.netty.channel.Channel;
import io.netty.channel.EventLoop;
import io.netty.util.concurrent.SingleThreadEventExecutor;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor;
import java.io.IOException;
import java.io.InterruptedIOException;

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
import io.netty.channel.Channel;
import io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import java.io.IOException;
import java.io.InterruptedIOException;

View File

@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.crypto.Encryptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
import io.netty.channel.Channel;
import io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class SecureAsyncProtobufLogWriter extends AsyncProtobufLogWriter {

View File

@ -17,16 +17,16 @@
*/
package org.apache.hadoop.hbase.util;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.ServerChannel;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.epoll.EpollServerSocketChannel;
import io.netty.channel.epoll.EpollSocketChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.ServerChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollServerSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioServerSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
import java.util.concurrent.ThreadFactory;

View File

@ -19,12 +19,12 @@ package org.apache.hadoop.hbase.wal;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
import io.netty.channel.Channel;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory;
import java.io.IOException;

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.wal;
import com.google.common.base.Preconditions;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import java.util.HashMap;
import java.util.Map;

View File

@ -24,11 +24,11 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;
import io.netty.channel.Channel;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import java.io.FileNotFoundException;
import java.io.IOException;

View File

@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.io.asyncfs;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import java.io.IOException;
import java.util.concurrent.ExecutionException;

View File

@ -31,11 +31,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIP
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
import io.netty.channel.Channel;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import java.io.File;
import java.io.IOException;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.ipc;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.epoll.EpollSocketChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.epoll.EpollSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import java.util.ArrayList;
import java.util.Collection;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.regionserver.wal;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import java.io.IOException;
import java.util.List;

View File

@ -19,10 +19,10 @@ package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import java.io.IOException;
import java.io.InterruptedIOException;

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.regionserver.wal;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioSocketChannel;
import java.io.IOException;

19
pom.xml
View File

@ -626,6 +626,17 @@
<redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
<systemPropertyVariables>
<test.build.classes>${test.build.classes}</test.build.classes>
<!--For shaded netty, to find the relocated .so.
Trick from
https://stackoverflow.com/questions/33825743/rename-files-inside-a-jar-using-some-maven-plugin
The netty jar has a .so in it. Shading requires rename of the .so and then passing a system
property so netty finds the renamed .so and associates it w/ the relocated netty files.
The relocated netty is in hbase-thirdparty dependency. Just set this propery globally rather
than per module.
-->
<org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>org.apache.hadoop.hbase.shaded.</org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>
</systemPropertyVariables>
<excludes>
@ -1394,7 +1405,8 @@
<clover.version>4.0.3</clover.version>
<jamon-runtime.version>2.4.1</jamon-runtime.version>
<jettison.version>1.3.8</jettison.version>
<netty.version>4.1.9.Final</netty.version>
<!--This property is for hadoops netty. HBase netty
comes in via hbase-thirdparty hbase-shaded-netty-->
<netty.hadoop.version>3.6.2.Final</netty.hadoop.version>
<!--Make sure these joni/jcodings are compatible with the versions used by jruby-->
<joni.version>2.1.11</joni.version>
@ -1789,11 +1801,6 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<version>${netty.version}</version>
</dependency>
<dependency>
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>